From 3cca8410385c216ced1c9366a8e8cda8503f3407 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 9 Feb 2024 18:55:21 +0100 Subject: [PATCH 001/158] Unite s3/hdfs/azure storage implementations into a single class on top of IObjectStorage --- src/Backups/BackupIO_AzureBlobStorage.cpp | 26 +- src/Backups/BackupIO_AzureBlobStorage.h | 46 +- .../registerBackupEngineAzureBlobStorage.cpp | 18 +- src/CMakeLists.txt | 1 + .../AzureBlobStorage/AzureObjectStorage.cpp | 8 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 14 +- src/Disks/ObjectStorages/IObjectStorage.h | 5 +- src/Disks/ObjectStorages/IObjectStorage_fwd.h | 3 + .../MetadataStorageFromPlainObjectStorage.cpp | 2 +- .../ObjectStorages/ObjectStorageIterator.cpp | 2 +- .../ObjectStorages/ObjectStorageIterator.h | 22 +- .../ObjectStorageIteratorAsync.cpp | 4 +- .../ObjectStorageIteratorAsync.h | 4 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 26 +- .../copyAzureBlobStorageFile.h | 3 +- src/Interpreters/InterpreterSystemQuery.cpp | 6 +- src/Server/TCPHandler.cpp | 2 +- .../DataLakes/DeltaLakeMetadataParser.cpp | 87 +- .../DataLakes/DeltaLakeMetadataParser.h | 10 +- src/Storages/DataLakes/HudiMetadataParser.cpp | 181 +- src/Storages/DataLakes/HudiMetadataParser.h | 15 +- src/Storages/DataLakes/IStorageDataLake.h | 144 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 65 +- .../DataLakes/Iceberg/IcebergMetadata.h | 27 +- .../DataLakes/Iceberg/StorageIceberg.cpp | 79 - .../DataLakes/Iceberg/StorageIceberg.h | 117 +- src/Storages/DataLakes/S3MetadataReader.cpp | 86 - src/Storages/DataLakes/S3MetadataReader.h | 25 - src/Storages/DataLakes/StorageDeltaLake.h | 7 +- src/Storages/DataLakes/StorageHudi.h | 7 +- src/Storages/DataLakes/registerDataLakes.cpp | 38 +- src/Storages/HDFS/StorageHDFS.cpp | 1117 ---------- src/Storages/HDFS/StorageHDFS.h | 179 -- src/Storages/HDFS/StorageHDFSCluster.cpp | 98 - src/Storages/HDFS/StorageHDFSCluster.h | 56 - src/Storages/IStorage.h | 9 +- .../ObjectStorage/AzureConfiguration.cpp | 451 ++++ .../ObjectStorage/AzureConfiguration.h | 54 + src/Storages/ObjectStorage/Configuration.h | 55 + .../ObjectStorage/HDFSConfiguration.h | 81 + .../ObjectStorage/ReadBufferIterator.h | 197 ++ .../ObjectStorage/ReadFromObjectStorage.h | 105 + .../ObjectStorage/S3Configuration.cpp | 491 +++++ src/Storages/ObjectStorage/S3Configuration.h | 46 + src/Storages/ObjectStorage/Settings.h | 86 + .../ObjectStorage/StorageObjectStorage.cpp | 303 +++ .../ObjectStorage/StorageObjectStorage.h | 116 + .../StorageObjectStorageCluster.cpp | 107 + .../StorageObjectStorageCluster.h | 72 + .../ObjectStorage/StorageObjectStorageSink.h | 155 ++ .../StorageObjectStorageSource.cpp | 464 ++++ .../StorageObjectStorageSource.h | 217 ++ .../registerStorageObjectStorage.cpp | 166 ++ src/Storages/ObjectStorageConfiguration.h | 0 src/Storages/S3Queue/S3QueueSource.cpp | 85 +- src/Storages/S3Queue/S3QueueSource.h | 42 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 3 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 7 +- src/Storages/S3Queue/StorageS3Queue.cpp | 101 +- src/Storages/S3Queue/StorageS3Queue.h | 14 +- src/Storages/StorageAzureBlob.cpp | 1478 ------------- src/Storages/StorageAzureBlob.h | 339 --- src/Storages/StorageAzureBlobCluster.cpp | 89 - src/Storages/StorageAzureBlobCluster.h | 56 - src/Storages/StorageS3.cpp | 1905 ----------------- src/Storages/StorageS3.h | 399 ---- src/Storages/StorageS3Cluster.cpp | 103 - src/Storages/StorageS3Cluster.h | 58 - .../StorageSystemSchemaInferenceCache.cpp | 6 +- src/Storages/registerStorages.cpp | 17 +- src/TableFunctions/ITableFunctionCluster.h | 6 +- src/TableFunctions/ITableFunctionDataLake.h | 22 +- .../TableFunctionAzureBlobStorage.cpp | 323 --- .../TableFunctionAzureBlobStorage.h | 80 - .../TableFunctionAzureBlobStorageCluster.cpp | 85 - .../TableFunctionAzureBlobStorageCluster.h | 55 - src/TableFunctions/TableFunctionDeltaLake.cpp | 24 +- src/TableFunctions/TableFunctionHDFS.cpp | 54 - src/TableFunctions/TableFunctionHDFS.h | 50 - .../TableFunctionHDFSCluster.cpp | 61 - src/TableFunctions/TableFunctionHDFSCluster.h | 54 - src/TableFunctions/TableFunctionHudi.cpp | 24 +- src/TableFunctions/TableFunctionIceberg.cpp | 7 +- .../TableFunctionObjectStorage.cpp | 224 ++ .../TableFunctionObjectStorage.h | 150 ++ .../TableFunctionObjectStorageCluster.cpp | 113 + .../TableFunctionObjectStorageCluster.h | 91 + src/TableFunctions/TableFunctionS3.cpp | 464 ---- src/TableFunctions/TableFunctionS3.h | 86 - src/TableFunctions/TableFunctionS3Cluster.cpp | 74 - src/TableFunctions/TableFunctionS3Cluster.h | 64 - src/TableFunctions/registerTableFunctions.cpp | 23 +- src/TableFunctions/registerTableFunctions.h | 9 +- .../test_storage_azure_blob_storage/test.py | 8 +- 94 files changed, 4403 insertions(+), 8155 deletions(-) delete mode 100644 src/Storages/DataLakes/S3MetadataReader.cpp delete mode 100644 src/Storages/DataLakes/S3MetadataReader.h delete mode 100644 src/Storages/HDFS/StorageHDFS.cpp delete mode 100644 src/Storages/HDFS/StorageHDFS.h delete mode 100644 src/Storages/HDFS/StorageHDFSCluster.cpp delete mode 100644 src/Storages/HDFS/StorageHDFSCluster.h create mode 100644 src/Storages/ObjectStorage/AzureConfiguration.cpp create mode 100644 src/Storages/ObjectStorage/AzureConfiguration.h create mode 100644 src/Storages/ObjectStorage/Configuration.h create mode 100644 src/Storages/ObjectStorage/HDFSConfiguration.h create mode 100644 src/Storages/ObjectStorage/ReadBufferIterator.h create mode 100644 src/Storages/ObjectStorage/ReadFromObjectStorage.h create mode 100644 src/Storages/ObjectStorage/S3Configuration.cpp create mode 100644 src/Storages/ObjectStorage/S3Configuration.h create mode 100644 src/Storages/ObjectStorage/Settings.h create mode 100644 src/Storages/ObjectStorage/StorageObjectStorage.cpp create mode 100644 src/Storages/ObjectStorage/StorageObjectStorage.h create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageCluster.h create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageSink.h create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageSource.cpp create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageSource.h create mode 100644 src/Storages/ObjectStorage/registerStorageObjectStorage.cpp create mode 100644 src/Storages/ObjectStorageConfiguration.h delete mode 100644 src/Storages/StorageAzureBlob.cpp delete mode 100644 src/Storages/StorageAzureBlob.h delete mode 100644 src/Storages/StorageAzureBlobCluster.cpp delete mode 100644 src/Storages/StorageAzureBlobCluster.h delete mode 100644 src/Storages/StorageS3.cpp delete mode 100644 src/Storages/StorageS3.h delete mode 100644 src/Storages/StorageS3Cluster.cpp delete mode 100644 src/Storages/StorageS3Cluster.h delete mode 100644 src/TableFunctions/TableFunctionAzureBlobStorage.cpp delete mode 100644 src/TableFunctions/TableFunctionAzureBlobStorage.h delete mode 100644 src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp delete mode 100644 src/TableFunctions/TableFunctionAzureBlobStorageCluster.h delete mode 100644 src/TableFunctions/TableFunctionHDFS.cpp delete mode 100644 src/TableFunctions/TableFunctionHDFS.h delete mode 100644 src/TableFunctions/TableFunctionHDFSCluster.cpp delete mode 100644 src/TableFunctions/TableFunctionHDFSCluster.h create mode 100644 src/TableFunctions/TableFunctionObjectStorage.cpp create mode 100644 src/TableFunctions/TableFunctionObjectStorage.h create mode 100644 src/TableFunctions/TableFunctionObjectStorageCluster.cpp create mode 100644 src/TableFunctions/TableFunctionObjectStorageCluster.h delete mode 100644 src/TableFunctions/TableFunctionS3.cpp delete mode 100644 src/TableFunctions/TableFunctionS3.h delete mode 100644 src/TableFunctions/TableFunctionS3Cluster.cpp delete mode 100644 src/TableFunctions/TableFunctionS3Cluster.h diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 52ce20d5108..dc636f90be7 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -29,7 +28,7 @@ namespace ErrorCodes } BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureBlobConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) @@ -37,10 +36,10 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = configuration.createClient(/* is_read_only */ false); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), - StorageAzureBlob::createSettings(context_), + configuration.createSettings(context_), configuration_.container); client = object_storage->getAzureBlobStorageClient(); settings = object_storage->getSettings(); @@ -137,7 +136,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( - StorageAzureBlob::Configuration configuration_, + const StorageAzureBlobConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) @@ -145,17 +144,22 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); + auto client_ptr = configuration.createClient(/* is_read_only */ false); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), - StorageAzureBlob::createSettings(context_), - configuration_.container); + configuration.createSettings(context_), + configuration.container); client = object_storage->getAzureBlobStorageClient(); settings = object_storage->getSettings(); } -void BackupWriterAzureBlobStorage::copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) +void BackupWriterAzureBlobStorage::copyFileFromDisk( + const String & path_in_backup, + DiskPtr src_disk, + const String & src_path, + bool copy_encrypted, + UInt64 start_pos, + UInt64 length) { /// Use the native copy as a more optimal way to copy a file from AzureBlobStorage to AzureBlobStorage if it's possible. auto source_data_source_description = src_disk->getDataSourceDescription(); @@ -241,7 +245,7 @@ UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) object_storage->listObjects(key,children,/*max_keys*/0); if (children.empty()) throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object must exist"); - return children[0].metadata.size_bytes; + return children[0]->metadata.size_bytes; } std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 95325044a62..99002c53769 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -5,8 +5,8 @@ #if USE_AZURE_BLOB_STORAGE #include #include -#include #include +#include namespace DB @@ -16,20 +16,30 @@ namespace DB class BackupReaderAzureBlobStorage : public BackupReaderDefault { public: - BackupReaderAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + BackupReaderAzureBlobStorage( + const StorageAzureBlobConfiguration & configuration_, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_); + ~BackupReaderAzureBlobStorage() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr readFile(const String & file_name) override; - void copyFileToDisk(const String & path_in_backup, size_t file_size, bool encrypted_in_backup, - DiskPtr destination_disk, const String & destination_path, WriteMode write_mode) override; + void copyFileToDisk( + const String & path_in_backup, + size_t file_size, + bool encrypted_in_backup, + DiskPtr destination_disk, + const String & destination_path, + WriteMode write_mode) override; private: const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + StorageAzureBlobConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; @@ -37,16 +47,31 @@ private: class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: - BackupWriterAzureBlobStorage(StorageAzureBlob::Configuration configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); + BackupWriterAzureBlobStorage( + const StorageAzureBlobConfiguration & configuration_, + const ReadSettings & read_settings_, + const WriteSettings & write_settings_, + const ContextPtr & context_); + ~BackupWriterAzureBlobStorage() override; bool fileExists(const String & file_name) override; UInt64 getFileSize(const String & file_name) override; std::unique_ptr writeFile(const String & file_name) override; - void copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) override; - void copyFileFromDisk(const String & path_in_backup, DiskPtr src_disk, const String & src_path, - bool copy_encrypted, UInt64 start_pos, UInt64 length) override; + void copyDataToFile( + const String & path_in_backup, + const CreateReadBufferFunction & create_read_buffer, + UInt64 start_pos, + UInt64 length) override; + + void copyFileFromDisk( + const String & path_in_backup, + DiskPtr src_disk, + const String & src_path, + bool copy_encrypted, + UInt64 start_pos, + UInt64 length) override; void copyFile(const String & destination, const String & source, size_t size) override; @@ -56,9 +81,10 @@ public: private: std::unique_ptr readFile(const String & file_name, size_t expected_file_size) override; void removeFilesBatch(const Strings & file_names); + const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlob::Configuration configuration; + StorageAzureBlobConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 48f66569304..9408c7ccdcf 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -5,11 +5,11 @@ #if USE_AZURE_BLOB_STORAGE #include -#include #include #include #include #include +#include #include #endif @@ -49,7 +49,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; - StorageAzureBlob::Configuration configuration; + StorageAzureBlobConfiguration configuration; if (!id_arg.empty()) { @@ -59,6 +59,9 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (!config.has(config_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", id_arg); + if (!config.has(config_prefix)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no `{}` in config", config_prefix); + if (config.has(config_prefix + ".connection_string")) { configuration.connection_url = config.getString(config_prefix + ".connection_string"); @@ -75,10 +78,11 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } if (args.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Backup AzureBlobStorage requires 1 or 2 arguments: named_collection, [filename]"); if (args.size() == 1) - configuration.blob_path = args[0].safeGet(); + configuration.setPath(args[0].safeGet()); } else @@ -110,12 +114,14 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) } BackupImpl::ArchiveParams archive_params; - if (hasRegisteredArchiveFileExtension(configuration.blob_path)) + if (hasRegisteredArchiveFileExtension(configuration.getPath())) { if (params.is_internal_backup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled"); - archive_params.archive_name = removeFileNameFromURL(configuration.blob_path); + auto path = configuration.getPath(); + configuration.setPath(removeFileNameFromURL(path)); + archive_params.archive_name = configuration.getPath(); archive_params.compression_method = params.compression_method; archive_params.compression_level = params.compression_level; archive_params.password = params.password; diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 08913ed1b5a..50130e6abd0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -119,6 +119,7 @@ endif() add_headers_and_sources(dbms Storages/DataLakes) add_headers_and_sources(dbms Storages/DataLakes/Iceberg) +add_headers_and_sources(dbms Storages/ObjectStorage) add_headers_and_sources(dbms Common/NamedCollections) if (TARGET ch_contrib::amqp_cpp) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 74389aedb64..2ca44137442 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -65,14 +65,14 @@ private: for (const auto & blob : blobs_list) { - batch.emplace_back( + batch.emplace_back(std::make_shared( blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), - {}}); + {}})); } if (!blob_list_response.NextPageToken.HasValue() || blob_list_response.NextPageToken.Value().empty()) @@ -156,14 +156,14 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith for (const auto & blob : blobs_list) { - children.emplace_back( + children.emplace_back(std::make_shared( blob.Name, ObjectMetadata{ static_cast(blob.BlobSize), Poco::Timestamp::fromEpochTime( std::chrono::duration_cast( static_cast(blob.Details.LastModified).time_since_epoch()).count()), - {}}); + {}})); } if (max_keys) diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 0314e0a7e92..cc9ee3db505 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -363,18 +363,18 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * for (const auto & object : objects) { - LOG_INFO(disk->log, "Calling restore for key for disk {}", object.relative_path); + LOG_INFO(disk->log, "Calling restore for key for disk {}", object->relative_path); /// Skip file operations objects. They will be processed separately. - if (object.relative_path.find("/operations/") != String::npos) + if (object->relative_path.find("/operations/") != String::npos) continue; - const auto [revision, _] = extractRevisionAndOperationFromKey(object.relative_path); + const auto [revision, _] = extractRevisionAndOperationFromKey(object->relative_path); /// Filter early if it's possible to get revision from key. if (revision > restore_information.revision) continue; - keys_names.push_back(object.relative_path); + keys_names.push_back(object->relative_path); } if (!keys_names.empty()) @@ -474,10 +474,10 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject for (const auto & object : objects) { - const auto [revision, operation] = extractRevisionAndOperationFromKey(object.relative_path); + const auto [revision, operation] = extractRevisionAndOperationFromKey(object->relative_path); if (revision == UNKNOWN_REVISION) { - LOG_WARNING(disk->log, "Skip key {} with unknown revision", object.relative_path); + LOG_WARNING(disk->log, "Skip key {} with unknown revision", object->relative_path); continue; } @@ -490,7 +490,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject if (send_metadata) revision_counter = revision - 1; - auto object_attributes = *(source_object_storage->getObjectMetadata(object.relative_path).attributes); + auto object_attributes = *(source_object_storage->getObjectMetadata(object->relative_path).attributes); if (operation == rename) { auto from_path = object_attributes["from_path"]; diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 049935ad60c..7d354e6383d 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -62,6 +62,8 @@ struct RelativePathWithMetadata : relative_path(std::move(relative_path_)) , metadata(std::move(metadata_)) {} + + virtual ~RelativePathWithMetadata() = default; }; struct ObjectKeyWithMetadata @@ -77,7 +79,8 @@ struct ObjectKeyWithMetadata {} }; -using RelativePathsWithMetadata = std::vector; +using RelativePathWithMetadataPtr = std::shared_ptr; +using RelativePathsWithMetadata = std::vector; using ObjectKeysWithMetadata = std::vector; class IObjectStorageIterator; diff --git a/src/Disks/ObjectStorages/IObjectStorage_fwd.h b/src/Disks/ObjectStorages/IObjectStorage_fwd.h index f6ebc883682..67efa4aae2b 100644 --- a/src/Disks/ObjectStorages/IObjectStorage_fwd.h +++ b/src/Disks/ObjectStorages/IObjectStorage_fwd.h @@ -10,4 +10,7 @@ using ObjectStoragePtr = std::shared_ptr; class IMetadataStorage; using MetadataStoragePtr = std::shared_ptr; +class IObjectStorageIterator; +using ObjectStorageIteratorPtr = std::shared_ptr; + } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index b03809f5b39..f07cf23106f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -77,7 +77,7 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co std::vector result; for (const auto & path_size : files) { - result.push_back(path_size.relative_path); + result.push_back(path_size->relative_path); } std::unordered_set duplicates_filter; diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp index 72ec6e0e500..3d939ce9230 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.cpp @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -RelativePathWithMetadata ObjectStorageIteratorFromList::current() +RelativePathWithMetadataPtr ObjectStorageIteratorFromList::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index 841b0ea6664..e934fc2056d 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -12,9 +12,9 @@ public: virtual void next() = 0; virtual void nextBatch() = 0; virtual bool isValid() = 0; - virtual RelativePathWithMetadata current() = 0; + virtual RelativePathWithMetadataPtr current() = 0; virtual RelativePathsWithMetadata currentBatch() = 0; - virtual std::optional getCurrrentBatchAndScheduleNext() = 0; + virtual std::optional getCurrentBatchAndScheduleNext() = 0; virtual size_t getAccumulatedSize() const = 0; virtual ~IObjectStorageIterator() = default; @@ -47,22 +47,14 @@ public: return batch_iterator != batch.end(); } - RelativePathWithMetadata current() override; + RelativePathWithMetadataPtr current() override; - RelativePathsWithMetadata currentBatch() override - { - return batch; - } + RelativePathsWithMetadata currentBatch() override { return batch; } - virtual std::optional getCurrrentBatchAndScheduleNext() override - { - return std::nullopt; - } + std::optional getCurrentBatchAndScheduleNext() override { return std::nullopt; } + + size_t getAccumulatedSize() const override { return batch.size(); } - size_t getAccumulatedSize() const override - { - return batch.size(); - } private: RelativePathsWithMetadata batch; RelativePathsWithMetadata::iterator batch_iterator; diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index 990e66fc4e5..b7729623a64 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -82,7 +82,7 @@ bool IObjectStorageIteratorAsync::isValid() return current_batch_iterator != current_batch.end(); } -RelativePathWithMetadata IObjectStorageIteratorAsync::current() +RelativePathWithMetadataPtr IObjectStorageIteratorAsync::current() { if (!isValid()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to access invalid iterator"); @@ -101,7 +101,7 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch() return current_batch; } -std::optional IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext() +std::optional IObjectStorageIteratorAsync::getCurrentBatchAndScheduleNext() { std::lock_guard lock(mutex); if (!is_initialized) diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index a6abe03bac9..8d155f7ec8d 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -26,10 +26,10 @@ public: void next() override; void nextBatch() override; bool isValid() override; - RelativePathWithMetadata current() override; + RelativePathWithMetadataPtr current() override; RelativePathsWithMetadata currentBatch() override; size_t getAccumulatedSize() const override; - std::optional getCurrrentBatchAndScheduleNext() override; + std::optional getCurrentBatchAndScheduleNext() override; ~IObjectStorageIteratorAsync() override { diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 4cc49288af6..cc138c43c71 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -120,25 +120,22 @@ private: { ProfileEvents::increment(ProfileEvents::S3ListObjects); - bool result = false; auto outcome = client->ListObjectsV2(request); + /// Outcome failure will be handled on the caller side. if (outcome.IsSuccess()) { + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + auto objects = outcome.GetResult().GetContents(); - - result = !objects.empty(); - for (const auto & object : objects) - batch.emplace_back( - object.GetKey(), - ObjectMetadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}} - ); + { + ObjectMetadata metadata{static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), {}}; + batch.emplace_back(std::make_shared(object.GetKey(), std::move(metadata))); + } - if (result) - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - - return result; + /// It returns false when all objects were returned + return outcome.GetResult().GetIsTruncated(); } throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", @@ -249,7 +246,6 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); - auto blob_storage_log = BlobStorageLogWriter::create(disk_name); if (blob_storage_log) blob_storage_log->local_path = object.local_path; @@ -300,12 +296,12 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet break; for (const auto & object : objects) - children.emplace_back( + children.emplace_back(std::make_shared( object.GetKey(), ObjectMetadata{ static_cast(object.GetSize()), Poco::Timestamp::fromEpochTime(object.GetLastModified().Seconds()), - {}}); + {}})); if (max_keys) { diff --git a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h index 83814f42693..cc23f604278 100644 --- a/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h +++ b/src/IO/AzureBlobStorage/copyAzureBlobStorageFile.h @@ -4,9 +4,8 @@ #if USE_AZURE_BLOB_STORAGE -#include -#include #include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9a80553f149..d697d90c8a6 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -52,11 +52,9 @@ #include #include #include -#include #include -#include +#include #include -#include #include #include #include @@ -482,7 +480,7 @@ BlockIO InterpreterSystemQuery::execute() StorageURL::getSchemaCache(getContext()).clear(); #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) - StorageAzureBlob::getSchemaCache(getContext()).clear(); + StorageAzureBlobStorage::getSchemaCache(getContext()).clear(); #endif break; } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index e1086ac5833..58672a72563 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -35,7 +35,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp index 3584f137225..55ff8fefdd5 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp @@ -4,8 +4,6 @@ #include #if USE_AWS_S3 && USE_PARQUET -#include -#include #include #include #include @@ -13,10 +11,10 @@ #include #include #include +#include #include #include #include -#include namespace fs = std::filesystem; @@ -29,8 +27,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -template -struct DeltaLakeMetadataParser::Impl +struct DeltaLakeMetadataParser::Impl { /** * Useful links: @@ -65,10 +62,13 @@ struct DeltaLakeMetadataParser::Impl * An action changes one aspect of the table's state, for example, adding or removing a file. * Note: it is not a valid json, but a list of json's, so we read it in a while cycle. */ - std::set processMetadataFiles(const Configuration & configuration, ContextPtr context) + std::set processMetadataFiles( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfiguration & configuration, + ContextPtr context) { std::set result_files; - const auto checkpoint_version = getCheckpointIfExists(result_files, configuration, context); + const auto checkpoint_version = getCheckpointIfExists(result_files, object_storage, configuration, context); if (checkpoint_version) { @@ -78,10 +78,10 @@ struct DeltaLakeMetadataParser::Impl const auto filename = withPadding(++current_version) + metadata_file_suffix; const auto file_path = fs::path(configuration.getPath()) / deltalake_metadata_directory / filename; - if (!MetadataReadHelper::exists(file_path, configuration)) + if (!object_storage->exists(StoredObject(file_path))) break; - processMetadataFile(file_path, result_files, configuration, context); + processMetadataFile(file_path, result_files, object_storage, configuration, context); } LOG_TRACE( @@ -90,16 +90,33 @@ struct DeltaLakeMetadataParser::Impl } else { - const auto keys = MetadataReadHelper::listFiles( - configuration, deltalake_metadata_directory, metadata_file_suffix); - + const auto keys = listFiles(object_storage, configuration, deltalake_metadata_directory, metadata_file_suffix); for (const String & key : keys) - processMetadataFile(key, result_files, configuration, context); + processMetadataFile(key, result_files, object_storage, configuration, context); } return result_files; } + std::vector listFiles( + const ObjectStoragePtr & object_storage, + const StorageObjectStorageConfiguration & configuration, + const String & prefix, const String & suffix) + { + auto key = std::filesystem::path(configuration.getPath()) / prefix; + RelativePathsWithMetadata files_with_metadata; + object_storage->listObjects(key, files_with_metadata, 0); + Strings res; + for (const auto & file_with_metadata : files_with_metadata) + { + const auto & filename = file_with_metadata->relative_path; + if (filename.ends_with(suffix)) + res.push_back(filename); + } + LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); + return res; + } + /** * Example of content of a single .json metadata file: * " @@ -132,10 +149,12 @@ struct DeltaLakeMetadataParser::Impl void processMetadataFile( const String & key, std::set & result, - const Configuration & configuration, + ObjectStoragePtr object_storage, + const StorageObjectStorageConfiguration & configuration, ContextPtr context) { - auto buf = MetadataReadHelper::createReadBuffer(key, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(key), read_settings); char c; while (!buf->eof()) @@ -180,14 +199,18 @@ struct DeltaLakeMetadataParser::Impl * * We need to get "version", which is the version of the checkpoint we need to read. */ - size_t readLastCheckpointIfExists(const Configuration & configuration, ContextPtr context) + size_t readLastCheckpointIfExists( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfiguration & configuration, + ContextPtr context) const { const auto last_checkpoint_file = fs::path(configuration.getPath()) / deltalake_metadata_directory / "_last_checkpoint"; - if (!MetadataReadHelper::exists(last_checkpoint_file, configuration)) + if (!object_storage->exists(StoredObject(last_checkpoint_file))) return 0; String json_str; - auto buf = MetadataReadHelper::createReadBuffer(last_checkpoint_file, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(last_checkpoint_file), read_settings); readJSONObjectPossiblyInvalid(json_str, *buf); const JSON json(json_str); @@ -237,9 +260,13 @@ struct DeltaLakeMetadataParser::Impl throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arrow error: {}", _s.ToString()); \ } while (false) - size_t getCheckpointIfExists(std::set & result, const Configuration & configuration, ContextPtr context) + size_t getCheckpointIfExists( + std::set & result, + ObjectStoragePtr object_storage, + const StorageObjectStorageConfiguration & configuration, + ContextPtr context) { - const auto version = readLastCheckpointIfExists(configuration, context); + const auto version = readLastCheckpointIfExists(object_storage, configuration, context); if (!version) return 0; @@ -248,7 +275,8 @@ struct DeltaLakeMetadataParser::Impl LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); - auto buf = MetadataReadHelper::createReadBuffer(checkpoint_path, context, configuration); + auto read_settings = context->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(checkpoint_path), read_settings); auto format_settings = getFormatSettings(context); /// Force nullable, because this parquet file for some reason does not have nullable @@ -317,22 +345,17 @@ struct DeltaLakeMetadataParser::Impl LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; +DeltaLakeMetadataParser::DeltaLakeMetadataParser() : impl(std::make_unique()) {} -template -DeltaLakeMetadataParser::DeltaLakeMetadataParser() : impl(std::make_unique()) +Strings DeltaLakeMetadataParser::getFiles( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context) { -} - -template -Strings DeltaLakeMetadataParser::getFiles(const Configuration & configuration, ContextPtr context) -{ - auto result = impl->processMetadataFiles(configuration, context); + auto result = impl->processMetadataFiles(object_storage, *configuration, context); return Strings(result.begin(), result.end()); } -template DeltaLakeMetadataParser::DeltaLakeMetadataParser(); -template Strings DeltaLakeMetadataParser::getFiles( - const StorageS3::Configuration & configuration, ContextPtr); } #endif diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.h b/src/Storages/DataLakes/DeltaLakeMetadataParser.h index df7276b90b4..f94024597d6 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.h +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.h @@ -2,17 +2,21 @@ #include #include +#include +#include namespace DB { -template struct DeltaLakeMetadataParser { public: - DeltaLakeMetadataParser(); + DeltaLakeMetadataParser(); - Strings getFiles(const Configuration & configuration, ContextPtr context); + Strings getFiles( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context); private: struct Impl; diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/DataLakes/HudiMetadataParser.cpp index 699dfe8fda0..8571c035b32 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/DataLakes/HudiMetadataParser.cpp @@ -1,16 +1,11 @@ #include +#include #include -#include #include #include #include "config.h" -#include #include -#if USE_AWS_S3 -#include -#include - namespace DB { @@ -19,98 +14,98 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -template -struct HudiMetadataParser::Impl -{ - /** - * Useful links: - * - https://hudi.apache.org/tech-specs/ - * - https://hudi.apache.org/docs/file_layouts/ - */ +/** + * Useful links: + * - https://hudi.apache.org/tech-specs/ + * - https://hudi.apache.org/docs/file_layouts/ + */ - /** - * Hudi tables store metadata files and data files. - * Metadata files are stored in .hoodie/metadata directory. Though unlike DeltaLake and Iceberg, - * metadata is not required in order to understand which files we need to read, moreover, - * for Hudi metadata does not always exist. - * - * There can be two types of data files - * 1. base files (columnar file formats like Apache Parquet/Orc) - * 2. log files - * Currently we support reading only `base files`. - * Data file name format: - * [File Id]_[File Write Token]_[Transaction timestamp].[File Extension] - * - * To find needed parts we need to find out latest part file for every file group for every partition. - * Explanation why: - * Hudi reads in and overwrites the entire table/partition with each update. - * Hudi controls the number of file groups under a single partition according to the - * hoodie.parquet.max.file.size option. Once a single Parquet file is too large, Hudi creates a second file group. - * Each file group is identified by File Id. - */ - Strings processMetadataFiles(const Configuration & configuration) +/** + * Hudi tables store metadata files and data files. + * Metadata files are stored in .hoodie/metadata directory. Though unlike DeltaLake and Iceberg, + * metadata is not required in order to understand which files we need to read, moreover, + * for Hudi metadata does not always exist. + * + * There can be two types of data files + * 1. base files (columnar file formats like Apache Parquet/Orc) + * 2. log files + * Currently we support reading only `base files`. + * Data file name format: + * [File Id]_[File Write Token]_[Transaction timestamp].[File Extension] + * + * To find needed parts we need to find out latest part file for every file group for every partition. + * Explanation why: + * Hudi reads in and overwrites the entire table/partition with each update. + * Hudi controls the number of file groups under a single partition according to the + * hoodie.parquet.max.file.size option. Once a single Parquet file is too large, Hudi creates a second file group. + * Each file group is identified by File Id. + */ +std::vector listFiles( + const ObjectStoragePtr & object_storage, + const StorageObjectStorageConfiguration & configuration, + const String & prefix, const String & suffix) +{ + auto key = std::filesystem::path(configuration.getPath()) / prefix; + RelativePathsWithMetadata files_with_metadata; + object_storage->listObjects(key, files_with_metadata, 0); + Strings res; + for (const auto & file_with_metadata : files_with_metadata) { - auto log = getLogger("HudiMetadataParser"); - - const auto keys = MetadataReadHelper::listFiles(configuration, "", Poco::toLower(configuration.format)); - - using Partition = std::string; - using FileID = std::string; - struct FileInfo - { - String key; - UInt64 timestamp = 0; - }; - std::unordered_map> data_files; - - for (const auto & key : keys) - { - auto key_file = std::filesystem::path(key); - Strings file_parts; - const String stem = key_file.stem(); - splitInto<'_'>(file_parts, stem); - if (file_parts.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); - - const auto partition = key_file.parent_path().stem(); - const auto & file_id = file_parts[0]; - const auto timestamp = parse(file_parts[2]); - - auto & file_info = data_files[partition][file_id]; - if (file_info.timestamp == 0 || file_info.timestamp < timestamp) - { - file_info.key = std::move(key); - file_info.timestamp = timestamp; - } - } - - Strings result; - for (auto & [partition, partition_data] : data_files) - { - LOG_TRACE(log, "Adding {} data files from partition {}", partition, partition_data.size()); - for (auto & [file_id, file_data] : partition_data) - result.push_back(std::move(file_data.key)); - } - return result; + const auto & filename = file_with_metadata->relative_path; + if (filename.ends_with(suffix)) + res.push_back(filename); } -}; + LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); + return res; +} - -template -HudiMetadataParser::HudiMetadataParser() : impl(std::make_unique()) +Strings HudiMetadataParser::getFiles( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr) { + auto log = getLogger("HudiMetadataParser"); + + const auto keys = listFiles(object_storage, *configuration, "", Poco::toLower(configuration->format)); + + using Partition = std::string; + using FileID = std::string; + struct FileInfo + { + String key; + UInt64 timestamp = 0; + }; + std::unordered_map> data_files; + + for (const auto & key : keys) + { + auto key_file = std::filesystem::path(key); + Strings file_parts; + const String stem = key_file.stem(); + splitInto<'_'>(file_parts, stem); + if (file_parts.size() != 3) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key); + + const auto partition = key_file.parent_path().stem(); + const auto & file_id = file_parts[0]; + const auto timestamp = parse(file_parts[2]); + + auto & file_info = data_files[partition][file_id]; + if (file_info.timestamp == 0 || file_info.timestamp < timestamp) + { + file_info.key = key; + file_info.timestamp = timestamp; + } + } + + Strings result; + for (auto & [partition, partition_data] : data_files) + { + LOG_TRACE(log, "Adding {} data files from partition {}", partition, partition_data.size()); + for (auto & [file_id, file_data] : partition_data) + result.push_back(std::move(file_data.key)); + } + return result; } -template -Strings HudiMetadataParser::getFiles(const Configuration & configuration, ContextPtr) -{ - return impl->processMetadataFiles(configuration); } - -template HudiMetadataParser::HudiMetadataParser(); -template Strings HudiMetadataParser::getFiles( - const StorageS3::Configuration & configuration, ContextPtr); - -} - -#endif diff --git a/src/Storages/DataLakes/HudiMetadataParser.h b/src/Storages/DataLakes/HudiMetadataParser.h index 6727ba2f718..2fc004595ca 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.h +++ b/src/Storages/DataLakes/HudiMetadataParser.h @@ -1,22 +1,17 @@ #pragma once #include -#include +#include +#include namespace DB { -template struct HudiMetadataParser { -public: - HudiMetadataParser(); - - Strings getFiles(const Configuration & configuration, ContextPtr context); - -private: - struct Impl; - std::shared_ptr impl; + Strings getFiles( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, ContextPtr context); }; } diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h index db3f835494f..934bf227c42 100644 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ b/src/Storages/DataLakes/IStorageDataLake.h @@ -8,127 +8,91 @@ #include #include #include -#include +#include +#include namespace DB { -template -class IStorageDataLake : public Storage +template +class IStorageDataLake : public StorageObjectStorage { public: static constexpr auto name = Name::name; - using Configuration = typename Storage::Configuration; - template - explicit IStorageDataLake(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) - : Storage(getConfigurationForDataRead(configuration_, context_, {}, attach), context_, std::forward(args)...) - , base_configuration(configuration_) - , log(getLogger(getName())) {} // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) + using Storage = StorageObjectStorage; + using ConfigurationPtr = Storage::ConfigurationPtr; - template - static StoragePtr create(const Configuration & configuration_, ContextPtr context_, bool attach, Args && ...args) + static StoragePtr create( + ConfigurationPtr base_configuration, + ContextPtr context, + const String & engine_name_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment_, + std::optional format_settings_, + bool /* attach */) { - return std::make_shared>(configuration_, context_, attach, std::forward(args)...); + auto object_storage = base_configuration->createOrUpdateObjectStorage(context); + + auto configuration = base_configuration->clone(); + configuration->getPaths() = MetadataParser().getFiles(object_storage, configuration, context); + + return std::make_shared>( + base_configuration, configuration, object_storage, engine_name_, context, + table_id_, columns_, constraints_, comment_, format_settings_); } String getName() const override { return name; } static ColumnsDescription getTableStructureFromData( - Configuration & base_configuration, - const std::optional & format_settings, + ObjectStoragePtr object_storage_, + ConfigurationPtr base_configuration, + const std::optional &, ContextPtr local_context) { - auto configuration = getConfigurationForDataRead(base_configuration, local_context); - return Storage::getTableStructureFromData(configuration, format_settings, local_context); + auto metadata = parseIcebergMetadata(object_storage_, base_configuration, local_context); + return ColumnsDescription(metadata->getTableSchema()); } - static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context) + std::pair updateConfigurationAndGetCopy(ContextPtr local_context) override { - return Storage::getConfiguration(engine_args, local_context, /* get_format_from_file */false); + std::lock_guard lock(Storage::configuration_update_mutex); + + auto new_object_storage = base_configuration->createOrUpdateObjectStorage(local_context); + bool updated = new_object_storage != nullptr; + if (updated) + Storage::object_storage = new_object_storage; + + auto new_keys = MetadataParser().getFiles(Storage::object_storage, base_configuration, local_context); + + if (updated || new_keys != Storage::configuration->getPaths()) + { + auto updated_configuration = base_configuration->clone(); + /// If metadata wasn't changed, we won't list data files again. + updated_configuration->getPaths() = new_keys; + Storage::configuration = updated_configuration; + } + return {Storage::configuration, Storage::object_storage}; } - Configuration updateConfigurationAndGetCopy(ContextPtr local_context) override + template + explicit IStorageDataLake( + ConfigurationPtr base_configuration_, + Args &&... args) + : Storage(std::forward(args)...) + , base_configuration(base_configuration_) { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - return Storage::getConfiguration(); - } - - void updateConfiguration(ContextPtr local_context) override - { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); } private: - static Configuration getConfigurationForDataRead( - const Configuration & base_configuration, ContextPtr local_context, const Strings & keys = {}, bool attach = false) - { - auto configuration{base_configuration}; - configuration.update(local_context); - configuration.static_configuration = true; - - try - { - if (keys.empty()) - configuration.keys = getDataFiles(configuration, local_context); - else - configuration.keys = keys; - - LOG_TRACE( - getLogger("DataLake"), - "New configuration path: {}, keys: {}", - configuration.getPath(), fmt::join(configuration.keys, ", ")); - - configuration.connect(local_context); - return configuration; - } - catch (...) - { - if (!attach) - throw; - tryLogCurrentException(__PRETTY_FUNCTION__); - return configuration; - } - } - - static Strings getDataFiles(const Configuration & configuration, ContextPtr local_context) - { - return MetadataParser().getFiles(configuration, local_context); - } - - void updateConfigurationImpl(ContextPtr local_context) - { - const bool updated = base_configuration.update(local_context); - auto new_keys = getDataFiles(base_configuration, local_context); - - if (!updated && new_keys == Storage::getConfiguration().keys) - return; - - Storage::useConfiguration(getConfigurationForDataRead(base_configuration, local_context, new_keys)); - } - - Configuration base_configuration; - std::mutex configuration_update_mutex; + ConfigurationPtr base_configuration; LoggerPtr log; }; -template -static StoragePtr createDataLakeStorage(const StorageFactory::Arguments & args) -{ - auto configuration = DataLake::getConfiguration(args.engine_args, args.getLocalContext()); - - /// Data lakes use parquet format, no need for schema inference. - if (configuration.format == "auto") - configuration.format = "Parquet"; - - return DataLake::create(configuration, args.getContext(), args.attach, args.table_id, args.columns, args.constraints, - args.comment, getFormatSettings(args.getContext())); -} - } #endif diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index df1536f53fc..08cebb3f396 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -21,11 +21,11 @@ #include #include #include +#include #include #include #include -#include -#include +#include #include #include @@ -44,7 +44,8 @@ namespace ErrorCodes } IcebergMetadata::IcebergMetadata( - const StorageS3::Configuration & configuration_, + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, DB::ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -52,6 +53,7 @@ IcebergMetadata::IcebergMetadata( Int32 current_schema_id_, DB::NamesAndTypesList schema_) : WithContext(context_) + , object_storage(object_storage_) , configuration(configuration_) , metadata_version(metadata_version_) , format_version(format_version_) @@ -331,21 +333,42 @@ MutableColumns parseAvro( return columns; } +std::vector listFiles( + const ObjectStoragePtr & object_storage, + const StorageObjectStorageConfiguration & configuration, + const String & prefix, const String & suffix) +{ + auto key = std::filesystem::path(configuration.getPath()) / prefix; + RelativePathsWithMetadata files_with_metadata; + object_storage->listObjects(key, files_with_metadata, 0); + Strings res; + for (const auto & file_with_metadata : files_with_metadata) + { + const auto & filename = file_with_metadata->relative_path; + if (filename.ends_with(suffix)) + res.push_back(filename); + } + LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); + return res; +} + /** * Each version of table metadata is stored in a `metadata` directory and * has one of 2 formats: * 1) v.metadata.json, where V - metadata version. * 2) -.metadata.json, where V - metadata version */ -std::pair getMetadataFileAndVersion(const StorageS3::Configuration & configuration) +std::pair getMetadataFileAndVersion( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfiguration & configuration) { - const auto metadata_files = S3DataLakeMetadataReadHelper::listFiles(configuration, "metadata", ".metadata.json"); + const auto metadata_files = listFiles(object_storage, configuration, "metadata", ".metadata.json"); if (metadata_files.empty()) { throw Exception( ErrorCodes::FILE_DOESNT_EXIST, "The metadata file for Iceberg table with path {} doesn't exist", - configuration.url.key); + configuration.getPath()); } std::vector> metadata_files_with_versions; @@ -372,11 +395,15 @@ std::pair getMetadataFileAndVersion(const StorageS3::Configuratio } -std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context_) +std::unique_ptr parseIcebergMetadata( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context_) { - const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(configuration); + const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); - auto buf = S3DataLakeMetadataReadHelper::createReadBuffer(metadata_file_path, context_, configuration); + auto read_settings = context_->getReadSettings(); + auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); @@ -397,12 +424,12 @@ std::unique_ptr parseIcebergMetadata(const StorageS3::Configura if (snapshot->getValue("snapshot-id") == current_snapshot_id) { const auto path = snapshot->getValue("manifest-list"); - manifest_list_file = std::filesystem::path(configuration.url.key) / "metadata" / std::filesystem::path(path).filename(); + manifest_list_file = std::filesystem::path(configuration->getPath()) / "metadata" / std::filesystem::path(path).filename(); break; } } - return std::make_unique(configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique(object_storage, configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -441,12 +468,14 @@ Strings IcebergMetadata::getDataFiles() LOG_TEST(log, "Collect manifest files from manifest list {}", manifest_list_file); - auto manifest_list_buf = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_list_file, getContext(), configuration); + auto context = getContext(); + auto read_settings = context->getReadSettings(); + auto manifest_list_buf = object_storage->readObject(StoredObject(manifest_list_file), read_settings); auto manifest_list_file_reader = std::make_unique(std::make_unique(*manifest_list_buf)); auto data_type = AvroSchemaReader::avroNodeToDataType(manifest_list_file_reader->dataSchema().root()->leafAt(0)); Block header{{data_type->createColumn(), data_type, "manifest_path"}}; - auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(getContext())); + auto columns = parseAvro(*manifest_list_file_reader, header, getFormatSettings(context)); auto & col = columns.at(0); if (col->getDataType() != TypeIndex::String) @@ -462,7 +491,7 @@ Strings IcebergMetadata::getDataFiles() { const auto file_path = col_str->getDataAt(i).toView(); const auto filename = std::filesystem::path(file_path).filename(); - manifest_files.emplace_back(std::filesystem::path(configuration.url.key) / "metadata" / filename); + manifest_files.emplace_back(std::filesystem::path(configuration->getPath()) / "metadata" / filename); } NameSet files; @@ -471,7 +500,7 @@ Strings IcebergMetadata::getDataFiles() { LOG_TEST(log, "Process manifest file {}", manifest_file); - auto buffer = S3DataLakeMetadataReadHelper::createReadBuffer(manifest_file, getContext(), configuration); + auto buffer = object_storage->readObject(StoredObject(manifest_file), read_settings); auto manifest_file_reader = std::make_unique(std::make_unique(*buffer)); /// Manifest file should always have table schema in avro file metadata. By now we don't support tables with evolved schema, @@ -482,7 +511,7 @@ Strings IcebergMetadata::getDataFiles() Poco::JSON::Parser parser; Poco::Dynamic::Var json = parser.parse(schema_json_string); Poco::JSON::Object::Ptr schema_object = json.extract(); - if (!getContext()->getSettingsRef().iceberg_engine_ignore_schema_evolution && schema_object->getValue("schema-id") != current_schema_id) + if (!context->getSettingsRef().iceberg_engine_ignore_schema_evolution && schema_object->getValue("schema-id") != current_schema_id) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, "Cannot read Iceberg table: the table schema has been changed at least 1 time, reading tables with evolved schema is not " @@ -595,9 +624,9 @@ Strings IcebergMetadata::getDataFiles() const auto status = status_int_column->getInt(i); const auto data_path = std::string(file_path_string_column->getDataAt(i).toView()); - const auto pos = data_path.find(configuration.url.key); + const auto pos = data_path.find(configuration->getPath()); if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration.url.key, data_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expected to find {} in data path: {}", configuration->getPath(), data_path); const auto file_path = data_path.substr(pos); diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h index 3e6a2ec3415..92946e4192b 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h @@ -2,9 +2,10 @@ #if USE_AWS_S3 && USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. -#include #include #include +#include +#include namespace DB { @@ -59,13 +60,15 @@ namespace DB class IcebergMetadata : WithContext { public: - IcebergMetadata(const StorageS3::Configuration & configuration_, - ContextPtr context_, - Int32 metadata_version_, - Int32 format_version_, - String manifest_list_file_, - Int32 current_schema_id_, - NamesAndTypesList schema_); + IcebergMetadata( + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + ContextPtr context_, + Int32 metadata_version_, + Int32 format_version_, + String manifest_list_file_, + Int32 current_schema_id_, + NamesAndTypesList schema_); /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. /// All subsequent calls will return saved list of files (because it cannot be changed without changing metadata file) @@ -77,7 +80,8 @@ public: size_t getVersion() const { return metadata_version; } private: - const StorageS3::Configuration configuration; + ObjectStoragePtr object_storage; + StorageObjectStorageConfigurationPtr configuration; Int32 metadata_version; Int32 format_version; String manifest_list_file; @@ -88,7 +92,10 @@ private: }; -std::unique_ptr parseIcebergMetadata(const StorageS3::Configuration & configuration, ContextPtr context); +std::unique_ptr parseIcebergMetadata( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + ContextPtr context); } diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp index 8a1a2cdbd8f..ad1a27c312b 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp @@ -5,85 +5,6 @@ namespace DB { -StoragePtr StorageIceberg::create( - const DB::StorageIceberg::Configuration & base_configuration, - DB::ContextPtr context_, - bool attach, - const DB::StorageID & table_id_, - const DB::ColumnsDescription & columns_, - const DB::ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_) -{ - auto configuration{base_configuration}; - configuration.update(context_); - std::unique_ptr metadata; - NamesAndTypesList schema_from_metadata; - try - { - metadata = parseIcebergMetadata(configuration, context_); - schema_from_metadata = metadata->getTableSchema(); - configuration.keys = metadata->getDataFiles(); - } - catch (...) - { - if (!attach) - throw; - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - return std::make_shared( - std::move(metadata), - configuration, - context_, - table_id_, - columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, - constraints_, - comment, - format_settings_); -} - -StorageIceberg::StorageIceberg( - std::unique_ptr metadata_, - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_) - : StorageS3(configuration_, context_, table_id_, columns_, constraints_, comment, format_settings_) - , current_metadata(std::move(metadata_)) - , base_configuration(configuration_) -{ -} - -ColumnsDescription StorageIceberg::getTableStructureFromData( - Configuration & base_configuration, - const std::optional &, - ContextPtr local_context) -{ - auto configuration{base_configuration}; - configuration.update(local_context); - auto metadata = parseIcebergMetadata(configuration, local_context); - return ColumnsDescription(metadata->getTableSchema()); -} - -void StorageIceberg::updateConfigurationImpl(ContextPtr local_context) -{ - const bool updated = base_configuration.update(local_context); - auto new_metadata = parseIcebergMetadata(base_configuration, local_context); - - if (!current_metadata || new_metadata->getVersion() != current_metadata->getVersion()) - current_metadata = std::move(new_metadata); - else if (!updated) - return; - - auto updated_configuration{base_configuration}; - /// If metadata wasn't changed, we won't list data files again. - updated_configuration.keys = current_metadata->getDataFiles(); - StorageS3::useConfiguration(updated_configuration); -} } diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/DataLakes/Iceberg/StorageIceberg.h index 4e63da5508a..bca6e3c868f 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ b/src/Storages/DataLakes/Iceberg/StorageIceberg.h @@ -4,13 +4,13 @@ #if USE_AWS_S3 && USE_AVRO -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -21,65 +21,100 @@ namespace DB /// many Iceberg features like schema evolution, partitioning, positional and equality deletes. /// TODO: Implement Iceberg as a separate storage using IObjectStorage /// (to support all object storages, not only S3) and add support for missing Iceberg features. -class StorageIceberg : public StorageS3 +template +class StorageIceberg : public StorageObjectStorage { public: static constexpr auto name = "Iceberg"; + using Storage = StorageObjectStorage; + using ConfigurationPtr = Storage::ConfigurationPtr; - using Configuration = StorageS3::Configuration; - - static StoragePtr create(const Configuration & base_configuration, - ContextPtr context_, - bool attach, + static StoragePtr create( + ConfigurationPtr base_configuration, + ContextPtr context, + const String & engine_name_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_); + const String & comment_, + std::optional format_settings_, + bool attach) + { + auto object_storage = base_configuration->createOrUpdateObjectStorage(context); + std::unique_ptr metadata; + NamesAndTypesList schema_from_metadata; + try + { + metadata = parseIcebergMetadata(object_storage, base_configuration, context); + schema_from_metadata = metadata->getTableSchema(); + } + catch (...) + { + if (!attach) + throw; + tryLogCurrentException(__PRETTY_FUNCTION__); + } - StorageIceberg( - std::unique_ptr metadata_, - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_); + auto configuration = base_configuration->clone(); + configuration->getPaths() = metadata->getDataFiles(); + + return std::make_shared>( + base_configuration, std::move(metadata), configuration, object_storage, engine_name_, context, + table_id_, + columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, + constraints_, comment_, format_settings_); + } String getName() const override { return name; } static ColumnsDescription getTableStructureFromData( - Configuration & base_configuration, + ObjectStoragePtr object_storage_, + ConfigurationPtr base_configuration, const std::optional &, - ContextPtr local_context); - - static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context) + ContextPtr local_context) { - return StorageS3::getConfiguration(engine_args, local_context, /* get_format_from_file */false); + auto metadata = parseIcebergMetadata(object_storage_, base_configuration, local_context); + return ColumnsDescription(metadata->getTableSchema()); } - Configuration updateConfigurationAndGetCopy(ContextPtr local_context) override + std::pair updateConfigurationAndGetCopy(ContextPtr local_context) override { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); - return StorageS3::getConfiguration(); + std::lock_guard lock(Storage::configuration_update_mutex); + + auto new_object_storage = base_configuration->createOrUpdateObjectStorage(local_context); + bool updated = new_object_storage != nullptr; + if (updated) + Storage::object_storage = new_object_storage; + + auto new_metadata = parseIcebergMetadata(Storage::object_storage, base_configuration, local_context); + + if (!current_metadata || new_metadata->getVersion() != current_metadata->getVersion()) + current_metadata = std::move(new_metadata); + else if (updated) + { + auto updated_configuration = base_configuration->clone(); + /// If metadata wasn't changed, we won't list data files again. + updated_configuration->getPaths() = current_metadata->getDataFiles(); + Storage::configuration = updated_configuration; + } + return {Storage::configuration, Storage::object_storage}; } - void updateConfiguration(ContextPtr local_context) override + template + StorageIceberg( + ConfigurationPtr base_configuration_, + std::unique_ptr metadata_, + Args &&... args) + : Storage(std::forward(args)...) + , base_configuration(base_configuration_) + , current_metadata(std::move(metadata_)) { - std::lock_guard lock(configuration_update_mutex); - updateConfigurationImpl(local_context); } private: - void updateConfigurationImpl(ContextPtr local_context); - + ConfigurationPtr base_configuration; std::unique_ptr current_metadata; - Configuration base_configuration; - std::mutex configuration_update_mutex; }; - } #endif diff --git a/src/Storages/DataLakes/S3MetadataReader.cpp b/src/Storages/DataLakes/S3MetadataReader.cpp deleted file mode 100644 index d66e21550a3..00000000000 --- a/src/Storages/DataLakes/S3MetadataReader.cpp +++ /dev/null @@ -1,86 +0,0 @@ -#include - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int S3_ERROR; -} - -std::shared_ptr -S3DataLakeMetadataReadHelper::createReadBuffer(const String & key, ContextPtr context, const StorageS3::Configuration & base_configuration) -{ - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = context->getSettingsRef().s3_max_single_read_retries; - return std::make_shared( - base_configuration.client, - base_configuration.url.bucket, - key, - base_configuration.url.version_id, - request_settings, - context->getReadSettings()); -} - -bool S3DataLakeMetadataReadHelper::exists(const String & key, const StorageS3::Configuration & configuration) -{ - return S3::objectExists(*configuration.client, configuration.url.bucket, key); -} - -std::vector S3DataLakeMetadataReadHelper::listFiles( - const StorageS3::Configuration & base_configuration, const String & prefix, const String & suffix) -{ - const auto & table_path = base_configuration.url.key; - const auto & bucket = base_configuration.url.bucket; - const auto & client = base_configuration.client; - - std::vector res; - S3::ListObjectsV2Request request; - Aws::S3::Model::ListObjectsV2Outcome outcome; - - request.SetBucket(bucket); - request.SetPrefix(std::filesystem::path(table_path) / prefix); - - bool is_finished{false}; - while (!is_finished) - { - outcome = client->ListObjectsV2(request); - if (!outcome.IsSuccess()) - throw S3Exception( - outcome.GetError().GetErrorType(), - "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", - quoteString(bucket), - quoteString(base_configuration.url.key), - backQuote(outcome.GetError().GetExceptionName()), - quoteString(outcome.GetError().GetMessage())); - - const auto & result_batch = outcome.GetResult().GetContents(); - for (const auto & obj : result_batch) - { - const auto & filename = obj.GetKey(); - if (filename.ends_with(suffix)) - res.push_back(filename); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - is_finished = !outcome.GetResult().GetIsTruncated(); - } - - LOG_TRACE(getLogger("S3DataLakeMetadataReadHelper"), "Listed {} files", res.size()); - - return res; -} - -} -#endif diff --git a/src/Storages/DataLakes/S3MetadataReader.h b/src/Storages/DataLakes/S3MetadataReader.h deleted file mode 100644 index cae7dd1fa3d..00000000000 --- a/src/Storages/DataLakes/S3MetadataReader.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once - -#include - -#if USE_AWS_S3 - -#include - -class ReadBuffer; - -namespace DB -{ - -struct S3DataLakeMetadataReadHelper -{ - static std::shared_ptr createReadBuffer( - const String & key, ContextPtr context, const StorageS3::Configuration & base_configuration); - - static bool exists(const String & key, const StorageS3::Configuration & configuration); - - static std::vector listFiles(const StorageS3::Configuration & configuration, const std::string & prefix = "", const std::string & suffix = ""); -}; -} - -#endif diff --git a/src/Storages/DataLakes/StorageDeltaLake.h b/src/Storages/DataLakes/StorageDeltaLake.h index 8b4ba28d6f7..07c2205d2df 100644 --- a/src/Storages/DataLakes/StorageDeltaLake.h +++ b/src/Storages/DataLakes/StorageDeltaLake.h @@ -5,11 +5,6 @@ #include #include "config.h" -#if USE_AWS_S3 -#include -#include -#endif - namespace DB { @@ -19,7 +14,7 @@ struct StorageDeltaLakeName }; #if USE_AWS_S3 && USE_PARQUET -using StorageDeltaLakeS3 = IStorageDataLake>; +using StorageDeltaLakeS3 = IStorageDataLake; #endif } diff --git a/src/Storages/DataLakes/StorageHudi.h b/src/Storages/DataLakes/StorageHudi.h index 84666f51405..3fd52c82d32 100644 --- a/src/Storages/DataLakes/StorageHudi.h +++ b/src/Storages/DataLakes/StorageHudi.h @@ -5,11 +5,6 @@ #include #include "config.h" -#if USE_AWS_S3 -#include -#include -#endif - namespace DB { @@ -19,7 +14,7 @@ struct StorageHudiName }; #if USE_AWS_S3 -using StorageHudiS3 = IStorageDataLake>; +using StorageHudiS3 = IStorageDataLake; #endif } diff --git a/src/Storages/DataLakes/registerDataLakes.cpp b/src/Storages/DataLakes/registerDataLakes.cpp index 118600f7212..2647fbce39d 100644 --- a/src/Storages/DataLakes/registerDataLakes.cpp +++ b/src/Storages/DataLakes/registerDataLakes.cpp @@ -6,43 +6,43 @@ #include #include #include +#include namespace DB { -#define REGISTER_DATA_LAKE_STORAGE(STORAGE, NAME) \ - factory.registerStorage( \ - NAME, \ - [](const StorageFactory::Arguments & args) \ - { \ - return createDataLakeStorage(args);\ - }, \ - { \ - .supports_settings = false, \ - .supports_schema_inference = true, \ - .source_access_type = AccessType::S3, \ - }); - #if USE_PARQUET -void registerStorageDeltaLake(StorageFactory & factory) +void registerStorageDeltaLake(StorageFactory & ) { - REGISTER_DATA_LAKE_STORAGE(StorageDeltaLakeS3, StorageDeltaLakeName::name) + // factory.registerStorage( + // StorageDeltaLakeName::name, + // [&](const StorageFactory::Arguments & args) + // { + // auto configuration = std::make_shared(); + // return IStorageDataLake::create( + // configuration, args.getContext(), "deltaLake", args.table_id, args.columns, + // args.constraints, args.comment, std::nullopt, args.attach); + // }, + // { + // .supports_settings = false, + // .supports_schema_inference = true, + // .source_access_type = AccessType::S3, + // }); } #endif #if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. -void registerStorageIceberg(StorageFactory & factory) +void registerStorageIceberg(StorageFactory &) { - REGISTER_DATA_LAKE_STORAGE(StorageIceberg, StorageIceberg::name) + // REGISTER_DATA_LAKE_STORAGE(StorageIceberg, StorageIceberg::name) } #endif -void registerStorageHudi(StorageFactory & factory) +void registerStorageHudi(StorageFactory &) { - REGISTER_DATA_LAKE_STORAGE(StorageHudiS3, StorageHudiName::name) } } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp deleted file mode 100644 index ab21c4946e4..00000000000 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ /dev/null @@ -1,1117 +0,0 @@ -#include "config.h" - -#if USE_HDFS - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - -namespace ProfileEvents -{ - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ACCESS_DENIED; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int CANNOT_COMPILE_REGEXP; -} -namespace -{ - struct HDFSFileInfoDeleter - { - /// Can have only one entry (see hdfsGetPathInfo()) - void operator()(hdfsFileInfo * info) { hdfsFreeFileInfo(info, 1); } - }; - using HDFSFileInfoPtr = std::unique_ptr; - - /* Recursive directory listing with matched paths as a result. - * Have the same method in StorageFile. - */ - std::vector LSWithRegexpMatching( - const String & path_for_ls, - const HDFSFSPtr & fs, - const String & for_match) - { - std::vector result; - - const size_t first_glob_pos = for_match.find_first_of("*?{"); - - if (first_glob_pos == std::string::npos) - { - const String path = fs::path(path_for_ls + for_match.substr(1)).lexically_normal(); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path.c_str())); - if (hdfs_info) // NOLINT - { - result.push_back(StorageHDFS::PathWithInfo{ - String(path), - StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}}); - } - return result; - } - - const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); - const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - - const size_t next_slash_after_glob_pos = suffix_with_globs.find('/', 1); - - const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); - - re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); - if (!matcher.ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); - - HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); - if (ls.file_info == nullptr && errno != ENOENT) // NOLINT - { - // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. - throw Exception( - ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", prefix_without_globs, String(hdfsGetLastError())); - } - - if (!ls.file_info && ls.length > 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); - for (int i = 0; i < ls.length; ++i) - { - const String full_path = fs::path(ls.file_info[i].mName).lexically_normal(); - const size_t last_slash = full_path.rfind('/'); - const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; - const bool is_directory = ls.file_info[i].mKind == 'D'; - /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if (!is_directory && !looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - result.push_back(StorageHDFS::PathWithInfo{ - String(full_path), - StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}}); - } - else if (is_directory && looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - { - std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, - suffix_with_globs.substr(next_slash_after_glob_pos)); - /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); - } - } - } - - return result; - } - - std::pair getPathFromUriAndUriWithoutPath(const String & uri) - { - auto pos = uri.find("//"); - if (pos != std::string::npos && pos + 2 < uri.length()) - { - pos = uri.find('/', pos + 2); - if (pos != std::string::npos) - return {uri.substr(pos), uri.substr(0, pos)}; - } - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage HDFS requires valid URL to be set"); - } - - std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context) - { - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - Strings paths = expandSelectionGlob(path_from_uri); - - std::vector res; - - for (const auto & path : paths) - { - auto part_of_res = LSWithRegexpMatching("/", fs, path); - res.insert(res.end(), part_of_res.begin(), part_of_res.end()); - } - return res; - } -} - -StorageHDFS::StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context_, - const String & compression_method_, - const bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , WithContext(context_) - , uris({uri_}) - , format_name(format_name_) - , compression_method(compression_method_) - , distributed_processing(distributed_processing_) - , partition_by(partition_by_) -{ - FormatFactory::instance().checkFormatName(format_name); - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - checkHDFSURL(uri_); - - String path = uri_.substr(uri_.find('/', uri_.find("//") + 2)); - is_path_with_globs = path.find_first_of("*?{") != std::string::npos; - - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - auto columns = getTableStructureFromData(format_name, uri_, compression_method, context_); - storage_metadata.setColumns(columns); - } - else - { - /// We don't allow special columns in HDFS storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine HDFS doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - const std::vector & paths_with_info_, - const String & uri_without_path_, - const String & format_, - const String & compression_method_, - const ContextPtr & context_) - : WithContext(context_) - , paths_with_info(paths_with_info_) - , uri_without_path(uri_without_path_) - , format(format_) - , compression_method(compression_method_) - { - } - - std::pair, std::optional> next() override - { - bool is_first = current_index == 0; - /// For default mode check cached columns for all paths on first iteration. - if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(paths_with_info)) - return {nullptr, cached_columns}; - } - - StorageHDFS::PathWithInfo path_with_info; - - while (true) - { - if (current_index == paths_with_info.size()) - { - if (is_first) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because all files are empty. " - "You must specify table structure manually", format); - return {nullptr, std::nullopt}; - } - - path_with_info = paths_with_info[current_index++]; - if (getContext()->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) - continue; - - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - std::vector paths = {path_with_info}; - if (auto cached_columns = tryGetColumnsFromCache(paths)) - return {nullptr, cached_columns}; - } - - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - if (!getContext()->getSettingsRef().hdfs_skip_empty_files || !impl->eof()) - { - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)), std::nullopt}; - } - } - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addColumns(key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - Strings sources; - sources.reserve(paths_with_info.size()); - std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const StorageHDFS::PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; }); - auto cache_keys = getKeysForSchemaCache(sources, format, {}, getContext()); - StorageHDFS::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - String getLastFileName() const override - { - if (current_index != 0) - return paths_with_info[current_index - 1].path; - - return ""; - } - - private: - std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) - return std::nullopt; - - auto & schema_cache = StorageHDFS::getSchemaCache(getContext()); - for (const auto & path_with_info : paths_with_info_) - { - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - - auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str())); - if (hdfs_info) - return hdfs_info->mLastMod; - - return std::nullopt; - }; - - String url = uri_without_path + path_with_info.path; - auto cache_key = getKeyForSchemaCache(url, format, {}, getContext()); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; - } - - const std::vector & paths_with_info; - const String & uri_without_path; - const String & format; - const String & compression_method; - size_t current_index = 0; - }; -} - -ColumnsDescription StorageHDFS::getTableStructureFromData( - const String & format, - const String & uri, - const String & compression_method, - ContextPtr ctx) -{ - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - auto paths_with_info = getPathsList(path_from_uri, uri, ctx); - - if (paths_with_info.empty() && !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format)) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because there are no files in HDFS with provided path." - " You must specify table structure manually", format); - - ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx); - return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths_with_info.size() > 1, ctx); -} - -class HDFSSource::DisclosedGlobIterator::Impl -{ -public: - Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - { - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - uris = getPathsList(path_from_uri, uri_without_path, context); - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & path_with_info : uris) - paths.push_back(path_with_info.path); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); - } - auto file_progress_callback = context->getFileProgressCallback(); - - for (auto & elem : uris) - { - elem.path = uri_without_path + elem.path; - if (file_progress_callback && elem.info) - file_progress_callback(FileProgress(0, elem.info->size)); - } - uris_iter = uris.begin(); - } - - StorageHDFS::PathWithInfo next() - { - std::lock_guard lock(mutex); - if (uris_iter != uris.end()) - { - auto answer = *uris_iter; - ++uris_iter; - return answer; - } - return {}; - } -private: - std::mutex mutex; - std::vector uris; - std::vector::iterator uris_iter; -}; - -class HDFSSource::URISIterator::Impl : WithContext -{ -public: - explicit Impl(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context_) - : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) - { - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & uri : uris) - paths.push_back(getPathFromUriAndUriWithoutPath(uri).first); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, getContext()); - } - - if (!uris.empty()) - { - auto path_and_uri = getPathFromUriAndUriWithoutPath(uris[0]); - builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); - fs = createHDFSFS(builder.get()); - } - } - - StorageHDFS::PathWithInfo next() - { - String uri; - HDFSFileInfoPtr hdfs_info; - do - { - size_t current_index = index.fetch_add(1); - if (current_index >= uris.size()) - return {"", {}}; - - uri = uris[current_index]; - auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - hdfs_info.reset(hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str())); - } - /// Skip non-existed files. - while (!hdfs_info && String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); - - std::optional info; - if (hdfs_info) - { - info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - if (file_progress_callback) - file_progress_callback(FileProgress(0, hdfs_info->mSize)); - } - - return {uri, info}; - } - -private: - std::atomic_size_t index = 0; - Strings uris; - HDFSBuilderWrapper builder; - HDFSFSPtr fs; - std::function file_progress_callback; -}; - -HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uri, predicate, virtual_columns, context)) {} - -StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::URISIterator::URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uris_, predicate, virtual_columns, context)) -{ -} - -StorageHDFS::PathWithInfo HDFSSource::URISIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::HDFSSource( - const ReadFromFormatInfo & info, - StorageHDFSPtr storage_, - ContextPtr context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_) - : ISource(info.source_header, false) - , WithContext(context_) - , storage(std::move(storage_)) - , block_for_format(info.format_header) - , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) - , max_block_size(max_block_size_) - , file_iterator(file_iterator_) - , columns_description(info.columns_description) - , need_only_count(need_only_count_) -{ - initialize(); -} - -bool HDFSSource::initialize() -{ - bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files; - StorageHDFS::PathWithInfo path_with_info; - while (true) - { - path_with_info = (*file_iterator)(); - if (path_with_info.path.empty()) - return false; - - if (path_with_info.info && skip_empty_files && path_with_info.info->size == 0) - continue; - - current_path = path_with_info.path; - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); - - std::optional file_size; - if (!path_with_info.info) - { - auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_from_uri.c_str())); - if (hdfs_info) - path_with_info.info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - } - - if (path_with_info.info) - file_size = path_with_info.info->size; - - auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); - auto impl = std::make_unique( - uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings(), 0, false, file_size); - if (!skip_empty_files || !impl->eof()) - { - impl->setProgressCallback(getContext()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); - break; - } - } - - current_path = path_with_info.path; - current_file_size = path_with_info.info ? std::optional(path_with_info.info->size) : std::nullopt; - - QueryPipelineBuilder builder; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(path_with_info) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use a special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - auto source = std::make_shared(block_for_format, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, std::nullopt, max_parsing_threads); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - if (columns_description.hasDefaults()) - { - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, columns_description, *input_format, getContext()); - }); - } - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from the chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return true; -} - -String HDFSSource::getName() const -{ - return "HDFSSource"; -} - -Chunk HDFSSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (input_format) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, current_path, current_file_size); - return chunk; - } - - if (input_format && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(current_path, total_rows_in_file); - - total_rows_in_file = 0; - - reader.reset(); - pipeline.reset(); - input_format.reset(); - read_buf.reset(); - - if (!initialize()) - break; - } - return {}; -} - -void HDFSSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - auto cache_key = getKeyForSchemaCache(path, storage->format_name, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional HDFSSource::tryGetNumRowsFromCache(const StorageHDFS::PathWithInfo & path_with_info) -{ - auto cache_key = getKeyForSchemaCache(path_with_info.path, storage->format_name, std::nullopt, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - return std::nullopt; - }; - - return StorageHDFS::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -class HDFSSink : public SinkToStorage -{ -public: - HDFSSink(const String & uri, - const String & format, - const Block & sample_block, - ContextPtr context, - const CompressionMethod compression_method) - : SinkToStorage(sample_block) - { - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique( - uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, context->getWriteSettings()), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); - } - - String getName() const override { return "HDFSSink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->sync(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } - - std::unique_ptr write_buf; - OutputFormatPtr writer; - std::mutex cancel_mutex; - bool cancelled = false; -}; - -class PartitionedHDFSSink : public PartitionedSink -{ -public: - PartitionedHDFSSink( - const ASTPtr & partition_by, - const String & uri_, - const String & format_, - const Block & sample_block_, - ContextPtr context_, - const CompressionMethod compression_method_) - : PartitionedSink(partition_by, context_, sample_block_) - , uri(uri_) - , format(format_) - , sample_block(sample_block_) - , context(context_) - , compression_method(compression_method_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto path = PartitionedSink::replaceWildcards(uri, partition_id); - PartitionedSink::validatePartitionKey(path, true); - return std::make_shared(path, format, sample_block, context, compression_method); - } - -private: - const String uri; - const String format; - const Block sample_block; - ContextPtr context; - const CompressionMethod compression_method; -}; - - -bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_); -} - -class ReadFromHDFS : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromHDFS"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void applyFilters() override; - - ReadFromHDFS( - Block sample_block, - ReadFromFormatInfo info_, - bool need_only_count_, - std::shared_ptr storage_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) - , info(std::move(info_)) - , need_only_count(need_only_count_) - , storage(std::move(storage_)) - , context(std::move(context_)) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - } - -private: - ReadFromFormatInfo info; - const bool need_only_count; - std::shared_ptr storage; - - ContextPtr context; - size_t max_block_size; - size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -void ReadFromHDFS::applyFilters() -{ - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void StorageHDFS::read( - 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) -{ - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), virtual_columns); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && context_->getSettingsRef().optimize_count_from_files; - - auto this_ptr = std::static_pointer_cast(shared_from_this()); - - auto reading = std::make_unique( - read_from_format_info.source_header, - std::move(read_from_format_info), - need_only_count, - std::move(this_ptr), - context_, - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - if (storage->distributed_processing) - { - iterator_wrapper = std::make_shared( - [callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { - return StorageHDFS::PathWithInfo{callback(), std::nullopt}; - }); - } - else if (storage->is_path_with_globs) - { - /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->virtual_columns, context); - iterator_wrapper = std::make_shared([glob_iterator]() - { - return glob_iterator->next(); - }); - } - else - { - auto uris_iterator = std::make_shared(storage->uris, predicate, storage->virtual_columns, context); - iterator_wrapper = std::make_shared([uris_iterator]() - { - return uris_iterator->next(); - }); - } -} - -void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - - Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - info, - storage, - context, - max_block_size, - iterator_wrapper, - need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) -{ - String current_uri = uris.back(); - - bool has_wildcards = current_uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; - const auto * insert_query = dynamic_cast(query.get()); - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && has_wildcards; - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } - else - { - if (is_path_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); - - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_uri); - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - bool truncate_on_insert = context_->getSettingsRef().hdfs_truncate_on_insert; - if (!truncate_on_insert && !hdfsExists(fs.get(), path_from_uri.c_str())) - { - if (context_->getSettingsRef().hdfs_create_new_file_on_insert) - { - auto pos = uris[0].find_first_of('.', uris[0].find_last_of('/')); - size_t index = uris.size(); - String new_uri; - do - { - new_uri = uris[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : uris[0].substr(pos)); - ++index; - } - while (!hdfsExists(fs.get(), new_uri.c_str())); - uris.push_back(new_uri); - current_uri = new_uri; - } - else - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " - "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", - path_from_uri); - } - - return std::make_shared(current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } -} - -void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) -{ - const size_t begin_of_path = uris[0].find('/', uris[0].find("//") + 2); - const String url = uris[0].substr(0, begin_of_path); - - HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", local_context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - - for (const auto & uri : uris) - { - const String path = uri.substr(begin_of_path); - int ret = hdfsDelete(fs.get(), path.data(), 0); - if (ret) - throw Exception(ErrorCodes::ACCESS_DENIED, "Unable to truncate hdfs table: {}", std::string(hdfsGetLastError())); - } -} - - -void registerStorageHDFS(StorageFactory & factory) -{ - factory.registerStorage("HDFS", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - - if (engine_args.empty() || engine_args.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage HDFS requires 1, 2 or 3 arguments: " - "url, name of used format (taken from file extension by default) and optional compression method."); - - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext()); - - String url = checkAndGetLiteralArgument(engine_args[0], "url"); - - String format_name = "auto"; - if (engine_args.size() > 1) - { - engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext()); - format_name = checkAndGetLiteralArgument(engine_args[1], "format_name"); - } - - if (format_name == "auto") - format_name = FormatFactory::instance().getFormatFromFileName(url, true); - - String compression_method; - if (engine_args.size() == 3) - { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext()); - compression_method = checkAndGetLiteralArgument(engine_args[2], "compression_method"); - } else compression_method = "auto"; - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - return std::make_shared( - url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, false, partition_by); - }, - { - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::HDFS, - }); -} - -NamesAndTypesList StorageHDFS::getVirtuals() const -{ - return virtual_columns; -} - -Names StorageHDFS::getVirtualColumnNames() -{ - return VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage({}).getNames(); -} - -SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_hdfs", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h deleted file mode 100644 index 7170763c959..00000000000 --- a/src/Storages/HDFS/StorageHDFS.h +++ /dev/null @@ -1,179 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class IInputFormat; - -/** - * This class represents table engine for external hdfs files. - * Read method is supported for now. - */ -class StorageHDFS final : public IStorage, WithContext -{ -public: - struct PathInfo - { - time_t last_mod_time; - size_t size; - }; - - struct PathWithInfo - { - PathWithInfo() = default; - PathWithInfo(const String & path_, const std::optional & info_) : path(path_), info(info_) {} - String path; - std::optional info; - }; - - StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - ContextPtr context_, - const String & compression_method_ = "", - bool distributed_processing_ = false, - ASTPtr partition_by = nullptr); - - String getName() const override { return "HDFS"; } - - void read( - 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 write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, bool async_insert) override; - - void truncate( - const ASTPtr & query, - const StorageMetadataPtr & metadata_snapshot, - ContextPtr local_context, - TableExclusiveLockHolder &) override; - - NamesAndTypesList getVirtuals() const override; - static Names getVirtualColumnNames(); - - bool supportsPartitionBy() const override { return true; } - - /// Check if the format is column-oriented. - /// Is is useful because column oriented formats could effectively skip unknown columns - /// So we can create a header of only required columns in read method and ask - /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. - bool supportsSubsetOfColumns(const ContextPtr & context_) const; - - bool supportsSubcolumns() const override { return true; } - - static ColumnsDescription getTableStructureFromData( - const String & format, - const String & uri, - const String & compression_method, - ContextPtr ctx); - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - bool supportsTrivialCountOptimization() const override { return true; } - -protected: - friend class HDFSSource; - friend class ReadFromHDFS; - -private: - std::vector uris; - String format_name; - String compression_method; - const bool distributed_processing; - ASTPtr partition_by; - bool is_path_with_globs; - NamesAndTypesList virtual_columns; - - LoggerPtr log = getLogger("StorageHDFS"); -}; - -class PullingPipelineExecutor; - -class HDFSSource : public ISource, WithContext -{ -public: - class DisclosedGlobIterator - { - public: - DisclosedGlobIterator(const String & uri_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); - StorageHDFS::PathWithInfo next(); - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class URISIterator - { - public: - URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context); - StorageHDFS::PathWithInfo next(); - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - using IteratorWrapper = std::function; - using StorageHDFSPtr = std::shared_ptr; - - HDFSSource( - const ReadFromFormatInfo & info, - StorageHDFSPtr storage_, - ContextPtr context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_); - - String getName() const override; - - Chunk generate() override; - -private: - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const StorageHDFS::PathWithInfo & path_with_info); - - StorageHDFSPtr storage; - Block block_for_format; - NamesAndTypesList requested_columns; - NamesAndTypesList requested_virtual_columns; - UInt64 max_block_size; - std::shared_ptr file_iterator; - ColumnsDescription columns_description; - bool need_only_count; - size_t total_rows_in_file = 0; - - std::unique_ptr read_buf; - std::shared_ptr input_format; - std::unique_ptr pipeline; - std::unique_ptr reader; - String current_path; - std::optional current_file_size; - - /// Recreate ReadBuffer and PullingPipelineExecutor for each file. - bool initialize(); -}; -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp deleted file mode 100644 index fad29436102..00000000000 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ /dev/null @@ -1,98 +0,0 @@ -#include "config.h" -#include "Interpreters/Context_fwd.h" - -#if USE_HDFS - -#include - -#include -#include -#include -#include -#include - -#include - -#include -#include -#include - -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageHDFSCluster::StorageHDFSCluster( - ContextPtr context_, - const String & cluster_name_, - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & compression_method_, - bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageHDFSCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) - , uri(uri_) - , format_name(format_name_) - , compression_method(compression_method_) -{ - checkHDFSURL(uri_); - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - auto columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_); - storage_metadata.setColumns(columns); - } - else - storage_metadata.setColumns(columns_); - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query)); - - TableFunctionHDFSCluster::addColumnsStructureToArguments(expression_list->children, structure, context); -} - - -RemoteQueryExecutor::Extension StorageHDFSCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared(uri, predicate, virtual_columns, context); - auto callback = std::make_shared>([iter = std::move(iterator)]() mutable -> String { return iter->next().path; }); - return RemoteQueryExecutor::Extension{.task_iterator = std::move(callback)}; -} - -NamesAndTypesList StorageHDFSCluster::getVirtuals() const -{ - return NamesAndTypesList{ - {"_path", std::make_shared(std::make_shared())}, - {"_file", std::make_shared(std::make_shared())}}; -} - -} - -#endif diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h deleted file mode 100644 index 7c4c41a573a..00000000000 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ /dev/null @@ -1,56 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include - -#include -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageHDFSCluster : public IStorageCluster -{ -public: - StorageHDFSCluster( - ContextPtr context_, - const String & cluster_name_, - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & compression_method_, - bool structure_argument_was_provided_); - - std::string getName() const override { return "HDFSCluster"; } - - NamesAndTypesList getVirtuals() const override; - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization() const override { return true; } - -private: - void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; - - String uri; - String format_name; - String compression_method; - NamesAndTypesList virtual_columns; -}; - - -} - -#endif diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 4fa6bfdd617..26301472f24 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -98,9 +98,14 @@ class IStorage : public std::enable_shared_from_this, public TypePromo public: IStorage() = delete; /// Storage metadata can be set separately in setInMemoryMetadata method - explicit IStorage(StorageID storage_id_) + explicit IStorage(StorageID storage_id_, std::unique_ptr metadata_ = nullptr) : storage_id(std::move(storage_id_)) - , metadata(std::make_unique()) {} + { + if (metadata_) + metadata.set(std::move(metadata_)); + else + metadata.set(std::make_unique()); + } IStorage(const IStorage &) = delete; IStorage & operator=(const IStorage &) = delete; diff --git a/src/Storages/ObjectStorage/AzureConfiguration.cpp b/src/Storages/ObjectStorage/AzureConfiguration.cpp new file mode 100644 index 00000000000..ba3e796223a --- /dev/null +++ b/src/Storages/ObjectStorage/AzureConfiguration.cpp @@ -0,0 +1,451 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +const std::unordered_set required_configuration_keys = { + "blob_path", + "container", +}; + +const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "structure", + "compression_method", + "account_name", + "account_key", + "connection_string", + "storage_account_url", +}; + +using AzureClient = Azure::Storage::Blobs::BlobContainerClient; +using AzureClientPtr = std::unique_ptr; + +namespace +{ + bool isConnectionString(const std::string & candidate) + { + return !candidate.starts_with("http"); + } + + bool containerExists(std::unique_ptr & blob_service_client, std::string container_name) + { + Azure::Storage::Blobs::ListBlobContainersOptions options; + options.Prefix = container_name; + options.PageSizeHint = 1; + + auto containers_list_response = blob_service_client->ListBlobContainers(options); + auto containers_list = containers_list_response.BlobContainers; + + for (const auto & container : containers_list) + { + if (container_name == container.Name) + return true; + } + return false; + } +} + +void StorageAzureBlobConfiguration::check(ContextPtr context) const +{ + Poco::URI url_to_check; + if (is_connection_string) + { + auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); + url_to_check = Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); + } + else + url_to_check = Poco::URI(connection_url); + + context->getGlobalContext()->getRemoteHostFilter().checkURL(url_to_check); +} + +StorageObjectStorageConfigurationPtr StorageAzureBlobConfiguration::clone() +{ + auto configuration = std::make_shared(); + configuration->connection_url = connection_url; + configuration->is_connection_string = is_connection_string; + configuration->account_name = account_name; + configuration->account_key = account_key; + configuration->container = container; + configuration->blob_path = blob_path; + configuration->blobs_paths = blobs_paths; + return configuration; +} + +AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(ContextPtr context) +{ + const auto & context_settings = context->getSettingsRef(); + auto settings_ptr = std::make_unique(); + settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; + settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; + settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + return settings_ptr; +} + +ObjectStoragePtr StorageAzureBlobConfiguration::createOrUpdateObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +{ + auto client = createClient(is_readonly); + auto settings = createSettings(context); + return std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), container); +} + +AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) +{ + using namespace Azure::Storage::Blobs; + + AzureClientPtr result; + + if (is_connection_string) + { + auto blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); + result = std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_url, container)); + bool container_exists = containerExists(blob_service_client, container); + + if (!container_exists) + { + if (is_read_only) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "AzureBlobStorage container does not exist '{}'", + container); + + try + { + result->CreateIfNotExists(); + } catch (const Azure::Storage::StorageException & e) + { + if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.")) + { + throw; + } + } + } + } + else + { + std::shared_ptr storage_shared_key_credential; + if (account_name.has_value() && account_key.has_value()) + { + storage_shared_key_credential = + std::make_shared(*account_name, *account_key); + } + + std::unique_ptr blob_service_client; + if (storage_shared_key_credential) + { + blob_service_client = std::make_unique(connection_url, storage_shared_key_credential); + } + else + { + blob_service_client = std::make_unique(connection_url); + } + + bool container_exists = containerExists(blob_service_client, container); + + std::string final_url; + size_t pos = connection_url.find('?'); + if (pos != std::string::npos) + { + auto url_without_sas = connection_url.substr(0, pos); + final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + container + + connection_url.substr(pos); + } + else + final_url + = connection_url + (connection_url.back() == '/' ? "" : "/") + container; + + if (container_exists) + { + if (storage_shared_key_credential) + result = std::make_unique(final_url, storage_shared_key_credential); + else + result = std::make_unique(final_url); + } + else + { + if (is_read_only) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "AzureBlobStorage container does not exist '{}'", + container); + try + { + result = std::make_unique(blob_service_client->CreateBlobContainer(container).Value); + } + catch (const Azure::Storage::StorageException & e) + { + if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.") + { + if (storage_shared_key_credential) + result = std::make_unique(final_url, storage_shared_key_credential); + else + result = std::make_unique(final_url); + } + else + { + throw; + } + } + } + } + + return result; +} + +void StorageAzureBlobConfiguration::fromNamedCollection(const NamedCollection & collection) +{ + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + + if (collection.has("connection_string")) + { + connection_url = collection.get("connection_string"); + is_connection_string = true; + } + + if (collection.has("storage_account_url")) + { + connection_url = collection.get("storage_account_url"); + is_connection_string = false; + } + + container = collection.get("container"); + blob_path = collection.get("blob_path"); + + if (collection.has("account_name")) + account_name = collection.get("account_name"); + + 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")); + + blobs_paths = {blob_path}; + if (format == "auto") + format = FormatFactory::instance().getFormatFromFileName(blob_path, true); +} + +void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) +{ + if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage AzureBlobStorage requires 3 to 7 arguments: " + "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure)])"); + } + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + std::unordered_map engine_args_to_idx; + + connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); + is_connection_string = isConnectionString(connection_url); + + container = checkAndGetLiteralArgument(engine_args[1], "container"); + blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); + + auto is_format_arg = [] (const std::string & s) -> bool + { + return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); + }; + + if (engine_args.size() == 4) + { + //'c1 UInt64, c2 UInt64 + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + format = fourth_arg; + } + else + { + if (with_structure) + structure = fourth_arg; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format or account name specified without account key"); + } + } + else if (engine_args.size() == 5) + { + 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"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + } + } + else if (engine_args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (is_format_arg(fourth_arg)) + { + if (with_structure) + { + format = fourth_arg; + compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); + structure = checkAndGetLiteralArgument(engine_args[5], "structure"); + } + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name"); + if (is_format_arg(sixth_arg)) + format = sixth_arg; + else + { + if (with_structure) + structure = sixth_arg; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format {}", sixth_arg); + } + } + } + else if (engine_args.size() == 7) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + if (!with_structure && is_format_arg(fourth_arg)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); + } + else + { + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + 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"); + } + } + else if (with_structure && engine_args.size() == 8) + { + auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); + account_name = fourth_arg; + account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); + 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"); + structure = checkAndGetLiteralArgument(engine_args[7], "structure"); + } + + blobs_paths = {blob_path}; + + if (format == "auto") + format = FormatFactory::instance().getFormatFromFileName(blob_path, true); +} + +void StorageAzureBlobConfiguration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + if (args.size() < 3 || args.size() > 8) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Azure requires 3 to 7 arguments: " + "StorageObjectStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); + } + + auto structure_literal = std::make_shared(structure_); + auto is_format_arg + = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; + + if (args.size() == 3) + { + /// Add format=auto & compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else if (args.size() == 4) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); + if (is_format_arg(fourth_arg)) + { + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else + { + args.back() = structure_literal; + } + } + else if (args.size() == 5) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + if (!is_format_arg(fourth_arg)) + { + /// Add format=auto & compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(std::make_shared("auto")); + } + args.push_back(structure_literal); + } + else if (args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + if (!is_format_arg(fourth_arg)) + { + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + else + { + args.back() = structure_literal; + } + } + else if (args.size() == 7) + { + args.push_back(structure_literal); + } + else if (args.size() == 8) + { + args.back() = structure_literal; + } + } +} + +} diff --git a/src/Storages/ObjectStorage/AzureConfiguration.h b/src/Storages/ObjectStorage/AzureConfiguration.h new file mode 100644 index 00000000000..40d718d7690 --- /dev/null +++ b/src/Storages/ObjectStorage/AzureConfiguration.h @@ -0,0 +1,54 @@ +#pragma once +#include +#include + +namespace DB +{ +class BackupFactory; + +class StorageAzureBlobConfiguration : public StorageObjectStorageConfiguration +{ + friend class BackupReaderAzureBlobStorage; + friend class BackupWriterAzureBlobStorage; + friend void registerBackupEngineAzureBlobStorage(BackupFactory & factory); + +public: + StorageAzureBlobConfiguration() = default; + StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other); + + Path getPath() const override { return blob_path; } + void setPath(const Path & path) override { blob_path = path; } + + const Paths & getPaths() const override { return blobs_paths; } + Paths & getPaths() override { return blobs_paths; } + + String getDataSourceDescription() override { return fs::path(connection_url) / container; } + String getNamespace() const override { return container; } + + void check(ContextPtr context) const override; + StorageObjectStorageConfigurationPtr clone() override; + ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + static void addStructureToArgs(ASTs & args, const String & structure, ContextPtr context); + +protected: + using AzureClient = Azure::Storage::Blobs::BlobContainerClient; + using AzureClientPtr = std::unique_ptr; + + std::string connection_url; + bool is_connection_string; + + std::optional account_name; + std::optional account_key; + + std::string container; + std::string blob_path; + std::vector blobs_paths; + + AzureClientPtr createClient(bool is_read_only); + AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); +}; + +} diff --git a/src/Storages/ObjectStorage/Configuration.h b/src/Storages/ObjectStorage/Configuration.h new file mode 100644 index 00000000000..708041980e3 --- /dev/null +++ b/src/Storages/ObjectStorage/Configuration.h @@ -0,0 +1,55 @@ +#pragma once +#include +#include + +namespace DB +{ + +class StorageObjectStorageConfiguration; +using StorageObjectStorageConfigurationPtr = std::shared_ptr; + +class StorageObjectStorageConfiguration +{ +public: + StorageObjectStorageConfiguration() = default; + virtual ~StorageObjectStorageConfiguration() = default; + + using Path = std::string; + using Paths = std::vector; + + virtual Path getPath() const = 0; + virtual void setPath(const Path & path) = 0; + + virtual const Paths & getPaths() const = 0; + virtual Paths & getPaths() = 0; + + virtual String getDataSourceDescription() = 0; + virtual String getNamespace() const = 0; + + bool isPathWithGlobs() const { return getPath().find_first_of("*?{") != std::string::npos; } + bool isNamespaceWithGlobs() const { return getNamespace().find_first_of("*?{") != std::string::npos; } + + std::string getPathWithoutGlob() const { return getPath().substr(0, getPath().find_first_of("*?{")); } + + virtual bool withWildcard() const + { + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return getPath().find(PARTITION_ID_WILDCARD) != String::npos; + } + + virtual void check(ContextPtr context) const = 0; + virtual StorageObjectStorageConfigurationPtr clone() = 0; + + virtual ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT + + virtual void fromNamedCollection(const NamedCollection & collection) = 0; + virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + + String format = "auto"; + String compression_method = "auto"; + String structure = "auto"; +}; + +using StorageObjectStorageConfigurationPtr = std::shared_ptr; + +} diff --git a/src/Storages/ObjectStorage/HDFSConfiguration.h b/src/Storages/ObjectStorage/HDFSConfiguration.h new file mode 100644 index 00000000000..f42cedf459d --- /dev/null +++ b/src/Storages/ObjectStorage/HDFSConfiguration.h @@ -0,0 +1,81 @@ +#pragma once +#include "config.h" + +#if USE_HDFS + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +class StorageHDFSConfiguration : public StorageObjectStorageConfiguration +{ +public: + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + Paths & getPaths() override { return paths; } + + String getNamespace() const override { return ""; } + String getDataSourceDescription() override { return url; } + + void check(ContextPtr context) const override + { + context->getRemoteHostFilter().checkURL(Poco::URI(url)); + checkHDFSURL(url); + } + StorageObjectStorageConfigurationPtr clone() override + { + auto configuration = std::make_shared(); + return configuration; + } + + ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override /// NOLINT + { + UNUSED(is_readonly); + auto settings = std::make_unique(); + return std::make_shared(url, std::move(settings), context->getConfigRef()); + } + + void fromNamedCollection(const NamedCollection &) override {} + void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override + { + url = checkAndGetLiteralArgument(args[0], "url"); + + String format_name = "auto"; + if (args.size() > 1) + format_name = checkAndGetLiteralArgument(args[1], "format_name"); + + if (format_name == "auto") + format_name = FormatFactory::instance().getFormatFromFileName(url, true); + + String compression_method; + if (args.size() == 3) + { + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + } else compression_method = "auto"; + + } + static void addStructureToArgs(ASTs &, const String &, ContextPtr) {} + +private: + String url; + String path; + std::vector paths; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h new file mode 100644 index 00000000000..248700e2edf --- /dev/null +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -0,0 +1,197 @@ +#pragma once +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + +} + +template +class ReadBufferIterator : public IReadBufferIterator, WithContext +{ +public: + using Storage = StorageObjectStorage; + using Source = StorageObjectStorageSource; + using FileIterator = std::shared_ptr; + using ObjectInfos = typename Storage::ObjectInfos; + + ReadBufferIterator( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const FileIterator & file_iterator_, + const std::optional & format_settings_, + ObjectInfos & read_keys_, + const ContextPtr & context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + , file_iterator(file_iterator_) + , format_settings(format_settings_) + , storage_settings(StorageSettings::create(context_->getSettingsRef())) + , read_keys(read_keys_) + , prev_read_keys_size(read_keys_.size()) + { + } + + std::pair, std::optional> next() override + { + /// For default mode check cached columns for currently read keys on first iteration. + if (first && storage_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + return {nullptr, cached_columns}; + } + + current_object_info = file_iterator->next(0); + if (current_object_info->relative_path.empty()) + { + if (first) + { + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, " + "because there are no files with provided path. " + "You must specify table structure manually", + configuration->format); + } + return {nullptr, std::nullopt}; + } + + first = false; + + /// File iterator could get new keys after new iteration, + /// check them in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT + && read_keys.size() > prev_read_keys_size) + { + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); + prev_read_keys_size = read_keys.size(); + if (columns_from_cache) + return {nullptr, columns_from_cache}; + } + else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + ObjectInfos paths = {current_object_info}; + if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) + return {nullptr, columns_from_cache}; + } + + first = false; + + std::unique_ptr read_buffer = object_storage->readObject( + StoredObject(current_object_info->relative_path), + getContext()->getReadSettings(), + {}, + current_object_info->metadata.size_bytes); + + read_buffer = wrapReadBufferWithCompressionMethod( + std::move(read_buffer), + chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), + static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + + return {std::move(read_buffer), std::nullopt}; + } + + void setNumRowsToLastFile(size_t num_rows) override + { + if (storage_settings.schema_inference_use_cache) + { + Storage::getSchemaCache(getContext()).addNumRows( + getKeyForSchemaCache(current_object_info->relative_path), num_rows); + } + } + + void setSchemaToLastFile(const ColumnsDescription & columns) override + { + if (storage_settings.schema_inference_use_cache + && storage_settings.schema_inference_mode == SchemaInferenceMode::UNION) + { + Storage::getSchemaCache(getContext()).addColumns( + getKeyForSchemaCache(current_object_info->relative_path), columns); + } + } + + void setResultingSchema(const ColumnsDescription & columns) override + { + if (storage_settings.schema_inference_use_cache + && storage_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + Storage::getSchemaCache(getContext()).addManyColumns(getPathsForSchemaCache(), columns); + } + } + + String getLastFileName() const override { return current_object_info->relative_path; } + +private: + SchemaCache::Key getKeyForSchemaCache(const String & path) const + { + auto source = fs::path(configuration->getDataSourceDescription()) / path; + return DB::getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); + } + + SchemaCache::Keys getPathsForSchemaCache() const + { + Strings sources; + sources.reserve(read_keys.size()); + std::transform( + read_keys.begin(), read_keys.end(), + std::back_inserter(sources), + [&](const auto & elem) + { + return fs::path(configuration->getDataSourceDescription()) / elem->relative_path; + }); + return DB::getKeysForSchemaCache(sources, configuration->format, format_settings, getContext()); + } + + std::optional tryGetColumnsFromCache( + const ObjectInfos::iterator & begin, + const ObjectInfos::iterator & end) + { + if (!storage_settings.schema_inference_use_cache) + return std::nullopt; + + auto & schema_cache = Storage::getSchemaCache(getContext()); + for (auto it = begin; it < end; ++it) + { + const auto & object_info = (*it); + auto get_last_mod_time = [&] -> std::optional + { + if (object_info->metadata.last_modified) + return object_info->metadata.last_modified->epochMicroseconds(); + else + { + object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + return object_info->metadata.last_modified->epochMicroseconds(); + } + }; + + auto cache_key = getKeyForSchemaCache(object_info->relative_path); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; + } + + ObjectStoragePtr object_storage; + const Storage::ConfigurationPtr configuration; + const FileIterator file_iterator; + const std::optional & format_settings; + const StorageObjectStorageSettings storage_settings; + ObjectInfos & read_keys; + + size_t prev_read_keys_size; + Storage::ObjectInfoPtr current_object_info; + bool first = true; +}; +} diff --git a/src/Storages/ObjectStorage/ReadFromObjectStorage.h b/src/Storages/ObjectStorage/ReadFromObjectStorage.h new file mode 100644 index 00000000000..9cb77dcc25e --- /dev/null +++ b/src/Storages/ObjectStorage/ReadFromObjectStorage.h @@ -0,0 +1,105 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +template +class ReadFromStorageObejctStorage : public SourceStepWithFilter +{ +public: + using Storage = StorageObjectStorage; + using Source = StorageObjectStorageSource; + + ReadFromStorageObejctStorage( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const String & name_, + const NamesAndTypesList & virtual_columns_, + const std::optional & format_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + const bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = info_.source_header}) + , object_storage(object_storage_) + , configuration(configuration_) + , context(std::move(context_)) + , info(std::move(info_)) + , virtual_columns(virtual_columns_) + , format_settings(format_settings_) + , name(name_ + "Source") + , need_only_count(need_only_count_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , distributed_processing(distributed_processing_) + { + } + + std::string getName() const override { return name; } + + void applyFilters() override + { + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); + } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + createIterator(nullptr); + + Pipes pipes; + for (size_t i = 0; i < num_streams; ++i) + { + pipes.emplace_back(std::make_shared( + getName(), object_storage, configuration, info, format_settings, + context, max_block_size, iterator_wrapper, need_only_count)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); + } + +private: + ObjectStoragePtr object_storage; + Storage::ConfigurationPtr configuration; + ContextPtr context; + + const ReadFromFormatInfo info; + const NamesAndTypesList virtual_columns; + const std::optional format_settings; + const String name; + const bool need_only_count; + const size_t max_block_size; + const size_t num_streams; + const bool distributed_processing; + + std::shared_ptr iterator_wrapper; + + void createIterator(const ActionsDAG::Node * predicate) + { + if (!iterator_wrapper) + { + iterator_wrapper = Source::createFileIterator( + configuration, object_storage, distributed_processing, context, + predicate, virtual_columns, nullptr, context->getFileProgressCallback()); + } + } +}; + +} diff --git a/src/Storages/ObjectStorage/S3Configuration.cpp b/src/Storages/ObjectStorage/S3Configuration.cpp new file mode 100644 index 00000000000..5a5412019f5 --- /dev/null +++ b/src/Storages/ObjectStorage/S3Configuration.cpp @@ -0,0 +1,491 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +static const std::unordered_set required_configuration_keys = { + "url", +}; + +static const std::unordered_set optional_configuration_keys = { + "format", + "compression", + "compression_method", + "structure", + "access_key_id", + "secret_access_key", + "session_token", + "filename", + "use_environment_credentials", + "max_single_read_retries", + "min_upload_part_size", + "upload_part_size_multiply_factor", + "upload_part_size_multiply_parts_count_threshold", + "max_single_part_upload_size", + "max_connections", + "expiration_window_seconds", + "no_sign_request" +}; + +String StorageS3Configuration::getDataSourceDescription() +{ + return fs::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; +} + +void StorageS3Configuration::check(ContextPtr context) const +{ + context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); + context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); +} + +StorageObjectStorageConfigurationPtr StorageS3Configuration::clone() +{ + auto configuration = std::make_shared(); + configuration->url = url; + configuration->auth_settings = auth_settings; + configuration->request_settings = request_settings; + configuration->static_configuration = static_configuration; + configuration->headers_from_ast = headers_from_ast; + configuration->keys = keys; + configuration->initialized = initialized; + return configuration; +} + +ObjectStoragePtr StorageS3Configuration::createOrUpdateObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT +{ + auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString()); + request_settings = s3_settings.request_settings; + request_settings.updateFromSettings(context->getSettings()); + + if (!initialized || (!static_configuration && auth_settings.hasUpdates(s3_settings.auth_settings))) + { + auth_settings.updateFrom(s3_settings.auth_settings); + keys[0] = url.key; + initialized = true; + } + + const auto & config = context->getConfigRef(); + auto s3_capabilities = S3Capabilities + { + .support_batch_delete = config.getBool("s3.support_batch_delete", true), + .support_proxy = config.getBool("s3.support_proxy", config.has("s3.proxy")), + }; + + auto s3_storage_settings = std::make_unique( + request_settings, + config.getUInt64("s3.min_bytes_for_seek", 1024 * 1024), + config.getInt("s3.list_object_keys_size", 1000), + config.getInt("s3.objects_chunk_size_to_delete", 1000), + config.getBool("s3.readonly", false)); + + auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); + auto client = createClient(context); + std::string disk_name = "StorageS3"; + + return std::make_shared( + std::move(client), std::move(s3_storage_settings), url, s3_capabilities, key_generator, /*disk_name*/disk_name); +} + +std::unique_ptr StorageS3Configuration::createClient(ContextPtr context) +{ + const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); + const Settings & local_settings = context->getSettingsRef(); + + auto client_configuration = S3::ClientFactory::instance().createClientConfiguration( + auth_settings.region, + context->getRemoteHostFilter(), + static_cast(global_settings.s3_max_redirects), + static_cast(global_settings.s3_retry_attempts), + global_settings.enable_s3_requests_logging, + /* for_disk_s3 = */ false, + request_settings.get_request_throttler, + request_settings.put_request_throttler, + url.uri.getScheme()); + + client_configuration.endpointOverride = url.endpoint; + client_configuration.maxConnections = static_cast(request_settings.max_connections); + client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; + + auto headers = auth_settings.headers; + if (!headers_from_ast.empty()) + headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); + + client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; + + S3::ClientSettings client_settings{ + .use_virtual_addressing = url.is_virtual_hosted_style, + .disable_checksum = local_settings.s3_disable_checksum, + .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), + }; + + auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, + auth_settings.secret_access_key, + auth_settings.session_token); + + auto credentials_configuration = S3::CredentialsConfiguration + { + auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), + auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + auth_settings.expiration_window_seconds.value_or(context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), + }; + + return S3::ClientFactory::instance().create( + client_configuration, + client_settings, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + auth_settings.server_side_encryption_customer_key_base64, + auth_settings.server_side_encryption_kms_config, + std::move(headers), + credentials_configuration); +} + +void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) +{ + validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); + + auto filename = collection.getOrDefault("filename", ""); + if (!filename.empty()) + url = S3::URI(std::filesystem::path(collection.get("url")) / filename); + else + url = S3::URI(collection.get("url")); + + auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); + auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); + auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); + auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", false); + auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); + + format = collection.getOrDefault("format", format); + compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + structure = collection.getOrDefault("structure", "auto"); + + request_settings = S3Settings::RequestSettings(collection); + + static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + + keys = {url.key}; + + //if (format == "auto" && get_format_from_file) + if (format == "auto") + format = FormatFactory::instance().getFormatFromFileName(url.key, true); +} + +void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + /// Supported signatures: S3('url') S3('url', 'format') S3('url', 'format', 'compression') S3('url', NOSIGN) S3('url', NOSIGN, 'format') S3('url', NOSIGN, 'format', 'compression') S3('url', 'aws_access_key_id', 'aws_secret_access_key') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression') + /// with optional headers() function + + size_t count = StorageURL::evalArgsAndCollectHeaders(args, headers_from_ast, context); + + if (count == 0 || count > (with_structure ? 7 : 6)) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage S3 requires 1 to 5 arguments: " + "url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]"); + + std::unordered_map engine_args_to_idx; + bool no_sign_request = false; + + /// For 2 arguments we support 2 possible variants: + /// - s3(source, format) + /// - s3(source, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + if (count == 2) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + no_sign_request = true; + else + engine_args_to_idx = {{"format", 1}}; + } + /// For 3 arguments we support 2 possible variants: + /// - s3(source, format, compression_method) + /// - s3(source, access_key_id, secret_access_key) + /// - s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. + else if (count == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}}; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + { + if (with_structure) + engine_args_to_idx = {{"format", 1}, {"structure", 2}}; + else + engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; + } + else + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; + } + /// For 4 arguments we support 3 possible variants: + /// if with_structure == 0: + /// - s3(source, access_key_id, secret_access_key, session_token) + /// - s3(source, access_key_id, secret_access_key, format) + /// - s3(source, NOSIGN, format, compression_method) + /// if with_structure == 1: + /// - s3(source, format, structure, compression_method), + /// - s3(source, access_key_id, secret_access_key, format), + /// - s3(source, access_key_id, secret_access_key, session_token) + /// - s3(source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. + else if (count == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "access_key_id/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + if (with_structure) + engine_args_to_idx = {{"format", 2}, {"structure", 3}}; + else + engine_args_to_idx = {{"format", 2}, {"compression_method", 3}}; + } + else if (with_structure && (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))) + { + engine_args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "session_token/format"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; + } + } + } + /// For 5 arguments we support 2 possible variants: + /// if with_structure == 0: + /// - s3(source, access_key_id, secret_access_key, session_token, format) + /// - s3(source, access_key_id, secret_access_key, format, compression) + /// if with_structure == 1: + /// - s3(source, access_key_id, secret_access_key, format, structure) + /// - s3(source, access_key_id, secret_access_key, session_token, format) + /// - s3(source, NOSIGN, format, structure, compression_method) + else if (count == 5) + { + if (with_structure) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); + if (boost::iequals(second_arg, "NOSIGN")) + { + no_sign_request = true; + engine_args_to_idx = {{"format", 2}, {"structure", 3}, {"compression_method", 4}}; + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; + } + } + } + else + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "session_token/format"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression_method", 4}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; + } + } + } + else if (count == 6) + { + if (with_structure) + { + /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) + /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) + /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); + if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}; + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}}; + } + } + else + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}; + } + } + else if (with_structure && count == 7) + { + engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}; + } + + /// This argument is always the first + url = S3::URI(checkAndGetLiteralArgument(args[0], "url")); + + if (engine_args_to_idx.contains("format")) + { + 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 (engine_args_to_idx.contains("structure")) + structure = 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"); + + if (engine_args_to_idx.contains("access_key_id")) + auth_settings.access_key_id = checkAndGetLiteralArgument(args[engine_args_to_idx["access_key_id"]], "access_key_id"); + + if (engine_args_to_idx.contains("secret_access_key")) + auth_settings.secret_access_key = checkAndGetLiteralArgument(args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); + + if (engine_args_to_idx.contains("session_token")) + auth_settings.session_token = checkAndGetLiteralArgument(args[engine_args_to_idx["session_token"]], "session_token"); + + if (no_sign_request) + auth_settings.no_sign_request = no_sign_request; + + static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); + auth_settings.no_sign_request = no_sign_request; + + keys = {url.key}; + + // if (format == "auto" && get_format_from_file) + if (format == "auto") + format = FormatFactory::instance().getFormatFromFileName(url.key, true); +} + +void StorageS3Configuration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) +{ + if (tryGetNamedCollectionWithOverrides(args, context)) + { + /// In case of named collection, just add key-value pair "structure='...'" + /// at the end of arguments to override existed structure. + ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure_)}; + auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); + args.push_back(equal_func); + } + else + { + HTTPHeaderEntries tmp_headers; + size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); + + if (count == 0 || count > 6) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to 6 arguments in table function, got {}", count); + + auto structure_literal = std::make_shared(structure_); + + /// s3(s3_url) + if (count == 1) + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3(s3_url, format) or s3(s3_url, NOSIGN) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. + else if (count == 2) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + /// If there is NOSIGN, add format=auto before structure. + if (boost::iequals(second_arg, "NOSIGN")) + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// s3(source, format, structure) or + /// s3(source, access_key_id, secret_access_key) or + /// s3(source, NOSIGN, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. + else if (count == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args.push_back(structure_literal); + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + { + args[count - 1] = structure_literal; + } + else + { + /// Add format=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + } + /// s3(source, format, structure, compression_method) or + /// s3(source, access_key_id, secret_access_key, format) or + /// s3(source, NOSIGN, format, structure) + /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. + else if (count == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(second_arg, "NOSIGN")) + { + args[count - 1] = structure_literal; + } + else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + { + args[count - 2] = structure_literal; + } + else + { + args.push_back(structure_literal); + } + } + /// s3(source, access_key_id, secret_access_key, format, structure) or + /// s3(source, NOSIGN, format, structure, compression_method) + /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. + else if (count == 5) + { + auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); + if (boost::iequals(sedond_arg, "NOSIGN")) + { + args[count - 2] = structure_literal; + } + else + { + args[count - 1] = structure_literal; + } + } + /// s3(source, access_key_id, secret_access_key, format, structure, compression) + else if (count == 6) + { + args[count - 2] = structure_literal; + } + } +} + +} diff --git a/src/Storages/ObjectStorage/S3Configuration.h b/src/Storages/ObjectStorage/S3Configuration.h new file mode 100644 index 00000000000..34f5735e02a --- /dev/null +++ b/src/Storages/ObjectStorage/S3Configuration.h @@ -0,0 +1,46 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class StorageS3Configuration : public StorageObjectStorageConfiguration +{ +public: + Path getPath() const override { return url.key; } + void setPath(const Path & path) override { url.key = path; } + + const Paths & getPaths() const override { return keys; } + Paths & getPaths() override { return keys; } + + String getNamespace() const override { return url.bucket; } + String getDataSourceDescription() override; + + void check(ContextPtr context) const override; + StorageObjectStorageConfigurationPtr clone() override; + + ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + static void addStructureToArgs(ASTs & args, const String & structure, ContextPtr context); + +private: + S3::URI url; + S3::AuthSettings auth_settings; + S3Settings::RequestSettings request_settings; + /// If s3 configuration was passed from ast, then it is static. + /// If from config - it can be changed with config reload. + bool static_configuration = true; + /// Headers from ast is a part of static configuration. + HTTPHeaderEntries headers_from_ast; + std::vector keys; + + std::unique_ptr createClient(ContextPtr context); + + bool initialized = false; +}; + +} diff --git a/src/Storages/ObjectStorage/Settings.h b/src/Storages/ObjectStorage/Settings.h new file mode 100644 index 00000000000..015cf9bc01d --- /dev/null +++ b/src/Storages/ObjectStorage/Settings.h @@ -0,0 +1,86 @@ +#pragma once +#include +#include +#include + +namespace CurrentMetrics +{ + extern const Metric ObjectStorageAzureThreads; + extern const Metric ObjectStorageAzureThreadsActive; + extern const Metric ObjectStorageAzureThreadsScheduled; + + extern const Metric ObjectStorageS3Threads; + extern const Metric ObjectStorageS3ThreadsActive; + extern const Metric ObjectStorageS3ThreadsScheduled; +} + +namespace DB +{ + +struct StorageObjectStorageSettings +{ + bool truncate_on_insert; + bool create_new_file_on_insert; + bool schema_inference_use_cache; + SchemaInferenceMode schema_inference_mode; +}; + +struct S3StorageSettings +{ + static StorageObjectStorageSettings create(const Settings & settings) + { + return StorageObjectStorageSettings{ + .truncate_on_insert = settings.s3_truncate_on_insert, + .create_new_file_on_insert = settings.s3_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_s3, + .schema_inference_mode = settings.schema_inference_mode, + }; + } + + static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_s3"; + + static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageS3Threads; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageS3ThreadsActive; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageS3ThreadsScheduled; } /// NOLINT +}; + +struct AzureStorageSettings +{ + static StorageObjectStorageSettings create(const Settings & settings) + { + return StorageObjectStorageSettings{ + .truncate_on_insert = settings.azure_truncate_on_insert, + .create_new_file_on_insert = settings.azure_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, + .schema_inference_mode = settings.schema_inference_mode, + }; + } + + static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_azure"; + + static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageAzureThreads; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageAzureThreadsActive; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageAzureThreadsScheduled; } /// NOLINT +}; + +struct HDFSStorageSettings +{ + static StorageObjectStorageSettings create(const Settings & settings) + { + return StorageObjectStorageSettings{ + .truncate_on_insert = settings.hdfs_truncate_on_insert, + .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, + .schema_inference_mode = settings.schema_inference_mode, + }; + } + + static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_hdfs"; + + /// TODO: s3 -> hdfs + static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageS3Threads; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageS3ThreadsActive; } /// NOLINT + static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageS3ThreadsScheduled; } /// NOLINT +}; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp new file mode 100644 index 00000000000..9250ab8ecbe --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -0,0 +1,303 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int DATABASE_ACCESS_DENIED; + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int LOGICAL_ERROR; + extern const int NOT_IMPLEMENTED; + +} + +template +std::unique_ptr getStorageMetadata( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfigurationPtr & configuration, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + std::optional format_settings, + const String & comment, + const std::string & engine_name, + const ContextPtr & context) +{ + auto storage_metadata = std::make_unique(); + if (columns.empty()) + { + auto fetched_columns = StorageObjectStorage::getTableStructureFromData( + object_storage, configuration, format_settings, context); + storage_metadata->setColumns(fetched_columns); + } + else + { + /// We don't allow special columns. + if (!columns.hasOnlyOrdinary()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Table engine {} doesn't support special columns " + "like MATERIALIZED, ALIAS or EPHEMERAL", + engine_name); + + storage_metadata->setColumns(columns); + } + + storage_metadata->setConstraints(constraints); + storage_metadata->setComment(comment); + return storage_metadata; +} + +template +StorageObjectStorage::StorageObjectStorage( + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const String & engine_name_, + ContextPtr context, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + bool distributed_processing_, + ASTPtr partition_by_) + : IStorage(table_id_, getStorageMetadata( + object_storage_, configuration_, columns_, constraints_, format_settings_, + comment, engine_name, context)) + , engine_name(engine_name_) + , virtual_columns(VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage( + getInMemoryMetadataPtr()->getSampleBlock().getNamesAndTypesList())) + , format_settings(format_settings_) + , partition_by(partition_by_) + , distributed_processing(distributed_processing_) + , object_storage(object_storage_) + , configuration(configuration_) +{ + FormatFactory::instance().checkFormatName(configuration->format); + configuration->check(context); + + StoredObjects objects; + for (const auto & key : configuration->getPaths()) + objects.emplace_back(key); +} + +template +Names StorageObjectStorage::getVirtualColumnNames() +{ + return VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage({}).getNames(); +} + +template +bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) const +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); +} + +template +bool StorageObjectStorage::prefersLargeBlocks() const +{ + return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration->format); +} + +template +bool StorageObjectStorage::parallelizeOutputAfterReading(ContextPtr context) const +{ + return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration->format, context); +} + +template +std::pair +StorageObjectStorage::updateConfigurationAndGetCopy(ContextPtr local_context) +{ + std::lock_guard lock(configuration_update_mutex); + auto new_object_storage = configuration->createOrUpdateObjectStorage(local_context); + if (new_object_storage) + object_storage = new_object_storage; + return {configuration, object_storage}; +} + +template +SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) +{ + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + StorageSettings::SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING, + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; +} + +template +void StorageObjectStorage::read( + QueryPlan & query_plan, + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + if (partition_by && configuration->withWildcard()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Reading from a partitioned {} storage is not implemented yet", + getName()); + } + + auto this_ptr = std::static_pointer_cast(shared_from_this()); + auto read_from_format_info = prepareReadingFromFormat( + column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); + bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + && local_context->getSettingsRef().optimize_count_from_files; + + auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); + auto reading = std::make_unique>( + query_object_storage, + query_configuration, + getName(), + virtual_columns, + format_settings, + distributed_processing, + std::move(read_from_format_info), + need_only_count, + local_context, + max_block_size, + num_streams); + + query_plan.addStep(std::move(reading)); +} + +template +SinkToStoragePtr StorageObjectStorage::write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + bool /* async_insert */) +{ + auto insert_query = std::dynamic_pointer_cast(query); + auto partition_by_ast = insert_query + ? (insert_query->partition_by ? insert_query->partition_by : partition_by) + : nullptr; + bool is_partitioned_implementation = partition_by_ast && configuration->withWildcard(); + + auto sample_block = metadata_snapshot->getSampleBlock(); + auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); + + if (is_partitioned_implementation) + { + return std::make_shared( + object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); + } + + if (configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs()) + { + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "{} key '{}' contains globs, so the table is in readonly mode", + getName(), configuration->getPath()); + } + + if (!storage_settings.truncate_on_insert + && object_storage->exists(StoredObject(configuration->getPath()))) + { + if (storage_settings.create_new_file_on_insert) + { + size_t index = configuration->getPaths().size(); + const auto & first_key = configuration->getPaths()[0]; + auto pos = first_key.find_first_of('.'); + String new_key; + + do + { + new_key = first_key.substr(0, pos) + + "." + + std::to_string(index) + + (pos == std::string::npos ? "" : first_key.substr(pos)); + ++index; + } + while (object_storage->exists(StoredObject(new_key))); + + configuration->getPaths().push_back(new_key); + } + else + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. " + "If you want to overwrite it, enable setting [engine_name]_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting [engine_name]_create_new_file_on_insert", + configuration->getNamespace(), configuration->getPaths().back()); + } + } + + return std::make_shared( + object_storage, configuration, format_settings, sample_block, local_context); +} + +template +void StorageObjectStorage::truncate( + const ASTPtr &, + const StorageMetadataPtr &, + ContextPtr, + TableExclusiveLockHolder &) +{ + if (configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs()) + { + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "{} key '{}' contains globs, so the table is in readonly mode and cannot be truncated", + getName(), configuration->getPath()); + } + + StoredObjects objects; + for (const auto & key : configuration->getPaths()) + objects.emplace_back(key); + + object_storage->removeObjectsIfExist(objects); +} + +template +ColumnsDescription StorageObjectStorage::getTableStructureFromData( + ObjectStoragePtr object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + ContextPtr context) +{ + using Source = StorageObjectStorageSource; + + ObjectInfos read_keys; + auto file_iterator = Source::createFileIterator( + configuration, object_storage, /* distributed_processing */false, + context, /* predicate */{}, /* virtual_columns */{}, &read_keys); + + ReadBufferIterator read_buffer_iterator( + object_storage, configuration, file_iterator, + format_settings, read_keys, context); + + const bool retry = configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs(); + return readSchemaFromFormat( + configuration->format, format_settings, + read_buffer_iterator, retry, context); +} + +template class StorageObjectStorage; +template class StorageObjectStorage; +template class StorageObjectStorage; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h new file mode 100644 index 00000000000..0b29845ba5c --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct SelectQueryInfo; +class StorageObjectStorageConfiguration; +struct S3StorageSettings; +struct HDFSStorageSettings; +struct AzureStorageSettings; +class PullingPipelineExecutor; +using ReadTaskCallback = std::function; +class IOutputFormat; +class IInputFormat; +class SchemaCache; + + +template +class StorageObjectStorage : public IStorage +{ +public: + using Configuration = StorageObjectStorageConfiguration; + using ConfigurationPtr = std::shared_ptr; + using ObjectInfo = RelativePathWithMetadata; + using ObjectInfoPtr = std::shared_ptr; + using ObjectInfos = std::vector; + + StorageObjectStorage( + ConfigurationPtr configuration_, + ObjectStoragePtr object_storage_, + const String & engine_name_, + ContextPtr context_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, + std::optional format_settings_, + bool distributed_processing_ = false, + ASTPtr partition_by_ = nullptr); + + String getName() const override { return engine_name; } + + void read( + QueryPlan & query_plan, + const Names &, + const StorageSnapshotPtr &, + SelectQueryInfo &, + ContextPtr, + QueryProcessingStage::Enum, + size_t, + size_t) override; + + SinkToStoragePtr write( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + bool async_insert) override; + + void truncate( + const ASTPtr & query, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr local_context, + TableExclusiveLockHolder &) override; + + NamesAndTypesList getVirtuals() const override { return virtual_columns; } + + static Names getVirtualColumnNames(); + + bool supportsPartitionBy() const override { return true; } + + bool supportsSubcolumns() const override { return true; } + + bool supportsTrivialCountOptimization() const override { return true; } + + bool supportsSubsetOfColumns(const ContextPtr & context) const; + + bool prefersLargeBlocks() const override; + + bool parallelizeOutputAfterReading(ContextPtr context) const override; + + static SchemaCache & getSchemaCache(const ContextPtr & context); + + static ColumnsDescription getTableStructureFromData( + ObjectStoragePtr object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + ContextPtr context); + +protected: + virtual std::pair + updateConfigurationAndGetCopy(ContextPtr local_context); + + const std::string engine_name; + const NamesAndTypesList virtual_columns; + std::optional format_settings; + const ASTPtr partition_by; + const bool distributed_processing; + + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::mutex configuration_update_mutex; +}; + +using StorageS3 = StorageObjectStorage; +using StorageAzureBlobStorage = StorageObjectStorage; +using StorageHDFS = StorageObjectStorage; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp new file mode 100644 index 00000000000..414932016f4 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -0,0 +1,107 @@ +#include "Storages/ObjectStorage/StorageObjectStorageCluster.h" + +#include "config.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +template +StorageObjectStorageCluster::StorageObjectStorageCluster( + const String & cluster_name_, + const Storage::ConfigurationPtr & configuration_, + ObjectStoragePtr object_storage_, + const String & engine_name_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + bool structure_argument_was_provided_) + : IStorageCluster(cluster_name_, + table_id_, + getLogger(fmt::format("{}({})", engine_name_, table_id_.table_name)), + structure_argument_was_provided_) + , engine_name(engine_name_) + , configuration{configuration_} + , object_storage(object_storage_) +{ + configuration->check(context_); + StorageInMemoryMetadata storage_metadata; + + if (columns_.empty()) + { + /// `format_settings` is set to std::nullopt, because StorageObjectStorageCluster is used only as table function + auto columns = StorageObjectStorage::getTableStructureFromData( + object_storage, configuration, /*format_settings=*/std::nullopt, context_); + storage_metadata.setColumns(columns); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); + + virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage( + storage_metadata.getSampleBlock().getNamesAndTypesList()); +} + +template +void StorageObjectStorageCluster::addColumnsStructureToQuery( + ASTPtr & query, + const String & structure, + const ContextPtr & context) +{ + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected SELECT query from table function {}, got '{}'", + engine_name, queryToString(query)); + } + using TableFunction = TableFunctionObjectStorageCluster; + TableFunction::addColumnsStructureToArguments(expression_list->children, structure, context); +} + +template +RemoteQueryExecutor::Extension +StorageObjectStorageCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr &) const +{ + auto iterator = std::make_shared( + object_storage, configuration, predicate, virtual_columns, nullptr); + + auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next(0)->relative_path; }); + return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; +} + + +#if USE_AWS_S3 +template class StorageObjectStorageCluster; +#endif + +#if USE_AZURE_BLOB_STORAGE +template class StorageObjectStorageCluster; +#endif + +#if USE_HDFS +template class StorageObjectStorageCluster; +#endif + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h new file mode 100644 index 00000000000..b1f9af14e03 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -0,0 +1,72 @@ +#pragma once + +#include "config.h" + +#include +#include +#include +#include +#include + +namespace DB +{ + +class StorageS3Settings; +class StorageAzureBlobSettings; + +class Context; + +template +class StorageObjectStorageCluster : public IStorageCluster +{ +public: + using Storage = StorageObjectStorage; + using Source = StorageObjectStorageSource; + + StorageObjectStorageCluster( + const String & cluster_name_, + const Storage::ConfigurationPtr & configuration_, + ObjectStoragePtr object_storage_, + const String & engine_name_, + const StorageID & table_id_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + ContextPtr context_, + bool structure_argument_was_provided_); + + std::string getName() const override { return engine_name; } + + NamesAndTypesList getVirtuals() const override { return virtual_columns; } + + RemoteQueryExecutor::Extension + getTaskIteratorExtension( + const ActionsDAG::Node * predicate, + const ContextPtr & context) const override; + + bool supportsSubcolumns() const override { return true; } + + bool supportsTrivialCountOptimization() const override { return true; } + +private: + void updateBeforeRead(const ContextPtr & /* context */) override {} + + void addColumnsStructureToQuery( + ASTPtr & query, + const String & structure, + const ContextPtr & context) override; + + const String & engine_name; + const Storage::ConfigurationPtr configuration; + const ObjectStoragePtr object_storage; + NamesAndTypesList virtual_columns; +}; + + +#if USE_AWS_S3 +using StorageS3Cluster = StorageObjectStorageCluster; +#endif +#if USE_AZURE_BLOB_STORAGE +using StorageAzureBlobCluster = StorageObjectStorageCluster; +#endif + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h new file mode 100644 index 00000000000..34ab8ebec66 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -0,0 +1,155 @@ +#pragma once +#include +#include +#include +#include +#include + +namespace DB +{ +class StorageObjectStorageSink : public SinkToStorage +{ +public: + StorageObjectStorageSink( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context, + const std::string & blob_path = "") + : SinkToStorage(sample_block_) + , sample_block(sample_block_) + , format_settings(format_settings_) + { + const auto & settings = context->getSettingsRef(); + const auto path = blob_path.empty() ? configuration->getPaths().back() : blob_path; + const auto chosen_compression_method = chooseCompressionMethod(path, configuration->compression_method); + + auto buffer = object_storage->writeObject( + StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + write_buf = wrapWriteBufferWithCompressionMethod( + std::move(buffer), + chosen_compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); + + writer = FormatFactory::instance().getOutputFormatParallelIfPossible( + configuration->format, *write_buf, sample_block, context, format_settings); + } + + String getName() const override { return "StorageObjectStorageSink"; } + + void consume(Chunk chunk) override + { + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); + } + + void onCancel() override + { + std::lock_guard lock(cancel_mutex); + finalize(); + cancelled = true; + } + + void onException(std::exception_ptr exception) override + { + std::lock_guard lock(cancel_mutex); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization. + release(); + } + } + + void onFinish() override + { + std::lock_guard lock(cancel_mutex); + finalize(); + } + +private: + const Block sample_block; + const std::optional format_settings; + + std::unique_ptr write_buf; + OutputFormatPtr writer; + bool cancelled = false; + std::mutex cancel_mutex; + + void finalize() + { + if (!writer) + return; + + try + { + writer->finalize(); + writer->flush(); + write_buf->finalize(); + } + catch (...) + { + /// Stop ParallelFormattingOutputFormat correctly. + release(); + throw; + } + } + + void release() + { + writer.reset(); + write_buf->finalize(); + } +}; + +class PartitionedStorageObjectStorageSink : public PartitionedSink +{ +public: + PartitionedStorageObjectStorageSink( + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context_, + const ASTPtr & partition_by) + : PartitionedSink(partition_by, context_, sample_block_) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + auto blob = configuration->getPaths().back(); + auto partition_key = replaceWildcards(blob, partition_id); + validatePartitionKey(partition_key, true); + return std::make_shared( + object_storage, + configuration, + format_settings, + sample_block, + context, + partition_key + ); + } + +private: + ObjectStoragePtr object_storage; + StorageObjectStorageConfigurationPtr configuration; + const std::optional format_settings; + const Block sample_block; + const ContextPtr context; +}; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp new file mode 100644 index 00000000000..9fc7925a6d1 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -0,0 +1,464 @@ +#include "StorageObjectStorageSource.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace ProfileEvents +{ + extern const Event EngineFileLikeReadFiles; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_COMPILE_REGEXP; +} + +template +std::shared_ptr::IIterator> +StorageObjectStorageSource::createFileIterator( + Storage::ConfigurationPtr configuration, + ObjectStoragePtr object_storage, + bool distributed_processing, + const ContextPtr & local_context, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ObjectInfos * read_keys, + std::function file_progress_callback) +{ + if (distributed_processing) + return std::make_shared(local_context->getReadTaskCallback()); + + if (configuration->isNamespaceWithGlobs()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); + + if (configuration->isPathWithGlobs()) + { + /// Iterate through disclosed globs and make a source for each file + return std::make_shared( + object_storage, configuration, predicate, virtual_columns, read_keys, file_progress_callback); + } + else + { + return std::make_shared( + object_storage, configuration, virtual_columns, read_keys, file_progress_callback); + } +} + +template +StorageObjectStorageSource::GlobIterator::GlobIterator( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + std::function file_progress_callback_) + : object_storage(object_storage_) + , configuration(configuration_) + , virtual_columns(virtual_columns_) + , read_keys(read_keys_) + , file_progress_callback(file_progress_callback_) +{ + if (configuration->isNamespaceWithGlobs()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); + } + else if (configuration->isPathWithGlobs()) + { + const auto key_with_globs = configuration_->getPath(); + const auto key_prefix = configuration->getPathWithoutGlob(); + object_storage_iterator = object_storage->iterate(key_prefix); + + matcher = std::make_unique(makeRegexpPatternFromGlobs(key_with_globs)); + if (matcher->ok()) + { + recursive = key_with_globs == "/**"; + filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + } + else + { + throw Exception( + ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", key_with_globs, matcher->error()); + } + } + else + { + const auto key_with_globs = configuration_->getPath(); + auto object_metadata = object_storage->getObjectMetadata(key_with_globs); + auto object_info = std::make_shared(key_with_globs, object_metadata); + + object_infos.emplace_back(object_info); + if (read_keys) + read_keys->emplace_back(object_info); + + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); + + is_finished = true; + } +} + +template +StorageObjectStorageSource::ObjectInfoPtr +StorageObjectStorageSource::GlobIterator::next(size_t /* processor */) +{ + std::lock_guard lock(next_mutex); + + if (is_finished && index >= object_infos.size()) + return {}; + + bool need_new_batch = object_infos.empty() || index >= object_infos.size(); + + if (need_new_batch) + { + ObjectInfos new_batch; + while (new_batch.empty()) + { + auto result = object_storage_iterator->getCurrentBatchAndScheduleNext(); + if (result.has_value()) + { + new_batch = result.value(); + } + else + { + is_finished = true; + return {}; + } + + for (auto it = new_batch.begin(); it != new_batch.end();) + { + if (!recursive && !re2::RE2::FullMatch((*it)->relative_path, *matcher)) + it = new_batch.erase(it); + else + ++it; + } + } + + index = 0; + + if (filter_dag) + { + std::vector paths; + paths.reserve(new_batch.size()); + for (auto & object_info : new_batch) + paths.push_back(fs::path(configuration->getNamespace()) / object_info->relative_path); + + VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); + } + + if (read_keys) + read_keys->insert(read_keys->end(), new_batch.begin(), new_batch.end()); + + object_infos = std::move(new_batch); + if (file_progress_callback) + { + for (const auto & object_info : object_infos) + { + file_progress_callback(FileProgress(0, object_info->metadata.size_bytes)); + } + } + } + + size_t current_index = index++; + if (current_index >= object_infos.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); + + return object_infos[current_index]; +} + +template +StorageObjectStorageSource::KeysIterator::KeysIterator( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + std::function file_progress_callback_) + : object_storage(object_storage_) + , configuration(configuration_) + , virtual_columns(virtual_columns_) + , file_progress_callback(file_progress_callback_) + , keys(configuration->getPaths()) +{ + if (read_keys_) + { + /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? + for (auto && key : keys) + { + auto object_info = std::make_shared(key, ObjectMetadata{}); + read_keys_->emplace_back(object_info); + } + } +} + +template +StorageObjectStorageSource::ObjectInfoPtr +StorageObjectStorageSource::KeysIterator::next(size_t /* processor */) +{ + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= keys.size()) + return {}; + + auto key = keys[current_index]; + + ObjectMetadata metadata{}; + if (file_progress_callback) + { + metadata = object_storage->getObjectMetadata(key); + file_progress_callback(FileProgress(0, metadata.size_bytes)); + } + + return std::make_shared(key, metadata); +} + +template +Chunk StorageObjectStorageSource::generate() +{ + while (true) + { + if (isCancelled() || !reader) + { + if (reader) + reader->cancel(); + break; + } + + Chunk chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + total_rows_in_file += num_rows; + size_t chunk_size = 0; + if (const auto * input_format = reader.getInputFormat()) + chunk_size = input_format->getApproxBytesReadForChunk(); + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); + + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, + read_from_format_info.requested_virtual_columns, + fs::path(configuration->getNamespace()) / reader.getRelativePath(), + reader.getObjectInfo().metadata.size_bytes); + + return chunk; + } + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) + addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); + + total_rows_in_file = 0; + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool.wait(); + reader_future = createReaderAsync(); + } + + return {}; +} + +template +void StorageObjectStorageSource::addNumRowsToCache(const String & path, size_t num_rows) +{ + String source = fs::path(configuration->getDataSourceDescription()) / path; + auto cache_key = getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); + Storage::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); +} + +template +std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfoPtr & object_info) +{ + String source = fs::path(configuration->getDataSourceDescription()) / object_info->relative_path; + auto cache_key = getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); + auto get_last_mod_time = [&]() -> std::optional + { + auto last_mod = object_info->metadata.last_modified; + if (last_mod) + return last_mod->epochTime(); + else + { + object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + return object_info->metadata.last_modified->epochMicroseconds(); + } + }; + return Storage::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); +} + +template +StorageObjectStorageSource::StorageObjectStorageSource( + String name_, + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const ReadFromFormatInfo & info, + std::optional format_settings_, + ContextPtr context_, + UInt64 max_block_size_, + std::shared_ptr file_iterator_, + bool need_only_count_) + :ISource(info.source_header, false) + , WithContext(context_) + , name(std::move(name_)) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , max_block_size(max_block_size_) + , need_only_count(need_only_count_) + , read_from_format_info(info) + , columns_desc(info.columns_description) + , file_iterator(file_iterator_) + , create_reader_pool(StorageSettings::ObjectStorageThreads(), + StorageSettings::ObjectStorageThreadsActive(), + StorageSettings::ObjectStorageThreadsScheduled(), 1) + , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "Reader")) +{ + reader = createReader(); + if (reader) + reader_future = createReaderAsync(); +} + +template +StorageObjectStorageSource::~StorageObjectStorageSource() +{ + create_reader_pool.wait(); +} + +template +StorageObjectStorageSource::ReaderHolder +StorageObjectStorageSource::createReader(size_t processor) +{ + auto object_info = file_iterator->next(processor); + if (object_info->relative_path.empty()) + return {}; + + if (object_info->metadata.size_bytes == 0) + object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + + QueryPipelineBuilder builder; + std::shared_ptr source; + std::unique_ptr read_buf; + std::optional num_rows_from_cache = need_only_count + && getContext()->getSettingsRef().use_cache_for_count_from_files + ? tryGetNumRowsFromCache(object_info) + : std::nullopt; + + if (num_rows_from_cache) + { + /// We should not return single chunk with all number of rows, + /// because there is a chance that this chunk will be materialized later + /// (it can cause memory problems even with default values in columns or when virtual columns are requested). + /// Instead, we use special ConstChunkGenerator that will generate chunks + /// with max_block_size rows until total number of rows is reached. + source = std::make_shared( + read_from_format_info.format_header, *num_rows_from_cache, max_block_size); + builder.init(Pipe(source)); + } + else + { + std::optional max_parsing_threads; + if (need_only_count) + max_parsing_threads = 1; + + auto compression_method = chooseCompressionMethod( + object_info->relative_path, configuration->compression_method); + + read_buf = createReadBuffer(object_info->relative_path, object_info->metadata.size_bytes); + + auto input_format = FormatFactory::instance().getInput( + configuration->format, *read_buf, read_from_format_info.format_header, + getContext(), max_block_size, format_settings, max_parsing_threads, + std::nullopt, /* is_remote_fs */ true, compression_method); + + if (need_only_count) + input_format->needOnlyCount(); + + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { + return std::make_shared(header, columns_desc, *input_format, getContext()); + }); + } + + source = input_format; + } + + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, read_from_format_info.requested_columns); + }); + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); + + return ReaderHolder{object_info, std::move(read_buf), + std::move(source), std::move(pipeline), std::move(current_reader)}; +} + +template +std::future::ReaderHolder> +StorageObjectStorageSource::createReaderAsync(size_t processor) +{ + return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); +} + +template +std::unique_ptr StorageObjectStorageSource::createReadBuffer(const String & key, size_t object_size) +{ + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; + + // auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; + // const bool object_too_small = object_size <= 2 * download_buffer_size; + + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + // if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) + // { + // LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + + // auto async_reader = object_storage->readObjects( + // StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, read_settings); + + // async_reader->setReadUntilEnd(); + // if (read_settings.remote_fs_prefetch) + // async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + // return async_reader; + // } + // else + return object_storage->readObject(StoredObject(key), read_settings); +} + +template class StorageObjectStorageSource; +template class StorageObjectStorageSource; +template class StorageObjectStorageSource; + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h new file mode 100644 index 00000000000..f68a5d47456 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -0,0 +1,217 @@ +#pragma once +#include +#include +#include + + +namespace DB +{ +template +class StorageObjectStorageSource : public ISource, WithContext +{ + friend class StorageS3QueueSource; +public: + using Source = StorageObjectStorageSource; + using Storage = StorageObjectStorage; + using ObjectInfo = Storage::ObjectInfo; + using ObjectInfoPtr = Storage::ObjectInfoPtr; + using ObjectInfos = Storage::ObjectInfos; + + class IIterator : public WithContext + { + public: + virtual ~IIterator() = default; + + virtual size_t estimatedKeysCount() = 0; + virtual ObjectInfoPtr next(size_t processor) = 0; + }; + + class ReadTaskIterator; + class GlobIterator; + class KeysIterator; + + StorageObjectStorageSource( + String name_, + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration, + const ReadFromFormatInfo & info, + std::optional format_settings_, + ContextPtr context_, + UInt64 max_block_size_, + std::shared_ptr file_iterator_, + bool need_only_count_); + + ~StorageObjectStorageSource() override; + + String getName() const override { return name; } + + Chunk generate() override; + + static std::shared_ptr createFileIterator( + Storage::ConfigurationPtr configuration, + ObjectStoragePtr object_storage, + bool distributed_processing, + const ContextPtr & local_context, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns, + ObjectInfos * read_keys, + std::function file_progress_callback = {}); + +protected: + void addNumRowsToCache(const String & path, size_t num_rows); + std::optional tryGetNumRowsFromCache(const ObjectInfoPtr & object_info); + + const String name; + ObjectStoragePtr object_storage; + const Storage::ConfigurationPtr configuration; + const std::optional format_settings; + const UInt64 max_block_size; + const bool need_only_count; + const ReadFromFormatInfo read_from_format_info; + + ColumnsDescription columns_desc; + std::shared_ptr file_iterator; + size_t total_rows_in_file = 0; + + struct ReaderHolder + { + public: + ReaderHolder( + ObjectInfoPtr object_info_, + std::unique_ptr read_buf_, + std::shared_ptr source_, + std::unique_ptr pipeline_, + std::unique_ptr reader_) + : object_info(std::move(object_info_)) + , read_buf(std::move(read_buf_)) + , source(std::move(source_)) + , pipeline(std::move(pipeline_)) + , reader(std::move(reader_)) + { + } + + ReaderHolder() = default; + ReaderHolder(const ReaderHolder & other) = delete; + ReaderHolder & operator=(const ReaderHolder & other) = delete; + ReaderHolder(ReaderHolder && other) noexcept { *this = std::move(other); } + + ReaderHolder & operator=(ReaderHolder && other) noexcept + { + /// The order of destruction is important. + /// reader uses pipeline, pipeline uses read_buf. + reader = std::move(other.reader); + pipeline = std::move(other.pipeline); + source = std::move(other.source); + read_buf = std::move(other.read_buf); + object_info = std::move(other.object_info); + return *this; + } + + explicit operator bool() const { return reader != nullptr; } + PullingPipelineExecutor * operator->() { return reader.get(); } + const PullingPipelineExecutor * operator->() const { return reader.get(); } + const String & getRelativePath() const { return object_info->relative_path; } + const ObjectInfo & getObjectInfo() const { return *object_info; } + const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } + + private: + ObjectInfoPtr object_info; + std::unique_ptr read_buf; + std::shared_ptr source; + std::unique_ptr pipeline; + std::unique_ptr reader; + }; + + ReaderHolder reader; + LoggerPtr log = getLogger("StorageObjectStorageSource"); + ThreadPool create_reader_pool; + ThreadPoolCallbackRunner create_reader_scheduler; + std::future reader_future; + + /// Recreate ReadBuffer and Pipeline for each file. + ReaderHolder createReader(size_t processor = 0); + std::future createReaderAsync(size_t processor = 0); + + std::unique_ptr createReadBuffer(const String & key, size_t object_size); +}; + +template +class StorageObjectStorageSource::ReadTaskIterator : public IIterator +{ +public: + explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {} + + size_t estimatedKeysCount() override { return 0; } /// TODO FIXME + + ObjectInfoPtr next(size_t) override { return std::make_shared( callback(), ObjectMetadata{} ); } + +private: + ReadTaskCallback callback; +}; + +template +class StorageObjectStorageSource::GlobIterator : public IIterator +{ +public: + GlobIterator( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const ActionsDAG::Node * predicate, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + std::function file_progress_callback_ = {}); + + ~GlobIterator() override = default; + + size_t estimatedKeysCount() override { return object_infos.size(); } + + ObjectInfoPtr next(size_t processor) override; + +private: + ObjectStoragePtr object_storage; + Storage::ConfigurationPtr configuration; + ActionsDAGPtr filter_dag; + NamesAndTypesList virtual_columns; + + size_t index = 0; + + ObjectInfos object_infos; + ObjectInfos * read_keys; + ObjectStorageIteratorPtr object_storage_iterator; + bool recursive{false}; + + std::unique_ptr matcher; + + void createFilterAST(const String & any_key); + bool is_finished = false; + std::mutex next_mutex; + + std::function file_progress_callback; +}; + +template +class StorageObjectStorageSource::KeysIterator : public IIterator +{ +public: + KeysIterator( + ObjectStoragePtr object_storage_, + Storage::ConfigurationPtr configuration_, + const NamesAndTypesList & virtual_columns_, + ObjectInfos * read_keys_, + std::function file_progress_callback = {}); + + ~KeysIterator() override = default; + + size_t estimatedKeysCount() override { return keys.size(); } + + ObjectInfoPtr next(size_t processor) override; + +private: + const ObjectStoragePtr object_storage; + const Storage::ConfigurationPtr configuration; + const NamesAndTypesList virtual_columns; + const std::function file_progress_callback; + const std::vector keys; + std::atomic index = 0; +}; +} diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp new file mode 100644 index 00000000000..bc9f93690f5 --- /dev/null +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -0,0 +1,166 @@ +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static void initializeConfiguration( + StorageObjectStorageConfiguration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); +} + +template +static std::shared_ptr> createStorageObjectStorage( + const StorageFactory::Arguments & args, + typename StorageObjectStorage::ConfigurationPtr configuration, + const String & engine_name, + ContextPtr context) +{ + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + // Use format settings from global server context + settings from + // the SETTINGS clause of the create query. Settings from current + // session and user are ignored. + std::optional format_settings; + if (args.storage_def->settings) + { + FormatFactorySettings user_format_settings; + + // Apply changed settings from global context, but ignore the + // unknown ones, because we only have the format settings here. + const auto & changes = context->getSettingsRef().changes(); + for (const auto & change : changes) + { + if (user_format_settings.has(change.name)) + user_format_settings.set(change.name, change.value); + } + + // Apply changes from SETTINGS clause, with validation. + user_format_settings.applyChanges(args.storage_def->settings->changes); + format_settings = getFormatSettings(context, user_format_settings); + } + else + { + format_settings = getFormatSettings(context); + } + + ASTPtr partition_by; + if (args.storage_def->partition_by) + partition_by = args.storage_def->partition_by->clone(); + + return std::make_shared>( + configuration, + configuration->createOrUpdateObjectStorage(context), + engine_name, + args.getContext(), + args.table_id, + args.columns, + args.constraints, + args.comment, + format_settings, + /* distributed_processing */ false, + partition_by); +} + +#if USE_AZURE_BLOB_STORAGE +void registerStorageAzure(StorageFactory & factory) +{ + factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) + { + auto context = args.getLocalContext(); + auto configuration = std::make_shared(); + initializeConfiguration(*configuration, args.engine_args, context, false); + return createStorageObjectStorage(args, configuration, "Azure", context); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::AZURE, + }); +} +#endif + +#if USE_AWS_S3 +void registerStorageS3Impl(const String & name, StorageFactory & factory) +{ + factory.registerStorage(name, [=](const StorageFactory::Arguments & args) + { + auto context = args.getLocalContext(); + auto configuration = std::make_shared(); + initializeConfiguration(*configuration, args.engine_args, context, false); + return createStorageObjectStorage(args, configuration, name, context); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +void registerStorageS3(StorageFactory & factory) +{ + return registerStorageS3Impl("S3", factory); +} + +void registerStorageCOS(StorageFactory & factory) +{ + return registerStorageS3Impl("COSN", factory); +} + +void registerStorageOSS(StorageFactory & factory) +{ + return registerStorageS3Impl("OSS", factory); +} + +#endif + +#if USE_HDFS +void registerStorageHDFS(StorageFactory & factory) +{ + factory.registerStorage("HDFS", [=](const StorageFactory::Arguments & args) + { + auto context = args.getLocalContext(); + auto configuration = std::make_shared(); + initializeConfiguration(*configuration, args.engine_args, context, false); + return createStorageObjectStorage(args, configuration, "HDFS", context); + }, + { + .supports_settings = true, + .supports_sort_order = true, // for partition by + .supports_schema_inference = true, + .source_access_type = AccessType::HDFS, + }); +} +#endif + +void registerStorageObjectStorage(StorageFactory & factory) +{ +#if USE_AWS_S3 + registerStorageS3(factory); + registerStorageCOS(factory); + registerStorageOSS(factory); +#endif +#if USE_AZURE_BLOB_STORAGE + registerStorageAzure(factory); +#endif +#if USE_HDFS + registerStorageHDFS(factory); +#endif +} + +} diff --git a/src/Storages/ObjectStorageConfiguration.h b/src/Storages/ObjectStorageConfiguration.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index b4f5f957f76..bd34d1ec093 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -5,9 +5,9 @@ #include #include #include -#include #include #include +#include namespace CurrentMetrics @@ -31,11 +31,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -StorageS3QueueSource::S3QueueKeyWithInfo::S3QueueKeyWithInfo( +StorageS3QueueSource::S3QueueObjectInfo::S3QueueObjectInfo( const std::string & key_, - std::optional info_, + const ObjectMetadata & object_metadata_, Metadata::ProcessingNodeHolderPtr processing_holder_) - : StorageS3Source::KeyWithInfo(key_, info_) + : Source::ObjectInfo(key_, object_metadata_) , processing_holder(processing_holder_) { } @@ -55,15 +55,15 @@ StorageS3QueueSource::FileIterator::FileIterator( if (sharded_processing) { for (const auto & id : metadata->getProcessingIdsForShard(current_shard)) - sharded_keys.emplace(id, std::deque{}); + sharded_keys.emplace(id, std::deque{}); } } -StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(size_t idx) +StorageS3QueueSource::Source::ObjectInfoPtr StorageS3QueueSource::FileIterator::next(size_t processor) { while (!shutdown_called) { - KeyWithInfoPtr val{nullptr}; + Source::ObjectInfoPtr val{nullptr}; { std::unique_lock lk(sharded_keys_mutex, std::defer_lock); @@ -73,7 +73,7 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si /// we need to check sharded_keys and to next() under lock. lk.lock(); - if (auto it = sharded_keys.find(idx); it != sharded_keys.end()) + if (auto it = sharded_keys.find(processor); it != sharded_keys.end()) { auto & keys = it->second; if (!keys.empty()) @@ -86,24 +86,24 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si { throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing id {} does not exist (Expected ids: {})", - idx, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); + processor, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); } } if (!val) { - val = glob_iterator->next(); + val = glob_iterator->next(processor); if (val && sharded_processing) { - const auto processing_id_for_key = metadata->getProcessingIdForPath(val->key); - if (idx != processing_id_for_key) + const auto processing_id_for_key = metadata->getProcessingIdForPath(val->relative_path); + if (processor != processing_id_for_key) { if (metadata->isProcessingIdBelongsToShard(processing_id_for_key, current_shard)) { LOG_TEST(log, "Putting key {} into queue of processor {} (total: {})", - val->key, processing_id_for_key, sharded_keys.size()); + val->relative_path, processing_id_for_key, sharded_keys.size()); - if (auto it = sharded_keys.find(idx); it != sharded_keys.end()) + if (auto it = sharded_keys.find(processor); it != sharded_keys.end()) { it->second.push_back(val); } @@ -111,7 +111,7 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si { throw Exception(ErrorCodes::LOGICAL_ERROR, "Processing id {} does not exist (Expected ids: {})", - idx, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); + processor, fmt::join(metadata->getProcessingIdsForShard(current_shard), ", ")); } } continue; @@ -129,25 +129,25 @@ StorageS3QueueSource::KeyWithInfoPtr StorageS3QueueSource::FileIterator::next(si return {}; } - auto processing_holder = metadata->trySetFileAsProcessing(val->key); + auto processing_holder = metadata->trySetFileAsProcessing(val->relative_path); if (shutdown_called) { LOG_TEST(log, "Shutdown was called, stopping file iterator"); return {}; } - LOG_TEST(log, "Checking if can process key {} for processing_id {}", val->key, idx); + LOG_TEST(log, "Checking if can process key {} for processing_id {}", val->relative_path, processor); if (processing_holder) { - return std::make_shared(val->key, val->info, processing_holder); + return std::make_shared(val->relative_path, val->metadata, processing_holder); } else if (sharded_processing - && metadata->getFileStatus(val->key)->state == S3QueueFilesMetadata::FileStatus::State::Processing) + && metadata->getFileStatus(val->relative_path)->state == S3QueueFilesMetadata::FileStatus::State::Processing) { throw Exception(ErrorCodes::LOGICAL_ERROR, "File {} is processing by someone else in sharded processing. " - "It is a bug", val->key); + "It is a bug", val->relative_path); } } return {}; @@ -161,7 +161,7 @@ size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() StorageS3QueueSource::StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -190,38 +190,19 @@ StorageS3QueueSource::StorageS3QueueSource( { } -StorageS3QueueSource::~StorageS3QueueSource() -{ - internal_source->create_reader_pool.wait(); -} - String StorageS3QueueSource::getName() const { return name; } -void StorageS3QueueSource::lazyInitialize() -{ - if (initialized) - return; - - internal_source->lazyInitialize(processing_id); - reader = std::move(internal_source->reader); - if (reader) - reader_future = std::move(internal_source->reader_future); - initialized = true; -} - Chunk StorageS3QueueSource::generate() { - lazyInitialize(); - while (true) { if (!reader) break; - const auto * key_with_info = dynamic_cast(&reader.getKeyWithInfo()); + const auto * key_with_info = dynamic_cast(&reader.getObjectInfo()); auto file_status = key_with_info->processing_holder->getFileStatus(); if (isCancelled()) @@ -239,7 +220,7 @@ Chunk StorageS3QueueSource::generate() tryLogCurrentException(__PRETTY_FUNCTION__); } - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); } break; @@ -254,7 +235,7 @@ Chunk StorageS3QueueSource::generate() { LOG_DEBUG( log, "Table is being dropped, {} rows are already processed from {}, but file is not fully processed", - processed_rows_from_file, reader.getFile()); + processed_rows_from_file, reader.getRelativePath()); try { @@ -265,7 +246,7 @@ Chunk StorageS3QueueSource::generate() tryLogCurrentException(__PRETTY_FUNCTION__); } - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); /// Leave the file half processed. Table is being dropped, so we do not care. break; @@ -273,7 +254,7 @@ Chunk StorageS3QueueSource::generate() LOG_DEBUG(log, "Shutdown called, but file {} is partially processed ({} rows). " "Will process the file fully and then shutdown", - reader.getFile(), processed_rows_from_file); + reader.getRelativePath(), processed_rows_from_file); } auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); @@ -287,30 +268,30 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; if (reader->pull(chunk)) { - LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getPath()); + LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getRelativePath()); file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath(), reader.getKeyWithInfo().info->size); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getRelativePath(), reader.getObjectInfo().metadata.size_bytes); return chunk; } } catch (...) { const auto message = getCurrentExceptionMessage(true); - LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getFile(), message); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getRelativePath(), message); files_metadata->setFileFailed(key_with_info->processing_holder, message); - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, false); + appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(key_with_info->processing_holder); - applyActionAfterProcessing(reader.getFile()); + applyActionAfterProcessing(reader.getRelativePath()); - appendLogElement(reader.getFile(), *file_status, processed_rows_from_file, true); + appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, true); file_status.reset(); processed_rows_from_file = 0; @@ -326,7 +307,7 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - file_status = files_metadata->getFileStatus(reader.getFile()); + file_status = files_metadata->getFileStatus(reader.getRelativePath()); /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 8fc7305ea08..fcf5c5c0160 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -5,7 +5,9 @@ #include #include #include -#include +#include +#include +#include #include @@ -14,28 +16,32 @@ namespace Poco { class Logger; } namespace DB { +struct ObjectMetadata; + class StorageS3QueueSource : public ISource, WithContext { public: - using IIterator = StorageS3Source::IIterator; - using KeyWithInfoPtr = StorageS3Source::KeyWithInfoPtr; - using GlobIterator = StorageS3Source::DisclosedGlobIterator; + using Storage = StorageObjectStorage; + using Source = StorageObjectStorageSource; + + using ConfigurationPtr = Storage::ConfigurationPtr; + using GlobIterator = Source::GlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; using Metadata = S3QueueFilesMetadata; - struct S3QueueKeyWithInfo : public StorageS3Source::KeyWithInfo + struct S3QueueObjectInfo : public Source::ObjectInfo { - S3QueueKeyWithInfo( - const std::string & key_, - std::optional info_, - Metadata::ProcessingNodeHolderPtr processing_holder_); + S3QueueObjectInfo( + const std::string & key_, + const ObjectMetadata & object_metadata_, + Metadata::ProcessingNodeHolderPtr processing_holder_); Metadata::ProcessingNodeHolderPtr processing_holder; }; - class FileIterator : public IIterator + class FileIterator : public Source::IIterator { public: FileIterator( @@ -47,7 +53,7 @@ public: /// Note: /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - KeyWithInfoPtr next(size_t idx) override; + Source::ObjectInfoPtr next(size_t processor) override; size_t estimatedKeysCount() override; @@ -60,14 +66,14 @@ public: const bool sharded_processing; const size_t current_shard; - std::unordered_map> sharded_keys; + std::unordered_map> sharded_keys; std::mutex sharded_keys_mutex; }; StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -80,8 +86,6 @@ public: const StorageID & storage_id_, LoggerPtr log_); - ~StorageS3QueueSource() override; - static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); String getName() const override; @@ -93,7 +97,7 @@ private: const S3QueueAction action; const size_t processing_id; const std::shared_ptr files_metadata; - const std::shared_ptr internal_source; + const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; const std::atomic & shutdown_called; const std::atomic & table_is_being_dropped; @@ -103,13 +107,11 @@ private: RemoveFileFunc remove_file_func; LoggerPtr log; - using ReaderHolder = StorageS3Source::ReaderHolder; - ReaderHolder reader; - std::future reader_future; + Source::ReaderHolder reader; + std::future reader_future; std::atomic initialized{false}; size_t processed_rows_from_file = 0; - void lazyInitialize(); void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); }; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 3ee2594135d..94816619aaa 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB @@ -33,7 +32,7 @@ namespace S3QueueTableMetadata::S3QueueTableMetadata( - const StorageS3::Configuration & configuration, + const StorageObjectStorageConfiguration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata) { diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 30642869930..942ce7973ef 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -3,7 +3,7 @@ #if USE_AWS_S3 #include -#include +#include #include namespace DB @@ -27,7 +27,10 @@ struct S3QueueTableMetadata UInt64 s3queue_processing_threads_num; S3QueueTableMetadata() = default; - S3QueueTableMetadata(const StorageS3::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); + S3QueueTableMetadata( + const StorageObjectStorageConfiguration & configuration, + const S3QueueSettings & engine_settings, + const StorageInMemoryMetadata & storage_metadata); void read(const String & metadata_str); static S3QueueTableMetadata parse(const String & metadata_str); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 0723205b544..fa7132f705a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -50,11 +51,6 @@ namespace ErrorCodes namespace { - bool containsGlobs(const S3::URI & url) - { - return url.key.find_first_of("*?{") != std::string::npos; - } - std::string chooseZooKeeperPath(const StorageID & table_id, const Settings & settings, const S3QueueSettings & s3queue_settings) { std::string zk_path_prefix = settings.s3queue_default_zookeeper_path.value; @@ -98,7 +94,7 @@ namespace StorageS3Queue::StorageS3Queue( std::unique_ptr s3queue_settings_, - const StorageS3::Configuration & configuration_, + const ConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -116,29 +112,29 @@ StorageS3Queue::StorageS3Queue( , reschedule_processing_interval_ms(s3queue_settings->s3queue_polling_min_timeout_ms) , log(getLogger("StorageS3Queue (" + table_id_.table_name + ")")) { - if (configuration.url.key.empty()) + if (configuration->getPath().empty()) { - configuration.url.key = "/*"; + configuration->setPath("/*"); } - else if (configuration.url.key.ends_with('/')) + else if (configuration->getPath().ends_with('/')) { - configuration.url.key += '*'; + configuration->setPath(configuration->getPath() + '*'); } - else if (!containsGlobs(configuration.url)) + else if (!configuration->isPathWithGlobs()) { throw Exception(ErrorCodes::QUERY_NOT_ALLOWED, "S3Queue url must either end with '/' or contain globs"); } checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef()); - configuration.update(context_); - FormatFactory::instance().checkFormatName(configuration.format); - context_->getRemoteHostFilter().checkURL(configuration.url.uri); + object_storage = configuration->createOrUpdateObjectStorage(context_); + FormatFactory::instance().checkFormatName(configuration->format); + configuration->check(context_); StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_); + auto columns = Storage::getTableStructureFromData(object_storage, configuration, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -226,7 +222,7 @@ void StorageS3Queue::drop() bool StorageS3Queue::supportsSubsetOfColumns(const ContextPtr & context_) const { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context_, format_settings); + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context_, format_settings); } class ReadFromS3Queue : public SourceStepWithFilter @@ -345,38 +341,20 @@ std::shared_ptr StorageS3Queue::createSource( size_t max_block_size, ContextPtr local_context) { - auto configuration_snapshot = updateConfigurationAndGetCopy(local_context); - - auto internal_source = std::make_unique( - info, configuration.format, getName(), local_context, format_settings, + auto internal_source = std::make_unique( + getName(), + object_storage, + configuration, + info, + format_settings, + local_context, max_block_size, - configuration_snapshot.request_settings, - configuration_snapshot.compression_method, - configuration_snapshot.client, - configuration_snapshot.url.bucket, - configuration_snapshot.url.version_id, - configuration_snapshot.url.uri.getHost() + std::to_string(configuration_snapshot.url.uri.getPort()), - file_iterator, local_context->getSettingsRef().max_download_threads, false); + file_iterator, + false); - auto file_deleter = [this, bucket = configuration_snapshot.url.bucket, client = configuration_snapshot.client, blob_storage_log = BlobStorageLogWriter::create()](const std::string & path) mutable + auto file_deleter = [=, this](const std::string & path) mutable { - S3::DeleteObjectRequest request; - request.WithKey(path).WithBucket(bucket); - auto outcome = client->DeleteObject(request); - if (blob_storage_log) - blob_storage_log->addEvent( - BlobStorageLogElement::EventType::Delete, - bucket, path, {}, 0, outcome.IsSuccess() ? nullptr : &outcome.GetError()); - - if (!outcome.IsSuccess()) - { - const auto & err = outcome.GetError(); - LOG_ERROR(log, "{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())); - } - else - { - LOG_TRACE(log, "Object with path {} was removed from S3", path); - } + object_storage->removeObject(StoredObject(path)); }; auto s3_queue_log = s3queue_settings->s3queue_enable_logging_to_s3queue_log ? local_context->getS3QueueLog() : nullptr; return std::make_shared( @@ -470,7 +448,6 @@ bool StorageS3Queue::streamToViews() auto s3queue_context = Context::createCopy(getContext()); s3queue_context->makeQueryContext(); - auto query_configuration = updateConfigurationAndGetCopy(s3queue_context); // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns @@ -505,12 +482,6 @@ bool StorageS3Queue::streamToViews() return rows > 0; } -StorageS3Queue::Configuration StorageS3Queue::updateConfigurationAndGetCopy(ContextPtr local_context) -{ - configuration.update(local_context); - return configuration; -} - zkutil::ZooKeeperPtr StorageS3Queue::getZooKeeper() const { return getContext()->getZooKeeper(); @@ -530,7 +501,7 @@ void StorageS3Queue::createOrCheckMetadata(const StorageInMemoryMetadata & stora } else { - std::string metadata = S3QueueTableMetadata(configuration, *s3queue_settings, storage_metadata).toString(); + std::string metadata = S3QueueTableMetadata(*configuration, *s3queue_settings, storage_metadata).toString(); requests.emplace_back(zkutil::makeCreateRequest(zk_path, "", zkutil::CreateMode::Persistent)); requests.emplace_back(zkutil::makeCreateRequest(zk_path / "processed", "", zkutil::CreateMode::Persistent)); requests.emplace_back(zkutil::makeCreateRequest(zk_path / "failed", "", zkutil::CreateMode::Persistent)); @@ -568,7 +539,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const String metadata_str = zookeeper->get(fs::path(zookeeper_prefix) / "metadata"); auto metadata_from_zk = S3QueueTableMetadata::parse(metadata_str); - S3QueueTableMetadata old_metadata(configuration, *s3queue_settings, storage_metadata); + S3QueueTableMetadata old_metadata(*configuration, *s3queue_settings, storage_metadata); old_metadata.checkEquals(metadata_from_zk); auto columns_from_zk = ColumnsDescription::parse(metadata_from_zk.columns); @@ -584,14 +555,25 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const } } -std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) +std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr , const ActionsDAG::Node * predicate) { - auto glob_iterator = std::make_unique( - *configuration.client, configuration.url, predicate, virtual_columns, local_context, - /* read_keys */nullptr, configuration.request_settings); + auto glob_iterator = std::make_unique(object_storage, configuration, predicate, virtual_columns, nullptr); + return std::make_shared(files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called); } +static void initializeConfiguration( + StorageObjectStorageConfiguration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); +} + void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( @@ -602,7 +584,8 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + auto configuration = std::make_shared(); + initializeConfiguration(*configuration, 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/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index fd3b4bb4914..88f9bd65093 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,11 +26,13 @@ class S3QueueFilesMetadata; class StorageS3Queue : public IStorage, WithContext { public: - using Configuration = typename StorageS3::Configuration; + using Storage = StorageObjectStorage; + using Source = StorageObjectStorageSource; + using ConfigurationPtr = Storage::ConfigurationPtr; StorageS3Queue( std::unique_ptr s3queue_settings_, - const Configuration & configuration_, + ConfigurationPtr configuration_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -53,8 +55,6 @@ public: NamesAndTypesList getVirtuals() const override { return virtual_columns; } - const auto & getFormatName() const { return configuration.format; } - const fs::path & getZooKeeperPath() const { return zk_path; } zkutil::ZooKeeperPtr getZooKeeper() const; @@ -68,7 +68,8 @@ private: const S3QueueAction after_processing; std::shared_ptr files_metadata; - Configuration configuration; + ConfigurationPtr configuration; + ObjectStoragePtr object_storage; const std::optional format_settings; NamesAndTypesList virtual_columns; @@ -103,7 +104,6 @@ private: void createOrCheckMetadata(const StorageInMemoryMetadata & storage_metadata); void checkTableStructure(const String & zookeeper_prefix, const StorageInMemoryMetadata & storage_metadata); - Configuration updateConfigurationAndGetCopy(ContextPtr local_context); }; } diff --git a/src/Storages/StorageAzureBlob.cpp b/src/Storages/StorageAzureBlob.cpp deleted file mode 100644 index c09db0bfb7b..00000000000 --- a/src/Storages/StorageAzureBlob.cpp +++ /dev/null @@ -1,1478 +0,0 @@ -#include - -#if USE_AZURE_BLOB_STORAGE -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include - -using namespace Azure::Storage::Blobs; - -namespace CurrentMetrics -{ - extern const Metric ObjectStorageAzureThreads; - extern const Metric ObjectStorageAzureThreadsActive; - extern const Metric ObjectStorageAzureThreadsScheduled; -} - -namespace ProfileEvents -{ - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; - -} - -namespace -{ - -const std::unordered_set required_configuration_keys = { - "blob_path", - "container", -}; - -const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "structure", - "compression_method", - "account_name", - "account_key", - "connection_string", - "storage_account_url", -}; - -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); -} - -} - -void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection) -{ - validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); - - if (collection.has("connection_string")) - { - configuration.connection_url = collection.get("connection_string"); - configuration.is_connection_string = true; - } - - if (collection.has("storage_account_url")) - { - configuration.connection_url = collection.get("storage_account_url"); - configuration.is_connection_string = false; - } - - configuration.container = collection.get("container"); - configuration.blob_path = collection.get("blob_path"); - - if (collection.has("account_name")) - configuration.account_name = collection.get("account_name"); - - if (collection.has("account_key")) - configuration.account_key = collection.get("account_key"); - - configuration.structure = collection.getOrDefault("structure", "auto"); - configuration.format = collection.getOrDefault("format", configuration.format); - configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); -} - - -StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context) -{ - StorageAzureBlob::Configuration configuration; - - /// Supported signatures: - /// - /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression]) - /// - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - processNamedCollectionResult(configuration, *named_collection); - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - - return configuration; - } - - if (engine_args.size() < 3 || engine_args.size() > 7) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage AzureBlobStorage requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression])"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - std::unordered_map engine_args_to_idx; - - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - - auto is_format_arg = [] (const std::string & s) -> bool - { - return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); - }; - - if (engine_args.size() == 4) - { - //'c1 UInt64, c2 UInt64 - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - } - else - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format or account name specified without account key"); - } - } - else if (engine_args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - } - } - else if (engine_args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - 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); - configuration.format = sixth_arg; - } - } - else if (engine_args.size() == 7) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Format and compression must be last arguments"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - 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); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } - } - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - - return configuration; -} - - -AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(ContextPtr local_context) -{ - const auto & context_settings = local_context->getSettingsRef(); - auto settings_ptr = std::make_unique(); - settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; - settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; - settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); - - return settings_ptr; -} - -void registerStorageAzureBlob(StorageFactory & factory) -{ - factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageAzureBlob::getConfiguration(engine_args, args.getLocalContext()); - auto client = StorageAzureBlob::createClient(configuration, /* is_read_only */ false); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - std::optional format_settings; - if (args.storage_def->settings) - { - FormatFactorySettings user_format_settings; - - // Apply changed settings from global context, but ignore the - // unknown ones, because we only have the format settings here. - const auto & changes = args.getContext()->getSettingsRef().changes(); - for (const auto & change : changes) - { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); - } - - // Apply changes from SETTINGS clause, with validation. - user_format_settings.applyChanges(args.storage_def->settings->changes); - format_settings = getFormatSettings(args.getContext(), user_format_settings); - } - else - { - format_settings = getFormatSettings(args.getContext()); - } - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - auto settings = StorageAzureBlob::createSettings(args.getContext()); - - return std::make_shared( - std::move(configuration), - std::make_unique("AzureBlobStorage", std::move(client), std::move(settings),configuration.container), - args.getContext(), - args.table_id, - args.columns, - args.constraints, - args.comment, - format_settings, - /* distributed_processing */ false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::AZURE, - }); -} - -static bool containerExists(std::unique_ptr &blob_service_client, std::string container_name) -{ - Azure::Storage::Blobs::ListBlobContainersOptions options; - options.Prefix = container_name; - options.PageSizeHint = 1; - - auto containers_list_response = blob_service_client->ListBlobContainers(options); - auto containers_list = containers_list_response.BlobContainers; - - for (const auto & container : containers_list) - { - if (container_name == container.Name) - return true; - } - return false; -} - -AzureClientPtr StorageAzureBlob::createClient(StorageAzureBlob::Configuration configuration, bool is_read_only) -{ - AzureClientPtr result; - - if (configuration.is_connection_string) - { - std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(configuration.connection_url)); - result = std::make_unique(BlobContainerClient::CreateFromConnectionString(configuration.connection_url, configuration.container)); - bool container_exists = containerExists(blob_service_client,configuration.container); - - if (!container_exists) - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - - try - { - result->CreateIfNotExists(); - } catch (const Azure::Storage::StorageException & e) - { - if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.")) - { - throw; - } - } - } - } - else - { - std::shared_ptr storage_shared_key_credential; - if (configuration.account_name.has_value() && configuration.account_key.has_value()) - { - storage_shared_key_credential - = std::make_shared(*configuration.account_name, *configuration.account_key); - } - - std::unique_ptr blob_service_client; - if (storage_shared_key_credential) - { - blob_service_client = std::make_unique(configuration.connection_url, storage_shared_key_credential); - } - else - { - blob_service_client = std::make_unique(configuration.connection_url); - } - - bool container_exists = containerExists(blob_service_client,configuration.container); - - std::string final_url; - size_t pos = configuration.connection_url.find('?'); - if (pos != std::string::npos) - { - auto url_without_sas = configuration.connection_url.substr(0, pos); - final_url = url_without_sas + (url_without_sas.back() == '/' ? "" : "/") + configuration.container - + configuration.connection_url.substr(pos); - } - else - final_url - = configuration.connection_url + (configuration.connection_url.back() == '/' ? "" : "/") + configuration.container; - - if (container_exists) - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - result = std::make_unique(final_url); - } - else - { - if (is_read_only) - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage container does not exist '{}'", - configuration.container); - try - { - result = std::make_unique(blob_service_client->CreateBlobContainer(configuration.container).Value); - } catch (const Azure::Storage::StorageException & e) - { - if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.") - { - if (storage_shared_key_credential) - result = std::make_unique(final_url, storage_shared_key_credential); - else - result = std::make_unique(final_url); - } - else - { - throw; - } - } - } - } - - return result; -} - -Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const -{ - if (!is_connection_string) - return Poco::URI(connection_url); - - auto parsed_connection_string = Azure::Storage::_internal::ParseConnectionString(connection_url); - return Poco::URI(parsed_connection_string.BlobServiceUrl.GetAbsoluteUrl()); -} - - -StorageAzureBlob::StorageAzureBlob( - const Configuration & configuration_, - std::unique_ptr && object_storage_, - ContextPtr context, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , name("AzureBlobStorage") - , configuration(configuration_) - , object_storage(std::move(object_storage_)) - , distributed_processing(distributed_processing_) - , format_settings(format_settings_) - , partition_by(partition_by_) -{ - FormatFactory::instance().checkFormatName(configuration.format); - context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.getConnectionURL()); - - StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - auto columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context, distributed_processing); - storage_metadata.setColumns(columns); - } - else - { - /// We don't allow special columns in File storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine AzureBlobStorage doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - - StoredObjects objects; - for (const auto & key : configuration.blobs_paths) - objects.emplace_back(key); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -void StorageAzureBlob::truncate(const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) -{ - if (configuration.withGlobs()) - { - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", - configuration.blob_path); - } - - StoredObjects objects; - for (const auto & key : configuration.blobs_paths) - objects.emplace_back(key); - - object_storage->removeObjectsIfExist(objects); -} - -namespace -{ - -class StorageAzureBlobSink : public SinkToStorage -{ -public: - StorageAzureBlobSink( - const String & format, - const Block & sample_block_, - ContextPtr context, - std::optional format_settings_, - const CompressionMethod compression_method, - AzureObjectStorage * object_storage, - const String & blob_path) - : SinkToStorage(sample_block_) - , sample_block(sample_block_) - , format_settings(format_settings_) - { - StoredObject object(blob_path); - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - object_storage->writeObject(object, WriteMode::Rewrite), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); - } - - String getName() const override { return "StorageAzureBlobSink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } - - Block sample_block; - std::optional format_settings; - std::unique_ptr write_buf; - OutputFormatPtr writer; - bool cancelled = false; - std::mutex cancel_mutex; -}; - -class PartitionedStorageAzureBlobSink : public PartitionedSink -{ -public: - PartitionedStorageAzureBlobSink( - const ASTPtr & partition_by, - const String & format_, - const Block & sample_block_, - ContextPtr context_, - std::optional format_settings_, - const CompressionMethod compression_method_, - AzureObjectStorage * object_storage_, - const String & blob_) - : PartitionedSink(partition_by, context_, sample_block_) - , format(format_) - , sample_block(sample_block_) - , context(context_) - , compression_method(compression_method_) - , object_storage(object_storage_) - , blob(blob_) - , format_settings(format_settings_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto partition_key = replaceWildcards(blob, partition_id); - validateKey(partition_key); - - return std::make_shared( - format, - sample_block, - context, - format_settings, - compression_method, - object_storage, - partition_key - ); - } - -private: - const String format; - const Block sample_block; - const ContextPtr context; - const CompressionMethod compression_method; - AzureObjectStorage * object_storage; - const String blob; - const std::optional format_settings; - - ExpressionActionsPtr partition_by_expr; - - static void validateKey(const String & str) - { - validatePartitionKey(str, true); - } -}; - -} - -class ReadFromAzureBlob : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromAzureBlob"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void applyFilters() override; - - ReadFromAzureBlob( - Block sample_block, - std::shared_ptr storage_, - ReadFromFormatInfo info_, - const bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) - , storage(std::move(storage_)) - , info(std::move(info_)) - , need_only_count(need_only_count_) - , context(std::move(context_)) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - } - -private: - std::shared_ptr storage; - ReadFromFormatInfo info; - const bool need_only_count; - - ContextPtr context; - - size_t max_block_size; - const size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -void ReadFromAzureBlob::applyFilters() -{ - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void StorageAzureBlob::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr local_context, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - if (partition_by && configuration.withWildcard()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned Azure storage is not implemented yet"); - - auto this_ptr = std::static_pointer_cast(shared_from_this()); - - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; - - auto reading = std::make_unique( - read_from_format_info.source_header, - std::move(this_ptr), - std::move(read_from_format_info), - need_only_count, - local_context, - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromAzureBlob::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - const auto & configuration = storage->configuration; - - if (storage->distributed_processing) - { - iterator_wrapper = std::make_shared(context, - context->getReadTaskCallback()); - } - else if (configuration.withGlobs()) - { - /// Iterate through disclosed globs and make a source for each file - iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blob_path, - predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback()); - } - else - { - iterator_wrapper = std::make_shared( - storage->object_storage.get(), configuration.container, configuration.blobs_paths, - predicate, storage->virtual_columns, context, nullptr, context->getFileProgressCallback()); - } -} - -void ReadFromAzureBlob::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - - const auto & configuration = storage->configuration; - Pipes pipes; - - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - info, - configuration.format, - getName(), - context, - storage->format_settings, - max_block_size, - configuration.compression_method, - storage->object_storage.get(), - configuration.container, - configuration.connection_url, - iterator_wrapper, - need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageAzureBlob::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) -{ - auto sample_block = metadata_snapshot->getSampleBlock(); - auto chosen_compression_method = chooseCompressionMethod(configuration.blobs_paths.back(), configuration.compression_method); - auto insert_query = std::dynamic_pointer_cast(query); - - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && configuration.withWildcard(); - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - object_storage.get(), - configuration.blobs_paths.back()); - } - else - { - if (configuration.withGlobs()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "AzureBlobStorage key '{}' contains globs, so the table is in readonly mode", configuration.blob_path); - - bool truncate_in_insert = local_context->getSettingsRef().azure_truncate_on_insert; - - if (!truncate_in_insert && object_storage->exists(StoredObject(configuration.blob_path))) - { - - if (local_context->getSettingsRef().azure_create_new_file_on_insert) - { - size_t index = configuration.blobs_paths.size(); - const auto & first_key = configuration.blobs_paths[0]; - auto pos = first_key.find_first_of('.'); - String new_key; - - do - { - new_key = first_key.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : first_key.substr(pos)); - ++index; - } - while (object_storage->exists(StoredObject(new_key))); - - configuration.blobs_paths.push_back(new_key); - } - else - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting azure_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting azure_create_new_file_on_insert", - configuration.container, configuration.blobs_paths.back()); - } - } - - return std::make_shared( - configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - object_storage.get(), - configuration.blobs_paths.back()); - } -} - -NamesAndTypesList StorageAzureBlob::getVirtuals() const -{ - return virtual_columns; -} - -Names StorageAzureBlob::getVirtualColumnNames() -{ - return VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage({}).getNames(); -} - -bool StorageAzureBlob::supportsPartitionBy() const -{ - return true; -} - -bool StorageAzureBlob::supportsSubsetOfColumns(const ContextPtr & context) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context, format_settings); -} - -bool StorageAzureBlob::prefersLargeBlocks() const -{ - return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format); -} - -bool StorageAzureBlob::parallelizeOutputAfterReading(ContextPtr context) const -{ - return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); -} - -StorageAzureBlobSource::GlobIterator::GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_) - : IIterator(context_) - , object_storage(object_storage_) - , container(container_) - , blob_path_with_globs(blob_path_with_globs_) - , virtual_columns(virtual_columns_) - , outer_blobs(outer_blobs_) - , file_progress_callback(file_progress_callback_) -{ - - const String key_prefix = blob_path_with_globs.substr(0, blob_path_with_globs.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == blob_path_with_globs.size()) - { - auto object_metadata = object_storage->getObjectMetadata(blob_path_with_globs); - blobs_with_metadata.emplace_back( - blob_path_with_globs, - object_metadata); - if (outer_blobs) - outer_blobs->emplace_back(blobs_with_metadata.back()); - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - is_finished = true; - return; - } - - object_storage_iterator = object_storage->iterate(key_prefix); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(blob_path_with_globs)); - - if (!matcher->ok()) - throw Exception( - ErrorCodes::CANNOT_COMPILE_REGEXP, "Cannot compile regex from glob ({}): {}", blob_path_with_globs, matcher->error()); - - recursive = blob_path_with_globs == "/**" ? true : false; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); -} - -RelativePathWithMetadata StorageAzureBlobSource::GlobIterator::next() -{ - std::lock_guard lock(next_mutex); - - if (is_finished && index >= blobs_with_metadata.size()) - { - return {}; - } - - bool need_new_batch = blobs_with_metadata.empty() || index >= blobs_with_metadata.size(); - - if (need_new_batch) - { - RelativePathsWithMetadata new_batch; - while (new_batch.empty()) - { - auto result = object_storage_iterator->getCurrrentBatchAndScheduleNext(); - if (result.has_value()) - { - new_batch = result.value(); - } - else - { - is_finished = true; - return {}; - } - - for (auto it = new_batch.begin(); it != new_batch.end();) - { - if (!recursive && !re2::RE2::FullMatch(it->relative_path, *matcher)) - it = new_batch.erase(it); - else - ++it; - } - } - - index = 0; - - if (filter_dag) - { - std::vector paths; - paths.reserve(new_batch.size()); - for (auto & path_with_metadata : new_batch) - paths.push_back(fs::path(container) / path_with_metadata.relative_path); - - VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); - } - - if (outer_blobs) - outer_blobs->insert(outer_blobs->end(), new_batch.begin(), new_batch.end()); - - blobs_with_metadata = std::move(new_batch); - if (file_progress_callback) - { - for (const auto & [relative_path, info] : blobs_with_metadata) - { - file_progress_callback(FileProgress(0, info.size_bytes)); - } - } - } - - size_t current_index = index++; - if (current_index >= blobs_with_metadata.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); - return blobs_with_metadata[current_index]; -} - -StorageAzureBlobSource::KeysIterator::KeysIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - const Strings & keys_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs, - std::function file_progress_callback) - : IIterator(context_) - , object_storage(object_storage_) - , container(container_) - , virtual_columns(virtual_columns_) -{ - Strings all_keys = keys_; - - ASTPtr filter_ast; - if (!all_keys.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - Strings paths; - paths.reserve(all_keys.size()); - for (const auto & key : all_keys) - paths.push_back(fs::path(container) / key); - - VirtualColumnUtils::filterByPathOrFile(all_keys, paths, filter_dag, virtual_columns, getContext()); - } - - for (auto && key : all_keys) - { - ObjectMetadata object_metadata = object_storage->getObjectMetadata(key); - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - keys.emplace_back(key, object_metadata); - } - - if (outer_blobs) - *outer_blobs = keys; -} - -RelativePathWithMetadata StorageAzureBlobSource::KeysIterator::next() -{ - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= keys.size()) - return {}; - - return keys[current_index]; -} - -Chunk StorageAzureBlobSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (const auto * input_format = reader.getInputFormat()) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - requested_virtual_columns, - fs::path(container) / reader.getRelativePath(), - reader.getRelativePathWithMetadata().metadata.size_bytes); - return chunk; - } - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); - - total_rows_in_file = 0; - - assert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - break; - - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); - } - - return {}; -} - -void StorageAzureBlobSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - String source = fs::path(connection_url) / container / path; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional StorageAzureBlobSource::tryGetNumRowsFromCache(const DB::RelativePathWithMetadata & path_with_metadata) -{ - String source = fs::path(connection_url) / container / path_with_metadata.relative_path; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - auto last_mod = path_with_metadata.metadata.last_modified; - if (last_mod) - return last_mod->epochTime(); - return std::nullopt; - }; - - return StorageAzureBlob::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -StorageAzureBlobSource::StorageAzureBlobSource( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - String compression_hint_, - AzureObjectStorage * object_storage_, - const String & container_, - const String & connection_url_, - std::shared_ptr file_iterator_, - bool need_only_count_) - :ISource(info.source_header, false) - , WithContext(context_) - , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) - , format(format_) - , name(std::move(name_)) - , sample_block(info.format_header) - , format_settings(format_settings_) - , columns_desc(info.columns_description) - , max_block_size(max_block_size_) - , compression_hint(compression_hint_) - , object_storage(std::move(object_storage_)) - , container(container_) - , connection_url(connection_url_) - , file_iterator(file_iterator_) - , need_only_count(need_only_count_) - , create_reader_pool(CurrentMetrics::ObjectStorageAzureThreads, CurrentMetrics::ObjectStorageAzureThreadsActive, CurrentMetrics::ObjectStorageAzureThreadsScheduled, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "AzureReader")) -{ - reader = createReader(); - if (reader) - reader_future = createReaderAsync(); -} - - -StorageAzureBlobSource::~StorageAzureBlobSource() -{ - create_reader_pool.wait(); -} - -String StorageAzureBlobSource::getName() const -{ - return name; -} - -StorageAzureBlobSource::ReaderHolder StorageAzureBlobSource::createReader() -{ - auto path_with_metadata = file_iterator->next(); - if (path_with_metadata.relative_path.empty()) - return {}; - - if (path_with_metadata.metadata.size_bytes == 0) - path_with_metadata.metadata = object_storage->getObjectMetadata(path_with_metadata.relative_path); - - QueryPipelineBuilder builder; - std::shared_ptr source; - std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files - ? tryGetNumRowsFromCache(path_with_metadata) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - source = std::make_shared(sample_block, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - auto compression_method = chooseCompressionMethod(path_with_metadata.relative_path, compression_hint); - read_buf = createAzureReadBuffer(path_with_metadata.relative_path, path_with_metadata.metadata.size_bytes); - auto input_format = FormatFactory::instance().getInput( - format, *read_buf, sample_block, getContext(), max_block_size, - format_settings, max_parsing_threads, std::nullopt, - /* is_remote_fs */ true, compression_method); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - source = input_format; - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - - return ReaderHolder{path_with_metadata, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageAzureBlobSource::createReaderAsync() -{ - return create_reader_scheduler([this] { return createReader(); }, Priority{}); -} - -std::unique_ptr StorageAzureBlobSource::createAzureReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from Azure with initial prefetch", object_size); - return createAsyncAzureReadBuffer(key, read_settings, object_size); - } - - return object_storage->readObject(StoredObject(key), read_settings, {}, object_size); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - const std::shared_ptr & file_iterator_, - AzureObjectStorage * object_storage_, - const StorageAzureBlob::Configuration & configuration_, - const std::optional & format_settings_, - const RelativePathsWithMetadata & read_keys_, - const ContextPtr & context_) - : WithContext(context_) - , file_iterator(file_iterator_) - , object_storage(object_storage_) - , configuration(configuration_) - , format_settings(format_settings_) - , read_keys(read_keys_) - , prev_read_keys_size(read_keys_.size()) - { - } - - std::pair, std::optional> next() override - { - /// For default mode check cached columns for currently read keys on first iteration. - if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns}; - } - - current_path_with_metadata = file_iterator->next(); - - if (current_path_with_metadata.relative_path.empty()) - { - if (first) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because there are no files with provided path " - "in AzureBlobStorage. You must specify table structure manually", configuration.format); - - return {nullptr, std::nullopt}; - } - - first = false; - - /// AzureBlobStorage file iterator could get new keys after new iteration, check them in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT && read_keys.size() > prev_read_keys_size) - { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - prev_read_keys_size = read_keys.size(); - if (columns_from_cache) - return {nullptr, columns_from_cache}; - } - else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - RelativePathsWithMetadata paths = {current_path_with_metadata}; - if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) - return {nullptr, columns_from_cache}; - } - - first = false; - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - return {wrapReadBufferWithCompressionMethod( - object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes), - chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method), - zstd_window_log_max), std::nullopt}; - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure) - return; - - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; - auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path; - auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_azure - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - Strings sources; - sources.reserve(read_keys.size()); - std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; }); - auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext()); - StorageAzureBlob::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - String getLastFileName() const override { return current_path_with_metadata.relative_path; } - - private: - std::optional tryGetColumnsFromCache(const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end) - { - auto & schema_cache = StorageAzureBlob::getSchemaCache(getContext()); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] -> std::optional - { - if (it->metadata.last_modified) - return it->metadata.last_modified->epochTime(); - return std::nullopt; - }; - - auto host_and_bucket = configuration.connection_url + '/' + configuration.container; - String source = host_and_bucket + '/' + it->relative_path; - auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; - } - - std::shared_ptr file_iterator; - AzureObjectStorage * object_storage; - const StorageAzureBlob::Configuration & configuration; - const std::optional & format_settings; - const RelativePathsWithMetadata & read_keys; - size_t prev_read_keys_size; - RelativePathWithMetadata current_path_with_metadata; - bool first = true; - }; -} - -ColumnsDescription StorageAzureBlob::getTableStructureFromData( - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx, - bool distributed_processing) -{ - RelativePathsWithMetadata read_keys; - std::shared_ptr file_iterator; - if (distributed_processing) - { - file_iterator = std::make_shared(ctx, - ctx->getReadTaskCallback()); - } - else if (configuration.withGlobs()) - { - file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blob_path, nullptr, NamesAndTypesList{}, ctx, &read_keys); - } - else - { - file_iterator = std::make_shared( - object_storage, configuration.container, configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys); - } - - ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, configuration, format_settings, read_keys, ctx); - return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); -} - -SchemaCache & StorageAzureBlob::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_azure", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - - -std::unique_ptr StorageAzureBlobSource::createAsyncAzureReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto modified_settings{read_settings}; - modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - auto async_reader = object_storage->readObjects(StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, modified_settings); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; -} - -} - -#endif diff --git a/src/Storages/StorageAzureBlob.h b/src/Storages/StorageAzureBlob.h deleted file mode 100644 index 6fc3c5ce592..00000000000 --- a/src/Storages/StorageAzureBlob.h +++ /dev/null @@ -1,339 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class StorageAzureBlob : public IStorage -{ -public: - - using AzureClient = Azure::Storage::Blobs::BlobContainerClient; - using AzureClientPtr = std::unique_ptr; - - struct Configuration : public StatelessTableEngineConfiguration - { - Configuration() = default; - - String getPath() const { return blob_path; } - - bool update(ContextPtr context); - - void connect(ContextPtr context); - - bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; } - - bool withWildcard() const - { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return blobs_paths.back().find(PARTITION_ID_WILDCARD) != String::npos; - } - - Poco::URI getConnectionURL() const; - - std::string connection_url; - bool is_connection_string; - - std::optional account_name; - std::optional account_key; - - std::string container; - std::string blob_path; - std::vector blobs_paths; - }; - - StorageAzureBlob( - const Configuration & configuration_, - std::unique_ptr && object_storage_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_); - - static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); - static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only); - - static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); - - static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection); - - String getName() const override - { - return name; - } - - void read( - QueryPlan & query_plan, - const Names &, - const StorageSnapshotPtr &, - SelectQueryInfo &, - ContextPtr, - QueryProcessingStage::Enum, - size_t, - size_t) override; - - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /* metadata_snapshot */, ContextPtr context, bool /*async_insert*/) override; - - void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - - NamesAndTypesList getVirtuals() const override; - static Names getVirtualColumnNames(); - - bool supportsPartitionBy() const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsSubsetOfColumns(const ContextPtr & context) const; - - bool supportsTrivialCountOptimization() const override { return true; } - - bool prefersLargeBlocks() const override; - - bool parallelizeOutputAfterReading(ContextPtr context) const override; - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - static ColumnsDescription getTableStructureFromData( - AzureObjectStorage * object_storage, - const Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx, - bool distributed_processing = false); - -private: - friend class ReadFromAzureBlob; - - std::string name; - Configuration configuration; - std::unique_ptr object_storage; - NamesAndTypesList virtual_columns; - - const bool distributed_processing; - std::optional format_settings; - ASTPtr partition_by; -}; - -class StorageAzureBlobSource : public ISource, WithContext -{ -public: - class IIterator : public WithContext - { - public: - IIterator(ContextPtr context_):WithContext(context_) {} - virtual ~IIterator() = default; - virtual RelativePathWithMetadata next() = 0; - - RelativePathWithMetadata operator ()() { return next(); } - }; - - class GlobIterator : public IIterator - { - public: - GlobIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - String blob_path_with_globs_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs_, - std::function file_progress_callback_ = {}); - - RelativePathWithMetadata next() override; - ~GlobIterator() override = default; - - private: - AzureObjectStorage * object_storage; - std::string container; - String blob_path_with_globs; - ActionsDAGPtr filter_dag; - NamesAndTypesList virtual_columns; - - size_t index = 0; - - RelativePathsWithMetadata blobs_with_metadata; - RelativePathsWithMetadata * outer_blobs; - ObjectStorageIteratorPtr object_storage_iterator; - bool recursive{false}; - - std::unique_ptr matcher; - - void createFilterAST(const String & any_key); - bool is_finished = false; - std::mutex next_mutex; - - std::function file_progress_callback; - }; - - class ReadIterator : public IIterator - { - public: - explicit ReadIterator(ContextPtr context_, - const ReadTaskCallback & callback_) - : IIterator(context_), callback(callback_) { } - RelativePathWithMetadata next() override - { - return {callback(), {}}; - } - - private: - ReadTaskCallback callback; - }; - - class KeysIterator : public IIterator - { - public: - KeysIterator( - AzureObjectStorage * object_storage_, - const std::string & container_, - const Strings & keys_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - RelativePathsWithMetadata * outer_blobs, - std::function file_progress_callback = {}); - - RelativePathWithMetadata next() override; - ~KeysIterator() override = default; - - private: - AzureObjectStorage * object_storage; - std::string container; - RelativePathsWithMetadata keys; - - ActionsDAGPtr filter_dag; - NamesAndTypesList virtual_columns; - - std::atomic index = 0; - }; - - StorageAzureBlobSource( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - String compression_hint_, - AzureObjectStorage * object_storage_, - const String & container_, - const String & connection_url_, - std::shared_ptr file_iterator_, - bool need_only_count_); - ~StorageAzureBlobSource() override; - - Chunk generate() override; - - String getName() const override; - -private: - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const RelativePathWithMetadata & path_with_metadata); - - NamesAndTypesList requested_columns; - NamesAndTypesList requested_virtual_columns; - String format; - String name; - Block sample_block; - std::optional format_settings; - ColumnsDescription columns_desc; - UInt64 max_block_size; - String compression_hint; - AzureObjectStorage * object_storage; - String container; - String connection_url; - std::shared_ptr file_iterator; - bool need_only_count; - size_t total_rows_in_file = 0; - - struct ReaderHolder - { - public: - ReaderHolder( - RelativePathWithMetadata relative_path_with_metadata_, - std::unique_ptr read_buf_, - std::shared_ptr source_, - std::unique_ptr pipeline_, - std::unique_ptr reader_) - : relative_path_with_metadata(std::move(relative_path_with_metadata_)) - , read_buf(std::move(read_buf_)) - , source(std::move(source_)) - , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) - { - } - - ReaderHolder() = default; - ReaderHolder(const ReaderHolder & other) = delete; - ReaderHolder & operator=(const ReaderHolder & other) = delete; - - ReaderHolder(ReaderHolder && other) noexcept - { - *this = std::move(other); - } - - ReaderHolder & operator=(ReaderHolder && other) noexcept - { - /// The order of destruction is important. - /// reader uses pipeline, pipeline uses read_buf. - reader = std::move(other.reader); - pipeline = std::move(other.pipeline); - source = std::move(other.source); - read_buf = std::move(other.read_buf); - relative_path_with_metadata = std::move(other.relative_path_with_metadata); - return *this; - } - - explicit operator bool() const { return reader != nullptr; } - PullingPipelineExecutor * operator->() { return reader.get(); } - const PullingPipelineExecutor * operator->() const { return reader.get(); } - const String & getRelativePath() const { return relative_path_with_metadata.relative_path; } - const RelativePathWithMetadata & getRelativePathWithMetadata() const { return relative_path_with_metadata; } - const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } - - private: - RelativePathWithMetadata relative_path_with_metadata; - std::unique_ptr read_buf; - std::shared_ptr source; - std::unique_ptr pipeline; - std::unique_ptr reader; - }; - - ReaderHolder reader; - - LoggerPtr log = getLogger("StorageAzureBlobSource"); - - ThreadPool create_reader_pool; - ThreadPoolCallbackRunner create_reader_scheduler; - std::future reader_future; - - /// Recreate ReadBuffer and Pipeline for each file. - ReaderHolder createReader(); - std::future createReaderAsync(); - - std::unique_ptr createAzureReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncAzureReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size); -}; - -} - -#endif diff --git a/src/Storages/StorageAzureBlobCluster.cpp b/src/Storages/StorageAzureBlobCluster.cpp deleted file mode 100644 index 1d587512f38..00000000000 --- a/src/Storages/StorageAzureBlobCluster.cpp +++ /dev/null @@ -1,89 +0,0 @@ -#include "Storages/StorageAzureBlobCluster.h" - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageAzureBlobCluster::StorageAzureBlobCluster( - const String & cluster_name_, - const StorageAzureBlob::Configuration & configuration_, - std::unique_ptr && object_storage_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageAzureBlobCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) - , configuration{configuration_} - , object_storage(std::move(object_storage_)) -{ - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL()); - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - /// `format_settings` is set to std::nullopt, because StorageAzureBlobCluster is used only as table function - auto columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context_, false); - storage_metadata.setColumns(columns); - } - else - storage_metadata.setColumns(columns_); - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); - - TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context); -} - -RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared( - object_storage.get(), configuration.container, configuration.blob_path, - predicate, virtual_columns, context, nullptr); - auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next().relative_path; }); - return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; -} - -NamesAndTypesList StorageAzureBlobCluster::getVirtuals() const -{ - return virtual_columns; -} - - -} - -#endif diff --git a/src/Storages/StorageAzureBlobCluster.h b/src/Storages/StorageAzureBlobCluster.h deleted file mode 100644 index 2831b94f825..00000000000 --- a/src/Storages/StorageAzureBlobCluster.h +++ /dev/null @@ -1,56 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include - -#include "Client/Connection.h" -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageAzureBlobCluster : public IStorageCluster -{ -public: - StorageAzureBlobCluster( - const String & cluster_name_, - const StorageAzureBlob::Configuration & configuration_, - std::unique_ptr && object_storage_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_); - - std::string getName() const override { return "AzureBlobStorageCluster"; } - - NamesAndTypesList getVirtuals() const override; - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization() const override { return true; } - -private: - void updateBeforeRead(const ContextPtr & /*context*/) override {} - - void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; - - StorageAzureBlob::Configuration configuration; - NamesAndTypesList virtual_columns; - std::unique_ptr object_storage; -}; - - -} - -#endif diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp deleted file mode 100644 index 4fde6fd04f3..00000000000 --- a/src/Storages/StorageS3.cpp +++ /dev/null @@ -1,1905 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include - -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include - - -#include -#include -#include - -#include - -#include - -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -#include - -#ifdef __clang__ -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" -#endif -#include -#ifdef __clang__ -# pragma clang diagnostic pop -#endif - -namespace fs = std::filesystem; - - -namespace CurrentMetrics -{ - extern const Metric StorageS3Threads; - extern const Metric StorageS3ThreadsActive; - extern const Metric StorageS3ThreadsScheduled; -} - -namespace ProfileEvents -{ - extern const Event S3DeleteObjects; - extern const Event S3ListObjects; - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ - -static const std::unordered_set required_configuration_keys = { - "url", -}; -static const std::unordered_set optional_configuration_keys = { - "format", - "compression", - "compression_method", - "structure", - "access_key_id", - "secret_access_key", - "session_token", - "filename", - "use_environment_credentials", - "max_single_read_retries", - "min_upload_part_size", - "upload_part_size_multiply_factor", - "upload_part_size_multiply_parts_count_threshold", - "max_single_part_upload_size", - "max_connections", - "expiration_window_seconds", - "no_sign_request" -}; - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_TEXT; - extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int S3_ERROR; - extern const int UNEXPECTED_EXPRESSION; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int NOT_IMPLEMENTED; - extern const int CANNOT_COMPILE_REGEXP; - extern const int FILE_DOESNT_EXIST; -} - - -class ReadFromStorageS3Step : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromStorageS3Step"; } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - - void applyFilters() override; - - ReadFromStorageS3Step( - Block sample_block, - const Names & column_names_, - StorageSnapshotPtr storage_snapshot_, - StorageS3 & storage_, - ReadFromFormatInfo read_from_format_info_, - bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = std::move(sample_block)}) - , column_names(column_names_) - , storage_snapshot(std::move(storage_snapshot_)) - , storage(storage_) - , read_from_format_info(std::move(read_from_format_info_)) - , need_only_count(need_only_count_) - , local_context(std::move(context_)) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - query_configuration = storage.updateConfigurationAndGetCopy(local_context); - virtual_columns = storage.getVirtuals(); - } - -private: - Names column_names; - StorageSnapshotPtr storage_snapshot; - StorageS3 & storage; - ReadFromFormatInfo read_from_format_info; - bool need_only_count; - StorageS3::Configuration query_configuration; - NamesAndTypesList virtual_columns; - - ContextPtr local_context; - - size_t max_block_size; - size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - - -class IOutputFormat; -using OutputFormatPtr = std::shared_ptr; - -class StorageS3Source::DisclosedGlobIterator::Impl : WithContext -{ -public: - Impl( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context_, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : WithContext(context_) - , client(client_.clone()) - , globbed_uri(globbed_uri_) - , virtual_columns(virtual_columns_) - , read_keys(read_keys_) - , request_settings(request_settings_) - , list_objects_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) - , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, "ListObjects")) - , file_progress_callback(file_progress_callback_) - { - if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) - throw Exception(ErrorCodes::UNEXPECTED_EXPRESSION, "Expression can not have wildcards inside bucket name"); - - const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{")); - - /// We don't have to list bucket, because there is no asterisks. - if (key_prefix.size() == globbed_uri.key.size()) - { - buffer.emplace_back(std::make_shared(globbed_uri.key, std::nullopt)); - buffer_iter = buffer.begin(); - is_finished = true; - return; - } - - request.SetBucket(globbed_uri.bucket); - request.SetPrefix(key_prefix); - request.SetMaxKeys(static_cast(request_settings.list_object_keys_size)); - - outcome_future = listObjectsAsync(); - - matcher = std::make_unique(makeRegexpPatternFromGlobs(globbed_uri.key)); - if (!matcher->ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", globbed_uri.key, matcher->error()); - - recursive = globbed_uri.key == "/**" ? true : false; - - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - fillInternalBufferAssumeLocked(); - } - - KeyWithInfoPtr next(size_t) - { - std::lock_guard lock(mutex); - return nextAssumeLocked(); - } - - size_t objectsCount() - { - return buffer.size(); - } - - ~Impl() - { - list_objects_pool.wait(); - } - -private: - using ListObjectsOutcome = Aws::S3::Model::ListObjectsV2Outcome; - - KeyWithInfoPtr nextAssumeLocked() - { - if (buffer_iter != buffer.end()) - { - auto answer = *buffer_iter; - ++buffer_iter; - - /// If url doesn't contain globs, we didn't list s3 bucket and didn't get object info for the key. - /// So we get object info lazily here on 'next()' request. - if (!answer->info) - { - answer->info = S3::getObjectInfo(*client, globbed_uri.bucket, answer->key, globbed_uri.version_id, request_settings); - if (file_progress_callback) - file_progress_callback(FileProgress(0, answer->info->size)); - } - - return answer; - } - - if (is_finished) - return {}; - - try - { - fillInternalBufferAssumeLocked(); - } - catch (...) - { - /// In case of exception thrown while listing new batch of files - /// iterator may be partially initialized and its further using may lead to UB. - /// Iterator is used by several processors from several threads and - /// it may take some time for threads to stop processors and they - /// may still use this iterator after exception is thrown. - /// To avoid this UB, reset the buffer and return defaults for further calls. - is_finished = true; - buffer.clear(); - buffer_iter = buffer.begin(); - throw; - } - - return nextAssumeLocked(); - } - - void fillInternalBufferAssumeLocked() - { - buffer.clear(); - assert(outcome_future.valid()); - auto outcome = outcome_future.get(); - - if (!outcome.IsSuccess()) - { - throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), - backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); - } - - const auto & result_batch = outcome.GetResult().GetContents(); - - /// It returns false when all objects were returned - is_finished = !outcome.GetResult().GetIsTruncated(); - - if (!is_finished) - { - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - list_objects_pool.wait(); - outcome_future = listObjectsAsync(); - } - - if (request_settings.throw_on_zero_files_match && result_batch.empty()) - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files using prefix {}", request.GetPrefix()); - - KeysWithInfo temp_buffer; - temp_buffer.reserve(result_batch.size()); - - for (const auto & row : result_batch) - { - String key = row.GetKey(); - if (recursive || re2::RE2::FullMatch(key, *matcher)) - { - S3::ObjectInfo info = - { - .size = size_t(row.GetSize()), - .last_modification_time = row.GetLastModified().Millis() / 1000, - }; - - temp_buffer.emplace_back(std::make_shared(std::move(key), std::move(info))); - } - } - - if (temp_buffer.empty()) - { - buffer_iter = buffer.begin(); - return; - } - - if (filter_dag) - { - std::vector paths; - paths.reserve(temp_buffer.size()); - for (const auto & key_with_info : temp_buffer) - paths.push_back(fs::path(globbed_uri.bucket) / key_with_info->key); - - VirtualColumnUtils::filterByPathOrFile(temp_buffer, paths, filter_dag, virtual_columns, getContext()); - } - - buffer = std::move(temp_buffer); - - if (file_progress_callback) - { - for (const auto & key_with_info : buffer) - file_progress_callback(FileProgress(0, key_with_info->info->size)); - } - - /// Set iterator only after the whole batch is processed - buffer_iter = buffer.begin(); - - if (read_keys) - read_keys->insert(read_keys->end(), buffer.begin(), buffer.end()); - } - - std::future listObjectsAsync() - { - return list_objects_scheduler([this] - { - ProfileEvents::increment(ProfileEvents::S3ListObjects); - auto outcome = client->ListObjectsV2(request); - - /// Outcome failure will be handled on the caller side. - if (outcome.IsSuccess()) - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - - return outcome; - }, Priority{}); - } - - std::mutex mutex; - - KeysWithInfo buffer; - KeysWithInfo::iterator buffer_iter; - - std::unique_ptr client; - S3::URI globbed_uri; - ASTPtr query; - NamesAndTypesList virtual_columns; - ActionsDAGPtr filter_dag; - std::unique_ptr matcher; - bool recursive{false}; - bool is_finished{false}; - KeysWithInfo * read_keys; - - S3::ListObjectsV2Request request; - S3Settings::RequestSettings request_settings; - - ThreadPool list_objects_pool; - ThreadPoolCallbackRunner list_objects_scheduler; - std::future outcome_future; - std::function file_progress_callback; -}; - -StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns_, - ContextPtr context, - KeysWithInfo * read_keys_, - const S3Settings::RequestSettings & request_settings_, - std::function file_progress_callback_) - : pimpl(std::make_shared(client_, globbed_uri_, predicate, virtual_columns_, context, read_keys_, request_settings_, file_progress_callback_)) -{ -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::DisclosedGlobIterator::next(size_t idx) /// NOLINT -{ - return pimpl->next(idx); -} - -size_t StorageS3Source::DisclosedGlobIterator::estimatedKeysCount() -{ - return pimpl->objectsCount(); -} - -class StorageS3Source::KeysIterator::Impl -{ -public: - explicit Impl( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys_, - std::function file_progress_callback_) - : keys(keys_) - , client(client_.clone()) - , version_id(version_id_) - , bucket(bucket_) - , request_settings(request_settings_) - , file_progress_callback(file_progress_callback_) - { - if (read_keys_) - { - for (const auto & key : keys) - read_keys_->push_back(std::make_shared(key)); - } - } - - KeyWithInfoPtr next(size_t) - { - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= keys.size()) - return {}; - auto key = keys[current_index]; - std::optional info; - if (file_progress_callback) - { - info = S3::getObjectInfo(*client, bucket, key, version_id, request_settings); - file_progress_callback(FileProgress(0, info->size)); - } - - return std::make_shared(key, info); - } - - size_t objectsCount() - { - return keys.size(); - } - -private: - Strings keys; - std::atomic_size_t index = 0; - std::unique_ptr client; - String version_id; - String bucket; - S3Settings::RequestSettings request_settings; - std::function file_progress_callback; -}; - -StorageS3Source::KeysIterator::KeysIterator( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys, - std::function file_progress_callback_) - : pimpl(std::make_shared( - client_, version_id_, keys_, bucket_, request_settings_, - read_keys, file_progress_callback_)) -{ -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::KeysIterator::next(size_t idx) /// NOLINT -{ - return pimpl->next(idx); -} - -size_t StorageS3Source::KeysIterator::estimatedKeysCount() -{ - return pimpl->objectsCount(); -} - -StorageS3Source::ReadTaskIterator::ReadTaskIterator( - const DB::ReadTaskCallback & callback_, - size_t max_threads_count) - : callback(callback_) -{ - ThreadPool pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, max_threads_count); - auto pool_scheduler = threadPoolCallbackRunner(pool, "S3ReadTaskItr"); - - std::vector> keys; - keys.reserve(max_threads_count); - for (size_t i = 0; i < max_threads_count; ++i) - keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); - - pool.wait(); - buffer.reserve(max_threads_count); - for (auto & key_future : keys) - buffer.emplace_back(std::make_shared(key_future.get(), std::nullopt)); -} - -StorageS3Source::KeyWithInfoPtr StorageS3Source::ReadTaskIterator::next(size_t) /// NOLINT -{ - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= buffer.size()) - return std::make_shared(callback()); - - return buffer[current_index]; -} - -size_t StorageS3Source::ReadTaskIterator::estimatedKeysCount() -{ - return buffer.size(); -} - -StorageS3Source::StorageS3Source( - const ReadFromFormatInfo & info, - const String & format_, - String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket_, - const String & version_id_, - const String & url_host_and_port_, - std::shared_ptr file_iterator_, - const size_t max_parsing_threads_, - bool need_only_count_) - : SourceWithKeyCondition(info.source_header, false) - , WithContext(context_) - , name(std::move(name_)) - , bucket(bucket_) - , version_id(version_id_) - , url_host_and_port(url_host_and_port_) - , format(format_) - , columns_desc(info.columns_description) - , requested_columns(info.requested_columns) - , max_block_size(max_block_size_) - , request_settings(request_settings_) - , compression_hint(std::move(compression_hint_)) - , client(client_) - , sample_block(info.format_header) - , format_settings(format_settings_) - , requested_virtual_columns(info.requested_virtual_columns) - , file_iterator(file_iterator_) - , max_parsing_threads(max_parsing_threads_) - , need_only_count(need_only_count_) - , create_reader_pool(CurrentMetrics::StorageS3Threads, CurrentMetrics::StorageS3ThreadsActive, CurrentMetrics::StorageS3ThreadsScheduled, 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) -{ -} - -void StorageS3Source::lazyInitialize(size_t idx) -{ - if (initialized) - return; - - reader = createReader(idx); - if (reader) - reader_future = createReaderAsync(idx); - initialized = true; -} - -StorageS3Source::ReaderHolder StorageS3Source::createReader(size_t idx) -{ - KeyWithInfoPtr key_with_info; - do - { - key_with_info = file_iterator->next(idx); - if (!key_with_info || key_with_info->key.empty()) - return {}; - - if (!key_with_info->info) - key_with_info->info = S3::getObjectInfo(*client, bucket, key_with_info->key, version_id, request_settings); - } - while (getContext()->getSettingsRef().s3_skip_empty_files && key_with_info->info->size == 0); - - QueryPipelineBuilder builder; - std::shared_ptr source; - std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(*key_with_info) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - source = std::make_shared(sample_block, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - auto compression_method = chooseCompressionMethod(key_with_info->key, compression_hint); - read_buf = createS3ReadBuffer(key_with_info->key, key_with_info->info->size); - - auto input_format = FormatFactory::instance().getInput( - format, - *read_buf, - sample_block, - getContext(), - max_block_size, - format_settings, - max_parsing_threads, - /* max_download_threads= */ std::nullopt, - /* is_remote_fs */ true, - compression_method, - need_only_count); - - if (key_condition) - input_format->setKeyCondition(key_condition); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { return std::make_shared(header, columns_desc, *input_format, getContext()); }); - } - - source = input_format; - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - - return ReaderHolder{key_with_info, bucket, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)}; -} - -std::future StorageS3Source::createReaderAsync(size_t idx) -{ - return create_reader_scheduler([=, this] { return createReader(idx); }, Priority{}); -} - -std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - { - LOG_TRACE(log, "Downloading object of size {} from S3 with initial prefetch", object_size); - return createAsyncS3ReadBuffer(key, read_settings, object_size); - } - - return std::make_unique( - client, bucket, key, version_id, request_settings, read_settings, - /*use_external_buffer*/ false, /*offset_*/ 0, /*read_until_position_*/ 0, - /*restricted_seek_*/ false, object_size); -} - -std::unique_ptr StorageS3Source::createAsyncS3ReadBuffer( - const String & key, const ReadSettings & read_settings, size_t object_size) -{ - auto context = getContext(); - auto read_buffer_creator = - [this, read_settings, object_size] - (const std::string & path, size_t read_until_position) -> std::unique_ptr - { - return std::make_unique( - client, - bucket, - path, - version_id, - request_settings, - read_settings, - /* use_external_buffer */true, - /* offset */0, - read_until_position, - /* restricted_seek */true, - object_size); - }; - - auto s3_impl = std::make_unique( - std::move(read_buffer_creator), - StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, - read_settings, - /* cache_log */nullptr, /* use_external_buffer */true); - - auto modified_settings{read_settings}; - /// FIXME: Changing this setting to default value breaks something around parquet reading - modified_settings.remote_read_min_bytes_for_seek = modified_settings.remote_fs_buffer_size; - - auto & pool_reader = context->getThreadPoolReader(FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); - auto async_reader = std::make_unique( - std::move(s3_impl), pool_reader, modified_settings, - context->getAsyncReadCounters(), context->getFilesystemReadPrefetchesLog()); - - async_reader->setReadUntilEnd(); - if (read_settings.remote_fs_prefetch) - async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - return async_reader; -} - -StorageS3Source::~StorageS3Source() -{ - create_reader_pool.wait(); -} - -String StorageS3Source::getName() const -{ - return name; -} - -Chunk StorageS3Source::generate() -{ - lazyInitialize(); - - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (const auto * input_format = reader.getInputFormat()) - chunk_size = reader.getInputFormat()->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getPath(), reader.getFileSize()); - return chunk; - } - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getFile(), total_rows_in_file); - - total_rows_in_file = 0; - - assert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - break; - - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); - } - - return {}; -} - -void StorageS3Source::addNumRowsToCache(const String & key, size_t num_rows) -{ - String source = fs::path(url_host_and_port) / bucket / key; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional StorageS3Source::tryGetNumRowsFromCache(const KeyWithInfo & key_with_info) -{ - String source = fs::path(url_host_and_port) / bucket / key_with_info.key; - auto cache_key = getKeyForSchemaCache(source, format, format_settings, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - return key_with_info.info->last_modification_time; - }; - - return StorageS3::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -class StorageS3Sink : public SinkToStorage -{ -public: - StorageS3Sink( - const String & format, - const Block & sample_block_, - ContextPtr context, - std::optional format_settings_, - const CompressionMethod compression_method, - const StorageS3::Configuration & configuration_, - const String & bucket, - const String & key) - : SinkToStorage(sample_block_) - , sample_block(sample_block_) - , format_settings(format_settings_) - { - BlobStorageLogWriterPtr blob_log = nullptr; - if (auto blob_storage_log = context->getBlobStorageLog()) - { - blob_log = std::make_shared(std::move(blob_storage_log)); - blob_log->query_id = context->getCurrentQueryId(); - } - - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique( - configuration_.client, - bucket, - key, - DBMS_DEFAULT_BUFFER_SIZE, - configuration_.request_settings, - std::move(blob_log), - std::nullopt, - threadPoolCallbackRunner(getIOThreadPool().get(), "S3ParallelWrite"), - context->getWriteSettings()), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer - = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, format_settings); - } - - String getName() const override { return "StorageS3Sink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf.reset(); - } - - Block sample_block; - std::optional format_settings; - std::unique_ptr write_buf; - OutputFormatPtr writer; - bool cancelled = false; - std::mutex cancel_mutex; -}; - - -class PartitionedStorageS3Sink : public PartitionedSink -{ -public: - PartitionedStorageS3Sink( - const ASTPtr & partition_by, - const String & format_, - const Block & sample_block_, - ContextPtr context_, - std::optional format_settings_, - const CompressionMethod compression_method_, - const StorageS3::Configuration & configuration_, - const String & bucket_, - const String & key_) - : PartitionedSink(partition_by, context_, sample_block_) - , format(format_) - , sample_block(sample_block_) - , context(context_) - , compression_method(compression_method_) - , configuration(configuration_) - , bucket(bucket_) - , key(key_) - , format_settings(format_settings_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto partition_bucket = replaceWildcards(bucket, partition_id); - validateBucket(partition_bucket); - - auto partition_key = replaceWildcards(key, partition_id); - validateKey(partition_key); - - return std::make_shared( - format, - sample_block, - context, - format_settings, - compression_method, - configuration, - partition_bucket, - partition_key - ); - } - -private: - const String format; - const Block sample_block; - const ContextPtr context; - const CompressionMethod compression_method; - const StorageS3::Configuration configuration; - const String bucket; - const String key; - const std::optional format_settings; - - static void validateBucket(const String & str) - { - S3::URI::validateBucket(str, {}); - - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name"); - - validatePartitionKey(str, false); - } - - static void validateKey(const String & str) - { - /// See: - /// - https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html - /// - https://cloud.ibm.com/apidocs/cos/cos-compatibility#putobject - - if (str.empty() || str.size() > 1024) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size()); - - if (!DB::UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key"); - - validatePartitionKey(str, true); - } -}; - - -StorageS3::StorageS3( - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , configuration(configuration_) - , name(configuration.url.storage_name) - , distributed_processing(distributed_processing_) - , format_settings(format_settings_) - , partition_by(partition_by_) -{ - updateConfiguration(context_); // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall) - - FormatFactory::instance().checkFormatName(configuration.format); - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri); - context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration.headers_from_ast); - - StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - auto columns = getTableStructureFromDataImpl(configuration, format_settings, context_); - storage_metadata.setColumns(columns); - } - else - { - /// We don't allow special columns in S3 storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine S3 doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -static std::shared_ptr createFileIterator( - const StorageS3::Configuration & configuration, - bool distributed_processing, - ContextPtr local_context, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns, - StorageS3::KeysWithInfo * read_keys = nullptr, - std::function file_progress_callback = {}) -{ - if (distributed_processing) - { - return std::make_shared(local_context->getReadTaskCallback(), local_context->getSettingsRef().max_threads); - } - else if (configuration.withGlobs()) - { - /// Iterate through disclosed globs and make a source for each file - return std::make_shared( - *configuration.client, configuration.url, predicate, virtual_columns, - local_context, read_keys, configuration.request_settings, file_progress_callback); - } - else - { - Strings keys = configuration.keys; - auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - if (filter_dag) - { - std::vector paths; - paths.reserve(keys.size()); - for (const auto & key : keys) - paths.push_back(fs::path(configuration.url.bucket) / key); - VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); - } - - return std::make_shared( - *configuration.client, configuration.url.version_id, keys, - configuration.url.bucket, configuration.request_settings, read_keys, file_progress_callback); - } -} - -bool StorageS3::supportsSubsetOfColumns(const ContextPtr & context) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context, format_settings); -} - -bool StorageS3::prefersLargeBlocks() const -{ - return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration.format); -} - -bool StorageS3::parallelizeOutputAfterReading(ContextPtr context) const -{ - return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration.format, context); -} - -void StorageS3::read( - QueryPlan & query_plan, - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr local_context, - QueryProcessingStage::Enum /*processed_stage*/, - size_t max_block_size, - size_t num_streams) -{ - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context), virtual_columns); - - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && local_context->getSettingsRef().optimize_count_from_files; - - auto reading = std::make_unique( - read_from_format_info.source_header, - column_names, - storage_snapshot, - *this, - std::move(read_from_format_info), - need_only_count, - local_context, - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromStorageS3Step::applyFilters() -{ - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void ReadFromStorageS3Step::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - iterator_wrapper = createFileIterator( - query_configuration, storage.distributed_processing, local_context, predicate, - virtual_columns, nullptr, local_context->getFileProgressCallback()); -} - -void ReadFromStorageS3Step::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - if (storage.partition_by && query_configuration.withWildcard()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned S3 storage is not implemented yet"); - - createIterator(nullptr); - - size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); - if (estimated_keys_count > 1) - num_streams = std::min(num_streams, estimated_keys_count); - else - /// Disclosed glob iterator can underestimate the amount of keys in some cases. We will keep one stream for this particular case. - num_streams = 1; - - const size_t max_threads = local_context->getSettingsRef().max_threads; - const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); - LOG_DEBUG(getLogger("StorageS3"), "Reading in {} streams, {} threads per stream", num_streams, max_parsing_threads); - - Pipes pipes; - pipes.reserve(num_streams); - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - read_from_format_info, - query_configuration.format, - storage.getName(), - local_context, - storage.format_settings, - max_block_size, - query_configuration.request_settings, - query_configuration.compression_method, - query_configuration.client, - query_configuration.url.bucket, - query_configuration.url.version_id, - query_configuration.url.uri.getHost() + std::to_string(query_configuration.url.uri.getPort()), - iterator_wrapper, - max_parsing_threads, - need_only_count); - - source->setKeyCondition(filter_nodes.nodes, local_context); - pipes.emplace_back(std::move(source)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(read_from_format_info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, bool /*async_insert*/) -{ - auto query_configuration = updateConfigurationAndGetCopy(local_context); - - auto sample_block = metadata_snapshot->getSampleBlock(); - auto chosen_compression_method = chooseCompressionMethod(query_configuration.keys.back(), query_configuration.compression_method); - auto insert_query = std::dynamic_pointer_cast(query); - - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && query_configuration.withWildcard(); - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - query_configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - query_configuration, - query_configuration.url.bucket, - query_configuration.keys.back()); - } - else - { - if (query_configuration.withGlobs()) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", query_configuration.url.key); - - bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; - - if (!truncate_in_insert && S3::objectExists(*query_configuration.client, query_configuration.url.bucket, query_configuration.keys.back(), query_configuration.url.version_id, query_configuration.request_settings)) - { - if (local_context->getSettingsRef().s3_create_new_file_on_insert) - { - size_t index = query_configuration.keys.size(); - const auto & first_key = query_configuration.keys[0]; - auto pos = first_key.find_first_of('.'); - String new_key; - do - { - new_key = first_key.substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : first_key.substr(pos)); - ++index; - } - while (S3::objectExists(*query_configuration.client, query_configuration.url.bucket, new_key, query_configuration.url.version_id, query_configuration.request_settings)); - - query_configuration.keys.push_back(new_key); - configuration.keys.push_back(new_key); - } - else - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", - query_configuration.url.bucket, query_configuration.keys.back()); - } - } - - return std::make_shared( - query_configuration.format, - sample_block, - local_context, - format_settings, - chosen_compression_method, - query_configuration, - query_configuration.url.bucket, - query_configuration.keys.back()); - } -} - -void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) -{ - auto query_configuration = updateConfigurationAndGetCopy(local_context); - - if (query_configuration.withGlobs()) - { - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "S3 key '{}' contains globs, so the table is in readonly mode", - query_configuration.url.key); - } - - Aws::S3::Model::Delete delkeys; - - for (const auto & key : query_configuration.keys) - { - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(key); - delkeys.AddObjects(std::move(obj)); - } - - ProfileEvents::increment(ProfileEvents::S3DeleteObjects); - S3::DeleteObjectsRequest request; - request.SetBucket(query_configuration.url.bucket); - request.SetDelete(delkeys); - - auto response = query_configuration.client->DeleteObjects(request); - - const auto * response_error = response.IsSuccess() ? nullptr : &response.GetError(); - auto time_now = std::chrono::system_clock::now(); - if (auto blob_storage_log = BlobStorageLogWriter::create()) - { - for (const auto & key : query_configuration.keys) - blob_storage_log->addEvent(BlobStorageLogElement::EventType::Delete, query_configuration.url.bucket, key, {}, 0, response_error, time_now); - } - - if (!response.IsSuccess()) - { - const auto & err = response.GetError(); - throw S3Exception(err.GetMessage(), err.GetErrorType()); - } - - for (const auto & error : response.GetResult().GetErrors()) - LOG_WARNING(getLogger("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage()); -} - -StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(ContextPtr local_context) -{ - std::lock_guard lock(configuration_update_mutex); - configuration.update(local_context); - return configuration; -} - -void StorageS3::updateConfiguration(ContextPtr local_context) -{ - std::lock_guard lock(configuration_update_mutex); - configuration.update(local_context); -} - -void StorageS3::useConfiguration(const Configuration & new_configuration) -{ - std::lock_guard lock(configuration_update_mutex); - configuration = new_configuration; -} - -const StorageS3::Configuration & StorageS3::getConfiguration() -{ - std::lock_guard lock(configuration_update_mutex); - return configuration; -} - -bool StorageS3::Configuration::update(ContextPtr context) -{ - auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString()); - request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context->getSettings()); - - if (client && (static_configuration || !auth_settings.hasUpdates(s3_settings.auth_settings))) - return false; - - auth_settings.updateFrom(s3_settings.auth_settings); - keys[0] = url.key; - connect(context); - return true; -} - -void StorageS3::Configuration::connect(ContextPtr context) -{ - const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); - const Settings & local_settings = context->getSettingsRef(); - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - auth_settings.region, - context->getRemoteHostFilter(), - static_cast(global_settings.s3_max_redirects), - static_cast(global_settings.s3_retry_attempts), - global_settings.enable_s3_requests_logging, - /* for_disk_s3 = */ false, - request_settings.get_request_throttler, - request_settings.put_request_throttler, - url.uri.getScheme()); - - client_configuration.endpointOverride = url.endpoint; - client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; - auto headers = auth_settings.headers; - if (!headers_from_ast.empty()) - headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); - - client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; - - S3::ClientSettings client_settings{ - .use_virtual_addressing = url.is_virtual_hosted_style, - .disable_checksum = local_settings.s3_disable_checksum, - .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), - }; - - auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, auth_settings.secret_access_key, auth_settings.session_token); - client = S3::ClientFactory::instance().create( - client_configuration, - client_settings, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - auth_settings.server_side_encryption_customer_key_base64, - auth_settings.server_side_encryption_kms_config, - std::move(headers), - S3::CredentialsConfiguration{ - auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), - auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - auth_settings.expiration_window_seconds.value_or( - context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), - }); -} - -void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection) -{ - validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); - - auto filename = collection.getOrDefault("filename", ""); - if (!filename.empty()) - configuration.url = S3::URI(std::filesystem::path(collection.get("url")) / filename); - else - configuration.url = S3::URI(collection.get("url")); - - configuration.auth_settings.access_key_id = collection.getOrDefault("access_key_id", ""); - configuration.auth_settings.secret_access_key = collection.getOrDefault("secret_access_key", ""); - configuration.auth_settings.use_environment_credentials = collection.getOrDefault("use_environment_credentials", 1); - configuration.auth_settings.no_sign_request = collection.getOrDefault("no_sign_request", false); - configuration.auth_settings.expiration_window_seconds = collection.getOrDefault("expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS); - - configuration.format = collection.getOrDefault("format", configuration.format); - configuration.compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); - configuration.structure = collection.getOrDefault("structure", "auto"); - - configuration.request_settings = S3Settings::RequestSettings(collection); -} - -StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file) -{ - StorageS3::Configuration configuration; - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - processNamedCollectionResult(configuration, *named_collection); - } - else - { - /// Supported signatures: - /// - /// S3('url') - /// S3('url', 'format') - /// S3('url', 'format', 'compression') - /// S3('url', NOSIGN) - /// S3('url', NOSIGN, 'format') - /// S3('url', NOSIGN, 'format', 'compression') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'session_token', 'format', 'compression') - /// with optional headers() function - - size_t count = StorageURL::evalArgsAndCollectHeaders(engine_args, configuration.headers_from_ast, local_context); - - if (count == 0 || count > 6) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage S3 requires 1 to 5 arguments: " - "url, [NOSIGN | access_key_id, secret_access_key], name of used format and [compression_method]"); - - std::unordered_map engine_args_to_idx; - bool no_sign_request = false; - - /// For 2 arguments we support 2 possible variants: - /// - s3(source, format) - /// - s3(source, NOSIGN) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - no_sign_request = true; - else - engine_args_to_idx = {{"format", 1}}; - } - /// For 3 arguments we support 2 possible variants: - /// - s3(source, format, compression_method) - /// - s3(source, access_key_id, secret_access_key) - /// - s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or format name. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - engine_args_to_idx = {{"format", 2}}; - } - else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - engine_args_to_idx = {{"format", 1}, {"compression_method", 2}}; - else - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; - } - /// For 4 arguments we support 3 possible variants: - /// - s3(source, access_key_id, secret_access_key, session_token) - /// - s3(source, access_key_id, secret_access_key, format) - /// - s3(source, NOSIGN, format, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN or not. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(engine_args[1], "access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - engine_args_to_idx = {{"format", 2}, {"compression_method", 3}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "session_token/format"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; - } - else - { - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; - } - } - } - /// For 5 arguments we support 2 possible variants: - /// - s3(source, access_key_id, secret_access_key, session_token, format) - /// - s3(source, access_key_id, secret_access_key, format, compression) - else if (count == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "session_token/format"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"compression", 4}}; - } - else - { - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; - } - } - else if (count == 6) - { - engine_args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"compression_method", 5}}; - } - - /// This argument is always the first - configuration.url = S3::URI(checkAndGetLiteralArgument(engine_args[0], "url")); - - if (engine_args_to_idx.contains("format")) - configuration.format = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["format"]], "format"); - - if (engine_args_to_idx.contains("compression_method")) - configuration.compression_method = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["compression_method"]], "compression_method"); - - if (engine_args_to_idx.contains("access_key_id")) - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["access_key_id"]], "access_key_id"); - - if (engine_args_to_idx.contains("secret_access_key")) - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["secret_access_key"]], "secret_access_key"); - - if (engine_args_to_idx.contains("session_token")) - configuration.auth_settings.session_token = checkAndGetLiteralArgument(engine_args[engine_args_to_idx["session_token"]], "session_token"); - - if (no_sign_request) - configuration.auth_settings.no_sign_request = no_sign_request; - } - - configuration.static_configuration = !configuration.auth_settings.access_key_id.empty() || configuration.auth_settings.no_sign_request.has_value(); - - configuration.keys = {configuration.url.key}; - - if (configuration.format == "auto" && get_format_from_file) - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.key, true); - - return configuration; -} - -ColumnsDescription StorageS3::getTableStructureFromData( - const StorageS3::Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx) -{ - return getTableStructureFromDataImpl(configuration, format_settings, ctx); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - std::shared_ptr file_iterator_, - const StorageS3Source::KeysWithInfo & read_keys_, - const StorageS3::Configuration & configuration_, - const std::optional & format_settings_, - const ContextPtr & context_) - : WithContext(context_) - , file_iterator(file_iterator_) - , read_keys(read_keys_) - , configuration(configuration_) - , format_settings(format_settings_) - , prev_read_keys_size(read_keys_.size()) - { - } - - std::pair, std::optional> next() override - { - /// For default mode check cached columns for currently read keys on first iteration. - if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns}; - } - - while (true) - { - current_key_with_info = (*file_iterator)(); - - if (!current_key_with_info || current_key_with_info->key.empty()) - { - if (first) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, because there are no files with provided path " - "in S3 or all files are empty. You must specify table structure manually", - configuration.format); - - return {nullptr, std::nullopt}; - } - - /// S3 file iterator could get new keys after new iteration, check them in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT && read_keys.size() > prev_read_keys_size) - { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - prev_read_keys_size = read_keys.size(); - if (columns_from_cache) - return {nullptr, columns_from_cache}; - } - - if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info->info && current_key_with_info->info->size == 0) - continue; - - /// In union mode, check cached columns only for current key. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - StorageS3::KeysWithInfo keys = {current_key_with_info}; - if (auto columns_from_cache = tryGetColumnsFromCache(keys.begin(), keys.end())) - { - first = false; - return {nullptr, columns_from_cache}; - } - } - - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - auto impl = std::make_unique(configuration.client, configuration.url.bucket, current_key_with_info->key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings()); - if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof()) - { - first = false; - return {wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max), std::nullopt}; - } - } - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) - return; - - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key; - auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key; - auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addColumns(cache_key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3 - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - auto host_and_bucket = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket; - Strings sources; - sources.reserve(read_keys.size()); - std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem->key; }); - auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext()); - StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - String getLastFileName() const override - { - if (current_key_with_info) - return current_key_with_info->key; - return ""; - } - - private: - std::optional tryGetColumnsFromCache( - const StorageS3::KeysWithInfo::const_iterator & begin, - const StorageS3::KeysWithInfo::const_iterator & end) - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3) - return std::nullopt; - - auto & schema_cache = StorageS3::getSchemaCache(getContext()); - for (auto it = begin; it < end; ++it) - { - auto get_last_mod_time = [&] - { - time_t last_modification_time = 0; - if ((*it)->info) - { - last_modification_time = (*it)->info->last_modification_time; - } - else - { - /// Note that in case of exception in getObjectInfo returned info will be empty, - /// but schema cache will handle this case and won't return columns from cache - /// because we can't say that it's valid without last modification time. - last_modification_time = S3::getObjectInfo( - *configuration.client, - configuration.url.bucket, - (*it)->key, - configuration.url.version_id, - configuration.request_settings, - /*with_metadata=*/ false, - /*for_disk_s3=*/ false, - /*throw_on_error= */ false).last_modification_time; - } - - return last_modification_time ? std::make_optional(last_modification_time) : std::nullopt; - }; - - String path = fs::path(configuration.url.bucket) / (*it)->key; - String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path; - auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext()); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; - } - - std::shared_ptr file_iterator; - const StorageS3Source::KeysWithInfo & read_keys; - const StorageS3::Configuration & configuration; - const std::optional & format_settings; - StorageS3Source::KeyWithInfoPtr current_key_with_info; - size_t prev_read_keys_size; - bool first = true; - }; - -} - -ColumnsDescription StorageS3::getTableStructureFromDataImpl( - const Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx) -{ - KeysWithInfo read_keys; - - auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys); - - ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx); - return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx); -} - -void registerStorageS3Impl(const String & name, StorageFactory & factory) -{ - factory.registerStorage(name, [](const StorageFactory::Arguments & args) - { - auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - // Use format settings from global server context + settings from - // the SETTINGS clause of the create query. Settings from current - // session and user are ignored. - std::optional format_settings; - if (args.storage_def->settings) - { - FormatFactorySettings user_format_settings; - - // Apply changed settings from global context, but ignore the - // unknown ones, because we only have the format settings here. - const auto & changes = args.getContext()->getSettingsRef().changes(); - for (const auto & change : changes) - { - if (user_format_settings.has(change.name)) - user_format_settings.set(change.name, change.value); - } - - // Apply changes from SETTINGS clause, with validation. - user_format_settings.applyChanges(args.storage_def->settings->changes); - format_settings = getFormatSettings(args.getContext(), user_format_settings); - } - else - { - format_settings = getFormatSettings(args.getContext()); - } - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - return std::make_shared( - std::move(configuration), - args.getContext(), - args.table_id, - args.columns, - args.constraints, - args.comment, - format_settings, - /* distributed_processing_ */false, - partition_by); - }, - { - .supports_settings = true, - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); -} - -void registerStorageS3(StorageFactory & factory) -{ - return registerStorageS3Impl("S3", factory); -} - -void registerStorageCOS(StorageFactory & factory) -{ - return registerStorageS3Impl("COSN", factory); -} - -void registerStorageOSS(StorageFactory & factory) -{ - return registerStorageS3Impl("OSS", factory); -} - -NamesAndTypesList StorageS3::getVirtuals() const -{ - return virtual_columns; -} - -Names StorageS3::getVirtualColumnNames() -{ - return VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage({}).getNames(); -} - -bool StorageS3::supportsPartitionBy() const -{ - return true; -} - -SchemaCache & StorageS3::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_s3", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - -} - -#endif diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h deleted file mode 100644 index 81a03cc5ad5..00000000000 --- a/src/Storages/StorageS3.h +++ /dev/null @@ -1,399 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include - -#include - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace Aws::S3 -{ - class Client; -} - -namespace DB -{ - -class PullingPipelineExecutor; -class NamedCollection; - -class StorageS3Source : public SourceWithKeyCondition, WithContext -{ -public: - - struct KeyWithInfo - { - KeyWithInfo() = default; - - explicit KeyWithInfo(String key_, std::optional info_ = std::nullopt) - : key(std::move(key_)), info(std::move(info_)) {} - - virtual ~KeyWithInfo() = default; - - String key; - std::optional info; - }; - using KeyWithInfoPtr = std::shared_ptr; - - using KeysWithInfo = std::vector; - - class IIterator - { - public: - virtual ~IIterator() = default; - virtual KeyWithInfoPtr next(size_t idx = 0) = 0; /// NOLINT - - /// Estimates how many streams we need to process all files. - /// If keys count >= max_threads_count, the returned number may not represent the actual number of the keys. - /// Intended to be called before any next() calls, may underestimate otherwise - /// fixme: May underestimate if the glob has a strong filter, so there are few matches among the first 1000 ListObjects results. - virtual size_t estimatedKeysCount() = 0; - - KeyWithInfoPtr operator ()() { return next(); } - }; - - class DisclosedGlobIterator : public IIterator - { - public: - DisclosedGlobIterator( - const S3::Client & client_, - const S3::URI & globbed_uri_, - const ActionsDAG::Node * predicate, - const NamesAndTypesList & virtual_columns, - ContextPtr context, - KeysWithInfo * read_keys_ = nullptr, - const S3Settings::RequestSettings & request_settings_ = {}, - std::function progress_callback_ = {}); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class KeysIterator : public IIterator - { - public: - explicit KeysIterator( - const S3::Client & client_, - const std::string & version_id_, - const std::vector & keys_, - const String & bucket_, - const S3Settings::RequestSettings & request_settings_, - KeysWithInfo * read_keys = nullptr, - std::function progress_callback_ = {}); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - - class ReadTaskIterator : public IIterator - { - public: - explicit ReadTaskIterator(const ReadTaskCallback & callback_, size_t max_threads_count); - - KeyWithInfoPtr next(size_t idx = 0) override; /// NOLINT - size_t estimatedKeysCount() override; - - private: - KeysWithInfo buffer; - std::atomic_size_t index = 0; - - ReadTaskCallback callback; - }; - - StorageS3Source( - const ReadFromFormatInfo & info, - const String & format, - String name_, - ContextPtr context_, - std::optional format_settings_, - UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, - String compression_hint_, - const std::shared_ptr & client_, - const String & bucket, - const String & version_id, - const String & url_host_and_port, - std::shared_ptr file_iterator_, - size_t max_parsing_threads, - bool need_only_count_); - - ~StorageS3Source() override; - - String getName() const override; - - void setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nodes, ContextPtr context_) override - { - setKeyConditionImpl(nodes, context_, sample_block); - } - - Chunk generate() override; - -private: - friend class StorageS3QueueSource; - - String name; - String bucket; - String version_id; - String url_host_and_port; - String format; - ColumnsDescription columns_desc; - NamesAndTypesList requested_columns; - UInt64 max_block_size; - S3Settings::RequestSettings request_settings; - String compression_hint; - std::shared_ptr client; - Block sample_block; - std::optional format_settings; - - struct ReaderHolder - { - public: - ReaderHolder( - KeyWithInfoPtr key_with_info_, - String bucket_, - std::unique_ptr read_buf_, - std::shared_ptr source_, - std::unique_ptr pipeline_, - std::unique_ptr reader_) - : key_with_info(key_with_info_) - , bucket(std::move(bucket_)) - , read_buf(std::move(read_buf_)) - , source(std::move(source_)) - , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) - { - } - - ReaderHolder() = default; - ReaderHolder(const ReaderHolder & other) = delete; - ReaderHolder & operator=(const ReaderHolder & other) = delete; - - ReaderHolder(ReaderHolder && other) noexcept - { - *this = std::move(other); - } - - ReaderHolder & operator=(ReaderHolder && other) noexcept - { - /// The order of destruction is important. - /// reader uses pipeline, pipeline uses read_buf. - reader = std::move(other.reader); - pipeline = std::move(other.pipeline); - source = std::move(other.source); - read_buf = std::move(other.read_buf); - key_with_info = std::move(other.key_with_info); - bucket = std::move(other.bucket); - return *this; - } - - explicit operator bool() const { return reader != nullptr; } - PullingPipelineExecutor * operator->() { return reader.get(); } - const PullingPipelineExecutor * operator->() const { return reader.get(); } - String getPath() const { return fs::path(bucket) / key_with_info->key; } - const String & getFile() const { return key_with_info->key; } - const KeyWithInfo & getKeyWithInfo() const { return *key_with_info; } - std::optional getFileSize() const { return key_with_info->info ? std::optional(key_with_info->info->size) : std::nullopt; } - - const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } - - private: - KeyWithInfoPtr key_with_info; - String bucket; - std::unique_ptr read_buf; - std::shared_ptr source; - std::unique_ptr pipeline; - std::unique_ptr reader; - }; - - ReaderHolder reader; - - NamesAndTypesList requested_virtual_columns; - std::shared_ptr file_iterator; - size_t max_parsing_threads = 1; - bool need_only_count; - - LoggerPtr log = getLogger("StorageS3Source"); - - ThreadPool create_reader_pool; - ThreadPoolCallbackRunner create_reader_scheduler; - std::future reader_future; - std::atomic initialized{false}; - - size_t total_rows_in_file = 0; - - /// Notice: we should initialize reader and future_reader lazily in generate to make sure key_condition - /// is set before createReader is invoked for key_condition is read in createReader. - void lazyInitialize(size_t idx = 0); - - /// Recreate ReadBuffer and Pipeline for each file. - ReaderHolder createReader(size_t idx = 0); - std::future createReaderAsync(size_t idx = 0); - - std::unique_ptr createS3ReadBuffer(const String & key, size_t object_size); - std::unique_ptr createAsyncS3ReadBuffer(const String & key, const ReadSettings & read_settings, size_t object_size); - - void addNumRowsToCache(const String & key, size_t num_rows); - std::optional tryGetNumRowsFromCache(const KeyWithInfo & key_with_info); -}; - -/** - * This class represents table engine for external S3 urls. - * It sends HTTP GET to server when select is called and - * HTTP PUT when insert is called. - */ -class StorageS3 : public IStorage -{ -public: - struct Configuration : public StatelessTableEngineConfiguration - { - Configuration() = default; - - String getPath() const { return url.key; } - - bool update(ContextPtr context); - - void connect(ContextPtr context); - - bool withGlobs() const { return url.key.find_first_of("*?{") != std::string::npos; } - - bool withWildcard() const - { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return url.bucket.find(PARTITION_ID_WILDCARD) != String::npos - || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; - } - - S3::URI url; - S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; - /// If s3 configuration was passed from ast, then it is static. - /// If from config - it can be changed with config reload. - bool static_configuration = true; - /// Headers from ast is a part of static configuration. - HTTPHeaderEntries headers_from_ast; - - std::shared_ptr client; - std::vector keys; - }; - - StorageS3( - const Configuration & configuration_, - ContextPtr context_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - std::optional format_settings_, - bool distributed_processing_ = false, - ASTPtr partition_by_ = nullptr); - - String getName() const override - { - return name; - } - - void read( - 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 write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context, bool async_insert) override; - - void truncate(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, TableExclusiveLockHolder &) override; - - NamesAndTypesList getVirtuals() const override; - static Names getVirtualColumnNames(); - - bool supportsPartitionBy() const override; - - static void processNamedCollectionResult(StorageS3::Configuration & configuration, const NamedCollection & collection); - - static SchemaCache & getSchemaCache(const ContextPtr & ctx); - - static StorageS3::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true); - - static ColumnsDescription getTableStructureFromData( - const StorageS3::Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx); - - using KeysWithInfo = StorageS3Source::KeysWithInfo; - - bool supportsTrivialCountOptimization() const override { return true; } - -protected: - virtual Configuration updateConfigurationAndGetCopy(ContextPtr local_context); - - virtual void updateConfiguration(ContextPtr local_context); - - void useConfiguration(const Configuration & new_configuration); - - const Configuration & getConfiguration(); - -private: - friend class StorageS3Cluster; - friend class TableFunctionS3Cluster; - friend class StorageS3Queue; - friend class ReadFromStorageS3Step; - - Configuration configuration; - std::mutex configuration_update_mutex; - NamesAndTypesList virtual_columns; - - String name; - const bool distributed_processing; - std::optional format_settings; - ASTPtr partition_by; - - static ColumnsDescription getTableStructureFromDataImpl( - const Configuration & configuration, - const std::optional & format_settings, - ContextPtr ctx); - - bool supportsSubcolumns() const override { return true; } - - bool supportsSubsetOfColumns(const ContextPtr & context) const; - - bool prefersLargeBlocks() const override; - - bool parallelizeOutputAfterReading(ContextPtr context) const override; -}; - -} - -#endif diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp deleted file mode 100644 index 25c2b42b766..00000000000 --- a/src/Storages/StorageS3Cluster.cpp +++ /dev/null @@ -1,103 +0,0 @@ -#include "Storages/StorageS3Cluster.h" - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -StorageS3Cluster::StorageS3Cluster( - const String & cluster_name_, - const StorageS3::Configuration & configuration_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_) - : IStorageCluster(cluster_name_, table_id_, getLogger("StorageS3Cluster (" + table_id_.table_name + ")"), structure_argument_was_provided_) - , s3_configuration{configuration_} -{ - context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri); - context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast); - - StorageInMemoryMetadata storage_metadata; - updateConfigurationIfChanged(context_); - - if (columns_.empty()) - { - /// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function - auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, /*format_settings=*/std::nullopt, context_); - storage_metadata.setColumns(columns); - } - else - storage_metadata.setColumns(columns_); - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) -{ - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query)); - - TableFunctionS3Cluster::addColumnsStructureToArguments(expression_list->children, structure, context); -} - -void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context) -{ - s3_configuration.update(local_context); -} - -RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const -{ - auto iterator = std::make_shared( - *s3_configuration.client, s3_configuration.url, predicate, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback()); - - auto callback = std::make_shared>([iterator]() mutable -> String - { - if (auto next = iterator->next()) - return next->key; - return ""; - }); - return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; -} - -NamesAndTypesList StorageS3Cluster::getVirtuals() const -{ - return virtual_columns; -} - - -} - -#endif diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h deleted file mode 100644 index c526f14834a..00000000000 --- a/src/Storages/StorageS3Cluster.h +++ /dev/null @@ -1,58 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include - -#include "Client/Connection.h" -#include -#include -#include -#include - -namespace DB -{ - -class Context; - -class StorageS3Cluster : public IStorageCluster -{ -public: - StorageS3Cluster( - const String & cluster_name_, - const StorageS3::Configuration & configuration_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_); - - std::string getName() const override { return "S3Cluster"; } - - NamesAndTypesList getVirtuals() const override; - - RemoteQueryExecutor::Extension getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const override; - - bool supportsSubcolumns() const override { return true; } - - bool supportsTrivialCountOptimization() const override { return true; } - -protected: - void updateConfigurationIfChanged(ContextPtr local_context); - -private: - void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); } - - void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override; - - StorageS3::Configuration s3_configuration; - NamesAndTypesList virtual_columns; -}; - - -} - -#endif diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index 1426ea83800..77d5be3698c 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -1,9 +1,7 @@ #include #include -#include #include -#include -#include +#include #include #include #include @@ -83,7 +81,7 @@ void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, C #endif fillDataImpl(res_columns, StorageURL::getSchemaCache(context), "URL"); #if USE_AZURE_BLOB_STORAGE - fillDataImpl(res_columns, StorageAzureBlob::getSchemaCache(context), "Azure"); + fillDataImpl(res_columns, StorageAzureBlobStorage::getSchemaCache(context), "Azure"); /// FIXME #endif } diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index dea9feaf28b..0b72d7e94fd 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -45,8 +45,6 @@ void registerStorageIceberg(StorageFactory & factory); #endif #if USE_HDFS -void registerStorageHDFS(StorageFactory & factory); - #if USE_HIVE void registerStorageHive(StorageFactory & factory); #endif @@ -99,9 +97,7 @@ void registerStorageSQLite(StorageFactory & factory); void registerStorageKeeperMap(StorageFactory & factory); -#if USE_AZURE_BLOB_STORAGE -void registerStorageAzureBlob(StorageFactory & factory); -#endif +void registerStorageObjectStorage(StorageFactory & factory); void registerStorages() { @@ -131,9 +127,7 @@ void registerStorages() #endif #if USE_AWS_S3 - registerStorageS3(factory); - registerStorageCOS(factory); - registerStorageOSS(factory); + // registerStorageS3(factory); registerStorageHudi(factory); registerStorageS3Queue(factory); @@ -148,12 +142,9 @@ void registerStorages() #endif #if USE_HDFS - registerStorageHDFS(factory); - #if USE_HIVE registerStorageHive(factory); #endif - #endif registerStorageODBC(factory); @@ -201,9 +192,7 @@ void registerStorages() registerStorageKeeperMap(factory); - #if USE_AZURE_BLOB_STORAGE - registerStorageAzureBlob(factory); - #endif + registerStorageObjectStorage(factory); } } diff --git a/src/TableFunctions/ITableFunctionCluster.h b/src/TableFunctions/ITableFunctionCluster.h index 7e81d6d21b7..0559472325b 100644 --- a/src/TableFunctions/ITableFunctionCluster.h +++ b/src/TableFunctions/ITableFunctionCluster.h @@ -1,14 +1,10 @@ #pragma once -#include "config.h" - #include #include -#include #include #include -#include -#include +#include namespace DB diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 961e5683fe2..884e1f5c4a2 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -10,6 +10,9 @@ # include # include # include +#include +#include +#include namespace DB { @@ -30,12 +33,13 @@ protected: bool /*is_insert_query*/) const override { ColumnsDescription columns; - if (TableFunction::configuration.structure != "auto") - columns = parseColumnsListFromString(TableFunction::configuration.structure, context); + if (TableFunction::configuration->structure != "auto") + columns = parseColumnsListFromString(TableFunction::configuration->structure, context); - StoragePtr storage = Storage::create( - TableFunction::configuration, context, false, StorageID(TableFunction::getDatabaseName(), table_name), - columns, ConstraintsDescription{}, String{}, std::nullopt); + StorageObjectStorageConfigurationPtr configuration = TableFunction::configuration; + StoragePtr storage = StorageIceberg>::create( + configuration, context, "", StorageID(TableFunction::getDatabaseName(), table_name), + columns, ConstraintsDescription{}, String{}, std::nullopt, false); storage->startup(); return storage; @@ -45,19 +49,19 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const override { - if (TableFunction::configuration.structure == "auto") + if (TableFunction::configuration->structure == "auto") { context->checkAccess(TableFunction::getSourceAccessType()); - return Storage::getTableStructureFromData(TableFunction::configuration, std::nullopt, context); + return Storage::getTableStructureFromData(TableFunction::object_storage, TableFunction::configuration, std::nullopt, context); } - return parseColumnsListFromString(TableFunction::configuration.structure, context); + return parseColumnsListFromString(TableFunction::configuration->structure, context); } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override { /// Set default format to Parquet if it's not specified in arguments. - TableFunction::configuration.format = "Parquet"; + TableFunction::configuration->format = "Parquet"; TableFunction::parseArguments(ast_function, context); } }; diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp b/src/TableFunctions/TableFunctionAzureBlobStorage.cpp deleted file mode 100644 index b098cac5144..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.cpp +++ /dev/null @@ -1,323 +0,0 @@ -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; -} - -namespace -{ - -bool isConnectionString(const std::string & candidate) -{ - return !candidate.starts_with("http"); -} - -} - -void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) -{ - /// Supported signatures: - /// - /// AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]) - /// - - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - { - StorageAzureBlob::processNamedCollectionResult(configuration, *named_collection); - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - } - else - { - if (engine_args.size() < 3 || engine_args.size() > 8) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - - for (auto & engine_arg : engine_args) - engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); - - std::unordered_map engine_args_to_idx; - - configuration.connection_url = checkAndGetLiteralArgument(engine_args[0], "connection_string/storage_account_url"); - configuration.is_connection_string = isConnectionString(configuration.connection_url); - - configuration.container = checkAndGetLiteralArgument(engine_args[1], "container"); - configuration.blob_path = checkAndGetLiteralArgument(engine_args[2], "blobpath"); - - auto is_format_arg - = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; - - if (engine_args.size() == 4) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name/structure"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - } - else - { - configuration.structure = fourth_arg; - } - } - else if (engine_args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - } - } - else if (engine_args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - if (is_format_arg(fourth_arg)) - { - configuration.format = fourth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[4], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[5], "structure"); - } - else - { - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - auto sixth_arg = checkAndGetLiteralArgument(engine_args[5], "format/account_name/structure"); - if (is_format_arg(sixth_arg)) - configuration.format = sixth_arg; - else - configuration.structure = sixth_arg; - } - } - else if (engine_args.size() == 7) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - 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); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - } - else if (engine_args.size() == 8) - { - auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); - configuration.account_name = fourth_arg; - configuration.account_key = checkAndGetLiteralArgument(engine_args[4], "account_key"); - 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); - configuration.format = sixth_arg; - configuration.compression_method = checkAndGetLiteralArgument(engine_args[6], "compression"); - configuration.structure = checkAndGetLiteralArgument(engine_args[7], "structure"); - } - - configuration.blobs_paths = {configuration.blob_path}; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true); - } -} - -void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - /// Clone ast function, because we can modify its arguments like removing headers. - auto ast_copy = ast_function->clone(); - - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - auto & args = args_func.at(0)->children; - - parseArgumentsImpl(args, context); -} - -void TableFunctionAzureBlobStorage::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) -{ - if (tryGetNamedCollectionWithOverrides(args, context)) - { - /// In case of named collection, just add key-value pair "structure='...'" - /// at the end of arguments to override existed structure. - ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure)}; - auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); - args.push_back(equal_func); - } - else - { - if (args.size() < 3 || args.size() > 8) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage Azure requires 3 to 7 arguments: " - "AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); - - auto structure_literal = std::make_shared(structure); - - auto is_format_arg - = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; - - - if (args.size() == 3) - { - /// Add format=auto & compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - else if (args.size() == 4) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); - if (is_format_arg(fourth_arg)) - { - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - else - { - args.back() = structure_literal; - } - } - else if (args.size() == 5) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - if (!is_format_arg(fourth_arg)) - { - /// Add format=auto & compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(std::make_shared("auto")); - } - args.push_back(structure_literal); - } - else if (args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - if (!is_format_arg(fourth_arg)) - { - /// Add compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - else - { - args.back() = structure_literal; - } - } - else if (args.size() == 7) - { - args.push_back(structure_literal); - } - else if (args.size() == 8) - { - args.back() = structure_literal; - } - } -} - -ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(ContextPtr context, bool is_insert_query) const -{ - if (configuration.structure == "auto") - { - context->checkAccess(getSourceAccessType()); - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - auto object_storage = std::make_unique("AzureBlobStorageTableFunction", std::move(client), std::move(settings), configuration.container); - return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context, false); - } - - return parseColumnsListFromString(configuration.structure, context); -} - -bool TableFunctionAzureBlobStorage::supportsReadingSubsetOfColumns(const ContextPtr & context) -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); -} - -std::unordered_set TableFunctionAzureBlobStorage::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageAzureBlob::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - -StoragePtr TableFunctionAzureBlobStorage::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const -{ - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - ColumnsDescription columns; - if (configuration.structure != "auto") - columns = parseColumnsListFromString(configuration.structure, context); - else if (!structure_hint.empty()) - columns = structure_hint; - - StoragePtr storage = std::make_shared( - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - String{}, - /// No format_settings for table function Azure - std::nullopt, - /* distributed_processing */ false, - nullptr); - - storage->startup(); - - return storage; -} - -void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory) -{ - 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])", ""}}}, - .allow_readonly = false}); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorage.h b/src/TableFunctions/TableFunctionAzureBlobStorage.h deleted file mode 100644 index 1a221f60c55..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorage.h +++ /dev/null @@ -1,80 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include - - -namespace DB -{ - -class Context; - -/* AzureBlob(source, [access_key_id, secret_access_key,] [format, compression, structure]) - creates a temporary storage for a file in AzureBlob. - */ -class TableFunctionAzureBlobStorage : public ITableFunction -{ -public: - static constexpr auto name = "azureBlobStorage"; - - static constexpr auto signature = " - connection_string, container_name, blobpath\n" - " - connection_string, container_name, blobpath, structure \n" - " - connection_string, container_name, blobpath, format \n" - " - connection_string, container_name, blobpath, format, compression \n" - " - connection_string, container_name, blobpath, format, compression, structure \n" - " - storage_account_url, container_name, blobpath, account_name, account_key\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" - " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; - - static size_t getMaxNumberOfArguments() { return 8; } - - String getName() const override - { - return name; - } - - virtual String getSignature() const - { - return signature; - } - - bool hasStaticStructure() const override { return configuration.structure != "auto"; } - - bool needStructureHint() const override { return configuration.structure == "auto"; } - - void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - - bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; - - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - - static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); - -protected: - - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "Azure"; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - mutable StorageAzureBlob::Configuration configuration; - ColumnsDescription structure_hint; -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp deleted file mode 100644 index 1c3b302a186..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.cpp +++ /dev/null @@ -1,85 +0,0 @@ -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include - -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl( - const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const -{ - StoragePtr storage; - ColumnsDescription columns; - bool structure_argument_was_provided = configuration.structure != "auto"; - - if (structure_argument_was_provided) - { - columns = parseColumnsListFromString(configuration.structure, context); - } - else if (!structure_hint.empty()) - { - columns = structure_hint; - } - - auto client = StorageAzureBlob::createClient(configuration, !is_insert_query); - auto settings = StorageAzureBlob::createSettings(context); - - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this filename won't contains globs - storage = std::make_shared( - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings - /* distributed_processing */ true, - /*partition_by_=*/nullptr); - } - else - { - storage = std::make_shared( - cluster_name, - configuration, - std::make_unique(table_name, std::move(client), std::move(settings), configuration.container), - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - context, - structure_argument_was_provided); - } - - storage->startup(); - - return storage; -} - - -void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory) -{ - 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.)", - .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, - .allow_readonly = false} - ); -} - - -} - -#endif diff --git a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h b/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h deleted file mode 100644 index 58f79328f63..00000000000 --- a/src/TableFunctions/TableFunctionAzureBlobStorageCluster.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AZURE_BLOB_STORAGE - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * azureBlobStorageCluster(cluster_name, source, [access_key_id, secret_access_key,] format, compression_method, structure) - * A table function, which allows to process many files from Azure Blob Storage on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in Azure Blob Storage file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionAzureBlobStorageCluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "azureBlobStorageCluster"; - static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "AzureBlobStorageCluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionDeltaLake.cpp b/src/TableFunctions/TableFunctionDeltaLake.cpp index b8bf810f6fa..08b62ed2612 100644 --- a/src/TableFunctions/TableFunctionDeltaLake.cpp +++ b/src/TableFunctions/TableFunctionDeltaLake.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include "registerTableFunctions.h" namespace DB @@ -16,17 +16,17 @@ struct TableFunctionDeltaLakeName static constexpr auto name = "deltaLake"; }; -using TableFunctionDeltaLake = ITableFunctionDataLake; - -void registerTableFunctionDeltaLake(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation = { - .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", - .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} +// using TableFunctionDeltaLake = ITableFunctionDataLake; +// +// void registerTableFunctionDeltaLake(TableFunctionFactory & factory) +// { +// factory.registerFunction( +// {.documentation = { +// .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", +// .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, +// .categories{"DataLake"}}, +// .allow_readonly = false}); +// } } diff --git a/src/TableFunctions/TableFunctionHDFS.cpp b/src/TableFunctions/TableFunctionHDFS.cpp deleted file mode 100644 index 8d48a7ba30e..00000000000 --- a/src/TableFunctions/TableFunctionHDFS.cpp +++ /dev/null @@ -1,54 +0,0 @@ -#include "config.h" -#include "registerTableFunctions.h" - -#if USE_HDFS -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -StoragePtr TableFunctionHDFS::getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const -{ - return std::make_shared( - source, - StorageID(getDatabaseName(), table_name), - format_, - columns, - ConstraintsDescription{}, - String{}, - global_context, - compression_method_); -} - -ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const -{ - if (structure == "auto") - { - context->checkAccess(getSourceAccessType()); - return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context); - } - - return parseColumnsListFromString(structure, context); -} - -std::unordered_set TableFunctionHDFS::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageHDFS::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - -void registerTableFunctionHDFS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} -#endif diff --git a/src/TableFunctions/TableFunctionHDFS.h b/src/TableFunctions/TableFunctionHDFS.h deleted file mode 100644 index 3a719496b26..00000000000 --- a/src/TableFunctions/TableFunctionHDFS.h +++ /dev/null @@ -1,50 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include - - -namespace DB -{ - -class Context; - -/* hdfs(URI, [format, structure, compression]) - creates a temporary storage from hdfs files - * - */ -class TableFunctionHDFS : public ITableFunctionFileLike -{ -public: - static constexpr auto name = "hdfs"; - static constexpr auto signature = " - uri\n" - " - uri, format\n" - " - uri, format, structure\n" - " - uri, format, structure, compression_method\n"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - -private: - StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const override; - const char * getStorageTypeName() const override { return "HDFS"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp deleted file mode 100644 index 6fb7ed0fce5..00000000000 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ /dev/null @@ -1,61 +0,0 @@ -#include "config.h" - -#if USE_HDFS - -#include -#include - -#include -#include -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionHDFSCluster::getStorage( - const String & /*source*/, const String & /*format_*/, const ColumnsDescription & columns, ContextPtr context, - const std::string & table_name, const String & /*compression_method_*/) const -{ - StoragePtr storage; - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this uri won't contains globs - storage = std::make_shared( - filename, - StorageID(getDatabaseName(), table_name), - format, - columns, - ConstraintsDescription{}, - String{}, - context, - compression_method, - /*distributed_processing=*/true, - nullptr); - } - else - { - storage = std::make_shared( - context, - cluster_name, - filename, - StorageID(getDatabaseName(), table_name), - format, - columns, - ConstraintsDescription{}, - compression_method, - structure != "auto"); - } - return storage; -} - -void registerTableFunctionHDFSCluster(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHDFSCluster.h b/src/TableFunctions/TableFunctionHDFSCluster.h deleted file mode 100644 index 0253217feb7..00000000000 --- a/src/TableFunctions/TableFunctionHDFSCluster.h +++ /dev/null @@ -1,54 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_HDFS - -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * hdfsCluster(cluster, URI, format, structure, compression_method) - * A table function, which allows to process many files from HDFS on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in HDFS file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionHDFSCluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "hdfsCluster"; - static constexpr auto signature = " - cluster_name, uri\n" - " - cluster_name, uri, format\n" - " - cluster_name, uri, format, structure\n" - " - cluster_name, uri, format, structure, compression_method\n"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr getStorage( - const String & source, const String & format_, const ColumnsDescription & columns, ContextPtr global_context, - const std::string & table_name, const String & compression_method_) const override; - - const char * getStorageTypeName() const override { return "HDFSCluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index 436e708b72d..c6d84504c40 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include "registerTableFunctions.h" namespace DB @@ -15,17 +15,17 @@ struct TableFunctionHudiName { static constexpr auto name = "hudi"; }; -using TableFunctionHudi = ITableFunctionDataLake; - -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)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} +// using TableFunctionHudi = ITableFunctionDataLake; +// +// 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)", ""}}, +// .categories{"DataLake"}}, +// .allow_readonly = false}); +// } } #endif diff --git a/src/TableFunctions/TableFunctionIceberg.cpp b/src/TableFunctions/TableFunctionIceberg.cpp index d37aace01c6..1a28f9292d1 100644 --- a/src/TableFunctions/TableFunctionIceberg.cpp +++ b/src/TableFunctions/TableFunctionIceberg.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include "registerTableFunctions.h" @@ -17,7 +17,10 @@ struct TableFunctionIcebergName static constexpr auto name = "iceberg"; }; -using TableFunctionIceberg = ITableFunctionDataLake; +using TableFunctionIceberg = ITableFunctionDataLake< + TableFunctionIcebergName, + StorageIceberg, + TableFunctionS3>; void registerTableFunctionIceberg(TableFunctionFactory & factory) { diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp new file mode 100644 index 00000000000..d009a9347f3 --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -0,0 +1,224 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +static void initializeConfiguration( + StorageObjectStorageConfiguration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); +} + +template +ObjectStoragePtr TableFunctionObjectStorage::getObjectStorage(const ContextPtr & context, bool create_readonly) const +{ + if (!object_storage) + object_storage = configuration->createOrUpdateObjectStorage(context, create_readonly); + return object_storage; +} + +template +std::vector TableFunctionObjectStorage::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +{ + auto & table_function_node = query_node_table_function->as(); + auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); + size_t table_function_arguments_size = table_function_arguments_nodes.size(); + + std::vector result; + for (size_t i = 0; i < table_function_arguments_size; ++i) + { + auto * function_node = table_function_arguments_nodes[i]->as(); + if (function_node && function_node->getFunctionName() == "headers") + result.push_back(i); + } + return result; +} + +template +void TableFunctionObjectStorage::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) +{ + Configuration::addStructureToArgs(args, structure, context); +} + +template +void TableFunctionObjectStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) +{ + configuration = std::make_shared(); + initializeConfiguration(*configuration, engine_args, local_context, true); +} + +template +void TableFunctionObjectStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Clone ast function, because we can modify its arguments like removing headers. + auto ast_copy = ast_function->clone(); + ASTs & args_func = ast_copy->children; + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); + + auto & args = args_func.at(0)->children; + parseArgumentsImpl(args, context); +} + +template +ColumnsDescription TableFunctionObjectStorage::getActualTableStructure(ContextPtr context, bool is_insert_query) const +{ + if (configuration->structure == "auto") + { + context->checkAccess(getSourceAccessType()); + auto storage = getObjectStorage(context, !is_insert_query); + return StorageObjectStorage::getTableStructureFromData(storage, configuration, std::nullopt, context); + } + + return parseColumnsListFromString(configuration->structure, context); +} + +template +bool TableFunctionObjectStorage::supportsReadingSubsetOfColumns(const ContextPtr & context) +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); +} + +template +std::unordered_set TableFunctionObjectStorage::getVirtualsToCheckBeforeUsingStructureHint() const +{ + auto virtual_column_names = StorageObjectStorage::getVirtualColumnNames(); + return {virtual_column_names.begin(), virtual_column_names.end()}; +} + +template +StoragePtr TableFunctionObjectStorage::executeImpl( + const ASTPtr & /* ast_function */, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const +{ + ColumnsDescription columns; + if (configuration->structure != "auto") + columns = parseColumnsListFromString(configuration->structure, context); + else if (!structure_hint.empty()) + columns = structure_hint; + else if (!cached_columns.empty()) + columns = cached_columns; + + StoragePtr storage = std::make_shared>( + configuration, + getObjectStorage(context, !is_insert_query), + Definition::storage_type_name, + context, + StorageID(getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + String{}, + /// No format_settings for table function Azure + std::nullopt, + /* distributed_processing */ false, + nullptr); + + storage->startup(); + return storage; +} + +void registerTableFunctionObjectStorage(TableFunctionFactory & 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)", ""} + }, + .categories{"DataLake"}}, + .allow_readonly = false + }); + + factory.registerFunction>( + { + .allow_readonly = false + }); + + factory.registerFunction>( + { + .allow_readonly = false + }); + factory.registerFunction>( + { + .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])", "" + }} + }, + .allow_readonly = false + }); +#endif +#if USE_HDFS + factory.registerFunction>( + { + .allow_readonly = false + }); +#endif +} + +#if USE_AZURE_BLOB_STORAGE +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; +#endif + +#if USE_HDFS +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +#endif + +} diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h new file mode 100644 index 00000000000..1df0ba2f843 --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -0,0 +1,150 @@ +#pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + +#include +#include +#include + + +namespace DB +{ + +class Context; +class StorageS3Configuration; +class StorageAzureBlobConfiguration; +class StorageHDFSConfiguration; +struct S3StorageSettings; +struct AzureStorageSettings; +struct HDFSStorageSettings; + +struct AzureDefinition +{ + static constexpr auto name = "azureBlobStorage"; + static constexpr auto storage_type_name = "Azure"; + static constexpr auto signature = " - connection_string, container_name, blobpath\n" + " - connection_string, container_name, blobpath, structure \n" + " - connection_string, container_name, blobpath, format \n" + " - connection_string, container_name, blobpath, format, compression \n" + " - connection_string, container_name, blobpath, format, compression, structure \n" + " - storage_account_url, container_name, blobpath, account_name, account_key\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, structure\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" + " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; +}; + +struct S3Definition +{ + static constexpr auto name = "s3"; + static constexpr auto storage_type_name = "S3"; + static constexpr auto signature = " - url\n" + " - url, format\n" + " - url, format, structure\n" + " - url, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key\n" + " - url, access_key_id, secret_access_key, session_token\n" + " - url, access_key_id, secret_access_key, format\n" + " - url, access_key_id, secret_access_key, session_token, format\n" + " - url, access_key_id, secret_access_key, format, structure\n" + " - url, access_key_id, secret_access_key, session_token, format, structure\n" + " - url, access_key_id, secret_access_key, format, structure, compression_method\n" + " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; +}; + +struct GCSDefinition +{ + static constexpr auto name = "gcs"; + static constexpr auto storage_type_name = "GCS"; + static constexpr auto signature = S3Definition::signature; +}; + +struct COSNDefinition +{ + static constexpr auto name = "cosn"; + static constexpr auto storage_type_name = "COSN"; + static constexpr auto signature = S3Definition::signature; +}; + +struct OSSDefinition +{ + static constexpr auto name = "oss"; + static constexpr auto storage_type_name = "OSS"; + static constexpr auto signature = S3Definition::signature; +}; + +struct HDFSDefinition +{ + static constexpr auto name = "hdfs"; + static constexpr auto storage_type_name = "HDFS"; + static constexpr auto signature = " - uri\n" + " - uri, format\n" + " - uri, format, structure\n" + " - uri, format, structure, compression_method\n"; +}; + +template +class TableFunctionObjectStorage : public ITableFunction +{ +public: + static constexpr auto name = Definition::name; + static constexpr auto signature = Definition::signature; + + static size_t getMaxNumberOfArguments() { return 8; } + + String getName() const override { return name; } + + virtual String getSignature() const { return signature; } + + bool hasStaticStructure() const override { return configuration->structure != "auto"; } + + bool needStructureHint() const override { return configuration->structure == "auto"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; + + std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; + + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + + static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return Definition::storage_type_name; } + + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + ObjectStoragePtr getObjectStorage(const ContextPtr & context, bool create_readonly) const; + + mutable typename StorageObjectStorage::ConfigurationPtr configuration; + mutable ObjectStoragePtr object_storage; + ColumnsDescription structure_hint; + + std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; +}; + +#if USE_AWS_S3 +using TableFunctionS3 = TableFunctionObjectStorage; +#endif + +#if USE_AZURE_BLOB_STORAGE +using TableFunctionAzureBlob = TableFunctionObjectStorage; +#endif + +#if USE_HDFS +using TableFunctionHDFS = TableFunctionObjectStorage; +#endif +} + +#endif diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp new file mode 100644 index 00000000000..1d27a857cea --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -0,0 +1,113 @@ +#include "config.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +template +StoragePtr TableFunctionObjectStorageCluster::executeImpl( + const ASTPtr & /*function*/, ContextPtr context, + const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const +{ + using Base = TableFunctionObjectStorage; + + StoragePtr storage; + ColumnsDescription columns; + bool structure_argument_was_provided = Base::configuration->structure != "auto"; + + if (structure_argument_was_provided) + { + columns = parseColumnsListFromString(Base::configuration->structure, context); + } + else if (!Base::structure_hint.empty()) + { + columns = Base::structure_hint; + } + + if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) + { + /// On worker node this filename won't contains globs + storage = std::make_shared>( + Base::configuration, + Base::configuration->createOrUpdateObjectStorage(context, !is_insert_query), + Definition::storage_type_name, + context, + StorageID(Base::getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + /* comment */String{}, + /* format_settings */std::nullopt, /// No format_settings + /* distributed_processing */ true, + /*partition_by_=*/nullptr); + } + else + { + storage = std::make_shared>( + ITableFunctionCluster::cluster_name, + Base::configuration, + Base::configuration->createOrUpdateObjectStorage(context, !is_insert_query), + Definition::storage_type_name, + StorageID(Base::getDatabaseName(), table_name), + columns, + ConstraintsDescription{}, + context, + structure_argument_was_provided); + } + + storage->startup(); + return storage; +} + + +void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) +{ +#if USE_AWS_S3 + 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.)", + .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, + .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.)", + .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, + .allow_readonly = false + } + ); +#endif + +#if USE_HDFS + factory.registerFunction(); +#endif +} + +#if USE_AWS_S3 +template class TableFunctionObjectStorageCluster; +#endif + +#if USE_AZURE_BLOB_STORAGE +template class TableFunctionObjectStorageCluster; +#endif + +#if USE_HDFS +template class TableFunctionObjectStorageCluster; +#endif +} diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h new file mode 100644 index 00000000000..461456e37df --- /dev/null +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -0,0 +1,91 @@ +#pragma once +#include "config.h" +#include +#include +#include + + +namespace DB +{ + +class Context; + +class StorageS3Settings; +class StorageAzureBlobSettings; +class StorageS3Configuration; +class StorageAzureBlobConfiguration; + +struct AzureClusterDefinition +{ + /** + * azureBlobStorageCluster(cluster_name, source, [access_key_id, secret_access_key,] format, compression_method, structure) + * A table function, which allows to process many files from Azure Blob Storage on a specific cluster + * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks + * in Azure Blob Storage file path and dispatch each file dynamically. + * On worker node it asks initiator about next task to process, processes it. + * This is repeated until the tasks are finished. + */ + static constexpr auto name = "azureBlobStorageCluster"; + static constexpr auto storage_type_name = "AzureBlobStorageCluster"; + static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; +}; + +struct S3ClusterDefinition +{ + static constexpr auto name = "s3Cluster"; + static constexpr auto storage_type_name = "S3Cluster"; + static constexpr auto signature = " - cluster, url\n" + " - cluster, url, format\n" + " - cluster, url, format, structure\n" + " - cluster, url, access_key_id, secret_access_key\n" + " - cluster, url, format, structure, compression_method\n" + " - cluster, url, access_key_id, secret_access_key, format\n" + " - cluster, url, access_key_id, secret_access_key, format, structure\n" + " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" + " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" + "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; +}; + +struct HDFSClusterDefinition +{ + static constexpr auto name = "hdfsCluster"; + static constexpr auto storage_type_name = "HDFSCluster"; + static constexpr auto signature = " - cluster_name, uri\n" + " - cluster_name, uri, format\n" + " - cluster_name, uri, format, structure\n" + " - cluster_name, uri, format, structure, compression_method\n"; +}; + +template +class TableFunctionObjectStorageCluster : public ITableFunctionCluster> +{ +public: + static constexpr auto name = Definition::name; + static constexpr auto signature = Definition::signature; + + String getName() const override { return name; } + String getSignature() const override { return signature; } + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return Definition::storage_type_name; } +}; + +#if USE_AWS_S3 +using TableFunctionS3Cluster = TableFunctionObjectStorageCluster; +#endif + +#if USE_AZURE_BLOB_STORAGE +using TableFunctionAzureBlobCluster = TableFunctionObjectStorageCluster; +#endif + +#if USE_HDFS +using TableFunctionHDFSCluster = TableFunctionObjectStorageCluster; +#endif +} diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp deleted file mode 100644 index a9c5a5c99f0..00000000000 --- a/src/TableFunctions/TableFunctionS3.cpp +++ /dev/null @@ -1,464 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include "registerTableFunctions.h" -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int LOGICAL_ERROR; -} - - -std::vector TableFunctionS3::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const -{ - auto & table_function_node = query_node_table_function->as(); - auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); - size_t table_function_arguments_size = table_function_arguments_nodes.size(); - - std::vector result; - - for (size_t i = 0; i < table_function_arguments_size; ++i) - { - auto * function_node = table_function_arguments_nodes[i]->as(); - if (function_node && function_node->getFunctionName() == "headers") - result.push_back(i); - } - - return result; -} - -/// This is needed to avoid copy-paste. Because s3Cluster arguments only differ in additional argument (first) - cluster name -void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(args, context)) - { - StorageS3::processNamedCollectionResult(configuration, *named_collection); - if (configuration.format == "auto") - { - String file_path = named_collection->getOrDefault("filename", Poco::URI(named_collection->get("url")).getPath()); - configuration.format = FormatFactory::instance().getFormatFromFileName(file_path, true); - } - } - else - { - - size_t count = StorageURL::evalArgsAndCollectHeaders(args, configuration.headers_from_ast, context); - - if (count == 0 || count > 7) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "The signature of table function {} shall be the following:\n{}", getName(), getSignature()); - - std::unordered_map args_to_idx; - - bool no_sign_request = false; - - /// For 2 arguments we support 2 possible variants: - /// - s3(source, format) - /// - s3(source, NOSIGN) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - no_sign_request = true; - else - args_to_idx = {{"format", 1}}; - } - /// For 3 arguments we support 3 possible variants: - /// - s3(source, format, structure) - /// - s3(source, access_key_id, secret_access_key) - /// - s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}}; - } - else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - args_to_idx = {{"format", 1}, {"structure", 2}}; - else - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; - } - /// For 4 arguments we support 4 possible variants: - /// - s3(source, format, structure, compression_method), - /// - s3(source, access_key_id, secret_access_key, format), - /// - s3(source, access_key_id, secret_access_key, session_token) - /// - s3(source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd and 4-th argument: check if it's a format name or not. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}, {"structure", 3}}; - } - else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - { - args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}}; - } - } - } - /// For 5 arguments we support 3 possible variants: - /// - s3(source, access_key_id, secret_access_key, format, structure) - /// - s3(source, access_key_id, secret_access_key, session_token, format) - /// - s3(source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or no, - /// and by the 4-th argument, check if it's a format name or not - else if (count == 5) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "NOSIGN/access_key_id"); - if (boost::iequals(second_arg, "NOSIGN")) - { - no_sign_request = true; - args_to_idx = {{"format", 2}, {"structure", 3}, {"compression_method", 4}}; - } - else - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}}; - } - } - } - // For 6 arguments we support 2 possible variants: - /// - s3(source, access_key_id, secret_access_key, format, structure, compression_method) - /// - s3(source, access_key_id, secret_access_key, session_token, format, structure) - /// We can distinguish them by looking at the 4-th argument: check if it's a format name or not - else if (count == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/session_token"); - if (fourth_arg == "auto" || FormatFactory::instance().getAllFormats().contains(fourth_arg)) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}; - } - else - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}}; - } - } - else if (count == 7) - { - args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"session_token", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}; - } - - /// This argument is always the first - String url = checkAndGetLiteralArgument(args[0], "url"); - configuration.url = S3::URI(url); - - if (args_to_idx.contains("format")) - { - auto format = checkAndGetLiteralArgument(args[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") - configuration.format = format; - } - - if (args_to_idx.contains("structure")) - configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); - - if (args_to_idx.contains("compression_method")) - configuration.compression_method = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); - - if (args_to_idx.contains("access_key_id")) - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); - - if (args_to_idx.contains("secret_access_key")) - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); - - if (args_to_idx.contains("session_token")) - configuration.auth_settings.session_token = checkAndGetLiteralArgument(args[args_to_idx["session_token"]], "session_token"); - - configuration.auth_settings.no_sign_request = no_sign_request; - - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(url).getPath(), true); - } - - configuration.keys = {configuration.url.key}; -} - -void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr context) -{ - /// Clone ast function, because we can modify its arguments like removing headers. - auto ast_copy = ast_function->clone(); - - /// Parse args - ASTs & args_func = ast_function->children; - - if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); - - auto & args = args_func.at(0)->children; - - parseArgumentsImpl(args, context); -} - -void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) -{ - if (tryGetNamedCollectionWithOverrides(args, context)) - { - /// In case of named collection, just add key-value pair "structure='...'" - /// at the end of arguments to override existed structure. - ASTs equal_func_args = {std::make_shared("structure"), std::make_shared(structure)}; - auto equal_func = makeASTFunction("equals", std::move(equal_func_args)); - args.push_back(equal_func); - } - else - { - HTTPHeaderEntries tmp_headers; - size_t count = StorageURL::evalArgsAndCollectHeaders(args, tmp_headers, context); - - if (count == 0 || count > getMaxNumberOfArguments()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), count); - - auto structure_literal = std::make_shared(structure); - - /// s3(s3_url) - if (count == 1) - { - /// Add format=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// s3(s3_url, format) or s3(s3_url, NOSIGN) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not. - else if (count == 2) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - /// If there is NOSIGN, add format=auto before structure. - if (boost::iequals(second_arg, "NOSIGN")) - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - /// s3(source, format, structure) or - /// s3(source, access_key_id, secret_access_key) or - /// s3(source, NOSIGN, format) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. - else if (count == 3) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - args.push_back(structure_literal); - } - else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - { - args[count - 1] = structure_literal; - } - else - { - /// Add format=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(structure_literal); - } - } - /// s3(source, format, structure, compression_method) or - /// s3(source, access_key_id, secret_access_key, format) or - /// s3(source, NOSIGN, format, structure) - /// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither. - else if (count == 4) - { - auto second_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(second_arg, "NOSIGN")) - { - args[count - 1] = structure_literal; - } - else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) - { - args[count - 2] = structure_literal; - } - else - { - args.push_back(structure_literal); - } - } - /// s3(source, access_key_id, secret_access_key, format, structure) or - /// s3(source, NOSIGN, format, structure, compression_method) - /// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not. - else if (count == 5) - { - auto sedond_arg = checkAndGetLiteralArgument(args[1], "format/NOSIGN"); - if (boost::iequals(sedond_arg, "NOSIGN")) - { - args[count - 2] = structure_literal; - } - else - { - args[count - 1] = structure_literal; - } - } - /// s3(source, access_key_id, secret_access_key, format, structure, compression) - else if (count == 6) - { - args[count - 2] = structure_literal; - } - } -} - -ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const -{ - if (configuration.structure == "auto") - { - context->checkAccess(getSourceAccessType()); - configuration.update(context); - return StorageS3::getTableStructureFromData(configuration, std::nullopt, context); - } - - return parseColumnsListFromString(configuration.structure, context); -} - -bool TableFunctionS3::supportsReadingSubsetOfColumns(const ContextPtr & context) -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format, context); -} - -std::unordered_set TableFunctionS3::getVirtualsToCheckBeforeUsingStructureHint() const -{ - auto virtual_column_names = StorageS3::getVirtualColumnNames(); - return {virtual_column_names.begin(), virtual_column_names.end()}; -} - -StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool /*is_insert_query*/) const -{ - S3::URI s3_uri (configuration.url); - - ColumnsDescription columns; - if (configuration.structure != "auto") - columns = parseColumnsListFromString(configuration.structure, context); - else if (!structure_hint.empty()) - columns = structure_hint; - else if (!cached_columns.empty()) - columns = cached_columns; - - StoragePtr storage = std::make_shared( - configuration, - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - String{}, - /// No format_settings for table function S3 - std::nullopt); - - storage->startup(); - - return storage; -} - - -class TableFunctionGCS : public TableFunctionS3 -{ -public: - static constexpr auto name = "gcs"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "GCS"; } -}; - -class TableFunctionCOS : public TableFunctionS3 -{ -public: - static constexpr auto name = "cosn"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "COSN"; } -}; - -class TableFunctionOSS : public TableFunctionS3 -{ -public: - static constexpr auto name = "oss"; - std::string getName() const override - { - return name; - } -private: - const char * getStorageTypeName() const override { return "OSS"; } -}; - - -void registerTableFunctionGCS(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the data stored on Google Cloud Storage.)", - .examples{{"gcs", "SELECT * FROM gcs(url, hmac_key, hmac_secret)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - -void registerTableFunctionS3(TableFunctionFactory & factory) -{ - 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)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - - -void registerTableFunctionCOS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -void registerTableFunctionOSS(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h deleted file mode 100644 index fa73c1d313e..00000000000 --- a/src/TableFunctions/TableFunctionS3.h +++ /dev/null @@ -1,86 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include - - -namespace DB -{ - -class Context; - -/* s3(source, [access_key_id, secret_access_key,] [format, structure, compression]) - creates a temporary storage for a file in S3. - */ -class TableFunctionS3 : public ITableFunction -{ -public: - static constexpr auto name = "s3"; - static constexpr auto signature = " - url\n" - " - url, format\n" - " - url, format, structure\n" - " - url, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key\n" - " - url, access_key_id, secret_access_key, session_token\n" - " - url, access_key_id, secret_access_key, format\n" - " - url, access_key_id, secret_access_key, session_token, format\n" - " - url, access_key_id, secret_access_key, format, structure\n" - " - url, access_key_id, secret_access_key, session_token, format, structure\n" - " - url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - - static size_t getMaxNumberOfArguments() { return 6; } - - String getName() const override - { - return name; - } - - virtual String getSignature() const - { - return signature; - } - - bool hasStaticStructure() const override { return configuration.structure != "auto"; } - - bool needStructureHint() const override { return configuration.structure == "auto"; } - - void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - - bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; - - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; - - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - - static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); - -protected: - - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "S3"; } - - ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; - void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - - mutable StorageS3::Configuration configuration; - ColumnsDescription structure_hint; - -private: - - std::vector skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr context) const override; -}; - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp deleted file mode 100644 index ce96f7f580b..00000000000 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ /dev/null @@ -1,74 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include - -#include "registerTableFunctions.h" - -#include - - -namespace DB -{ - -StoragePtr TableFunctionS3Cluster::executeImpl( - const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const -{ - StoragePtr storage; - ColumnsDescription columns; - bool structure_argument_was_provided = configuration.structure != "auto"; - - if (structure_argument_was_provided) - { - columns = parseColumnsListFromString(configuration.structure, context); - } - else if (!structure_hint.empty()) - { - columns = structure_hint; - } - - if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) - { - /// On worker node this filename won't contains globs - storage = std::make_shared( - configuration, - context, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - /* comment */String{}, - /* format_settings */std::nullopt, /// No format_settings for S3Cluster - /*distributed_processing=*/true); - } - else - { - storage = std::make_shared( - cluster_name, - configuration, - StorageID(getDatabaseName(), table_name), - columns, - ConstraintsDescription{}, - context, - structure_argument_was_provided); - } - - storage->startup(); - - return storage; -} - - -void registerTableFunctionS3Cluster(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - - -} - -#endif diff --git a/src/TableFunctions/TableFunctionS3Cluster.h b/src/TableFunctions/TableFunctionS3Cluster.h deleted file mode 100644 index 718b0d90de8..00000000000 --- a/src/TableFunctions/TableFunctionS3Cluster.h +++ /dev/null @@ -1,64 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include - - -namespace DB -{ - -class Context; - -/** - * s3cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure, compression_method) - * A table function, which allows to process many files from S3 on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in S3 file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ -class TableFunctionS3Cluster : public ITableFunctionCluster -{ -public: - static constexpr auto name = "s3Cluster"; - static constexpr auto signature = " - cluster, url\n" - " - cluster, url, format\n" - " - cluster, url, format, structure\n" - " - cluster, url, access_key_id, secret_access_key\n" - " - cluster, url, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, format\n" - " - cluster, url, access_key_id, secret_access_key, format, structure\n" - " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" - " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" - "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; - - String getName() const override - { - return name; - } - - String getSignature() const override - { - return signature; - } - -protected: - StoragePtr executeImpl( - const ASTPtr & ast_function, - ContextPtr context, - const std::string & table_name, - ColumnsDescription cached_columns, - bool is_insert_query) const override; - - const char * getStorageTypeName() const override { return "S3Cluster"; } -}; - -} - -#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 8c18c298f45..627d945fbf3 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -28,26 +28,17 @@ void registerTableFunctions() #endif #if USE_AWS_S3 - registerTableFunctionS3(factory); - registerTableFunctionS3Cluster(factory); - registerTableFunctionCOS(factory); - registerTableFunctionOSS(factory); - registerTableFunctionGCS(factory); - registerTableFunctionHudi(factory); + // registerTableFunctionS3Cluster(factory); + // registerTableFunctionHudi(factory); #if USE_PARQUET - registerTableFunctionDeltaLake(factory); + // registerTableFunctionDeltaLake(factory); #endif #if USE_AVRO - registerTableFunctionIceberg(factory); + // registerTableFunctionIceberg(factory); #endif #endif -#if USE_HDFS - registerTableFunctionHDFS(factory); - registerTableFunctionHDFSCluster(factory); -#endif - #if USE_HIVE registerTableFunctionHive(factory); #endif @@ -75,10 +66,8 @@ void registerTableFunctions() registerTableFunctionFormat(factory); registerTableFunctionExplain(factory); -#if USE_AZURE_BLOB_STORAGE - registerTableFunctionAzureBlobStorage(factory); - registerTableFunctionAzureBlobStorageCluster(factory); -#endif + registerTableFunctionObjectStorage(factory); + registerTableFunctionObjectStorageCluster(factory); } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index fae763e7dc8..cefb198273e 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -39,11 +39,6 @@ void registerTableFunctionIceberg(TableFunctionFactory & factory); #endif #endif -#if USE_HDFS -void registerTableFunctionHDFS(TableFunctionFactory & factory); -void registerTableFunctionHDFSCluster(TableFunctionFactory & factory); -#endif - #if USE_HIVE void registerTableFunctionHive(TableFunctionFactory & factory); #endif @@ -73,8 +68,8 @@ void registerTableFunctionFormat(TableFunctionFactory & factory); void registerTableFunctionExplain(TableFunctionFactory & factory); #if USE_AZURE_BLOB_STORAGE -void registerTableFunctionAzureBlobStorage(TableFunctionFactory & factory); -void registerTableFunctionAzureBlobStorageCluster(TableFunctionFactory & factory); +void registerTableFunctionObjectStorage(TableFunctionFactory & factory); +void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); #endif void registerTableFunctions(); diff --git a/tests/integration/test_storage_azure_blob_storage/test.py b/tests/integration/test_storage_azure_blob_storage/test.py index 3cccd07c134..41218e41069 100644 --- a/tests/integration/test_storage_azure_blob_storage/test.py +++ b/tests/integration/test_storage_azure_blob_storage/test.py @@ -29,6 +29,8 @@ def cluster(): with_azurite=True, ) cluster.start() + container_client = cluster.blob_service_client.get_container_client("cont") + container_client.create_container() yield cluster finally: cluster.shutdown() @@ -129,8 +131,10 @@ def test_create_table_connection_string(cluster): node = cluster.instances["node"] azure_query( node, - f"CREATE TABLE test_create_table_conn_string (key UInt64, data String) Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}'," - f"'cont', 'test_create_connection_string', 'CSV')", + f""" + CREATE TABLE test_create_table_conn_string (key UInt64, data String) + Engine = AzureBlobStorage('{cluster.env_variables['AZURITE_CONNECTION_STRING']}', 'cont', 'test_create_connection_string', 'CSV') + """, ) From 6d91d92601c04f160ba95a743fca270371b65eb8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 12 Feb 2024 18:17:22 +0100 Subject: [PATCH 002/158] Better --- src/Backups/BackupIO_AzureBlobStorage.cpp | 13 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 8 +- .../AzureBlobStorage/AzureObjectStorage.h | 4 +- .../Cached/CachedObjectStorage.cpp | 2 +- .../Cached/CachedObjectStorage.h | 2 +- src/Disks/ObjectStorages/IObjectStorage.cpp | 6 +- src/Disks/ObjectStorages/IObjectStorage.h | 8 +- .../ObjectStorageIteratorAsync.cpp | 63 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 19 +- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 +- .../DataLakes/DeltaLakeMetadataParser.h | 2 +- src/Storages/DataLakes/HudiMetadataParser.h | 3 +- .../DataLakes/Iceberg/IcebergMetadata.cpp | 1 - .../DataLakes/Iceberg/IcebergMetadata.h | 2 +- .../ObjectStorage/AzureConfiguration.cpp | 11 + .../ObjectStorage/AzureConfiguration.h | 2 +- .../ObjectStorage/HDFSConfiguration.h | 2 +- .../ObjectStorage/ReadBufferIterator.cpp | 179 ++++++ .../ObjectStorage/ReadBufferIterator.h | 179 +----- .../ObjectStorage/ReadFromObjectStorage.h | 105 ---- .../ReadFromStorageObjectStorage.cpp | 94 +++ .../ReadFromStorageObjectStorage.h | 60 ++ src/Storages/ObjectStorage/S3Configuration.h | 2 +- ....h => StorageObejctStorageConfiguration.h} | 28 +- .../ObjectStorage/StorageObjectStorage.cpp | 91 +-- .../StorageObjectStorageCluster.cpp | 9 +- .../StorageObjectStorageCluster.h | 1 - .../StorageObjectStorageConfiguration.cpp | 40 ++ ....h => StorageObjectStorageQuerySettings.h} | 8 + .../ObjectStorage/StorageObjectStorageSink.h | 2 +- .../StorageObjectStorageSource.cpp | 539 +++++++++--------- .../StorageObjectStorageSource.h | 98 ++-- .../StorageObjectStorage_fwd_internal.h | 11 + .../registerStorageObjectStorage.cpp | 18 +- src/Storages/S3Queue/S3QueueSource.cpp | 17 +- src/Storages/S3Queue/S3QueueSource.h | 25 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 32 +- src/Storages/S3Queue/StorageS3Queue.h | 1 - src/TableFunctions/ITableFunctionDataLake.h | 2 +- .../TableFunctionObjectStorage.cpp | 55 +- .../TableFunctionObjectStorageCluster.cpp | 14 +- 42 files changed, 973 insertions(+), 791 deletions(-) create mode 100644 src/Storages/ObjectStorage/ReadBufferIterator.cpp delete mode 100644 src/Storages/ObjectStorage/ReadFromObjectStorage.h create mode 100644 src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp create mode 100644 src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h rename src/Storages/ObjectStorage/{Configuration.h => StorageObejctStorageConfiguration.h} (73%) create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp rename src/Storages/ObjectStorage/{Settings.h => StorageObjectStorageQuerySettings.h} (86%) create mode 100644 src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index dc636f90be7..f12cc4c1d58 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -208,10 +208,15 @@ void BackupWriterAzureBlobStorage::copyFile(const String & destination, const St /* for_disk_azure_blob_storage= */ true); } -void BackupWriterAzureBlobStorage::copyDataToFile(const String & path_in_backup, const CreateReadBufferFunction & create_read_buffer, UInt64 start_pos, UInt64 length) +void BackupWriterAzureBlobStorage::copyDataToFile( + const String & path_in_backup, + const CreateReadBufferFunction & create_read_buffer, + UInt64 start_pos, + UInt64 length) { - copyDataToAzureBlobStorageFile(create_read_buffer, start_pos, length, client, configuration.container, path_in_backup, settings, - threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); + copyDataToAzureBlobStorageFile( + create_read_buffer, start_pos, length, client, configuration.container, + path_in_backup, settings, threadPoolCallbackRunner(getBackupsIOThreadPool().get(), "BackupWRAzure")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; @@ -245,7 +250,7 @@ UInt64 BackupWriterAzureBlobStorage::getFileSize(const String & file_name) object_storage->listObjects(key,children,/*max_keys*/0); if (children.empty()) throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Object must exist"); - return children[0]->metadata.size_bytes; + return children[0]->metadata->size_bytes; } std::unique_ptr BackupWriterAzureBlobStorage::readFile(const String & file_name, size_t /*expected_file_size*/) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 2ca44137442..bbbb5357505 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -128,15 +128,15 @@ bool AzureObjectStorage::exists(const StoredObject & object) const return false; } -ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr AzureObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { auto settings_ptr = settings.get(); auto client_ptr = client.get(); - return std::make_shared(path_prefix, client_ptr, settings_ptr->list_object_keys_size); + return std::make_shared(path_prefix, client_ptr, max_keys); } -void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { auto client_ptr = client.get(); @@ -168,7 +168,7 @@ void AzureObjectStorage::listObjects(const std::string & path, RelativePathsWith if (max_keys) { - int keys_left = max_keys - static_cast(children.size()); + size_t keys_left = max_keys - children.size(); if (keys_left <= 0) break; options.PageSizeHint = keys_left; diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index f16c35fb52c..31eb78924f9 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -69,9 +69,9 @@ public: SettingsPtr && settings_, const String & container_); - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; - ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override; std::string getName() const override { return "AzureObjectStorage"; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 1444f4c9c76..9f195b787a8 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -180,7 +180,7 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void CachedObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { object_storage->listObjects(path, children, max_keys); } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 437baead7be..ec116b63d01 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -80,7 +80,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 78fbdcaddfa..d36ef4f414a 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -24,16 +24,16 @@ bool IObjectStorage::existsOrHasAnyChild(const std::string & path) const return !files.empty(); } -void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, int) const +void IObjectStorage::listObjects(const std::string &, RelativePathsWithMetadata &, size_t) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listObjects() is not supported"); } -ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr IObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { RelativePathsWithMetadata files; - listObjects(path_prefix, files, 0); + listObjects(path_prefix, files, max_keys); return std::make_shared(std::move(files)); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 7d354e6383d..4955b0e6924 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -54,11 +54,11 @@ struct ObjectMetadata struct RelativePathWithMetadata { String relative_path; - ObjectMetadata metadata; + std::optional metadata; RelativePathWithMetadata() = default; - RelativePathWithMetadata(String relative_path_, ObjectMetadata metadata_) + explicit RelativePathWithMetadata(String relative_path_, std::optional metadata_ = std::nullopt) : relative_path(std::move(relative_path_)) , metadata(std::move(metadata_)) {} @@ -111,9 +111,9 @@ public: /// /, /a, /a/b, /a/b/c, /a/b/c/d while exists will return true only for /a/b/c/d virtual bool existsOrHasAnyChild(const std::string & path) const; - virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const; + virtual void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const; - virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const; + virtual ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index b7729623a64..62bdd0ed0c8 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -14,27 +14,32 @@ namespace ErrorCodes void IObjectStorageIteratorAsync::nextBatch() { std::lock_guard lock(mutex); - if (!is_finished) + if (is_finished) { + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 3"); + current_batch.clear(); + current_batch_iterator = current_batch.begin(); + } + else + { + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 4"); if (!is_initialized) { outcome_future = scheduleBatch(); is_initialized = true; } - BatchAndHasNext next_batch = outcome_future.get(); - current_batch = std::move(next_batch.batch); - accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - current_batch_iterator = current_batch.begin(); - if (next_batch.has_next) - outcome_future = scheduleBatch(); - else - is_finished = true; - } - else - { - current_batch.clear(); + chassert(outcome_future.valid()); + auto [batch, has_next] = outcome_future.get(); + current_batch = std::move(batch); current_batch_iterator = current_batch.begin(); + + accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); + + if (has_next) + outcome_future = scheduleBatch(); + else + is_finished = true; } } @@ -42,24 +47,10 @@ void IObjectStorageIteratorAsync::next() { std::lock_guard lock(mutex); - if (current_batch_iterator != current_batch.end()) - { + if (current_batch_iterator == current_batch.end()) + nextBatch(); + else ++current_batch_iterator; - } - else if (!is_finished) - { - if (outcome_future.valid()) - { - BatchAndHasNext next_batch = outcome_future.get(); - current_batch = std::move(next_batch.batch); - accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - current_batch_iterator = current_batch.begin(); - if (next_batch.has_next) - outcome_future = scheduleBatch(); - else - is_finished = true; - } - } } std::future IObjectStorageIteratorAsync::scheduleBatch() @@ -107,14 +98,16 @@ std::optional IObjectStorageIteratorAsync::getCurrent if (!is_initialized) nextBatch(); - if (current_batch_iterator != current_batch.end()) + if (current_batch_iterator == current_batch.end()) { - auto temp_current_batch = current_batch; - nextBatch(); - return temp_current_batch; + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 2"); + return std::nullopt; } - return std::nullopt; + auto temp_current_batch = std::move(current_batch); + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 1: {}", temp_current_batch.size()); + nextBatch(); + return temp_current_batch; } size_t IObjectStorageIteratorAsync::getAccumulatedSize() const diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index cc138c43c71..a9bd520e6e9 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -138,9 +138,10 @@ private: return outcome.GetResult().GetIsTruncated(); } - throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), - backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); + throw S3Exception(outcome.GetError().GetErrorType(), + "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", + quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } std::shared_ptr client; @@ -263,13 +264,13 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN } -ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix) const +ObjectStorageIteratorPtr S3ObjectStorage::iterate(const std::string & path_prefix, size_t max_keys) const { auto settings_ptr = s3_settings.get(); - return std::make_shared(uri.bucket, path_prefix, client.get(), settings_ptr->list_object_keys_size); + return std::make_shared(uri.bucket, path_prefix, client.get(), max_keys); } -void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const +void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { auto settings_ptr = s3_settings.get(); @@ -277,7 +278,7 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet request.SetBucket(uri.bucket); request.SetPrefix(path); if (max_keys) - request.SetMaxKeys(max_keys); + request.SetMaxKeys(static_cast(max_keys)); else request.SetMaxKeys(settings_ptr->list_object_keys_size); @@ -305,10 +306,10 @@ void S3ObjectStorage::listObjects(const std::string & path, RelativePathsWithMet if (max_keys) { - int keys_left = max_keys - static_cast(children.size()); + size_t keys_left = max_keys - children.size(); if (keys_left <= 0) break; - request.SetMaxKeys(keys_left); + request.SetMaxKeys(static_cast(keys_left)); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index ab0fa5bed68..a6843a383e5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -100,9 +100,9 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listObjects(const std::string & path, RelativePathsWithMetadata & children, int max_keys) const override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; - ObjectStorageIteratorPtr iterate(const std::string & path_prefix) const override; + ObjectStorageIteratorPtr iterate(const std::string & path_prefix, size_t max_keys) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.h b/src/Storages/DataLakes/DeltaLakeMetadataParser.h index f94024597d6..251ea3e3f15 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.h +++ b/src/Storages/DataLakes/DeltaLakeMetadataParser.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/DataLakes/HudiMetadataParser.h b/src/Storages/DataLakes/HudiMetadataParser.h index 2fc004595ca..72766a95876 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.h +++ b/src/Storages/DataLakes/HudiMetadataParser.h @@ -2,7 +2,8 @@ #include #include -#include +#include +#include namespace DB { diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp index 08cebb3f396..5543e60e7a7 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h index 92946e4192b..a289715848f 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/DataLakes/Iceberg/IcebergMetadata.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Storages/ObjectStorage/AzureConfiguration.cpp b/src/Storages/ObjectStorage/AzureConfiguration.cpp index ba3e796223a..04f6f26111b 100644 --- a/src/Storages/ObjectStorage/AzureConfiguration.cpp +++ b/src/Storages/ObjectStorage/AzureConfiguration.cpp @@ -89,6 +89,17 @@ StorageObjectStorageConfigurationPtr StorageAzureBlobConfiguration::clone() return configuration; } +StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other) +{ + connection_url = other.connection_url; + is_connection_string = other.is_connection_string; + account_name = other.account_name; + account_key = other.account_key; + container = other.container; + blob_path = other.blob_path; + blobs_paths = other.blobs_paths; +} + AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(ContextPtr context) { const auto & context_settings = context->getSettingsRef(); diff --git a/src/Storages/ObjectStorage/AzureConfiguration.h b/src/Storages/ObjectStorage/AzureConfiguration.h index 40d718d7690..4f285128241 100644 --- a/src/Storages/ObjectStorage/AzureConfiguration.h +++ b/src/Storages/ObjectStorage/AzureConfiguration.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Storages/ObjectStorage/HDFSConfiguration.h b/src/Storages/ObjectStorage/HDFSConfiguration.h index f42cedf459d..aa45c634042 100644 --- a/src/Storages/ObjectStorage/HDFSConfiguration.h +++ b/src/Storages/ObjectStorage/HDFSConfiguration.h @@ -3,7 +3,7 @@ #if USE_HDFS -#include +#include #include #include #include diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp new file mode 100644 index 00000000000..dcdf36dbcf5 --- /dev/null +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -0,0 +1,179 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + +} + +ReadBufferIterator::ReadBufferIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const FileIterator & file_iterator_, + const std::optional & format_settings_, + const StorageObjectStorageSettings & query_settings_, + SchemaCache & schema_cache_, + ObjectInfos & read_keys_, + const ContextPtr & context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + , file_iterator(file_iterator_) + , format_settings(format_settings_) + , query_settings(query_settings_) + , schema_cache(schema_cache_) + , read_keys(read_keys_) + , prev_read_keys_size(read_keys_.size()) +{ +} + +SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const String & path) const +{ + auto source = fs::path(configuration->getDataSourceDescription()) / path; + return DB::getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); +} + +SchemaCache::Keys ReadBufferIterator::getPathsForSchemaCache() const +{ + Strings sources; + sources.reserve(read_keys.size()); + std::transform( + read_keys.begin(), read_keys.end(), + std::back_inserter(sources), + [&](const auto & elem) + { + return fs::path(configuration->getDataSourceDescription()) / elem->relative_path; + }); + return DB::getKeysForSchemaCache(sources, configuration->format, format_settings, getContext()); +} + +std::optional ReadBufferIterator::tryGetColumnsFromCache( + const ObjectInfos::iterator & begin, + const ObjectInfos::iterator & end) +{ + if (!query_settings.schema_inference_use_cache) + return std::nullopt; + + for (auto it = begin; it < end; ++it) + { + const auto & object_info = (*it); + auto get_last_mod_time = [&] -> std::optional + { + if (object_info->metadata) + return object_info->metadata->last_modified->epochMicroseconds(); + else + { + object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + return object_info->metadata->last_modified->epochMicroseconds(); + } + }; + + auto cache_key = getKeyForSchemaCache(object_info->relative_path); + auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); + if (columns) + return columns; + } + + return std::nullopt; +} + +void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) +{ + if (query_settings.schema_inference_use_cache) + schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->relative_path), num_rows); +} + +void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) +{ + if (query_settings.schema_inference_use_cache + && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) + { + schema_cache.addColumns(getKeyForSchemaCache(current_object_info->relative_path), columns); + } +} + +void ReadBufferIterator::setResultingSchema(const ColumnsDescription & columns) +{ + if (query_settings.schema_inference_use_cache + && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + schema_cache.addManyColumns(getPathsForSchemaCache(), columns); + } +} + +String ReadBufferIterator::getLastFileName() const +{ + if (current_object_info) + return current_object_info->relative_path; + else + return ""; +} + +std::pair, std::optional> ReadBufferIterator::next() +{ + /// For default mode check cached columns for currently read keys on first iteration. + if (first && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + return {nullptr, cached_columns}; + } + + current_object_info = file_iterator->next(0); + if (!current_object_info || current_object_info->relative_path.empty()) + { + if (first) + { + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "Cannot extract table structure from {} format file, " + "because there are no files with provided path. " + "You must specify table structure manually", + configuration->format); + } + return {nullptr, std::nullopt}; + } + + first = false; + + /// File iterator could get new keys after new iteration, + /// check them in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT + && read_keys.size() > prev_read_keys_size) + { + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); + prev_read_keys_size = read_keys.size(); + if (columns_from_cache) + return {nullptr, columns_from_cache}; + } + else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + ObjectInfos paths = {current_object_info}; + if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) + return {nullptr, columns_from_cache}; + } + + first = false; + + chassert(current_object_info->metadata); + std::unique_ptr read_buffer = object_storage->readObject( + StoredObject(current_object_info->relative_path), + getContext()->getReadSettings(), + {}, + current_object_info->metadata->size_bytes); + + read_buffer = wrapReadBufferWithCompressionMethod( + std::move(read_buffer), + chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), + static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + + return {std::move(read_buffer), std::nullopt}; +} + +} diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 248700e2edf..4e9b8cfcfca 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -1,197 +1,54 @@ #pragma once #include -#include +#include #include -#include -#include #include namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; -} - -template class ReadBufferIterator : public IReadBufferIterator, WithContext { public: - using Storage = StorageObjectStorage; - using Source = StorageObjectStorageSource; - using FileIterator = std::shared_ptr; - using ObjectInfos = typename Storage::ObjectInfos; + using FileIterator = std::shared_ptr; ReadBufferIterator( ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, + ConfigurationPtr configuration_, const FileIterator & file_iterator_, const std::optional & format_settings_, + const StorageObjectStorageSettings & query_settings_, + SchemaCache & schema_cache_, ObjectInfos & read_keys_, - const ContextPtr & context_) - : WithContext(context_) - , object_storage(object_storage_) - , configuration(configuration_) - , file_iterator(file_iterator_) - , format_settings(format_settings_) - , storage_settings(StorageSettings::create(context_->getSettingsRef())) - , read_keys(read_keys_) - , prev_read_keys_size(read_keys_.size()) - { - } + const ContextPtr & context_); - std::pair, std::optional> next() override - { - /// For default mode check cached columns for currently read keys on first iteration. - if (first && storage_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns}; - } + std::pair, std::optional> next() override; - current_object_info = file_iterator->next(0); - if (current_object_info->relative_path.empty()) - { - if (first) - { - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, " - "because there are no files with provided path. " - "You must specify table structure manually", - configuration->format); - } - return {nullptr, std::nullopt}; - } + void setNumRowsToLastFile(size_t num_rows) override; - first = false; + void setSchemaToLastFile(const ColumnsDescription & columns) override; - /// File iterator could get new keys after new iteration, - /// check them in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT - && read_keys.size() > prev_read_keys_size) - { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - prev_read_keys_size = read_keys.size(); - if (columns_from_cache) - return {nullptr, columns_from_cache}; - } - else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - ObjectInfos paths = {current_object_info}; - if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) - return {nullptr, columns_from_cache}; - } + void setResultingSchema(const ColumnsDescription & columns) override; - first = false; - - std::unique_ptr read_buffer = object_storage->readObject( - StoredObject(current_object_info->relative_path), - getContext()->getReadSettings(), - {}, - current_object_info->metadata.size_bytes); - - read_buffer = wrapReadBufferWithCompressionMethod( - std::move(read_buffer), - chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), - static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - - return {std::move(read_buffer), std::nullopt}; - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (storage_settings.schema_inference_use_cache) - { - Storage::getSchemaCache(getContext()).addNumRows( - getKeyForSchemaCache(current_object_info->relative_path), num_rows); - } - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (storage_settings.schema_inference_use_cache - && storage_settings.schema_inference_mode == SchemaInferenceMode::UNION) - { - Storage::getSchemaCache(getContext()).addColumns( - getKeyForSchemaCache(current_object_info->relative_path), columns); - } - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (storage_settings.schema_inference_use_cache - && storage_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - Storage::getSchemaCache(getContext()).addManyColumns(getPathsForSchemaCache(), columns); - } - } - - String getLastFileName() const override { return current_object_info->relative_path; } + String getLastFileName() const override; private: - SchemaCache::Key getKeyForSchemaCache(const String & path) const - { - auto source = fs::path(configuration->getDataSourceDescription()) / path; - return DB::getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); - } - - SchemaCache::Keys getPathsForSchemaCache() const - { - Strings sources; - sources.reserve(read_keys.size()); - std::transform( - read_keys.begin(), read_keys.end(), - std::back_inserter(sources), - [&](const auto & elem) - { - return fs::path(configuration->getDataSourceDescription()) / elem->relative_path; - }); - return DB::getKeysForSchemaCache(sources, configuration->format, format_settings, getContext()); - } - + SchemaCache::Key getKeyForSchemaCache(const String & path) const; + SchemaCache::Keys getPathsForSchemaCache() const; std::optional tryGetColumnsFromCache( - const ObjectInfos::iterator & begin, - const ObjectInfos::iterator & end) - { - if (!storage_settings.schema_inference_use_cache) - return std::nullopt; - - auto & schema_cache = Storage::getSchemaCache(getContext()); - for (auto it = begin; it < end; ++it) - { - const auto & object_info = (*it); - auto get_last_mod_time = [&] -> std::optional - { - if (object_info->metadata.last_modified) - return object_info->metadata.last_modified->epochMicroseconds(); - else - { - object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - return object_info->metadata.last_modified->epochMicroseconds(); - } - }; - - auto cache_key = getKeyForSchemaCache(object_info->relative_path); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; - } - - return std::nullopt; - } + const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); ObjectStoragePtr object_storage; - const Storage::ConfigurationPtr configuration; + const ConfigurationPtr configuration; const FileIterator file_iterator; const std::optional & format_settings; - const StorageObjectStorageSettings storage_settings; + const StorageObjectStorageSettings query_settings; + SchemaCache & schema_cache; ObjectInfos & read_keys; size_t prev_read_keys_size; - Storage::ObjectInfoPtr current_object_info; + ObjectInfoPtr current_object_info; bool first = true; }; } diff --git a/src/Storages/ObjectStorage/ReadFromObjectStorage.h b/src/Storages/ObjectStorage/ReadFromObjectStorage.h deleted file mode 100644 index 9cb77dcc25e..00000000000 --- a/src/Storages/ObjectStorage/ReadFromObjectStorage.h +++ /dev/null @@ -1,105 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -template -class ReadFromStorageObejctStorage : public SourceStepWithFilter -{ -public: - using Storage = StorageObjectStorage; - using Source = StorageObjectStorageSource; - - ReadFromStorageObejctStorage( - ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, - const String & name_, - const NamesAndTypesList & virtual_columns_, - const std::optional & format_settings_, - bool distributed_processing_, - ReadFromFormatInfo info_, - const bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = info_.source_header}) - , object_storage(object_storage_) - , configuration(configuration_) - , context(std::move(context_)) - , info(std::move(info_)) - , virtual_columns(virtual_columns_) - , format_settings(format_settings_) - , name(name_ + "Source") - , need_only_count(need_only_count_) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - , distributed_processing(distributed_processing_) - { - } - - std::string getName() const override { return name; } - - void applyFilters() override - { - auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); - } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override - { - createIterator(nullptr); - - Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - getName(), object_storage, configuration, info, format_settings, - context, max_block_size, iterator_wrapper, need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); - } - -private: - ObjectStoragePtr object_storage; - Storage::ConfigurationPtr configuration; - ContextPtr context; - - const ReadFromFormatInfo info; - const NamesAndTypesList virtual_columns; - const std::optional format_settings; - const String name; - const bool need_only_count; - const size_t max_block_size; - const size_t num_streams; - const bool distributed_processing; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate) - { - if (!iterator_wrapper) - { - iterator_wrapper = Source::createFileIterator( - configuration, object_storage, distributed_processing, context, - predicate, virtual_columns, nullptr, context->getFileProgressCallback()); - } - } -}; - -} diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp new file mode 100644 index 00000000000..2c27c816078 --- /dev/null +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp @@ -0,0 +1,94 @@ +#include +#include +#include + +namespace DB +{ + +ReadFromStorageObejctStorage::ReadFromStorageObejctStorage( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const NamesAndTypesList & virtual_columns_, + const std::optional & format_settings_, + const StorageObjectStorageSettings & query_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + SchemaCache & schema_cache_, + const bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + CurrentMetrics::Metric metric_threads_count_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_) + : SourceStepWithFilter(DataStream{.header = info_.source_header}) + , WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + , info(std::move(info_)) + , virtual_columns(virtual_columns_) + , format_settings(format_settings_) + , query_settings(query_settings_) + , schema_cache(schema_cache_) + , name(name_ + "Source") + , need_only_count(need_only_count_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , distributed_processing(distributed_processing_) + , metric_threads_count(metric_threads_count_) + , metric_threads_active(metric_threads_active_) + , metric_threads_scheduled(metric_threads_scheduled_) +{ +} + +void ReadFromStorageObejctStorage::createIterator(const ActionsDAG::Node * predicate) +{ + if (!iterator_wrapper) + { + auto context = getContext(); + iterator_wrapper = StorageObjectStorageSource::createFileIterator( + configuration, object_storage, distributed_processing, context, predicate, + virtual_columns, nullptr, query_settings.list_object_keys_size, context->getFileProgressCallback()); + } +} + +void ReadFromStorageObejctStorage::applyFilters() +{ + auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + + createIterator(predicate); +} + +void ReadFromStorageObejctStorage::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + createIterator(nullptr); + auto context = getContext(); + + Pipes pipes; + for (size_t i = 0; i < num_streams; ++i) + { + auto threadpool = std::make_shared( + metric_threads_count, metric_threads_active, metric_threads_scheduled, /* max_threads */1); + + auto source = std::make_shared( + getName(), object_storage, configuration, info, format_settings, query_settings, + context, max_block_size, iterator_wrapper, need_only_count, schema_cache, std::move(threadpool)); + + pipes.emplace_back(std::move(source)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); +} + +} diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h new file mode 100644 index 00000000000..f5e057d297f --- /dev/null +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h @@ -0,0 +1,60 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class ReadFromStorageObejctStorage : public SourceStepWithFilter, WithContext +{ +public: + using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + + ReadFromStorageObejctStorage( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const NamesAndTypesList & virtual_columns_, + const std::optional & format_settings_, + const StorageObjectStorageSettings & query_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + SchemaCache & schema_cache_, + bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_, + CurrentMetrics::Metric metric_threads_count_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_); + + std::string getName() const override { return name; } + + void applyFilters() override; + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + +private: + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::shared_ptr iterator_wrapper; + + const ReadFromFormatInfo info; + const NamesAndTypesList virtual_columns; + const std::optional format_settings; + const StorageObjectStorageSettings query_settings; + SchemaCache & schema_cache; + const String name; + const bool need_only_count; + const size_t max_block_size; + const size_t num_streams; + const bool distributed_processing; + const CurrentMetrics::Metric metric_threads_count; + const CurrentMetrics::Metric metric_threads_active; + const CurrentMetrics::Metric metric_threads_scheduled; + + void createIterator(const ActionsDAG::Node * predicate); +}; + +} diff --git a/src/Storages/ObjectStorage/S3Configuration.h b/src/Storages/ObjectStorage/S3Configuration.h index 34f5735e02a..c953bc25c4e 100644 --- a/src/Storages/ObjectStorage/S3Configuration.h +++ b/src/Storages/ObjectStorage/S3Configuration.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { diff --git a/src/Storages/ObjectStorage/Configuration.h b/src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h similarity index 73% rename from src/Storages/ObjectStorage/Configuration.h rename to src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h index 708041980e3..427d6a8d453 100644 --- a/src/Storages/ObjectStorage/Configuration.h +++ b/src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h @@ -17,6 +17,12 @@ public: using Path = std::string; using Paths = std::vector; + static void initialize( + StorageObjectStorageConfiguration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure); + virtual Path getPath() const = 0; virtual void setPath(const Path & path) = 0; @@ -26,28 +32,24 @@ public: virtual String getDataSourceDescription() = 0; virtual String getNamespace() const = 0; - bool isPathWithGlobs() const { return getPath().find_first_of("*?{") != std::string::npos; } - bool isNamespaceWithGlobs() const { return getNamespace().find_first_of("*?{") != std::string::npos; } - - std::string getPathWithoutGlob() const { return getPath().substr(0, getPath().find_first_of("*?{")); } - - virtual bool withWildcard() const - { - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return getPath().find(PARTITION_ID_WILDCARD) != String::npos; - } + bool withWildcard() const; + bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } + bool isPathWithGlobs() const; + bool isNamespaceWithGlobs() const; + std::string getPathWithoutGlob() const; virtual void check(ContextPtr context) const = 0; virtual StorageObjectStorageConfigurationPtr clone() = 0; virtual ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT - virtual void fromNamedCollection(const NamedCollection & collection) = 0; - virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; - String format = "auto"; String compression_method = "auto"; String structure = "auto"; + +protected: + virtual void fromNamedCollection(const NamedCollection & collection) = 0; + virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; }; using StorageObjectStorageConfigurationPtr = std::shared_ptr; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 9250ab8ecbe..9a7260ea47c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -9,12 +9,12 @@ #include #include #include -#include -#include +#include +#include #include #include #include -#include +#include namespace DB @@ -154,34 +154,38 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - if (partition_by && configuration->withWildcard()) + auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); + if (partition_by && query_configuration->withWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned {} storage is not implemented yet", getName()); } - auto this_ptr = std::static_pointer_cast(shared_from_this()); - auto read_from_format_info = prepareReadingFromFormat( + const auto read_from_format_info = prepareReadingFromFormat( column_names, storage_snapshot, supportsSubsetOfColumns(local_context), getVirtuals()); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) + const bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; - auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); - auto reading = std::make_unique>( + auto read_step = std::make_unique( query_object_storage, query_configuration, getName(), virtual_columns, format_settings, + StorageSettings::create(local_context->getSettingsRef()), distributed_processing, std::move(read_from_format_info), + getSchemaCache(local_context), need_only_count, local_context, max_block_size, - num_streams); + num_streams, + StorageSettings::ObjectStorageThreads(), + StorageSettings::ObjectStorageThreadsActive(), + StorageSettings::ObjectStorageThreadsScheduled()); - query_plan.addStep(std::move(reading)); + query_plan.addStep(std::move(read_step)); } template @@ -191,35 +195,43 @@ SinkToStoragePtr StorageObjectStorage::write( ContextPtr local_context, bool /* async_insert */) { - auto insert_query = std::dynamic_pointer_cast(query); - auto partition_by_ast = insert_query - ? (insert_query->partition_by ? insert_query->partition_by : partition_by) - : nullptr; - bool is_partitioned_implementation = partition_by_ast && configuration->withWildcard(); + auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); + const auto sample_block = metadata_snapshot->getSampleBlock(); - auto sample_block = metadata_snapshot->getSampleBlock(); - auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); - - if (is_partitioned_implementation) + if (query_configuration->withWildcard()) { - return std::make_shared( - object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); + ASTPtr partition_by_ast = nullptr; + if (auto insert_query = std::dynamic_pointer_cast(query)) + { + if (insert_query->partition_by) + partition_by_ast = insert_query->partition_by; + else + partition_by_ast = partition_by; + } + + if (partition_by_ast) + { + return std::make_shared( + object_storage, query_configuration, format_settings, sample_block, local_context, partition_by_ast); + } } - if (configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs()) + if (query_configuration->withGlobs()) { throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "{} key '{}' contains globs, so the table is in readonly mode", - getName(), configuration->getPath()); + getName(), query_configuration->getPath()); } + const auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); if (!storage_settings.truncate_on_insert - && object_storage->exists(StoredObject(configuration->getPath()))) + && object_storage->exists(StoredObject(query_configuration->getPath()))) { if (storage_settings.create_new_file_on_insert) { - size_t index = configuration->getPaths().size(); - const auto & first_key = configuration->getPaths()[0]; + auto & paths = query_configuration->getPaths(); + size_t index = paths.size(); + const auto & first_key = paths[0]; auto pos = first_key.find_first_of('.'); String new_key; @@ -233,7 +245,7 @@ SinkToStoragePtr StorageObjectStorage::write( } while (object_storage->exists(StoredObject(new_key))); - configuration->getPaths().push_back(new_key); + paths.push_back(new_key); } else { @@ -242,12 +254,12 @@ SinkToStoragePtr StorageObjectStorage::write( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting [engine_name]_truncate_on_insert, if you " "want to create a new file on each insert, enable setting [engine_name]_create_new_file_on_insert", - configuration->getNamespace(), configuration->getPaths().back()); + query_configuration->getNamespace(), query_configuration->getPaths().back()); } } return std::make_shared( - object_storage, configuration, format_settings, sample_block, local_context); + object_storage, query_configuration, format_settings, sample_block, local_context); } template @@ -257,7 +269,7 @@ void StorageObjectStorage::truncate( ContextPtr, TableExclusiveLockHolder &) { - if (configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs()) + if (configuration->withGlobs()) { throw Exception( ErrorCodes::DATABASE_ACCESS_DENIED, @@ -279,21 +291,18 @@ ColumnsDescription StorageObjectStorage::getTableStructureFromD const std::optional & format_settings, ContextPtr context) { - using Source = StorageObjectStorageSource; - ObjectInfos read_keys; - auto file_iterator = Source::createFileIterator( + const auto settings = StorageSettings::create(context->getSettingsRef()); + auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, object_storage, /* distributed_processing */false, - context, /* predicate */{}, /* virtual_columns */{}, &read_keys); + context, /* predicate */{}, /* virtual_columns */{}, &read_keys, settings.list_object_keys_size); - ReadBufferIterator read_buffer_iterator( + ReadBufferIterator read_buffer_iterator( object_storage, configuration, file_iterator, - format_settings, read_keys, context); + format_settings, StorageSettings::create(context->getSettingsRef()), getSchemaCache(context), read_keys, context); - const bool retry = configuration->isPathWithGlobs() || configuration->isNamespaceWithGlobs(); - return readSchemaFromFormat( - configuration->format, format_settings, - read_buffer_iterator, retry, context); + const bool retry = configuration->withGlobs(); + return readSchemaFromFormat(configuration->format, format_settings, read_buffer_iterator, retry, context); } template class StorageObjectStorage; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 414932016f4..39cd5d8eca6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -11,8 +11,8 @@ #include #include #include -#include #include +#include #include #include @@ -82,10 +82,11 @@ void StorageObjectStorageCluster::ad template RemoteQueryExecutor::Extension -StorageObjectStorageCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr &) const +StorageObjectStorageCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { - auto iterator = std::make_shared( - object_storage, configuration, predicate, virtual_columns, nullptr); + const auto settings = StorageSettings::create(local_context->getSettingsRef()); + auto iterator = std::make_shared( + object_storage, configuration, predicate, virtual_columns, local_context, nullptr, settings.list_object_keys_size); auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next(0)->relative_path; }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index b1f9af14e03..aae8f704a73 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -21,7 +21,6 @@ class StorageObjectStorageCluster : public IStorageCluster { public: using Storage = StorageObjectStorage; - using Source = StorageObjectStorageSource; StorageObjectStorageCluster( const String & cluster_name_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp new file mode 100644 index 00000000000..2d5760ed9d8 --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -0,0 +1,40 @@ +#include + + +namespace DB +{ + +void StorageObjectStorageConfiguration::initialize( + StorageObjectStorageConfiguration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); +} + +bool StorageObjectStorageConfiguration::withWildcard() const +{ + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return getPath().find(PARTITION_ID_WILDCARD) != String::npos; +} + +bool StorageObjectStorageConfiguration::isPathWithGlobs() const +{ + return getPath().find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorageConfiguration::isNamespaceWithGlobs() const +{ + return getNamespace().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorageConfiguration::getPathWithoutGlob() const +{ + return getPath().substr(0, getPath().find_first_of("*?{")); +} + +} diff --git a/src/Storages/ObjectStorage/Settings.h b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h similarity index 86% rename from src/Storages/ObjectStorage/Settings.h rename to src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h index 015cf9bc01d..454da7c355f 100644 --- a/src/Storages/ObjectStorage/Settings.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h @@ -23,6 +23,8 @@ struct StorageObjectStorageSettings bool create_new_file_on_insert; bool schema_inference_use_cache; SchemaInferenceMode schema_inference_mode; + bool skip_empty_files; + size_t list_object_keys_size; }; struct S3StorageSettings @@ -34,6 +36,8 @@ struct S3StorageSettings .create_new_file_on_insert = settings.s3_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_s3, .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, + .list_object_keys_size = settings.s3_list_object_keys_size, }; } @@ -53,6 +57,8 @@ struct AzureStorageSettings .create_new_file_on_insert = settings.azure_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure + .list_object_keys_size = settings.azure_list_object_keys_size, }; } @@ -72,6 +78,8 @@ struct HDFSStorageSettings .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for hdfs + .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 34ab8ebec66..a2d42d7fa9f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -1,6 +1,6 @@ #pragma once #include -#include +#include #include #include #include diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 9fc7925a6d1..f170a46112f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include #include #include @@ -28,20 +28,55 @@ namespace ErrorCodes extern const int CANNOT_COMPILE_REGEXP; } -template -std::shared_ptr::IIterator> -StorageObjectStorageSource::createFileIterator( - Storage::ConfigurationPtr configuration, +StorageObjectStorageSource::StorageObjectStorageSource( + String name_, + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const ReadFromFormatInfo & info, + std::optional format_settings_, + const StorageObjectStorageSettings & query_settings_, + ContextPtr context_, + UInt64 max_block_size_, + std::shared_ptr file_iterator_, + bool need_only_count_, + SchemaCache & schema_cache_, + std::shared_ptr reader_pool_) + : SourceWithKeyCondition(info.source_header, false) + , WithContext(context_) + , name(std::move(name_)) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , query_settings(query_settings_) + , max_block_size(max_block_size_) + , need_only_count(need_only_count_) + , read_from_format_info(info) + , create_reader_pool(reader_pool_) + , columns_desc(info.columns_description) + , file_iterator(file_iterator_) + , schema_cache(schema_cache_) + , create_reader_scheduler(threadPoolCallbackRunner(*create_reader_pool, "Reader")) +{ +} + +StorageObjectStorageSource::~StorageObjectStorageSource() +{ + create_reader_pool->wait(); +} + +std::shared_ptr StorageObjectStorageSource::createFileIterator( + ConfigurationPtr configuration, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, + size_t list_object_keys_size, std::function file_progress_callback) { if (distributed_processing) - return std::make_shared(local_context->getReadTaskCallback()); + return std::make_shared(local_context->getReadTaskCallback()); if (configuration->isNamespaceWithGlobs()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); @@ -49,25 +84,240 @@ StorageObjectStorageSource::createFileIterator( if (configuration->isPathWithGlobs()) { /// Iterate through disclosed globs and make a source for each file - return std::make_shared( - object_storage, configuration, predicate, virtual_columns, read_keys, file_progress_callback); + return std::make_shared( + object_storage, configuration, predicate, virtual_columns, local_context, read_keys, list_object_keys_size, file_progress_callback); } else { - return std::make_shared( + return std::make_shared( object_storage, configuration, virtual_columns, read_keys, file_progress_callback); } } -template -StorageObjectStorageSource::GlobIterator::GlobIterator( +void StorageObjectStorageSource::lazyInitialize(size_t processor) +{ + if (initialized) + return; + + reader = createReader(processor); + if (reader) + reader_future = createReaderAsync(processor); + initialized = true; +} + +Chunk StorageObjectStorageSource::generate() +{ + lazyInitialize(0); + + while (true) + { + if (isCancelled() || !reader) + { + if (reader) + reader->cancel(); + break; + } + + Chunk chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + total_rows_in_file += num_rows; + + size_t chunk_size = 0; + if (const auto * input_format = reader.getInputFormat()) + chunk_size = input_format->getApproxBytesReadForChunk(); + + progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); + + const auto & object_info = reader.getObjectInfo(); + chassert(object_info.metadata); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, + read_from_format_info.requested_virtual_columns, + fs::path(configuration->getNamespace()) / reader.getRelativePath(), + object_info.metadata->size_bytes); + + return chunk; + } + + if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) + addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); + + total_rows_in_file = 0; + + assert(reader_future.valid()); + reader = reader_future.get(); + + if (!reader) + break; + + /// Even if task is finished the thread may be not freed in pool. + /// So wait until it will be freed before scheduling a new task. + create_reader_pool->wait(); + reader_future = createReaderAsync(); + } + + return {}; +} + +void StorageObjectStorageSource::addNumRowsToCache(const String & path, size_t num_rows) +{ + const auto cache_key = getKeyForSchemaCache( + fs::path(configuration->getDataSourceDescription()) / path, + configuration->format, + format_settings, + getContext()); + + schema_cache.addNumRows(cache_key, num_rows); +} + +std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfoPtr & object_info) +{ + const auto cache_key = getKeyForSchemaCache( + fs::path(configuration->getDataSourceDescription()) / object_info->relative_path, + configuration->format, + format_settings, + getContext()); + + auto get_last_mod_time = [&]() -> std::optional + { + return object_info->metadata && object_info->metadata->last_modified + ? object_info->metadata->last_modified->epochMicroseconds() + : 0; + }; + return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); +} + +StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader(size_t processor) +{ + ObjectInfoPtr object_info; + do + { + object_info = file_iterator->next(processor); + if (!object_info || object_info->relative_path.empty()) + return {}; + + if (!object_info->metadata) + object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + } + while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); + + QueryPipelineBuilder builder; + std::shared_ptr source; + std::unique_ptr read_buf; + + std::optional num_rows_from_cache = need_only_count + && getContext()->getSettingsRef().use_cache_for_count_from_files + ? tryGetNumRowsFromCache(object_info) + : std::nullopt; + + if (num_rows_from_cache) + { + /// We should not return single chunk with all number of rows, + /// because there is a chance that this chunk will be materialized later + /// (it can cause memory problems even with default values in columns or when virtual columns are requested). + /// Instead, we use special ConstChunkGenerator that will generate chunks + /// with max_block_size rows until total number of rows is reached. + builder.init(Pipe(std::make_shared( + read_from_format_info.format_header, *num_rows_from_cache, max_block_size))); + } + else + { + const auto compression_method = chooseCompressionMethod(object_info->relative_path, configuration->compression_method); + const auto max_parsing_threads = need_only_count ? std::optional(1) : std::nullopt; + read_buf = createReadBuffer(object_info->relative_path, object_info->metadata->size_bytes); + + auto input_format = FormatFactory::instance().getInput( + configuration->format, *read_buf, read_from_format_info.format_header, + getContext(), max_block_size, format_settings, max_parsing_threads, + std::nullopt, /* is_remote_fs */ true, compression_method); + + if (key_condition) + input_format->setKeyCondition(key_condition); + + if (need_only_count) + input_format->needOnlyCount(); + + builder.init(Pipe(input_format)); + + if (columns_desc.hasDefaults()) + { + builder.addSimpleTransform( + [&](const Block & header) + { + return std::make_shared(header, columns_desc, *input_format, getContext()); + }); + } + + source = input_format; + } + + /// Add ExtractColumnsTransform to extract requested columns/subcolumns + /// from chunk read by IInputFormat. + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, read_from_format_info.requested_columns); + }); + + auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + auto current_reader = std::make_unique(*pipeline); + + ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); + + return ReaderHolder( + object_info, std::move(read_buf), std::move(source), std::move(pipeline), std::move(current_reader)); +} + +std::future StorageObjectStorageSource::createReaderAsync(size_t processor) +{ + return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); +} + +std::unique_ptr StorageObjectStorageSource::createReadBuffer(const String & key, size_t object_size) +{ + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); + read_settings.enable_filesystem_cache = false; + read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; + + const bool object_too_small = object_size <= 2 * getContext()->getSettings().max_download_buffer_size; + const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; + read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; + + // Create a read buffer that will prefetch the first ~1 MB of the file. + // When reading lots of tiny files, this prefetching almost doubles the throughput. + // For bigger files, parallel reading is more useful. + if (use_prefetch) + { + LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); + + auto async_reader = object_storage->readObjects( + StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, read_settings); + + async_reader->setReadUntilEnd(); + if (read_settings.remote_fs_prefetch) + async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); + + return async_reader; + } + else + { + /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. + return object_storage->readObject(StoredObject(key), read_settings); + } +} + +StorageObjectStorageSource::GlobIterator::GlobIterator( ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, + ConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, + ContextPtr context_, ObjectInfos * read_keys_, + size_t list_object_keys_size, std::function file_progress_callback_) - : object_storage(object_storage_) + : WithContext(context_) + , object_storage(object_storage_) , configuration(configuration_) , virtual_columns(virtual_columns_) , read_keys(read_keys_) @@ -81,7 +331,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( { const auto key_with_globs = configuration_->getPath(); const auto key_prefix = configuration->getPathWithoutGlob(); - object_storage_iterator = object_storage->iterate(key_prefix); + object_storage_iterator = object_storage->iterate(key_prefix, list_object_keys_size); matcher = std::make_unique(makeRegexpPatternFromGlobs(key_with_globs)); if (matcher->ok()) @@ -113,13 +363,11 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } } -template -StorageObjectStorageSource::ObjectInfoPtr -StorageObjectStorageSource::GlobIterator::next(size_t /* processor */) +ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor */) { std::lock_guard lock(next_mutex); - if (is_finished && index >= object_infos.size()) + if (is_finished) return {}; bool need_new_batch = object_infos.empty() || index >= object_infos.size(); @@ -130,9 +378,10 @@ StorageObjectStorageSource::GlobIterator::next(size_t /* proces while (new_batch.empty()) { auto result = object_storage_iterator->getCurrentBatchAndScheduleNext(); + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {}", result.has_value()); if (result.has_value()) { - new_batch = result.value(); + new_batch = std::move(result.value()); } else { @@ -169,7 +418,8 @@ StorageObjectStorageSource::GlobIterator::next(size_t /* proces { for (const auto & object_info : object_infos) { - file_progress_callback(FileProgress(0, object_info->metadata.size_bytes)); + chassert(object_info->metadata); + file_progress_callback(FileProgress(0, object_info->metadata->size_bytes)); } } } @@ -181,10 +431,9 @@ StorageObjectStorageSource::GlobIterator::next(size_t /* proces return object_infos[current_index]; } -template -StorageObjectStorageSource::KeysIterator::KeysIterator( +StorageObjectStorageSource::KeysIterator::KeysIterator( ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, + ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, std::function file_progress_callback_) @@ -199,15 +448,13 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( /// TODO: should we add metadata if we anyway fetch it if file_progress_callback is passed? for (auto && key : keys) { - auto object_info = std::make_shared(key, ObjectMetadata{}); + auto object_info = std::make_shared(key); read_keys_->emplace_back(object_info); } } } -template -StorageObjectStorageSource::ObjectInfoPtr -StorageObjectStorageSource::KeysIterator::next(size_t /* processor */) +ObjectInfoPtr StorageObjectStorageSource::KeysIterator::next(size_t /* processor */) { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= keys.size()) @@ -225,240 +472,4 @@ StorageObjectStorageSource::KeysIterator::next(size_t /* proces return std::make_shared(key, metadata); } -template -Chunk StorageObjectStorageSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (const auto * input_format = reader.getInputFormat()) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, - read_from_format_info.requested_virtual_columns, - fs::path(configuration->getNamespace()) / reader.getRelativePath(), - reader.getObjectInfo().metadata.size_bytes); - - return chunk; - } - - if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); - - total_rows_in_file = 0; - - assert(reader_future.valid()); - reader = reader_future.get(); - - if (!reader) - break; - - /// Even if task is finished the thread may be not freed in pool. - /// So wait until it will be freed before scheduling a new task. - create_reader_pool.wait(); - reader_future = createReaderAsync(); - } - - return {}; -} - -template -void StorageObjectStorageSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - String source = fs::path(configuration->getDataSourceDescription()) / path; - auto cache_key = getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); - Storage::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -template -std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfoPtr & object_info) -{ - String source = fs::path(configuration->getDataSourceDescription()) / object_info->relative_path; - auto cache_key = getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - auto last_mod = object_info->metadata.last_modified; - if (last_mod) - return last_mod->epochTime(); - else - { - object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - return object_info->metadata.last_modified->epochMicroseconds(); - } - }; - return Storage::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -template -StorageObjectStorageSource::StorageObjectStorageSource( - String name_, - ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, - const ReadFromFormatInfo & info, - std::optional format_settings_, - ContextPtr context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_) - :ISource(info.source_header, false) - , WithContext(context_) - , name(std::move(name_)) - , object_storage(object_storage_) - , configuration(configuration_) - , format_settings(format_settings_) - , max_block_size(max_block_size_) - , need_only_count(need_only_count_) - , read_from_format_info(info) - , columns_desc(info.columns_description) - , file_iterator(file_iterator_) - , create_reader_pool(StorageSettings::ObjectStorageThreads(), - StorageSettings::ObjectStorageThreadsActive(), - StorageSettings::ObjectStorageThreadsScheduled(), 1) - , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "Reader")) -{ - reader = createReader(); - if (reader) - reader_future = createReaderAsync(); -} - -template -StorageObjectStorageSource::~StorageObjectStorageSource() -{ - create_reader_pool.wait(); -} - -template -StorageObjectStorageSource::ReaderHolder -StorageObjectStorageSource::createReader(size_t processor) -{ - auto object_info = file_iterator->next(processor); - if (object_info->relative_path.empty()) - return {}; - - if (object_info->metadata.size_bytes == 0) - object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - - QueryPipelineBuilder builder; - std::shared_ptr source; - std::unique_ptr read_buf; - std::optional num_rows_from_cache = need_only_count - && getContext()->getSettingsRef().use_cache_for_count_from_files - ? tryGetNumRowsFromCache(object_info) - : std::nullopt; - - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - source = std::make_shared( - read_from_format_info.format_header, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - auto compression_method = chooseCompressionMethod( - object_info->relative_path, configuration->compression_method); - - read_buf = createReadBuffer(object_info->relative_path, object_info->metadata.size_bytes); - - auto input_format = FormatFactory::instance().getInput( - configuration->format, *read_buf, read_from_format_info.format_header, - getContext(), max_block_size, format_settings, max_parsing_threads, - std::nullopt, /* is_remote_fs */ true, compression_method); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - - if (columns_desc.hasDefaults()) - { - builder.addSimpleTransform( - [&](const Block & header) - { - return std::make_shared(header, columns_desc, *input_format, getContext()); - }); - } - - source = input_format; - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, read_from_format_info.requested_columns); - }); - - auto pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - auto current_reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - - return ReaderHolder{object_info, std::move(read_buf), - std::move(source), std::move(pipeline), std::move(current_reader)}; -} - -template -std::future::ReaderHolder> -StorageObjectStorageSource::createReaderAsync(size_t processor) -{ - return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); -} - -template -std::unique_ptr StorageObjectStorageSource::createReadBuffer(const String & key, size_t object_size) -{ - auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); - read_settings.enable_filesystem_cache = false; - read_settings.remote_read_min_bytes_for_seek = read_settings.remote_fs_buffer_size; - - // auto download_buffer_size = getContext()->getSettings().max_download_buffer_size; - // const bool object_too_small = object_size <= 2 * download_buffer_size; - - // Create a read buffer that will prefetch the first ~1 MB of the file. - // When reading lots of tiny files, this prefetching almost doubles the throughput. - // For bigger files, parallel reading is more useful. - // if (object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) - // { - // LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); - - // auto async_reader = object_storage->readObjects( - // StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, read_settings); - - // async_reader->setReadUntilEnd(); - // if (read_settings.remote_fs_prefetch) - // async_reader->prefetch(DEFAULT_PREFETCH_PRIORITY); - - // return async_reader; - // } - // else - return object_storage->readObject(StoredObject(key), read_settings); -} - -template class StorageObjectStorageSource; -template class StorageObjectStorageSource; -template class StorageObjectStorageSource; - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index f68a5d47456..0d6a6b71271 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -1,31 +1,19 @@ #pragma once -#include +#include +#include #include #include +#include +#include namespace DB { -template -class StorageObjectStorageSource : public ISource, WithContext +class StorageObjectStorageSource : public SourceWithKeyCondition, WithContext { friend class StorageS3QueueSource; public: - using Source = StorageObjectStorageSource; - using Storage = StorageObjectStorage; - using ObjectInfo = Storage::ObjectInfo; - using ObjectInfoPtr = Storage::ObjectInfoPtr; - using ObjectInfos = Storage::ObjectInfos; - - class IIterator : public WithContext - { - public: - virtual ~IIterator() = default; - - virtual size_t estimatedKeysCount() = 0; - virtual ObjectInfoPtr next(size_t processor) = 0; - }; - + class IIterator; class ReadTaskIterator; class GlobIterator; class KeysIterator; @@ -33,13 +21,16 @@ public: StorageObjectStorageSource( String name_, ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration, + ConfigurationPtr configuration, const ReadFromFormatInfo & info, std::optional format_settings_, + const StorageObjectStorageSettings & query_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - bool need_only_count_); + bool need_only_count_, + SchemaCache & schema_cache_, + std::shared_ptr reader_pool_); ~StorageObjectStorageSource() override; @@ -48,32 +39,35 @@ public: Chunk generate() override; static std::shared_ptr createFileIterator( - Storage::ConfigurationPtr configuration, + ConfigurationPtr configuration, ObjectStoragePtr object_storage, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, + size_t list_object_keys_size, std::function file_progress_callback = {}); protected: - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const ObjectInfoPtr & object_info); - const String name; ObjectStoragePtr object_storage; - const Storage::ConfigurationPtr configuration; + const ConfigurationPtr configuration; const std::optional format_settings; + const StorageObjectStorageSettings query_settings; const UInt64 max_block_size; const bool need_only_count; const ReadFromFormatInfo read_from_format_info; - + const std::shared_ptr create_reader_pool; ColumnsDescription columns_desc; std::shared_ptr file_iterator; - size_t total_rows_in_file = 0; + SchemaCache & schema_cache; + bool initialized = false; - struct ReaderHolder + size_t total_rows_in_file = 0; + LoggerPtr log = getLogger("StorageObjectStorageSource"); + + struct ReaderHolder : private boost::noncopyable { public: ReaderHolder( @@ -86,15 +80,15 @@ protected: , read_buf(std::move(read_buf_)) , source(std::move(source_)) , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) - { - } + , reader(std::move(reader_)) {} ReaderHolder() = default; - ReaderHolder(const ReaderHolder & other) = delete; - ReaderHolder & operator=(const ReaderHolder & other) = delete; ReaderHolder(ReaderHolder && other) noexcept { *this = std::move(other); } + explicit operator bool() const { return reader != nullptr; } + PullingPipelineExecutor * operator->() { return reader.get(); } + const PullingPipelineExecutor * operator->() const { return reader.get(); } + ReaderHolder & operator=(ReaderHolder && other) noexcept { /// The order of destruction is important. @@ -107,9 +101,6 @@ protected: return *this; } - explicit operator bool() const { return reader != nullptr; } - PullingPipelineExecutor * operator->() { return reader.get(); } - const PullingPipelineExecutor * operator->() const { return reader.get(); } const String & getRelativePath() const { return object_info->relative_path; } const ObjectInfo & getObjectInfo() const { return *object_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } @@ -123,20 +114,29 @@ protected: }; ReaderHolder reader; - LoggerPtr log = getLogger("StorageObjectStorageSource"); - ThreadPool create_reader_pool; ThreadPoolCallbackRunner create_reader_scheduler; std::future reader_future; /// Recreate ReadBuffer and Pipeline for each file. ReaderHolder createReader(size_t processor = 0); std::future createReaderAsync(size_t processor = 0); - std::unique_ptr createReadBuffer(const String & key, size_t object_size); + + void addNumRowsToCache(const String & path, size_t num_rows); + std::optional tryGetNumRowsFromCache(const ObjectInfoPtr & object_info); + void lazyInitialize(size_t processor); }; -template -class StorageObjectStorageSource::ReadTaskIterator : public IIterator +class StorageObjectStorageSource::IIterator +{ +public: + virtual ~IIterator() = default; + + virtual size_t estimatedKeysCount() = 0; + virtual ObjectInfoPtr next(size_t processor) = 0; +}; + +class StorageObjectStorageSource::ReadTaskIterator : public IIterator { public: explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {} @@ -149,16 +149,17 @@ private: ReadTaskCallback callback; }; -template -class StorageObjectStorageSource::GlobIterator : public IIterator +class StorageObjectStorageSource::GlobIterator : public IIterator, WithContext { public: GlobIterator( ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, + ConfigurationPtr configuration_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns_, + ContextPtr context_, ObjectInfos * read_keys_, + size_t list_object_keys_size, std::function file_progress_callback_ = {}); ~GlobIterator() override = default; @@ -169,7 +170,7 @@ public: private: ObjectStoragePtr object_storage; - Storage::ConfigurationPtr configuration; + ConfigurationPtr configuration; ActionsDAGPtr filter_dag; NamesAndTypesList virtual_columns; @@ -189,13 +190,12 @@ private: std::function file_progress_callback; }; -template -class StorageObjectStorageSource::KeysIterator : public IIterator +class StorageObjectStorageSource::KeysIterator : public IIterator { public: KeysIterator( ObjectStoragePtr object_storage_, - Storage::ConfigurationPtr configuration_, + ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, std::function file_progress_callback = {}); @@ -208,7 +208,7 @@ public: private: const ObjectStoragePtr object_storage; - const Storage::ConfigurationPtr configuration; + const ConfigurationPtr configuration; const NamesAndTypesList virtual_columns; const std::function file_progress_callback; const std::vector keys; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h b/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h new file mode 100644 index 00000000000..51be7419e1c --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h @@ -0,0 +1,11 @@ +#include + +namespace DB +{ + +using ConfigurationPtr = StorageObjectStorageConfigurationPtr; +using ObjectInfo = RelativePathWithMetadata; +using ObjectInfoPtr = std::shared_ptr; +using ObjectInfos = std::vector; + +} diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index bc9f93690f5..f7ab37490e1 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -8,18 +8,6 @@ namespace DB { -static void initializeConfiguration( - StorageObjectStorageConfiguration & configuration, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); - else - configuration.fromAST(engine_args, local_context, with_table_structure); -} - template static std::shared_ptr> createStorageObjectStorage( const StorageFactory::Arguments & args, @@ -82,7 +70,7 @@ void registerStorageAzure(StorageFactory & factory) { auto context = args.getLocalContext(); auto configuration = std::make_shared(); - initializeConfiguration(*configuration, args.engine_args, context, false); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); return createStorageObjectStorage(args, configuration, "Azure", context); }, { @@ -101,7 +89,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) { auto context = args.getLocalContext(); auto configuration = std::make_shared(); - initializeConfiguration(*configuration, args.engine_args, context, false); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); return createStorageObjectStorage(args, configuration, name, context); }, { @@ -136,7 +124,7 @@ void registerStorageHDFS(StorageFactory & factory) { auto context = args.getLocalContext(); auto configuration = std::make_shared(); - initializeConfiguration(*configuration, args.engine_args, context, false); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); return createStorageObjectStorage(args, configuration, "HDFS", context); }, { diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index bd34d1ec093..b64aa23d47c 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -35,7 +35,7 @@ StorageS3QueueSource::S3QueueObjectInfo::S3QueueObjectInfo( const std::string & key_, const ObjectMetadata & object_metadata_, Metadata::ProcessingNodeHolderPtr processing_holder_) - : Source::ObjectInfo(key_, object_metadata_) + : ObjectInfo(key_, object_metadata_) , processing_holder(processing_holder_) { } @@ -55,15 +55,15 @@ StorageS3QueueSource::FileIterator::FileIterator( if (sharded_processing) { for (const auto & id : metadata->getProcessingIdsForShard(current_shard)) - sharded_keys.emplace(id, std::deque{}); + sharded_keys.emplace(id, std::deque{}); } } -StorageS3QueueSource::Source::ObjectInfoPtr StorageS3QueueSource::FileIterator::next(size_t processor) +StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::next(size_t processor) { while (!shutdown_called) { - Source::ObjectInfoPtr val{nullptr}; + ObjectInfoPtr val{nullptr}; { std::unique_lock lk(sharded_keys_mutex, std::defer_lock); @@ -140,7 +140,7 @@ StorageS3QueueSource::Source::ObjectInfoPtr StorageS3QueueSource::FileIterator:: if (processing_holder) { - return std::make_shared(val->relative_path, val->metadata, processing_holder); + return std::make_shared(val->relative_path, val->metadata.value(), processing_holder); } else if (sharded_processing && metadata->getFileStatus(val->relative_path)->state == S3QueueFilesMetadata::FileStatus::State::Processing) @@ -161,7 +161,7 @@ size_t StorageS3QueueSource::FileIterator::estimatedKeysCount() StorageS3QueueSource::StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -273,7 +273,8 @@ Chunk StorageS3QueueSource::generate() file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, reader.getRelativePath(), reader.getObjectInfo().metadata.size_bytes); + VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( + chunk, requested_virtual_columns, reader.getRelativePath(), reader.getObjectInfo().metadata->size_bytes); return chunk; } } @@ -311,7 +312,7 @@ Chunk StorageS3QueueSource::generate() /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. - internal_source->create_reader_pool.wait(); + internal_source->create_reader_pool->wait(); reader_future = internal_source->createReaderAsync(processing_id); } diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fcf5c5c0160..2bdac7f2311 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include @@ -22,16 +22,19 @@ class StorageS3QueueSource : public ISource, WithContext { public: using Storage = StorageObjectStorage; - using Source = StorageObjectStorageSource; using ConfigurationPtr = Storage::ConfigurationPtr; - using GlobIterator = Source::GlobIterator; + using GlobIterator = StorageObjectStorageSource::GlobIterator; using ZooKeeperGetter = std::function; using RemoveFileFunc = std::function; using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; + using ReaderHolder = StorageObjectStorageSource::ReaderHolder; using Metadata = S3QueueFilesMetadata; + using ObjectInfo = RelativePathWithMetadata; + using ObjectInfoPtr = std::shared_ptr; + using ObjectInfos = std::vector; - struct S3QueueObjectInfo : public Source::ObjectInfo + struct S3QueueObjectInfo : public ObjectInfo { S3QueueObjectInfo( const std::string & key_, @@ -41,7 +44,7 @@ public: Metadata::ProcessingNodeHolderPtr processing_holder; }; - class FileIterator : public Source::IIterator + class FileIterator : public StorageObjectStorageSource::IIterator { public: FileIterator( @@ -53,7 +56,7 @@ public: /// Note: /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - Source::ObjectInfoPtr next(size_t processor) override; + ObjectInfoPtr next(size_t processor) override; size_t estimatedKeysCount() override; @@ -66,14 +69,14 @@ public: const bool sharded_processing; const size_t current_shard; - std::unordered_map> sharded_keys; + std::unordered_map> sharded_keys; std::mutex sharded_keys_mutex; }; StorageS3QueueSource( String name_, const Block & header_, - std::unique_ptr internal_source_, + std::unique_ptr internal_source_, std::shared_ptr files_metadata_, size_t processing_id_, const S3QueueAction & action_, @@ -97,7 +100,7 @@ private: const S3QueueAction action; const size_t processing_id; const std::shared_ptr files_metadata; - const std::shared_ptr internal_source; + const std::shared_ptr internal_source; const NamesAndTypesList requested_virtual_columns; const std::atomic & shutdown_called; const std::atomic & table_is_being_dropped; @@ -107,8 +110,8 @@ private: RemoveFileFunc remove_file_func; LoggerPtr log; - Source::ReaderHolder reader; - std::future reader_future; + ReaderHolder reader; + std::future reader_future; std::atomic initialized{false}; size_t processed_rows_from_file = 0; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 942ce7973ef..70dd8f27d71 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -3,7 +3,7 @@ #if USE_AWS_S3 #include -#include +#include #include namespace DB diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index fa7132f705a..fc4ef77ebb9 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -341,16 +341,23 @@ std::shared_ptr StorageS3Queue::createSource( size_t max_block_size, ContextPtr local_context) { - auto internal_source = std::make_unique( + auto threadpool = std::make_shared(CurrentMetrics::ObjectStorageS3Threads, + CurrentMetrics::ObjectStorageS3ThreadsActive, + CurrentMetrics::ObjectStorageS3ThreadsScheduled, + /* max_threads */1); + auto internal_source = std::make_unique( getName(), object_storage, configuration, info, format_settings, + S3StorageSettings::create(local_context->getSettingsRef()), local_context, max_block_size, file_iterator, - false); + false, + Storage::getSchemaCache(local_context), + threadpool); auto file_deleter = [=, this](const std::string & path) mutable { @@ -555,25 +562,14 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const } } -std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr , const ActionsDAG::Node * predicate) +std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { - auto glob_iterator = std::make_unique(object_storage, configuration, predicate, virtual_columns, nullptr); - + auto settings = S3StorageSettings::create(local_context->getSettingsRef()); + auto glob_iterator = std::make_unique( + object_storage, configuration, predicate, virtual_columns, local_context, nullptr, settings.list_object_keys_size); return std::make_shared(files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called); } -static void initializeConfiguration( - StorageObjectStorageConfiguration & configuration, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); - else - configuration.fromAST(engine_args, local_context, with_table_structure); -} - void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) { factory.registerStorage( @@ -585,7 +581,7 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = std::make_shared(); - initializeConfiguration(*configuration, args.engine_args, args.getContext(), false); + StorageObjectStorageConfiguration::initialize(*configuration, 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/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 88f9bd65093..46a8b8d82c1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -27,7 +27,6 @@ class StorageS3Queue : public IStorage, WithContext { public: using Storage = StorageObjectStorage; - using Source = StorageObjectStorageSource; using ConfigurationPtr = Storage::ConfigurationPtr; StorageS3Queue( diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 884e1f5c4a2..0ffa1460d78 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -10,7 +10,7 @@ # include # include # include -#include +#include #include #include diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index d009a9347f3..de46c13af37 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include @@ -27,20 +27,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -static void initializeConfiguration( - StorageObjectStorageConfiguration & configuration, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); - else - configuration.fromAST(engine_args, local_context, with_table_structure); -} - template -ObjectStoragePtr TableFunctionObjectStorage::getObjectStorage(const ContextPtr & context, bool create_readonly) const +ObjectStoragePtr TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::getObjectStorage(const ContextPtr & context, bool create_readonly) const { if (!object_storage) object_storage = configuration->createOrUpdateObjectStorage(context, create_readonly); @@ -48,7 +37,8 @@ ObjectStoragePtr TableFunctionObjectStorage -std::vector TableFunctionObjectStorage::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +std::vector TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const { auto & table_function_node = query_node_table_function->as(); auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); @@ -65,16 +55,18 @@ std::vector TableFunctionObjectStorage -void TableFunctionObjectStorage::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) +void TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) { Configuration::addStructureToArgs(args, structure, context); } template -void TableFunctionObjectStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) +void TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) { configuration = std::make_shared(); - initializeConfiguration(*configuration, engine_args, local_context, true); + StorageObjectStorageConfiguration::initialize(*configuration, engine_args, local_context, true); } template @@ -91,7 +83,8 @@ void TableFunctionObjectStorage::par } template -ColumnsDescription TableFunctionObjectStorage::getActualTableStructure(ContextPtr context, bool is_insert_query) const +ColumnsDescription TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const { if (configuration->structure == "auto") { @@ -104,13 +97,15 @@ ColumnsDescription TableFunctionObjectStorage -bool TableFunctionObjectStorage::supportsReadingSubsetOfColumns(const ContextPtr & context) +bool TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::supportsReadingSubsetOfColumns(const ContextPtr & context) { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); } template -std::unordered_set TableFunctionObjectStorage::getVirtualsToCheckBeforeUsingStructureHint() const +std::unordered_set TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::getVirtualsToCheckBeforeUsingStructureHint() const { auto virtual_column_names = StorageObjectStorage::getVirtualColumnNames(); return {virtual_column_names.begin(), virtual_column_names.end()}; @@ -166,15 +161,33 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) factory.registerFunction>( { + .documentation = + { + .description=R"(The table function can be used to read the data stored on GCS.)", + .examples{{"gcs", "SELECT * FROM gcs(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, .allow_readonly = false }); factory.registerFunction>( { + .documentation = + { + .description=R"(The table function can be used to read the data stored on COSN.)", + .examples{{"cosn", "SELECT * FROM cosn(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, .allow_readonly = false }); factory.registerFunction>( { + .documentation = + { + .description=R"(The table function can be used to read the data stored on OSS.)", + .examples{{"oss", "SELECT * FROM oss(url, access_key_id, secret_access_key)", ""} + }, + .categories{"DataLake"}}, .allow_readonly = false }); #endif diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 1d27a857cea..8e6c96a3f2a 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -76,8 +75,8 @@ void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) 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.)", - .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, + .description=R"(The table function can be used to read the data stored on S3 in parallel for many nodes in a specified cluster.)", + .examples{{"s3Cluster", "SELECT * FROM s3Cluster(cluster, url, format, structure)", ""}}}, .allow_readonly = false } ); @@ -95,7 +94,14 @@ void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) #endif #if USE_HDFS - factory.registerFunction(); + factory.registerFunction( + { + .documentation = { + .description=R"(The table function can be used to read the data stored on HDFS in parallel for many nodes in a specified cluster.)", + .examples{{"HDFSCluster", "SELECT * FROM HDFSCluster(cluster_name, uri, format)", ""}}}, + .allow_readonly = false + } + ); #endif } From 84b0fe670a4d73cc0b5c26bb922e90369025dae6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 13 Feb 2024 17:03:11 +0100 Subject: [PATCH 003/158] Refactor data lakes --- src/Backups/BackupIO_AzureBlobStorage.h | 2 +- .../registerBackupEngineAzureBlobStorage.cpp | 5 +- src/CMakeLists.txt | 7 +- .../AzureBlobStorage/AzureObjectStorage.cpp | 6 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 28 ++-- src/Disks/ObjectStorages/IObjectStorage.h | 4 +- .../ObjectStorageIteratorAsync.cpp | 30 ++++- .../ObjectStorageIteratorAsync.h | 6 +- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- .../DataLakes/DeltaLakeMetadataParser.h | 26 ---- src/Storages/DataLakes/HudiMetadataParser.h | 18 --- src/Storages/DataLakes/IStorageDataLake.h | 98 -------------- .../DataLakes/Iceberg/StorageIceberg.cpp | 11 -- src/Storages/DataLakes/StorageDeltaLake.h | 20 --- src/Storages/DataLakes/StorageHudi.h | 20 --- src/Storages/DataLakes/registerDataLakes.cpp | 50 ------- .../Configuration.cpp} | 49 +++---- .../Configuration.h} | 11 +- .../ObjectStorage/DataLakes/Common.cpp | 28 ++++ src/Storages/ObjectStorage/DataLakes/Common.h | 15 +++ .../DataLakes/DeltaLakeMetadata.cpp} | 110 +++++++-------- .../DataLakes/DeltaLakeMetadata.h | 48 +++++++ .../DataLakes/HudiMetadata.cpp} | 55 ++++---- .../ObjectStorage/DataLakes/HudiMetadata.h | 51 +++++++ .../DataLakes/IDataLakeMetadata.h | 19 +++ .../DataLakes/IStorageDataLake.h} | 58 ++++---- .../DataLakes}/IcebergMetadata.cpp | 36 ++--- .../DataLakes}/IcebergMetadata.h | 40 +++--- .../DataLakes/registerDataLakeStorages.cpp | 83 ++++++++++++ .../ObjectStorage/HDFS/Configuration.cpp | 57 ++++++++ .../ObjectStorage/HDFS/Configuration.h | 45 +++++++ .../ObjectStorage/HDFSConfiguration.h | 81 ----------- .../ObjectStorage/ReadBufferIterator.cpp | 4 +- .../ReadFromStorageObjectStorage.cpp | 1 - .../Configuration.cpp} | 30 +++-- .../{S3Configuration.h => S3/Configuration.h} | 15 ++- .../ObjectStorage/StorageObjectStorage.cpp | 10 +- .../ObjectStorage/StorageObjectStorage.h | 5 +- .../StorageObjectStorageCluster.cpp | 2 +- .../StorageObjectStorageCluster.h | 3 + .../StorageObjectStorageConfiguration.cpp | 2 +- ....h => StorageObjectStorageConfiguration.h} | 3 +- .../StorageObjectStorageSink.cpp | 127 ++++++++++++++++++ .../ObjectStorage/StorageObjectStorageSink.h | 113 ++-------------- .../StorageObjectStorageSource.cpp | 33 ++++- .../StorageObjectStorageSource.h | 22 +-- .../StorageObjectStorage_fwd_internal.h | 3 +- .../registerStorageObjectStorage.cpp | 12 +- src/Storages/ObjectStorageConfiguration.h | 0 src/Storages/S3Queue/S3QueueTableMetadata.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 9 +- .../StorageSystemSchemaInferenceCache.cpp | 2 +- src/TableFunctions/ITableFunctionDataLake.h | 76 +++++++---- src/TableFunctions/TableFunctionDeltaLake.cpp | 33 ----- src/TableFunctions/TableFunctionHudi.cpp | 31 ----- src/TableFunctions/TableFunctionIceberg.cpp | 37 ----- .../TableFunctionObjectStorage.cpp | 22 ++- .../TableFunctionObjectStorage.h | 13 +- .../TableFunctionObjectStorageCluster.cpp | 8 +- .../registerDataLakeTableFunctions.cpp | 69 ++++++++++ src/TableFunctions/registerTableFunctions.cpp | 3 +- src/TableFunctions/registerTableFunctions.h | 10 +- 62 files changed, 946 insertions(+), 873 deletions(-) delete mode 100644 src/Storages/DataLakes/DeltaLakeMetadataParser.h delete mode 100644 src/Storages/DataLakes/HudiMetadataParser.h delete mode 100644 src/Storages/DataLakes/IStorageDataLake.h delete mode 100644 src/Storages/DataLakes/Iceberg/StorageIceberg.cpp delete mode 100644 src/Storages/DataLakes/StorageDeltaLake.h delete mode 100644 src/Storages/DataLakes/StorageHudi.h delete mode 100644 src/Storages/DataLakes/registerDataLakes.cpp rename src/Storages/ObjectStorage/{AzureConfiguration.cpp => AzureBlob/Configuration.cpp} (92%) rename src/Storages/ObjectStorage/{AzureConfiguration.h => AzureBlob/Configuration.h} (88%) create mode 100644 src/Storages/ObjectStorage/DataLakes/Common.cpp create mode 100644 src/Storages/ObjectStorage/DataLakes/Common.h rename src/Storages/{DataLakes/DeltaLakeMetadataParser.cpp => ObjectStorage/DataLakes/DeltaLakeMetadata.cpp} (79%) create mode 100644 src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h rename src/Storages/{DataLakes/HudiMetadataParser.cpp => ObjectStorage/DataLakes/HudiMetadata.cpp} (68%) create mode 100644 src/Storages/ObjectStorage/DataLakes/HudiMetadata.h create mode 100644 src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h rename src/Storages/{DataLakes/Iceberg/StorageIceberg.h => ObjectStorage/DataLakes/IStorageDataLake.h} (61%) rename src/Storages/{DataLakes/Iceberg => ObjectStorage/DataLakes}/IcebergMetadata.cpp (96%) rename src/Storages/{DataLakes/Iceberg => ObjectStorage/DataLakes}/IcebergMetadata.h (76%) create mode 100644 src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp create mode 100644 src/Storages/ObjectStorage/HDFS/Configuration.cpp create mode 100644 src/Storages/ObjectStorage/HDFS/Configuration.h delete mode 100644 src/Storages/ObjectStorage/HDFSConfiguration.h rename src/Storages/ObjectStorage/{S3Configuration.cpp => S3/Configuration.cpp} (97%) rename src/Storages/ObjectStorage/{S3Configuration.h => S3/Configuration.h} (81%) rename src/Storages/ObjectStorage/{StorageObejctStorageConfiguration.h => StorageObjectStorageConfiguration.h} (99%) create mode 100644 src/Storages/ObjectStorage/StorageObjectStorageSink.cpp delete mode 100644 src/Storages/ObjectStorageConfiguration.h delete mode 100644 src/TableFunctions/TableFunctionDeltaLake.cpp delete mode 100644 src/TableFunctions/TableFunctionHudi.cpp delete mode 100644 src/TableFunctions/TableFunctionIceberg.cpp create mode 100644 src/TableFunctions/registerDataLakeTableFunctions.cpp diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 99002c53769..9f1702cb3a3 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 9408c7ccdcf..c4c04bbc057 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #endif @@ -59,9 +59,6 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) if (!config.has(config_prefix)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", id_arg); - if (!config.has(config_prefix)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no `{}` in config", config_prefix); - if (config.has(config_prefix + ".connection_string")) { configuration.connection_url = config.getString(config_prefix + ".connection_string"); diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 50130e6abd0..118e0131b37 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -105,6 +105,7 @@ add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhous add_headers_and_sources(dbms Disks/IO) add_headers_and_sources(dbms Disks/ObjectStorages) +add_headers_and_sources(dbms Disks/ObjectStorages) if (TARGET ch_contrib::sqlite) add_headers_and_sources(dbms Databases/SQLite) endif() @@ -117,9 +118,11 @@ if (TARGET ch_contrib::nats_io) add_headers_and_sources(dbms Storages/NATS) endif() -add_headers_and_sources(dbms Storages/DataLakes) -add_headers_and_sources(dbms Storages/DataLakes/Iceberg) add_headers_and_sources(dbms Storages/ObjectStorage) +add_headers_and_sources(dbms Storages/ObjectStorage/AzureBlob) +add_headers_and_sources(dbms Storages/ObjectStorage/S3) +add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) +add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) add_headers_and_sources(dbms Common/NamedCollections) if (TARGET ch_contrib::amqp_cpp) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index bbbb5357505..bcc75f91e2a 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -323,10 +323,8 @@ void AzureObjectStorage::removeObjectsIfExist(const StoredObjects & objects) { removeObjectIfExists(object); } - } - ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) const { auto client_ptr = client.get(); @@ -338,9 +336,9 @@ ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) c { result.attributes.emplace(); for (const auto & [key, value] : properties.Metadata) - (*result.attributes)[key] = value; + result.attributes[key] = value; } - result.last_modified.emplace(static_cast(properties.LastModified).time_since_epoch().count()); + result.last_modified = static_cast(properties.LastModified).time_since_epoch().count(); return result; } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index cc9ee3db505..9f9efad9615 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -404,26 +404,20 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( { for (const auto & key : keys) { - auto meta = source_object_storage->getObjectMetadata(key); - auto object_attributes = meta.attributes; + auto metadata = source_object_storage->getObjectMetadata(key); + auto object_attributes = metadata.attributes; String path; - if (object_attributes.has_value()) + /// Restore file if object has 'path' in metadata. + auto path_entry = object_attributes.find("path"); + if (path_entry == object_attributes.end()) { - /// Restore file if object has 'path' in metadata. - auto path_entry = object_attributes->find("path"); - if (path_entry == object_attributes->end()) - { - /// Such keys can remain after migration, we can skip them. - LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key); - continue; - } - - path = path_entry->second; - } - else + /// Such keys can remain after migration, we can skip them. + LOG_WARNING(disk->log, "Skip key {} because it doesn't have 'path' in metadata", key); continue; + } + path = path_entry->second; disk->createDirectories(directoryPath(path)); auto object_key = ObjectStorageKey::createAsRelative(disk->object_key_prefix, shrinkKey(source_path, key)); @@ -435,7 +429,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::processRestoreFiles( source_object_storage->copyObjectToAnotherObjectStorage(object_from, object_to, read_settings, write_settings, *disk->object_storage); auto tx = disk->metadata_storage->createTransaction(); - tx->addBlobToMetadata(path, object_key, meta.size_bytes); + tx->addBlobToMetadata(path, object_key, metadata.size_bytes); tx->commit(); LOG_TRACE(disk->log, "Restored file {}", path); @@ -490,7 +484,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject if (send_metadata) revision_counter = revision - 1; - auto object_attributes = *(source_object_storage->getObjectMetadata(object->relative_path).attributes); + auto object_attributes = source_object_storage->getObjectMetadata(object->relative_path).attributes; if (operation == rename) { auto from_path = object_attributes["from_path"]; diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 4955b0e6924..8a5352e71ca 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -47,8 +47,8 @@ using ObjectAttributes = std::map; struct ObjectMetadata { uint64_t size_bytes = 0; - std::optional last_modified; - std::optional attributes; + Poco::Timestamp last_modified; + ObjectAttributes attributes; }; struct RelativePathWithMetadata diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index 62bdd0ed0c8..f441b18d59d 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -11,18 +11,26 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +IObjectStorageIteratorAsync::IObjectStorageIteratorAsync( + CurrentMetrics::Metric threads_metric, + CurrentMetrics::Metric threads_active_metric, + CurrentMetrics::Metric threads_scheduled_metric, + const std::string & thread_name) + : list_objects_pool(threads_metric, threads_active_metric, threads_scheduled_metric, 1) + , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) +{ +} + void IObjectStorageIteratorAsync::nextBatch() { std::lock_guard lock(mutex); if (is_finished) { - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 3"); current_batch.clear(); current_batch_iterator = current_batch.begin(); } else { - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 4"); if (!is_initialized) { outcome_future = scheduleBatch(); @@ -30,13 +38,23 @@ void IObjectStorageIteratorAsync::nextBatch() } chassert(outcome_future.valid()); - auto [batch, has_next] = outcome_future.get(); - current_batch = std::move(batch); + BatchAndHasNext result; + try + { + result = outcome_future.get(); + } + catch (...) + { + is_finished = true; + throw; + } + + current_batch = std::move(result.batch); current_batch_iterator = current_batch.begin(); accumulated_size.fetch_add(current_batch.size(), std::memory_order_relaxed); - if (has_next) + if (result.has_next) outcome_future = scheduleBatch(); else is_finished = true; @@ -100,12 +118,10 @@ std::optional IObjectStorageIteratorAsync::getCurrent if (current_batch_iterator == current_batch.end()) { - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 2"); return std::nullopt; } auto temp_current_batch = std::move(current_batch); - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: here 1: {}", temp_current_batch.size()); nextBatch(); return temp_current_batch; } diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index 8d155f7ec8d..86e5feb3010 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -17,11 +17,7 @@ public: CurrentMetrics::Metric threads_metric, CurrentMetrics::Metric threads_active_metric, CurrentMetrics::Metric threads_scheduled_metric, - const std::string & thread_name) - : list_objects_pool(threads_metric, threads_active_metric, threads_scheduled_metric, 1) - , list_objects_scheduler(threadPoolCallbackRunner(list_objects_pool, thread_name)) - { - } + const std::string & thread_name); void next() override; void nextBatch() override; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d697d90c8a6..36f5bd73ca6 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -480,7 +480,7 @@ BlockIO InterpreterSystemQuery::execute() StorageURL::getSchemaCache(getContext()).clear(); #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) - StorageAzureBlobStorage::getSchemaCache(getContext()).clear(); + StorageAzureBlob::getSchemaCache(getContext()).clear(); #endif break; } diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.h b/src/Storages/DataLakes/DeltaLakeMetadataParser.h deleted file mode 100644 index 251ea3e3f15..00000000000 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.h +++ /dev/null @@ -1,26 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -struct DeltaLakeMetadataParser -{ -public: - DeltaLakeMetadataParser(); - - Strings getFiles( - ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, - ContextPtr context); - -private: - struct Impl; - std::shared_ptr impl; -}; - -} diff --git a/src/Storages/DataLakes/HudiMetadataParser.h b/src/Storages/DataLakes/HudiMetadataParser.h deleted file mode 100644 index 72766a95876..00000000000 --- a/src/Storages/DataLakes/HudiMetadataParser.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -struct HudiMetadataParser -{ - Strings getFiles( - ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, ContextPtr context); -}; - -} diff --git a/src/Storages/DataLakes/IStorageDataLake.h b/src/Storages/DataLakes/IStorageDataLake.h deleted file mode 100644 index 934bf227c42..00000000000 --- a/src/Storages/DataLakes/IStorageDataLake.h +++ /dev/null @@ -1,98 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -template -class IStorageDataLake : public StorageObjectStorage -{ -public: - static constexpr auto name = Name::name; - - using Storage = StorageObjectStorage; - using ConfigurationPtr = Storage::ConfigurationPtr; - - static StoragePtr create( - ConfigurationPtr base_configuration, - ContextPtr context, - const String & engine_name_, - const StorageID & table_id_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment_, - std::optional format_settings_, - bool /* attach */) - { - auto object_storage = base_configuration->createOrUpdateObjectStorage(context); - - auto configuration = base_configuration->clone(); - configuration->getPaths() = MetadataParser().getFiles(object_storage, configuration, context); - - return std::make_shared>( - base_configuration, configuration, object_storage, engine_name_, context, - table_id_, columns_, constraints_, comment_, format_settings_); - } - - String getName() const override { return name; } - - static ColumnsDescription getTableStructureFromData( - ObjectStoragePtr object_storage_, - ConfigurationPtr base_configuration, - const std::optional &, - ContextPtr local_context) - { - auto metadata = parseIcebergMetadata(object_storage_, base_configuration, local_context); - return ColumnsDescription(metadata->getTableSchema()); - } - - std::pair updateConfigurationAndGetCopy(ContextPtr local_context) override - { - std::lock_guard lock(Storage::configuration_update_mutex); - - auto new_object_storage = base_configuration->createOrUpdateObjectStorage(local_context); - bool updated = new_object_storage != nullptr; - if (updated) - Storage::object_storage = new_object_storage; - - auto new_keys = MetadataParser().getFiles(Storage::object_storage, base_configuration, local_context); - - if (updated || new_keys != Storage::configuration->getPaths()) - { - auto updated_configuration = base_configuration->clone(); - /// If metadata wasn't changed, we won't list data files again. - updated_configuration->getPaths() = new_keys; - Storage::configuration = updated_configuration; - } - return {Storage::configuration, Storage::object_storage}; - } - - template - explicit IStorageDataLake( - ConfigurationPtr base_configuration_, - Args &&... args) - : Storage(std::forward(args)...) - , base_configuration(base_configuration_) - { - } - -private: - ConfigurationPtr base_configuration; - LoggerPtr log; -}; - - -} - -#endif diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp b/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp deleted file mode 100644 index ad1a27c312b..00000000000 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.cpp +++ /dev/null @@ -1,11 +0,0 @@ -#include - -#if USE_AWS_S3 && USE_AVRO - -namespace DB -{ - - -} - -#endif diff --git a/src/Storages/DataLakes/StorageDeltaLake.h b/src/Storages/DataLakes/StorageDeltaLake.h deleted file mode 100644 index 07c2205d2df..00000000000 --- a/src/Storages/DataLakes/StorageDeltaLake.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include -#include -#include -#include "config.h" - -namespace DB -{ - -struct StorageDeltaLakeName -{ - static constexpr auto name = "DeltaLake"; -}; - -#if USE_AWS_S3 && USE_PARQUET -using StorageDeltaLakeS3 = IStorageDataLake; -#endif - -} diff --git a/src/Storages/DataLakes/StorageHudi.h b/src/Storages/DataLakes/StorageHudi.h deleted file mode 100644 index 3fd52c82d32..00000000000 --- a/src/Storages/DataLakes/StorageHudi.h +++ /dev/null @@ -1,20 +0,0 @@ -#pragma once - -#include -#include -#include -#include "config.h" - -namespace DB -{ - -struct StorageHudiName -{ - static constexpr auto name = "Hudi"; -}; - -#if USE_AWS_S3 -using StorageHudiS3 = IStorageDataLake; -#endif - -} diff --git a/src/Storages/DataLakes/registerDataLakes.cpp b/src/Storages/DataLakes/registerDataLakes.cpp deleted file mode 100644 index 2647fbce39d..00000000000 --- a/src/Storages/DataLakes/registerDataLakes.cpp +++ /dev/null @@ -1,50 +0,0 @@ -#include -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include - - -namespace DB -{ - -#if USE_PARQUET -void registerStorageDeltaLake(StorageFactory & ) -{ - // factory.registerStorage( - // StorageDeltaLakeName::name, - // [&](const StorageFactory::Arguments & args) - // { - // auto configuration = std::make_shared(); - // return IStorageDataLake::create( - // configuration, args.getContext(), "deltaLake", args.table_id, args.columns, - // args.constraints, args.comment, std::nullopt, args.attach); - // }, - // { - // .supports_settings = false, - // .supports_schema_inference = true, - // .source_access_type = AccessType::S3, - // }); -} -#endif - -#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. - -void registerStorageIceberg(StorageFactory &) -{ - // REGISTER_DATA_LAKE_STORAGE(StorageIceberg, StorageIceberg::name) -} - -#endif - -void registerStorageHudi(StorageFactory &) -{ -} - -} - -#endif diff --git a/src/Storages/ObjectStorage/AzureConfiguration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp similarity index 92% rename from src/Storages/ObjectStorage/AzureConfiguration.cpp rename to src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 04f6f26111b..109918dfc8b 100644 --- a/src/Storages/ObjectStorage/AzureConfiguration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -1,4 +1,7 @@ -#include +#include + +#if USE_AZURE_BLOB_STORAGE + #include #include #include @@ -44,21 +47,19 @@ namespace return !candidate.starts_with("http"); } - bool containerExists(std::unique_ptr & blob_service_client, std::string container_name) + bool containerExists(Azure::Storage::Blobs::BlobServiceClient & blob_service_client, std::string container_name) { Azure::Storage::Blobs::ListBlobContainersOptions options; options.Prefix = container_name; options.PageSizeHint = 1; - auto containers_list_response = blob_service_client->ListBlobContainers(options); + auto containers_list_response = blob_service_client.ListBlobContainers(options); auto containers_list = containers_list_response.BlobContainers; - for (const auto & container : containers_list) - { - if (container_name == container.Name) - return true; - } - return false; + auto it = std::find_if( + containers_list.begin(), containers_list.end(), + [&](const auto & c) { return c.Name == container_name; }); + return it != containers_list.end(); } } @@ -76,19 +77,6 @@ void StorageAzureBlobConfiguration::check(ContextPtr context) const context->getGlobalContext()->getRemoteHostFilter().checkURL(url_to_check); } -StorageObjectStorageConfigurationPtr StorageAzureBlobConfiguration::clone() -{ - auto configuration = std::make_shared(); - configuration->connection_url = connection_url; - configuration->is_connection_string = is_connection_string; - configuration->account_name = account_name; - configuration->account_key = account_key; - configuration->container = container; - configuration->blob_path = blob_path; - configuration->blobs_paths = blobs_paths; - return configuration; -} - StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other) { connection_url = other.connection_url; @@ -98,6 +86,10 @@ StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureB container = other.container; blob_path = other.blob_path; blobs_paths = other.blobs_paths; + + format = other.format; + compression_method = other.compression_method; + structure = other.structure; } AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(ContextPtr context) @@ -127,7 +119,7 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) { auto blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_url, container)); - bool container_exists = containerExists(blob_service_client, container); + bool container_exists = containerExists(*blob_service_client, container); if (!container_exists) { @@ -140,10 +132,11 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) try { result->CreateIfNotExists(); - } catch (const Azure::Storage::StorageException & e) + } + catch (const Azure::Storage::StorageException & e) { - if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict - && e.ReasonPhrase == "The specified container already exists.")) + if (e.StatusCode != Azure::Core::Http::HttpStatusCode::Conflict + || e.ReasonPhrase != "The specified container already exists.") { throw; } @@ -169,7 +162,7 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) blob_service_client = std::make_unique(connection_url); } - bool container_exists = containerExists(blob_service_client, container); + bool container_exists = containerExists(*blob_service_client, container); std::string final_url; size_t pos = connection_url.find('?'); @@ -460,3 +453,5 @@ void StorageAzureBlobConfiguration::addStructureToArgs(ASTs & args, const String } } + +#endif diff --git a/src/Storages/ObjectStorage/AzureConfiguration.h b/src/Storages/ObjectStorage/AzureBlob/Configuration.h similarity index 88% rename from src/Storages/ObjectStorage/AzureConfiguration.h rename to src/Storages/ObjectStorage/AzureBlob/Configuration.h index 4f285128241..deeb365d012 100644 --- a/src/Storages/ObjectStorage/AzureConfiguration.h +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.h @@ -1,6 +1,11 @@ #pragma once + +#include "config.h" + +#if USE_AZURE_BLOB_STORAGE + #include -#include +#include namespace DB { @@ -26,8 +31,8 @@ public: String getNamespace() const override { return container; } void check(ContextPtr context) const override; - StorageObjectStorageConfigurationPtr clone() override; ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } void fromNamedCollection(const NamedCollection & collection) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; @@ -52,3 +57,5 @@ protected: }; } + +#endif diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp new file mode 100644 index 00000000000..5f0138078d4 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -0,0 +1,28 @@ +#include "Common.h" +#include +#include +#include + +namespace DB +{ + +std::vector listFiles( + const IObjectStorage & object_storage, + const StorageObjectStorageConfiguration & configuration, + const String & prefix, const String & suffix) +{ + auto key = std::filesystem::path(configuration.getPath()) / prefix; + RelativePathsWithMetadata files_with_metadata; + object_storage.listObjects(key, files_with_metadata, 0); + Strings res; + for (const auto & file_with_metadata : files_with_metadata) + { + const auto & filename = file_with_metadata->relative_path; + if (filename.ends_with(suffix)) + res.push_back(filename); + } + LOG_TRACE(getLogger("DataLakeCommon"), "Listed {} files", res.size()); + return res; +} + +} diff --git a/src/Storages/ObjectStorage/DataLakes/Common.h b/src/Storages/ObjectStorage/DataLakes/Common.h new file mode 100644 index 00000000000..ae3767f2eec --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/Common.h @@ -0,0 +1,15 @@ +#pragma once +#include + +namespace DB +{ + +class IObjectStorage; +class StorageObjectStorageConfiguration; + +std::vector listFiles( + const IObjectStorage & object_storage, + const StorageObjectStorageConfiguration & configuration, + const String & prefix, const String & suffix); + +} diff --git a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp similarity index 79% rename from src/Storages/DataLakes/DeltaLakeMetadataParser.cpp rename to src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 55ff8fefdd5..903558b73ab 100644 --- a/src/Storages/DataLakes/DeltaLakeMetadataParser.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include "config.h" #include @@ -15,8 +15,7 @@ #include #include #include - -namespace fs = std::filesystem; +#include namespace DB { @@ -27,12 +26,23 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct DeltaLakeMetadataParser::Impl +struct DeltaLakeMetadata::Impl final : private WithContext { + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + /** * Useful links: * - https://github.com/delta-io/delta/blob/master/PROTOCOL.md#data-files */ + Impl(ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) + { + } /** * DeltaLake tables store metadata files and data files. @@ -62,13 +72,10 @@ struct DeltaLakeMetadataParser::Impl * An action changes one aspect of the table's state, for example, adding or removing a file. * Note: it is not a valid json, but a list of json's, so we read it in a while cycle. */ - std::set processMetadataFiles( - ObjectStoragePtr object_storage, - const StorageObjectStorageConfiguration & configuration, - ContextPtr context) + std::set processMetadataFiles() { std::set result_files; - const auto checkpoint_version = getCheckpointIfExists(result_files, object_storage, configuration, context); + const auto checkpoint_version = getCheckpointIfExists(result_files); if (checkpoint_version) { @@ -76,12 +83,12 @@ struct DeltaLakeMetadataParser::Impl while (true) { const auto filename = withPadding(++current_version) + metadata_file_suffix; - const auto file_path = fs::path(configuration.getPath()) / deltalake_metadata_directory / filename; + const auto file_path = fs::path(configuration->getPath()) / deltalake_metadata_directory / filename; if (!object_storage->exists(StoredObject(file_path))) break; - processMetadataFile(file_path, result_files, object_storage, configuration, context); + processMetadataFile(file_path, result_files); } LOG_TRACE( @@ -90,33 +97,14 @@ struct DeltaLakeMetadataParser::Impl } else { - const auto keys = listFiles(object_storage, configuration, deltalake_metadata_directory, metadata_file_suffix); + const auto keys = listFiles(*object_storage, *configuration, deltalake_metadata_directory, metadata_file_suffix); for (const String & key : keys) - processMetadataFile(key, result_files, object_storage, configuration, context); + processMetadataFile(key, result_files); } return result_files; } - std::vector listFiles( - const ObjectStoragePtr & object_storage, - const StorageObjectStorageConfiguration & configuration, - const String & prefix, const String & suffix) - { - auto key = std::filesystem::path(configuration.getPath()) / prefix; - RelativePathsWithMetadata files_with_metadata; - object_storage->listObjects(key, files_with_metadata, 0); - Strings res; - for (const auto & file_with_metadata : files_with_metadata) - { - const auto & filename = file_with_metadata->relative_path; - if (filename.ends_with(suffix)) - res.push_back(filename); - } - LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); - return res; - } - /** * Example of content of a single .json metadata file: * " @@ -146,14 +134,9 @@ struct DeltaLakeMetadataParser::Impl * \"nullCount\":{\"col-6c990940-59bb-4709-8f2e-17083a82c01a\":0,\"col-763cd7e2-7627-4d8e-9fb7-9e85d0c8845b\":0}}"}} * " */ - void processMetadataFile( - const String & key, - std::set & result, - ObjectStoragePtr object_storage, - const StorageObjectStorageConfiguration & configuration, - ContextPtr context) + void processMetadataFile(const String & key, std::set & result) { - auto read_settings = context->getReadSettings(); + auto read_settings = getContext()->getReadSettings(); auto buf = object_storage->readObject(StoredObject(key), read_settings); char c; @@ -176,12 +159,12 @@ struct DeltaLakeMetadataParser::Impl if (json.has("add")) { const auto path = json["add"]["path"].getString(); - result.insert(fs::path(configuration.getPath()) / path); + result.insert(fs::path(configuration->getPath()) / path); } else if (json.has("remove")) { const auto path = json["remove"]["path"].getString(); - result.erase(fs::path(configuration.getPath()) / path); + result.erase(fs::path(configuration->getPath()) / path); } } } @@ -199,17 +182,14 @@ struct DeltaLakeMetadataParser::Impl * * We need to get "version", which is the version of the checkpoint we need to read. */ - size_t readLastCheckpointIfExists( - ObjectStoragePtr object_storage, - const StorageObjectStorageConfiguration & configuration, - ContextPtr context) const + size_t readLastCheckpointIfExists() { - const auto last_checkpoint_file = fs::path(configuration.getPath()) / deltalake_metadata_directory / "_last_checkpoint"; + const auto last_checkpoint_file = fs::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; if (!object_storage->exists(StoredObject(last_checkpoint_file))) return 0; String json_str; - auto read_settings = context->getReadSettings(); + auto read_settings = getContext()->getReadSettings(); auto buf = object_storage->readObject(StoredObject(last_checkpoint_file), read_settings); readJSONObjectPossiblyInvalid(json_str, *buf); @@ -260,21 +240,18 @@ struct DeltaLakeMetadataParser::Impl throw Exception(ErrorCodes::BAD_ARGUMENTS, "Arrow error: {}", _s.ToString()); \ } while (false) - size_t getCheckpointIfExists( - std::set & result, - ObjectStoragePtr object_storage, - const StorageObjectStorageConfiguration & configuration, - ContextPtr context) + size_t getCheckpointIfExists(std::set & result) { - const auto version = readLastCheckpointIfExists(object_storage, configuration, context); + const auto version = readLastCheckpointIfExists(); if (!version) return 0; const auto checkpoint_filename = withPadding(version) + ".checkpoint.parquet"; - const auto checkpoint_path = fs::path(configuration.getPath()) / deltalake_metadata_directory / checkpoint_filename; + const auto checkpoint_path = fs::path(configuration->getPath()) / deltalake_metadata_directory / checkpoint_filename; LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); + auto context = getContext(); auto read_settings = context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(checkpoint_path), read_settings); auto format_settings = getFormatSettings(context); @@ -334,7 +311,7 @@ struct DeltaLakeMetadataParser::Impl if (filename.empty()) continue; LOG_TEST(log, "Adding {}", filename); - const auto [_, inserted] = result.insert(fs::path(configuration.getPath()) / filename); + const auto [_, inserted] = result.insert(fs::path(configuration->getPath()) / filename); if (!inserted) throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", filename); } @@ -345,15 +322,22 @@ struct DeltaLakeMetadataParser::Impl LoggerPtr log = getLogger("DeltaLakeMetadataParser"); }; -DeltaLakeMetadataParser::DeltaLakeMetadataParser() : impl(std::make_unique()) {} - -Strings DeltaLakeMetadataParser::getFiles( - ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, - ContextPtr context) +DeltaLakeMetadata::DeltaLakeMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : impl(std::make_unique(object_storage_, configuration_, context_)) { - auto result = impl->processMetadataFiles(object_storage, *configuration, context); - return Strings(result.begin(), result.end()); +} + +Strings DeltaLakeMetadata::getDataFiles() const +{ + if (!data_files.empty()) + return data_files; + + auto result = impl->processMetadataFiles(); + data_files = Strings(result.begin(), result.end()); + return data_files; } } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h new file mode 100644 index 00000000000..1a5bb85586a --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class DeltaLakeMetadata final : public IDataLakeMetadata, private WithContext +{ +public: + using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + + static constexpr auto name = "DeltaLake"; + + DeltaLakeMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_); + + Strings getDataFiles() const override; + + NamesAndTypesList getTableSchema() const override { return {}; } + + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * deltalake_metadata = dynamic_cast(&other); + return deltalake_metadata && getDataFiles() == deltalake_metadata->getDataFiles(); + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context) + { + return std::make_unique(object_storage, configuration, local_context); + } + +private: + struct Impl; + const std::shared_ptr impl; + mutable Strings data_files; +}; + +} diff --git a/src/Storages/DataLakes/HudiMetadataParser.cpp b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp similarity index 68% rename from src/Storages/DataLakes/HudiMetadataParser.cpp rename to src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp index 8571c035b32..91a586ccbf9 100644 --- a/src/Storages/DataLakes/HudiMetadataParser.cpp +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.cpp @@ -1,4 +1,5 @@ -#include +#include +#include #include #include #include @@ -40,33 +41,10 @@ namespace ErrorCodes * hoodie.parquet.max.file.size option. Once a single Parquet file is too large, Hudi creates a second file group. * Each file group is identified by File Id. */ -std::vector listFiles( - const ObjectStoragePtr & object_storage, - const StorageObjectStorageConfiguration & configuration, - const String & prefix, const String & suffix) +Strings HudiMetadata::getDataFilesImpl() const { - auto key = std::filesystem::path(configuration.getPath()) / prefix; - RelativePathsWithMetadata files_with_metadata; - object_storage->listObjects(key, files_with_metadata, 0); - Strings res; - for (const auto & file_with_metadata : files_with_metadata) - { - const auto & filename = file_with_metadata->relative_path; - if (filename.ends_with(suffix)) - res.push_back(filename); - } - LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); - return res; -} - -Strings HudiMetadataParser::getFiles( - ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, - ContextPtr) -{ - auto log = getLogger("HudiMetadataParser"); - - const auto keys = listFiles(object_storage, *configuration, "", Poco::toLower(configuration->format)); + auto log = getLogger("HudiMetadata"); + const auto keys = listFiles(*object_storage, *configuration, "", Poco::toLower(configuration->format)); using Partition = std::string; using FileID = std::string; @@ -75,7 +53,7 @@ Strings HudiMetadataParser::getFiles( String key; UInt64 timestamp = 0; }; - std::unordered_map> data_files; + std::unordered_map> files; for (const auto & key : keys) { @@ -90,7 +68,7 @@ Strings HudiMetadataParser::getFiles( const auto & file_id = file_parts[0]; const auto timestamp = parse(file_parts[2]); - auto & file_info = data_files[partition][file_id]; + auto & file_info = files[partition][file_id]; if (file_info.timestamp == 0 || file_info.timestamp < timestamp) { file_info.key = key; @@ -99,7 +77,7 @@ Strings HudiMetadataParser::getFiles( } Strings result; - for (auto & [partition, partition_data] : data_files) + for (auto & [partition, partition_data] : files) { LOG_TRACE(log, "Adding {} data files from partition {}", partition, partition_data.size()); for (auto & [file_id, file_data] : partition_data) @@ -108,4 +86,21 @@ Strings HudiMetadataParser::getFiles( return result; } +HudiMetadata::HudiMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_) + : WithContext(context_) + , object_storage(object_storage_) + , configuration(configuration_) +{ +} + +Strings HudiMetadata::getDataFiles() const +{ + if (data_files.empty()) + data_files = getDataFilesImpl(); + return data_files; +} + } diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h new file mode 100644 index 00000000000..ee8b1ea4978 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class HudiMetadata final : public IDataLakeMetadata, private WithContext +{ +public: + using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + + static constexpr auto name = "Hudi"; + + HudiMetadata( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + ContextPtr context_); + + Strings getDataFiles() const override; + + NamesAndTypesList getTableSchema() const override { return {}; } + + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * hudi_metadata = dynamic_cast(&other); + return hudi_metadata && getDataFiles() == hudi_metadata->getDataFiles(); + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context) + { + return std::make_unique(object_storage, configuration, local_context); + } + +private: + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + mutable Strings data_files; + + Strings getDataFilesImpl() const; +}; + +} diff --git a/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h new file mode 100644 index 00000000000..a2bd5adb947 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/IDataLakeMetadata.h @@ -0,0 +1,19 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +class IDataLakeMetadata : boost::noncopyable +{ +public: + virtual ~IDataLakeMetadata() = default; + virtual Strings getDataFiles() const = 0; + virtual NamesAndTypesList getTableSchema() const = 0; + virtual bool operator==(const IDataLakeMetadata & other) const = 0; +}; +using DataLakeMetadataPtr = std::unique_ptr; + +} diff --git a/src/Storages/DataLakes/Iceberg/StorageIceberg.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h similarity index 61% rename from src/Storages/DataLakes/Iceberg/StorageIceberg.h rename to src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index bca6e3c868f..95196cdd000 100644 --- a/src/Storages/DataLakes/Iceberg/StorageIceberg.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -5,11 +5,13 @@ #if USE_AWS_S3 && USE_AVRO #include -#include #include #include #include -#include +#include +#include +#include +#include #include @@ -19,13 +21,10 @@ namespace DB /// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) /// Right now it's implemented on top of StorageS3 and right now it doesn't support /// many Iceberg features like schema evolution, partitioning, positional and equality deletes. -/// TODO: Implement Iceberg as a separate storage using IObjectStorage -/// (to support all object storages, not only S3) and add support for missing Iceberg features. -template -class StorageIceberg : public StorageObjectStorage +template +class IStorageDataLake final : public StorageObjectStorage { public: - static constexpr auto name = "Iceberg"; using Storage = StorageObjectStorage; using ConfigurationPtr = Storage::ConfigurationPtr; @@ -41,12 +40,14 @@ public: bool attach) { auto object_storage = base_configuration->createOrUpdateObjectStorage(context); - std::unique_ptr metadata; + DataLakeMetadataPtr metadata; NamesAndTypesList schema_from_metadata; + ConfigurationPtr configuration = base_configuration->clone(); try { - metadata = parseIcebergMetadata(object_storage, base_configuration, context); + metadata = DataLakeMetadata::create(object_storage, base_configuration, context); schema_from_metadata = metadata->getTableSchema(); + configuration->getPaths() = metadata->getDataFiles(); } catch (...) { @@ -55,17 +56,14 @@ public: tryLogCurrentException(__PRETTY_FUNCTION__); } - auto configuration = base_configuration->clone(); - configuration->getPaths() = metadata->getDataFiles(); - - return std::make_shared>( + return std::make_shared>( base_configuration, std::move(metadata), configuration, object_storage, engine_name_, context, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment_, format_settings_); } - String getName() const override { return name; } + String getName() const override { return DataLakeMetadata::name; } static ColumnsDescription getTableStructureFromData( ObjectStoragePtr object_storage_, @@ -73,7 +71,7 @@ public: const std::optional &, ContextPtr local_context) { - auto metadata = parseIcebergMetadata(object_storage_, base_configuration, local_context); + auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); return ColumnsDescription(metadata->getTableSchema()); } @@ -86,24 +84,25 @@ public: if (updated) Storage::object_storage = new_object_storage; - auto new_metadata = parseIcebergMetadata(Storage::object_storage, base_configuration, local_context); + auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - if (!current_metadata || new_metadata->getVersion() != current_metadata->getVersion()) + if (!current_metadata || !(*current_metadata == *new_metadata)) current_metadata = std::move(new_metadata); - else if (updated) - { - auto updated_configuration = base_configuration->clone(); - /// If metadata wasn't changed, we won't list data files again. - updated_configuration->getPaths() = current_metadata->getDataFiles(); - Storage::configuration = updated_configuration; - } + else if (!updated) + return {Storage::configuration, Storage::object_storage}; + + auto updated_configuration = base_configuration->clone(); + /// If metadata wasn't changed, we won't list data files again. + updated_configuration->getPaths() = current_metadata->getDataFiles(); + Storage::configuration = updated_configuration; + return {Storage::configuration, Storage::object_storage}; } template - StorageIceberg( + IStorageDataLake( ConfigurationPtr base_configuration_, - std::unique_ptr metadata_, + DataLakeMetadataPtr metadata_, Args &&... args) : Storage(std::forward(args)...) , base_configuration(base_configuration_) @@ -113,8 +112,13 @@ public: private: ConfigurationPtr base_configuration; - std::unique_ptr current_metadata; + DataLakeMetadataPtr current_metadata; }; + +using StorageIceberg = IStorageDataLake; +using StorageDeltaLake = IStorageDataLake; +using StorageHudi = IStorageDataLake; + } #endif diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp similarity index 96% rename from src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp rename to src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 5543e60e7a7..8ee6f002ca6 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -24,7 +24,8 @@ #include #include #include -#include +#include +#include #include #include @@ -332,25 +333,6 @@ MutableColumns parseAvro( return columns; } -std::vector listFiles( - const ObjectStoragePtr & object_storage, - const StorageObjectStorageConfiguration & configuration, - const String & prefix, const String & suffix) -{ - auto key = std::filesystem::path(configuration.getPath()) / prefix; - RelativePathsWithMetadata files_with_metadata; - object_storage->listObjects(key, files_with_metadata, 0); - Strings res; - for (const auto & file_with_metadata : files_with_metadata) - { - const auto & filename = file_with_metadata->relative_path; - if (filename.ends_with(suffix)) - res.push_back(filename); - } - LOG_TRACE(getLogger("DataLakeMetadataReadHelper"), "Listed {} files", res.size()); - return res; -} - /** * Each version of table metadata is stored in a `metadata` directory and * has one of 2 formats: @@ -361,7 +343,7 @@ std::pair getMetadataFileAndVersion( ObjectStoragePtr object_storage, const StorageObjectStorageConfiguration & configuration) { - const auto metadata_files = listFiles(object_storage, configuration, "metadata", ".metadata.json"); + const auto metadata_files = listFiles(*object_storage, configuration, "metadata", ".metadata.json"); if (metadata_files.empty()) { throw Exception( @@ -394,14 +376,14 @@ std::pair getMetadataFileAndVersion( } -std::unique_ptr parseIcebergMetadata( +DataLakeMetadataPtr IcebergMetadata::create( ObjectStoragePtr object_storage, StorageObjectStorageConfigurationPtr configuration, - ContextPtr context_) + ContextPtr local_context) { const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); LOG_DEBUG(getLogger("IcebergMetadata"), "Parse metadata {}", metadata_file_path); - auto read_settings = context_->getReadSettings(); + auto read_settings = local_context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(metadata_file_path), read_settings); String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); @@ -411,7 +393,7 @@ std::unique_ptr parseIcebergMetadata( Poco::JSON::Object::Ptr object = json.extract(); auto format_version = object->getValue("format-version"); - auto [schema, schema_id] = parseTableSchema(object, format_version, context_->getSettingsRef().iceberg_engine_ignore_schema_evolution); + auto [schema, schema_id] = parseTableSchema(object, format_version, local_context->getSettingsRef().iceberg_engine_ignore_schema_evolution); auto current_snapshot_id = object->getValue("current-snapshot-id"); auto snapshots = object->get("snapshots").extract(); @@ -428,7 +410,7 @@ std::unique_ptr parseIcebergMetadata( } } - return std::make_unique(object_storage, configuration, context_, metadata_version, format_version, manifest_list_file, schema_id, schema); + return std::make_unique(object_storage, configuration, local_context, metadata_version, format_version, manifest_list_file, schema_id, schema); } /** @@ -456,7 +438,7 @@ std::unique_ptr parseIcebergMetadata( * │ 1 │ 2252246380142525104 │ ('/iceberg_data/db/table_name/data/a=2/00000-1-c9535a00-2f4f-405c-bcfa-6d4f9f477235-00003.parquet','PARQUET',(2),1,631,67108864,[(1,46),(2,48)],[(1,1),(2,1)],[(1,0),(2,0)],[],[(1,'\0\0\0\0\0\0\0'),(2,'3')],[(1,'\0\0\0\0\0\0\0'),(2,'3')],NULL,[4],0) │ * └────────┴─────────────────────┴────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ */ -Strings IcebergMetadata::getDataFiles() +Strings IcebergMetadata::getDataFiles() const { if (!data_files.empty()) return data_files; diff --git a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h similarity index 76% rename from src/Storages/DataLakes/Iceberg/IcebergMetadata.h rename to src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index a289715848f..f88e3eecc67 100644 --- a/src/Storages/DataLakes/Iceberg/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -5,7 +5,8 @@ #include #include #include -#include +#include +#include namespace DB { @@ -57,12 +58,16 @@ namespace DB * "metadata-log" : [ ] * } */ -class IcebergMetadata : WithContext +class IcebergMetadata : public IDataLakeMetadata, private WithContext { public: + using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + + static constexpr auto name = "Iceberg"; + IcebergMetadata( ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + ConfigurationPtr configuration_, ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -72,31 +77,36 @@ public: /// Get data files. On first request it reads manifest_list file and iterates through manifest files to find all data files. /// All subsequent calls will return saved list of files (because it cannot be changed without changing metadata file) - Strings getDataFiles(); + Strings getDataFiles() const override; /// Get table schema parsed from metadata. - NamesAndTypesList getTableSchema() const { return schema; } + NamesAndTypesList getTableSchema() const override { return schema; } - size_t getVersion() const { return metadata_version; } + bool operator ==(const IDataLakeMetadata & other) const override + { + const auto * iceberg_metadata = dynamic_cast(&other); + return iceberg_metadata && getVersion() == iceberg_metadata->getVersion(); + } + + static DataLakeMetadataPtr create( + ObjectStoragePtr object_storage, + ConfigurationPtr configuration, + ContextPtr local_context); private: - ObjectStoragePtr object_storage; - StorageObjectStorageConfigurationPtr configuration; + size_t getVersion() const { return metadata_version; } + + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; Int32 metadata_version; Int32 format_version; String manifest_list_file; Int32 current_schema_id; NamesAndTypesList schema; - Strings data_files; + mutable Strings data_files; LoggerPtr log; - }; -std::unique_ptr parseIcebergMetadata( - ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, - ContextPtr context); - } #endif diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp new file mode 100644 index 00000000000..d93c14dfe32 --- /dev/null +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -0,0 +1,83 @@ +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include + + +namespace DB +{ + +#if USE_AVRO /// StorageIceberg depending on Avro to parse metadata with Avro format. + +void registerStorageIceberg(StorageFactory & factory) +{ + factory.registerStorage( + "Iceberg", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageIceberg::create( + configuration, args.getContext(), "Iceberg", args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.attach); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +#endif + +#if USE_PARQUET +void registerStorageDeltaLake(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageDeltaLake::create( + configuration, args.getContext(), "DeltaLake", args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.attach); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} +#endif + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage( + "Hudi", + [&](const StorageFactory::Arguments & args) + { + auto configuration = std::make_shared(); + StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + + return StorageHudi::create( + configuration, args.getContext(), "Hudi", args.table_id, args.columns, + args.constraints, args.comment, std::nullopt, args.attach); + }, + { + .supports_settings = false, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} + +#endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp new file mode 100644 index 00000000000..c80237b3055 --- /dev/null +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -0,0 +1,57 @@ +#include + +#if USE_HDFS +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) +{ + url = other.url; + path = other.path; + paths = other.paths; + format = other.format; + compression_method = other.compression_method; + structure = other.structure; +} + +void StorageHDFSConfiguration::check(ContextPtr context) const +{ + context->getRemoteHostFilter().checkURL(Poco::URI(url)); + checkHDFSURL(url); +} + +ObjectStoragePtr StorageHDFSConfiguration::createOrUpdateObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +{ + UNUSED(is_readonly); + auto settings = std::make_unique(); + return std::make_shared(url, std::move(settings), context->getConfigRef()); +} + +void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr, bool /* with_structure */) +{ + url = checkAndGetLiteralArgument(args[0], "url"); + + String format_name = "auto"; + if (args.size() > 1) + format_name = checkAndGetLiteralArgument(args[1], "format_name"); + + if (format_name == "auto") + format_name = FormatFactory::instance().getFormatFromFileName(url, true); + + String compression_method; + if (args.size() == 3) + compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); + else + compression_method = "auto"; + +} +} + +#endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h new file mode 100644 index 00000000000..03fb0824123 --- /dev/null +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -0,0 +1,45 @@ +#pragma once +#include "config.h" + +#if USE_HDFS +#include +#include +#include +#include + +namespace DB +{ + +class StorageHDFSConfiguration : public StorageObjectStorageConfiguration +{ +public: + StorageHDFSConfiguration() = default; + StorageHDFSConfiguration(const StorageHDFSConfiguration & other); + + Path getPath() const override { return path; } + void setPath(const Path & path_) override { path = path_; } + + const Paths & getPaths() const override { return paths; } + Paths & getPaths() override { return paths; } + + String getNamespace() const override { return ""; } + String getDataSourceDescription() override { return url; } + + void check(ContextPtr context) const override; + ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } + + void fromNamedCollection(const NamedCollection &) override {} + void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; + + static void addStructureToArgs(ASTs &, const String &, ContextPtr) {} + +private: + String url; + String path; + std::vector paths; +}; + +} + +#endif diff --git a/src/Storages/ObjectStorage/HDFSConfiguration.h b/src/Storages/ObjectStorage/HDFSConfiguration.h deleted file mode 100644 index aa45c634042..00000000000 --- a/src/Storages/ObjectStorage/HDFSConfiguration.h +++ /dev/null @@ -1,81 +0,0 @@ -#pragma once -#include "config.h" - -#if USE_HDFS - -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - -class StorageHDFSConfiguration : public StorageObjectStorageConfiguration -{ -public: - Path getPath() const override { return path; } - void setPath(const Path & path_) override { path = path_; } - - const Paths & getPaths() const override { return paths; } - Paths & getPaths() override { return paths; } - - String getNamespace() const override { return ""; } - String getDataSourceDescription() override { return url; } - - void check(ContextPtr context) const override - { - context->getRemoteHostFilter().checkURL(Poco::URI(url)); - checkHDFSURL(url); - } - StorageObjectStorageConfigurationPtr clone() override - { - auto configuration = std::make_shared(); - return configuration; - } - - ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override /// NOLINT - { - UNUSED(is_readonly); - auto settings = std::make_unique(); - return std::make_shared(url, std::move(settings), context->getConfigRef()); - } - - void fromNamedCollection(const NamedCollection &) override {} - void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override - { - url = checkAndGetLiteralArgument(args[0], "url"); - - String format_name = "auto"; - if (args.size() > 1) - format_name = checkAndGetLiteralArgument(args[1], "format_name"); - - if (format_name == "auto") - format_name = FormatFactory::instance().getFormatFromFileName(url, true); - - String compression_method; - if (args.size() == 3) - { - compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); - } else compression_method = "auto"; - - } - static void addStructureToArgs(ASTs &, const String &, ContextPtr) {} - -private: - String url; - String path; - std::vector paths; -}; - -} - -#endif diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index dcdf36dbcf5..a3e19b907bc 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -67,11 +67,11 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( auto get_last_mod_time = [&] -> std::optional { if (object_info->metadata) - return object_info->metadata->last_modified->epochMicroseconds(); + return object_info->metadata->last_modified.epochMicroseconds(); else { object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - return object_info->metadata->last_modified->epochMicroseconds(); + return object_info->metadata->last_modified.epochMicroseconds(); } }; diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp index 2c27c816078..b33eea7d354 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp @@ -59,7 +59,6 @@ void ReadFromStorageObejctStorage::applyFilters() const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); - createIterator(predicate); } diff --git a/src/Storages/ObjectStorage/S3Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp similarity index 97% rename from src/Storages/ObjectStorage/S3Configuration.cpp rename to src/Storages/ObjectStorage/S3/Configuration.cpp index 5a5412019f5..f057745d669 100644 --- a/src/Storages/ObjectStorage/S3Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -1,4 +1,7 @@ -#include +#include + +#if USE_AWS_S3 + #include #include #include @@ -14,6 +17,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } static const std::unordered_set required_configuration_keys = { @@ -51,17 +55,19 @@ void StorageS3Configuration::check(ContextPtr context) const context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); } -StorageObjectStorageConfigurationPtr StorageS3Configuration::clone() +StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) { - auto configuration = std::make_shared(); - configuration->url = url; - configuration->auth_settings = auth_settings; - configuration->request_settings = request_settings; - configuration->static_configuration = static_configuration; - configuration->headers_from_ast = headers_from_ast; - configuration->keys = keys; - configuration->initialized = initialized; - return configuration; + url = other.url; + auth_settings = other.auth_settings; + request_settings = other.request_settings; + static_configuration = other.static_configuration; + headers_from_ast = other.headers_from_ast; + keys = other.keys; + initialized = other.initialized; + + format = other.format; + compression_method = other.compression_method; + structure = other.structure; } ObjectStoragePtr StorageS3Configuration::createOrUpdateObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT @@ -489,3 +495,5 @@ void StorageS3Configuration::addStructureToArgs(ASTs & args, const String & stru } } + +#endif diff --git a/src/Storages/ObjectStorage/S3Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h similarity index 81% rename from src/Storages/ObjectStorage/S3Configuration.h rename to src/Storages/ObjectStorage/S3/Configuration.h index c953bc25c4e..037cf2eae87 100644 --- a/src/Storages/ObjectStorage/S3Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -1,7 +1,12 @@ #pragma once + +#include "config.h" + +#if USE_AWS_S3 + #include #include -#include +#include namespace DB { @@ -9,6 +14,9 @@ namespace DB class StorageS3Configuration : public StorageObjectStorageConfiguration { public: + StorageS3Configuration() = default; + StorageS3Configuration(const StorageS3Configuration & other); + Path getPath() const override { return url.key; } void setPath(const Path & path) override { url.key = path; } @@ -19,9 +27,8 @@ public: String getDataSourceDescription() override; void check(ContextPtr context) const override; - StorageObjectStorageConfigurationPtr clone() override; - ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } void fromNamedCollection(const NamedCollection & collection) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; @@ -44,3 +51,5 @@ private: }; } + +#endif diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 9a7260ea47c..08d7c9d0014 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -24,8 +24,6 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; } @@ -59,7 +57,6 @@ std::unique_ptr getStorageMetadata( storage_metadata->setColumns(columns); } - storage_metadata->setConstraints(constraints); storage_metadata->setComment(comment); return storage_metadata; @@ -264,10 +261,7 @@ SinkToStoragePtr StorageObjectStorage::write( template void StorageObjectStorage::truncate( - const ASTPtr &, - const StorageMetadataPtr &, - ContextPtr, - TableExclusiveLockHolder &) + const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) { if (configuration->withGlobs()) { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 0b29845ba5c..6f18153c7af 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -95,8 +95,7 @@ public: ContextPtr context); protected: - virtual std::pair - updateConfigurationAndGetCopy(ContextPtr local_context); + virtual std::pair updateConfigurationAndGetCopy(ContextPtr local_context); const std::string engine_name; const NamesAndTypesList virtual_columns; @@ -110,7 +109,7 @@ protected: }; using StorageS3 = StorageObjectStorage; -using StorageAzureBlobStorage = StorageObjectStorage; +using StorageAzureBlob = StorageObjectStorage; using StorageHDFS = StorageObjectStorage; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 39cd5d8eca6..c03bbd1a45d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index aae8f704a73..507de20e888 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -67,5 +67,8 @@ using StorageS3Cluster = StorageObjectStorageCluster; #endif +#if USE_HDFS +using StorageHDFSCluster = StorageObjectStorageCluster; +#endif } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 2d5760ed9d8..651f1d25ec1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB diff --git a/src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h similarity index 99% rename from src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h rename to src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 427d6a8d453..04b2d8e8fd9 100644 --- a/src/Storages/ObjectStorage/StorageObejctStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -39,9 +39,8 @@ public: std::string getPathWithoutGlob() const; virtual void check(ContextPtr context) const = 0; - virtual StorageObjectStorageConfigurationPtr clone() = 0; - virtual ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT + virtual StorageObjectStorageConfigurationPtr clone() = 0; String format = "auto"; String compression_method = "auto"; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp new file mode 100644 index 00000000000..37f93a2b82f --- /dev/null +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -0,0 +1,127 @@ +#include "StorageObjectStorageSink.h" +#include +#include + +namespace DB +{ + +StorageObjectStorageSink::StorageObjectStorageSink( + ObjectStoragePtr object_storage, + StorageObjectStorageConfigurationPtr configuration, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context, + const std::string & blob_path) + : SinkToStorage(sample_block_) + , sample_block(sample_block_) + , format_settings(format_settings_) +{ + const auto & settings = context->getSettingsRef(); + const auto path = blob_path.empty() ? configuration->getPaths().back() : blob_path; + const auto chosen_compression_method = chooseCompressionMethod(path, configuration->compression_method); + + auto buffer = object_storage->writeObject( + StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); + + write_buf = wrapWriteBufferWithCompressionMethod( + std::move(buffer), + chosen_compression_method, + static_cast(settings.output_format_compression_level), + static_cast(settings.output_format_compression_zstd_window_log)); + + writer = FormatFactory::instance().getOutputFormatParallelIfPossible( + configuration->format, *write_buf, sample_block, context, format_settings); +} + +void StorageObjectStorageSink::consume(Chunk chunk) +{ + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); +} + +void StorageObjectStorageSink::onCancel() +{ + std::lock_guard lock(cancel_mutex); + finalize(); + cancelled = true; +} + +void StorageObjectStorageSink::onException(std::exception_ptr exception) +{ + std::lock_guard lock(cancel_mutex); + try + { + std::rethrow_exception(exception); + } + catch (...) + { + /// An exception context is needed to proper delete write buffers without finalization. + release(); + } +} + +void StorageObjectStorageSink::onFinish() +{ + std::lock_guard lock(cancel_mutex); + finalize(); +} + +void StorageObjectStorageSink::finalize() +{ + if (!writer) + return; + + try + { + writer->finalize(); + writer->flush(); + write_buf->finalize(); + } + catch (...) + { + /// Stop ParallelFormattingOutputFormat correctly. + release(); + throw; + } +} + +void StorageObjectStorageSink::release() +{ + writer.reset(); + write_buf->finalize(); +} + +PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( + ObjectStoragePtr object_storage_, + StorageObjectStorageConfigurationPtr configuration_, + std::optional format_settings_, + const Block & sample_block_, + ContextPtr context_, + const ASTPtr & partition_by) + : PartitionedSink(partition_by, context_, sample_block_) + , object_storage(object_storage_) + , configuration(configuration_) + , format_settings(format_settings_) + , sample_block(sample_block_) + , context(context_) +{ +} + +SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) +{ + auto blob = configuration->getPaths().back(); + auto partition_key = replaceWildcards(blob, partition_id); + validatePartitionKey(partition_key, true); + return std::make_shared( + object_storage, + configuration, + format_settings, + sample_block, + context, + partition_key + ); +} + +} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index a2d42d7fa9f..14298376d0e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -1,9 +1,8 @@ #pragma once #include -#include -#include +#include #include -#include +#include namespace DB { @@ -16,64 +15,17 @@ public: std::optional format_settings_, const Block & sample_block_, ContextPtr context, - const std::string & blob_path = "") - : SinkToStorage(sample_block_) - , sample_block(sample_block_) - , format_settings(format_settings_) - { - const auto & settings = context->getSettingsRef(); - const auto path = blob_path.empty() ? configuration->getPaths().back() : blob_path; - const auto chosen_compression_method = chooseCompressionMethod(path, configuration->compression_method); - - auto buffer = object_storage->writeObject( - StoredObject(path), WriteMode::Rewrite, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, context->getWriteSettings()); - - write_buf = wrapWriteBufferWithCompressionMethod( - std::move(buffer), - chosen_compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - - writer = FormatFactory::instance().getOutputFormatParallelIfPossible( - configuration->format, *write_buf, sample_block, context, format_settings); - } + const std::string & blob_path = ""); String getName() const override { return "StorageObjectStorageSink"; } - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } + void consume(Chunk chunk) override; - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } + void onCancel() override; - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization. - release(); - } - } + void onException(std::exception_ptr exception) override; - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } + void onFinish() override; private: const Block sample_block; @@ -84,30 +36,8 @@ private: bool cancelled = false; std::mutex cancel_mutex; - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } + void finalize(); + void release(); }; class PartitionedStorageObjectStorageSink : public PartitionedSink @@ -119,30 +49,9 @@ public: std::optional format_settings_, const Block & sample_block_, ContextPtr context_, - const ASTPtr & partition_by) - : PartitionedSink(partition_by, context_, sample_block_) - , object_storage(object_storage_) - , configuration(configuration_) - , format_settings(format_settings_) - , sample_block(sample_block_) - , context(context_) - { - } + const ASTPtr & partition_by); - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto blob = configuration->getPaths().back(); - auto partition_key = replaceWildcards(blob, partition_id); - validatePartitionKey(partition_key, true); - return std::make_shared( - object_storage, - configuration, - format_settings, - sample_block, - context, - partition_key - ); - } + SinkPtr createSinkForPartition(const String & partition_id) override; private: ObjectStoragePtr object_storage; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index f170a46112f..1fda75897f9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,6 +26,8 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_COMPILE_REGEXP; + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } StorageObjectStorageSource::StorageObjectStorageSource( @@ -182,8 +184,8 @@ std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const O auto get_last_mod_time = [&]() -> std::optional { - return object_info->metadata && object_info->metadata->last_modified - ? object_info->metadata->last_modified->epochMicroseconds() + return object_info->metadata + ? object_info->metadata->last_modified.epochMicroseconds() : 0; }; return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); @@ -472,4 +474,29 @@ ObjectInfoPtr StorageObjectStorageSource::KeysIterator::next(size_t /* processor return std::make_shared(key, metadata); } +StorageObjectStorageSource::ReaderHolder::ReaderHolder( + ObjectInfoPtr object_info_, + std::unique_ptr read_buf_, + std::shared_ptr source_, + std::unique_ptr pipeline_, + std::unique_ptr reader_) + : object_info(std::move(object_info_)) + , read_buf(std::move(read_buf_)) + , source(std::move(source_)) + , pipeline(std::move(pipeline_)) + , reader(std::move(reader_)) +{ +} + +StorageObjectStorageSource::ReaderHolder & StorageObjectStorageSource::ReaderHolder::operator=(ReaderHolder && other) noexcept +{ + /// The order of destruction is important. + /// reader uses pipeline, pipeline uses read_buf. + reader = std::move(other.reader); + pipeline = std::move(other.pipeline); + source = std::move(other.source); + read_buf = std::move(other.read_buf); + object_info = std::move(other.object_info); + return *this; +} } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 0d6a6b71271..214a7de14d6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -75,32 +75,16 @@ protected: std::unique_ptr read_buf_, std::shared_ptr source_, std::unique_ptr pipeline_, - std::unique_ptr reader_) - : object_info(std::move(object_info_)) - , read_buf(std::move(read_buf_)) - , source(std::move(source_)) - , pipeline(std::move(pipeline_)) - , reader(std::move(reader_)) {} + std::unique_ptr reader_); ReaderHolder() = default; ReaderHolder(ReaderHolder && other) noexcept { *this = std::move(other); } + ReaderHolder & operator=(ReaderHolder && other) noexcept; explicit operator bool() const { return reader != nullptr; } PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - ReaderHolder & operator=(ReaderHolder && other) noexcept - { - /// The order of destruction is important. - /// reader uses pipeline, pipeline uses read_buf. - reader = std::move(other.reader); - pipeline = std::move(other.pipeline); - source = std::move(other.source); - read_buf = std::move(other.read_buf); - object_info = std::move(other.object_info); - return *this; - } - const String & getRelativePath() const { return object_info->relative_path; } const ObjectInfo & getObjectInfo() const { return *object_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } @@ -143,7 +127,7 @@ public: size_t estimatedKeysCount() override { return 0; } /// TODO FIXME - ObjectInfoPtr next(size_t) override { return std::make_shared( callback(), ObjectMetadata{} ); } + ObjectInfoPtr next(size_t) override { return std::make_shared(callback(), ObjectMetadata{}); } private: ReadTaskCallback callback; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h b/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h index 51be7419e1c..241e2f20962 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h @@ -1,4 +1,5 @@ -#include +#pragma once +#include namespace DB { diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index f7ab37490e1..e23457c04e9 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,6 +1,6 @@ -#include -#include -#include +#include +#include +#include #include #include #include @@ -8,6 +8,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + template static std::shared_ptr> createStorageObjectStorage( const StorageFactory::Arguments & args, @@ -149,6 +154,7 @@ void registerStorageObjectStorage(StorageFactory & factory) #if USE_HDFS registerStorageHDFS(factory); #endif + UNUSED(factory); } } diff --git a/src/Storages/ObjectStorageConfiguration.h b/src/Storages/ObjectStorageConfiguration.h deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 70dd8f27d71..9502a3c5e70 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -3,7 +3,7 @@ #if USE_AWS_S3 #include -#include +#include #include namespace DB diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index fc4ef77ebb9..b03224cedff 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include #include #include @@ -36,6 +36,13 @@ namespace ProfileEvents extern const Event S3ListObjects; } +namespace CurrentMetrics +{ + extern const Metric ObjectStorageS3Threads; + extern const Metric ObjectStorageS3ThreadsActive; + extern const Metric ObjectStorageS3ThreadsScheduled; +} + namespace DB { diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index 77d5be3698c..a53ce440c3f 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -81,7 +81,7 @@ void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, C #endif fillDataImpl(res_columns, StorageURL::getSchemaCache(context), "URL"); #if USE_AZURE_BLOB_STORAGE - fillDataImpl(res_columns, StorageAzureBlobStorage::getSchemaCache(context), "Azure"); /// FIXME + fillDataImpl(res_columns, StorageAzureBlob::getSchemaCache(context), "Azure"); /// FIXME #endif } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 0ffa1460d78..8edba4e6e4b 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -1,18 +1,17 @@ #pragma once #include "config.h" - -#if USE_AWS_S3 - -# include -# include -# include -# include -# include -# include -#include +#include +#include +#include +#include +#include +#include #include -#include +#include +#include +#include + namespace DB { @@ -26,18 +25,20 @@ public: protected: StoragePtr executeImpl( - const ASTPtr & /*ast_function*/, + const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, - ColumnsDescription /*cached_columns*/, + ColumnsDescription cached_columns, bool /*is_insert_query*/) const override { ColumnsDescription columns; - if (TableFunction::configuration->structure != "auto") - columns = parseColumnsListFromString(TableFunction::configuration->structure, context); + auto configuration = TableFunction::getConfiguration(); + if (configuration->structure != "auto") + columns = parseColumnsListFromString(configuration->structure, context); + else if (!cached_columns.empty()) + columns = cached_columns; - StorageObjectStorageConfigurationPtr configuration = TableFunction::configuration; - StoragePtr storage = StorageIceberg>::create( + StoragePtr storage = Storage::create( configuration, context, "", StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt, false); @@ -45,26 +46,53 @@ protected: return storage; } - const char * getStorageTypeName() const override { return Storage::name; } + const char * getStorageTypeName() const override { return name; } - ColumnsDescription getActualTableStructure(ContextPtr context, bool /*is_insert_query*/) const override + ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override { - if (TableFunction::configuration->structure == "auto") + auto configuration = TableFunction::getConfiguration(); + if (configuration->structure == "auto") { context->checkAccess(TableFunction::getSourceAccessType()); - return Storage::getTableStructureFromData(TableFunction::object_storage, TableFunction::configuration, std::nullopt, context); + auto object_storage = TableFunction::getObjectStorage(context, !is_insert_query); + return Storage::getTableStructureFromData(object_storage, configuration, std::nullopt, context); } - return parseColumnsListFromString(TableFunction::configuration->structure, context); + return parseColumnsListFromString(configuration->structure, context); } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override { + auto configuration = TableFunction::getConfiguration(); + configuration->format = "Parquet"; /// Set default format to Parquet if it's not specified in arguments. - TableFunction::configuration->format = "Parquet"; TableFunction::parseArguments(ast_function, context); } }; -} +struct TableFunctionIcebergName +{ + static constexpr auto name = "iceberg"; +}; + +struct TableFunctionDeltaLakeName +{ + static constexpr auto name = "deltaLake"; +}; + +struct TableFunctionHudiName +{ + static constexpr auto name = "hudi"; +}; + +#if USE_AWS_S3 +#if USE_AVRO +using TableFunctionIceberg = ITableFunctionDataLake; #endif +#if USE_PARQUET +using TableFunctionDeltaLake = ITableFunctionDataLake; +#endif +using TableFunctionHudi = ITableFunctionDataLake; +#endif + +} diff --git a/src/TableFunctions/TableFunctionDeltaLake.cpp b/src/TableFunctions/TableFunctionDeltaLake.cpp deleted file mode 100644 index 08b62ed2612..00000000000 --- a/src/TableFunctions/TableFunctionDeltaLake.cpp +++ /dev/null @@ -1,33 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 && USE_PARQUET - -#include -#include -#include -#include -#include "registerTableFunctions.h" - -namespace DB -{ - -struct TableFunctionDeltaLakeName -{ - static constexpr auto name = "deltaLake"; -}; - -// using TableFunctionDeltaLake = ITableFunctionDataLake; -// -// void registerTableFunctionDeltaLake(TableFunctionFactory & factory) -// { -// factory.registerFunction( -// {.documentation = { -// .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", -// .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, -// .categories{"DataLake"}}, -// .allow_readonly = false}); -// } - -} - -#endif diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp deleted file mode 100644 index c6d84504c40..00000000000 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ /dev/null @@ -1,31 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include "registerTableFunctions.h" - -namespace DB -{ - -struct TableFunctionHudiName -{ - static constexpr auto name = "hudi"; -}; -// using TableFunctionHudi = ITableFunctionDataLake; -// -// 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)", ""}}, -// .categories{"DataLake"}}, -// .allow_readonly = false}); -// } -} - -#endif diff --git a/src/TableFunctions/TableFunctionIceberg.cpp b/src/TableFunctions/TableFunctionIceberg.cpp deleted file mode 100644 index 1a28f9292d1..00000000000 --- a/src/TableFunctions/TableFunctionIceberg.cpp +++ /dev/null @@ -1,37 +0,0 @@ -#include "config.h" - -#if USE_AWS_S3 && USE_AVRO - -#include -#include -#include -#include -#include "registerTableFunctions.h" - - -namespace DB -{ - -struct TableFunctionIcebergName -{ - static constexpr auto name = "iceberg"; -}; - -using TableFunctionIceberg = ITableFunctionDataLake< - TableFunctionIcebergName, - StorageIceberg, - TableFunctionS3>; - -void registerTableFunctionIceberg(TableFunctionFactory & factory) -{ - factory.registerFunction( - {.documentation - = {.description=R"(The table function can be used to read the Iceberg table stored on object store.)", - .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, - .categories{"DataLake"}}, - .allow_readonly = false}); -} - -} - -#endif diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index de46c13af37..a948102ac2b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -7,10 +7,10 @@ #include #include #include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include @@ -24,7 +24,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; } template @@ -36,6 +35,15 @@ ObjectStoragePtr TableFunctionObjectStorage< return object_storage; } +template +StorageObjectStorageConfigurationPtr TableFunctionObjectStorage< + Definition, StorageSettings, Configuration>::getConfiguration() const +{ + if (!configuration) + configuration = std::make_shared(); + return configuration; +} + template std::vector TableFunctionObjectStorage< Definition, StorageSettings, Configuration>::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const @@ -65,8 +73,7 @@ template void TableFunctionObjectStorage< Definition, StorageSettings, Configuration>::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) { - configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, engine_args, local_context, true); + StorageObjectStorageConfiguration::initialize(*getConfiguration(), engine_args, local_context, true); } template @@ -147,6 +154,7 @@ StoragePtr TableFunctionObjectStorage>( { diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 1df0ba2f843..5e180301862 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -2,10 +2,9 @@ #include "config.h" -#if USE_AZURE_BLOB_STORAGE - #include #include +#include #include @@ -114,6 +113,8 @@ public: static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); protected: + using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, @@ -125,9 +126,11 @@ protected: ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override; void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; - ObjectStoragePtr getObjectStorage(const ContextPtr & context, bool create_readonly) const; - mutable typename StorageObjectStorage::ConfigurationPtr configuration; + ObjectStoragePtr getObjectStorage(const ContextPtr & context, bool create_readonly) const; + ConfigurationPtr getConfiguration() const; + + mutable ConfigurationPtr configuration; mutable ObjectStoragePtr object_storage; ColumnsDescription structure_hint; @@ -146,5 +149,3 @@ using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif } - -#endif diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 8e6c96a3f2a..c93d816dc07 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -6,9 +6,9 @@ #include #include #include -#include -#include -#include +#include +#include +#include namespace DB @@ -103,6 +103,8 @@ void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) } ); #endif + + UNUSED(factory); } #if USE_AWS_S3 diff --git a/src/TableFunctions/registerDataLakeTableFunctions.cpp b/src/TableFunctions/registerDataLakeTableFunctions.cpp new file mode 100644 index 00000000000..15a6668f434 --- /dev/null +++ b/src/TableFunctions/registerDataLakeTableFunctions.cpp @@ -0,0 +1,69 @@ +#include +#include + +namespace DB +{ + +#if USE_AWS_S3 +#if USE_AVRO +void registerTableFunctionIceberg(TableFunctionFactory & factory) +{ + factory.registerFunction( + { + .documentation = + { + .description=R"(The table function can be used to read the Iceberg table stored on object store.)", + .examples{{"iceberg", "SELECT * FROM iceberg(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +#if USE_PARQUET +void registerTableFunctionDeltaLake(TableFunctionFactory & factory) +{ + factory.registerFunction( + { + .documentation = + { + .description=R"(The table function can be used to read the DeltaLake table stored on object store.)", + .examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +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)", ""}}, + .categories{"DataLake"} + }, + .allow_readonly = false + }); +} +#endif + +void registerDataLakeTableFunctions(TableFunctionFactory & factory) +{ + UNUSED(factory); +#if USE_AWS_S3 +#if USE_AVRO + registerTableFunctionIceberg(factory); +#endif +#if USE_PARQUET + registerTableFunctionDeltaLake(factory); +#endif + registerTableFunctionHudi(factory); +#endif +} + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 627d945fbf3..05fe147e076 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -68,8 +68,7 @@ void registerTableFunctions() registerTableFunctionObjectStorage(factory); registerTableFunctionObjectStorageCluster(factory); - - + registerDataLakeTableFunctions(factory); } } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index cefb198273e..7998a4b49d9 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -30,13 +30,6 @@ void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory); void registerTableFunctionOSS(TableFunctionFactory & factory); void registerTableFunctionGCS(TableFunctionFactory & factory); -void registerTableFunctionHudi(TableFunctionFactory & factory); -#if USE_PARQUET -void registerTableFunctionDeltaLake(TableFunctionFactory & factory); -#endif -#if USE_AVRO -void registerTableFunctionIceberg(TableFunctionFactory & factory); -#endif #endif #if USE_HIVE @@ -67,10 +60,9 @@ void registerTableFunctionFormat(TableFunctionFactory & factory); void registerTableFunctionExplain(TableFunctionFactory & factory); -#if USE_AZURE_BLOB_STORAGE void registerTableFunctionObjectStorage(TableFunctionFactory & factory); void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory); -#endif +void registerDataLakeTableFunctions(TableFunctionFactory & factory); void registerTableFunctions(); From 7577257df558fb3bd74e862e7da7b0f1b485ffeb Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 14 Feb 2024 17:29:03 +0100 Subject: [PATCH 004/158] Fix cluster functions --- .../ReadFromStorageObjectStorage.cpp | 6 +- .../ObjectStorage/StorageObjectStorage.cpp | 3 +- .../StorageObjectStorageCluster.cpp | 9 ++- .../StorageObjectStorageSource.cpp | 63 ++++++++++++++++--- .../StorageObjectStorageSource.h | 25 ++++++-- src/Storages/S3Queue/StorageS3Queue.cpp | 5 +- .../TableFunctionObjectStorage.cpp | 3 + .../TableFunctionObjectStorageCluster.cpp | 27 ++++---- 8 files changed, 110 insertions(+), 31 deletions(-) diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp index b33eea7d354..9c58fcdaa9a 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp @@ -49,7 +49,8 @@ void ReadFromStorageObejctStorage::createIterator(const ActionsDAG::Node * predi auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( configuration, object_storage, distributed_processing, context, predicate, - virtual_columns, nullptr, query_settings.list_object_keys_size, context->getFileProgressCallback()); + virtual_columns, nullptr, query_settings.list_object_keys_size, metric_threads_count, + metric_threads_active, metric_threads_scheduled, context->getFileProgressCallback()); } } @@ -75,7 +76,8 @@ void ReadFromStorageObejctStorage::initializePipeline(QueryPipelineBuilder & pip auto source = std::make_shared( getName(), object_storage, configuration, info, format_settings, query_settings, - context, max_block_size, iterator_wrapper, need_only_count, schema_cache, std::move(threadpool)); + context, max_block_size, iterator_wrapper, need_only_count, schema_cache, + std::move(threadpool), metric_threads_count, metric_threads_active, metric_threads_scheduled); pipes.emplace_back(std::move(source)); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 08d7c9d0014..2e834da5529 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -289,7 +289,8 @@ ColumnsDescription StorageObjectStorage::getTableStructureFromD const auto settings = StorageSettings::create(context->getSettingsRef()); auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, object_storage, /* distributed_processing */false, - context, /* predicate */{}, /* virtual_columns */{}, &read_keys, settings.list_object_keys_size); + context, /* predicate */{}, /* virtual_columns */{}, &read_keys, settings.list_object_keys_size, + StorageSettings::ObjectStorageThreads(), StorageSettings::ObjectStorageThreadsActive(), StorageSettings::ObjectStorageThreadsScheduled()); ReadBufferIterator read_buffer_iterator( object_storage, configuration, file_iterator, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index c03bbd1a45d..f0d9ea400c4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -88,7 +88,14 @@ StorageObjectStorageCluster::getTask auto iterator = std::make_shared( object_storage, configuration, predicate, virtual_columns, local_context, nullptr, settings.list_object_keys_size); - auto callback = std::make_shared>([iterator]() mutable -> String{ return iterator->next(0)->relative_path; }); + auto callback = std::make_shared>([iterator]() mutable -> String + { + auto object_info = iterator->next(0); + if (object_info) + return object_info->relative_path; + else + return ""; + }); return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 1fda75897f9..a8bde4cd56f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -42,7 +42,10 @@ StorageObjectStorageSource::StorageObjectStorageSource( std::shared_ptr file_iterator_, bool need_only_count_, SchemaCache & schema_cache_, - std::shared_ptr reader_pool_) + std::shared_ptr reader_pool_, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_) : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) , name(std::move(name_)) @@ -57,6 +60,9 @@ StorageObjectStorageSource::StorageObjectStorageSource( , columns_desc(info.columns_description) , file_iterator(file_iterator_) , schema_cache(schema_cache_) + , metric_threads(metric_threads_) + , metric_threads_active(metric_threads_active_) + , metric_threads_scheduled(metric_threads_scheduled_) , create_reader_scheduler(threadPoolCallbackRunner(*create_reader_pool, "Reader")) { } @@ -75,10 +81,16 @@ std::shared_ptr StorageObjectStorageSourc const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, size_t list_object_keys_size, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_, std::function file_progress_callback) { if (distributed_processing) - return std::make_shared(local_context->getReadTaskCallback()); + return std::make_shared( + local_context->getReadTaskCallback(), + local_context->getSettingsRef().max_threads, + metric_threads_, metric_threads_active_, metric_threads_scheduled_); if (configuration->isNamespaceWithGlobs()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); @@ -380,19 +392,16 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor while (new_batch.empty()) { auto result = object_storage_iterator->getCurrentBatchAndScheduleNext(); - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {}", result.has_value()); - if (result.has_value()) - { - new_batch = std::move(result.value()); - } - else + if (!result.has_value()) { is_finished = true; return {}; } + new_batch = std::move(result.value()); for (auto it = new_batch.begin(); it != new_batch.end();) { + chassert(*it); if (!recursive && !re2::RE2::FullMatch((*it)->relative_path, *matcher)) it = new_batch.erase(it); else @@ -406,8 +415,11 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor { std::vector paths; paths.reserve(new_batch.size()); - for (auto & object_info : new_batch) + for (const auto & object_info : new_batch) + { + chassert(object_info); paths.push_back(fs::path(configuration->getNamespace()) / object_info->relative_path); + } VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); } @@ -416,6 +428,7 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor read_keys->insert(read_keys->end(), new_batch.begin(), new_batch.end()); object_infos = std::move(new_batch); + if (file_progress_callback) { for (const auto & object_info : object_infos) @@ -499,4 +512,36 @@ StorageObjectStorageSource::ReaderHolder & StorageObjectStorageSource::ReaderHol object_info = std::move(other.object_info); return *this; } + +StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( + const ReadTaskCallback & callback_, + size_t max_threads_count, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_) + : callback(callback_) +{ + ThreadPool pool(metric_threads_, metric_threads_active_, metric_threads_scheduled_, max_threads_count); + auto pool_scheduler = threadPoolCallbackRunner(pool, "ReadTaskIter"); + + std::vector> keys; + keys.reserve(max_threads_count); + for (size_t i = 0; i < max_threads_count; ++i) + keys.push_back(pool_scheduler([this] { return callback(); }, Priority{})); + + pool.wait(); + buffer.reserve(max_threads_count); + for (auto & key_future : keys) + buffer.emplace_back(std::make_shared(key_future.get(), std::nullopt)); +} + +ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::next(size_t) +{ + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= buffer.size()) + return std::make_shared(callback()); + + return buffer[current_index]; +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 214a7de14d6..14e59312c8c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -30,7 +30,10 @@ public: std::shared_ptr file_iterator_, bool need_only_count_, SchemaCache & schema_cache_, - std::shared_ptr reader_pool_); + std::shared_ptr reader_pool_, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_); ~StorageObjectStorageSource() override; @@ -47,6 +50,9 @@ public: const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, size_t list_object_keys_size, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_, std::function file_progress_callback = {}); protected: @@ -64,6 +70,10 @@ protected: SchemaCache & schema_cache; bool initialized = false; + const CurrentMetrics::Metric metric_threads; + const CurrentMetrics::Metric metric_threads_active; + const CurrentMetrics::Metric metric_threads_scheduled; + size_t total_rows_in_file = 0; LoggerPtr log = getLogger("StorageObjectStorageSource"); @@ -123,14 +133,21 @@ public: class StorageObjectStorageSource::ReadTaskIterator : public IIterator { public: - explicit ReadTaskIterator(const ReadTaskCallback & callback_) : callback(callback_) {} + ReadTaskIterator( + const ReadTaskCallback & callback_, + size_t max_threads_count, + CurrentMetrics::Metric metric_threads_, + CurrentMetrics::Metric metric_threads_active_, + CurrentMetrics::Metric metric_threads_scheduled_); - size_t estimatedKeysCount() override { return 0; } /// TODO FIXME + size_t estimatedKeysCount() override { return buffer.size(); } - ObjectInfoPtr next(size_t) override { return std::make_shared(callback(), ObjectMetadata{}); } + ObjectInfoPtr next(size_t) override; private: ReadTaskCallback callback; + ObjectInfos buffer; + std::atomic_size_t index = 0; }; class StorageObjectStorageSource::GlobIterator : public IIterator, WithContext diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index b03224cedff..b256f030da1 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -364,7 +364,10 @@ std::shared_ptr StorageS3Queue::createSource( file_iterator, false, Storage::getSchemaCache(local_context), - threadpool); + threadpool, + CurrentMetrics::ObjectStorageS3Threads, + CurrentMetrics::ObjectStorageS3ThreadsActive, + CurrentMetrics::ObjectStorageS3ThreadsScheduled); auto file_deleter = [=, this](const std::string & path) mutable { diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index a948102ac2b..a48c95469d0 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -93,6 +93,7 @@ template ColumnsDescription TableFunctionObjectStorage< Definition, StorageSettings, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const { + chassert(configuration); if (configuration->structure == "auto") { context->checkAccess(getSourceAccessType()); @@ -107,6 +108,7 @@ template bool TableFunctionObjectStorage< Definition, StorageSettings, Configuration>::supportsReadingSubsetOfColumns(const ContextPtr & context) { + chassert(configuration); return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); } @@ -127,6 +129,7 @@ StoragePtr TableFunctionObjectStoragestructure != "auto") columns = parseColumnsListFromString(configuration->structure, context); else if (!structure_hint.empty()) diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index c93d816dc07..5a29a693431 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -21,25 +21,23 @@ StoragePtr TableFunctionObjectStorageCluster; - StoragePtr storage; + auto configuration = Base::getConfiguration(); + bool structure_argument_was_provided = configuration->structure != "auto"; + ColumnsDescription columns; - bool structure_argument_was_provided = Base::configuration->structure != "auto"; - if (structure_argument_was_provided) - { - columns = parseColumnsListFromString(Base::configuration->structure, context); - } + columns = parseColumnsListFromString(configuration->structure, context); else if (!Base::structure_hint.empty()) - { columns = Base::structure_hint; - } + auto object_storage = Base::getObjectStorage(context, !is_insert_query); + StoragePtr storage; if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { /// On worker node this filename won't contains globs storage = std::make_shared>( - Base::configuration, - Base::configuration->createOrUpdateObjectStorage(context, !is_insert_query), + configuration, + object_storage, Definition::storage_type_name, context, StorageID(Base::getDatabaseName(), table_name), @@ -54,8 +52,8 @@ StoragePtr TableFunctionObjectStorageCluster>( ITableFunctionCluster::cluster_name, - Base::configuration, - Base::configuration->createOrUpdateObjectStorage(context, !is_insert_query), + configuration, + object_storage, Definition::storage_type_name, StorageID(Base::getDatabaseName(), table_name), columns, @@ -87,7 +85,10 @@ void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) { .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.)", - .examples{{"azureBlobStorageCluster", "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])", ""}}}, + .examples{{ + "azureBlobStorageCluster", + "SELECT * FROM azureBlobStorageCluster(cluster, connection_string|storage_account_url, container_name, blobpath, " + "[account_name, account_key, format, compression, structure])", ""}}}, .allow_readonly = false } ); From 0552f44f70d76f25f268259a09cbbb10dc3781d7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 19 Feb 2024 10:45:56 +0100 Subject: [PATCH 005/158] Fixes after merge with master, move some part of code to object storage --- src/Backups/BackupIO_S3.cpp | 8 +- src/Disks/ObjectStorages/IObjectStorage.h | 3 +- .../ObjectStorages/ObjectStorageFactory.cpp | 4 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 57 ++++- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 12 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 110 +++++---- src/Disks/ObjectStorages/S3/diskSettings.h | 13 +- src/IO/S3Common.cpp | 7 +- src/Storages/Cache/SchemaCache.cpp | 2 + .../ObjectStorage/AzureBlob/Configuration.cpp | 7 +- .../ObjectStorage/AzureBlob/Configuration.h | 2 +- .../DataLakes/IStorageDataLake.h | 18 +- .../ObjectStorage/HDFS/Configuration.cpp | 7 +- .../ObjectStorage/HDFS/Configuration.h | 2 +- .../ObjectStorage/ReadBufferIterator.cpp | 210 +++++++++++++----- .../ObjectStorage/ReadBufferIterator.h | 12 +- .../ObjectStorage/S3/Configuration.cpp | 108 ++------- src/Storages/ObjectStorage/S3/Configuration.h | 18 +- .../ObjectStorage/StorageObjectStorage.cpp | 109 ++++++--- .../ObjectStorage/StorageObjectStorage.h | 20 +- .../StorageObjectStorageCluster.cpp | 30 ++- .../StorageObjectStorageCluster.h | 8 +- .../StorageObjectStorageConfiguration.cpp | 6 +- .../StorageObjectStorageConfiguration.h | 3 +- .../StorageObjectStorageSource.h | 4 + .../registerStorageObjectStorage.cpp | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 8 +- src/Storages/StorageS3Settings.cpp | 11 +- src/Storages/StorageS3Settings.h | 8 +- .../TableFunctionObjectStorage.cpp | 6 +- .../TableFunctionObjectStorage.h | 6 +- .../TableFunctionObjectStorageCluster.cpp | 7 +- 32 files changed, 498 insertions(+), 330 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index fa4c1af3698..6c7b3674fb7 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -127,10 +127,10 @@ BackupReaderS3::BackupReaderS3( : BackupReaderDefault(read_settings_, write_settings_, getLogger("BackupReaderS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString())) + , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef()); + request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; client = makeS3Client(s3_uri_, access_key_id_, secret_access_key_, s3_settings, context_); @@ -217,10 +217,10 @@ BackupWriterS3::BackupWriterS3( : BackupWriterDefault(read_settings_, write_settings_, getLogger("BackupWriterS3")) , s3_uri(s3_uri_) , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::S3, MetadataStorageType::None, s3_uri.endpoint, false, false} - , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString())) + , s3_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).value_or(S3Settings{})) { auto & request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context_->getSettingsRef()); + request_settings.updateFromSettingsIfChanged(context_->getSettingsRef()); request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint request_settings.allow_native_copy = allow_s3_native_copy; request_settings.setStorageClassName(storage_class_name); diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 8a5352e71ca..5ff618e08eb 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -193,8 +193,7 @@ public: virtual void applyNewSettings( const Poco::Util::AbstractConfiguration &, const std::string & /*config_prefix*/, - ContextPtr) - {} + ContextPtr) {} /// Sometimes object storages have something similar to chroot or namespace, for example /// buckets in S3. If object storage doesn't have any namepaces return empty string. diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index b3626135177..0855ba54d2f 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -126,7 +126,7 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); + auto client = getClient(config, config_prefix, context, *settings, true); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); auto object_storage = std::make_shared( @@ -162,7 +162,7 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory) auto uri = getS3URI(config, config_prefix, context); auto s3_capabilities = getCapabilitiesFromConfig(config, config_prefix); auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); + auto client = getClient(config, config_prefix, context, *settings, true); auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix); auto object_storage = std::make_shared( diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a9bd520e6e9..7e856b45aea 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -242,7 +242,12 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN if (mode != WriteMode::Rewrite) throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files"); - auto settings_ptr = s3_settings.get(); + S3Settings::RequestSettings request_settings = s3_settings.get()->request_settings; + if (auto query_context = CurrentThread::getQueryContext()) + { + request_settings.updateFromSettingsIfChanged(query_context->getSettingsRef()); + } + ThreadPoolCallbackRunner scheduler; if (write_settings.s3_allow_parallel_part_upload) scheduler = threadPoolCallbackRunner(getThreadPoolWriter(), "VFSWrite"); @@ -256,7 +261,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN uri.bucket, object.remote_path, buf_size, - settings_ptr->request_settings, + request_settings, std::move(blob_storage_log), attributes, std::move(scheduler), @@ -534,19 +539,57 @@ void S3ObjectStorage::startup() const_cast(*client.get()).EnableRequestProcessing(); } -void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) +void S3ObjectStorage::applyNewSettings( + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ContextPtr context) { auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + if (!static_headers.empty()) + { + new_s3_settings->auth_settings.headers.insert( + new_s3_settings->auth_settings.headers.end(), + static_headers.begin(), static_headers.end()); + } + + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString())) + new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + + auto current_s3_settings = s3_settings.get(); + if (current_s3_settings->auth_settings.hasUpdates(new_s3_settings->auth_settings) || for_disk_s3) + { + auto new_client = getClient(config, config_prefix, context, *new_s3_settings, for_disk_s3, &uri); + client.set(std::move(new_client)); + } + s3_settings.set(std::move(new_s3_settings)); - client.set(std::move(new_client)); } +// void S3ObjectStorage::applyNewSettings(ContextPtr context) +// { +// auto settings = s3_settings.get(); +// if (!endpoint_settings || !settings->auth_settings.hasUpdates(endpoint_settings->auth_settings)) +// return; +// +// const auto & config = context->getConfigRef(); +// auto new_s3_settings = getSettings(uri, config, "s3.", context); +// +// new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); +// +// auto new_client = getClient(config, "s3.", context, *new_s3_settings, false); +// +// s3_settings.set(std::move(new_s3_settings)); +// client.set(std::move(new_client)); +// } + std::unique_ptr S3ObjectStorage::cloneObjectStorage( - const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) + const std::string & new_namespace, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + ContextPtr context) { auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + auto new_client = getClient(config, config_prefix, context, *new_s3_settings, true); String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); auto new_uri{uri}; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index a6843a383e5..187cdb58447 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -21,11 +21,13 @@ struct S3ObjectStorageSettings S3ObjectStorageSettings( const S3Settings::RequestSettings & request_settings_, + const S3::AuthSettings & auth_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, int32_t objects_chunk_size_to_delete_, bool read_only_) : request_settings(request_settings_) + , auth_settings(auth_settings_) , min_bytes_for_seek(min_bytes_for_seek_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) @@ -33,6 +35,7 @@ struct S3ObjectStorageSettings {} S3Settings::RequestSettings request_settings; + S3::AuthSettings auth_settings; uint64_t min_bytes_for_seek; int32_t list_object_keys_size; @@ -52,7 +55,9 @@ private: S3::URI uri_, const S3Capabilities & s3_capabilities_, ObjectStorageKeysGeneratorPtr key_generator_, - const String & disk_name_) + const String & disk_name_, + bool for_disk_s3_ = true, + const HTTPHeaderEntries & static_headers_ = {}) : uri(uri_) , key_generator(std::move(key_generator_)) , disk_name(disk_name_) @@ -60,6 +65,8 @@ private: , s3_settings(std::move(s3_settings_)) , s3_capabilities(s3_capabilities_) , log(getLogger(logger_name)) + , for_disk_s3(for_disk_s3_) + , static_headers(static_headers_) { } @@ -180,6 +187,9 @@ private: S3Capabilities s3_capabilities; LoggerPtr log; + + const bool for_disk_s3; + const HTTPHeaderEntries static_headers; }; /// Do not encode keys, store as-is, and do not require separate disk for metadata. diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 4fd4b17aabe..cb2bb690292 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -10,8 +10,6 @@ #include #include #include -#include "Disks/DiskFactory.h" - #include #include #include @@ -25,13 +23,19 @@ namespace DB { -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +std::unique_ptr getSettings( + const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { const Settings & settings = context->getSettingsRef(); S3Settings::RequestSettings request_settings(config, config_prefix, settings, "s3_"); + /// TODO: add request settings prefix, becausse for StorageS3 it should be "s3." + + S3::AuthSettings auth_settings; + auth_settings.loadFromConfig(config_prefix, config); return std::make_unique( request_settings, + auth_settings, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), @@ -42,78 +46,92 @@ std::unique_ptr getClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, - const S3ObjectStorageSettings & settings) + const S3ObjectStorageSettings & settings, + bool for_disk_s3, + const S3::URI * url_) { const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); const Settings & local_settings = context->getSettingsRef(); - String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - S3::URI uri(endpoint); - if (!uri.key.ends_with('/')) - uri.key.push_back('/'); + const auto & auth_settings = settings.auth_settings; + const auto & request_settings = settings.request_settings; + + S3::URI url; + if (for_disk_s3) + { + String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); + S3::URI uri(endpoint); + if (!uri.key.ends_with('/')) + uri.key.push_back('/'); + } + else + { + if (!url_) + throw Exception(ErrorCodes::LOGICAL_ERROR, "URL not passed"); + url = *url_; + } S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - config.getString(config_prefix + ".region", ""), + auth_settings.region, context->getRemoteHostFilter(), static_cast(global_settings.s3_max_redirects), static_cast(global_settings.s3_retry_attempts), global_settings.enable_s3_requests_logging, - /* for_disk_s3 = */ true, + for_disk_s3, settings.request_settings.get_request_throttler, settings.request_settings.put_request_throttler, - uri.uri.getScheme()); + url.uri.getScheme()); + client_configuration.endpointOverride = url.endpoint; + client_configuration.maxConnections = static_cast(request_settings.max_connections); client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS); client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); - client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); - client_configuration.endpointOverride = uri.endpoint; - client_configuration.http_keep_alive_timeout_ms = config.getUInt( - config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); - client_configuration.http_connection_pool_size = config.getUInt(config_prefix + ".http_connection_pool_size", 1000); - client_configuration.wait_on_pool_size_limit = false; - client_configuration.s3_use_adaptive_timeouts = config.getBool( - config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); - /* - * Override proxy configuration for backwards compatibility with old configuration format. - * */ - auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( - ProxyConfiguration::protocolFromString(uri.uri.getScheme()), - config_prefix, - config - ); - if (proxy_config) + client_configuration.http_keep_alive_timeout_ms = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); + client_configuration.http_connection_pool_size = config.getUInt( + config_prefix + ".http_connection_pool_size", static_cast(global_settings.s3_http_connection_pool_size.value)); + client_configuration.s3_use_adaptive_timeouts = config.getBool(config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); + client_configuration.wait_on_pool_size_limit = for_disk_s3; + + if (for_disk_s3) { - client_configuration.per_request_configuration - = [proxy_config]() { return proxy_config->resolve(); }; - client_configuration.error_report - = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + /* + * Override proxy configuration for backwards compatibility with old configuration format. + * */ + if (auto proxy_config = DB::ProxyConfigurationResolverProvider::getFromOldSettingsFormat( + ProxyConfiguration::protocolFromString(url.uri.getScheme()), config_prefix, config)) + { + client_configuration.per_request_configuration + = [proxy_config]() { return proxy_config->resolve(); }; + client_configuration.error_report + = [proxy_config](const auto & request_config) { proxy_config->errorReport(request_config); }; + } } - HTTPHeaderEntries headers = S3::getHTTPHeaders(config_prefix, config); S3::ServerSideEncryptionKMSConfig sse_kms_config = S3::getSSEKMSConfig(config_prefix, config); - S3::ClientSettings client_settings{ - .use_virtual_addressing = uri.is_virtual_hosted_style, + .use_virtual_addressing = url.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), }; + auto credentials_configuration = S3::CredentialsConfiguration + { + auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), + auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), + auth_settings.expiration_window_seconds.value_or(context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), + auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), + }; + return S3::ClientFactory::instance().create( client_configuration, client_settings, - config.getString(config_prefix + ".access_key_id", ""), - config.getString(config_prefix + ".secret_access_key", ""), - config.getString(config_prefix + ".server_side_encryption_customer_key_base64", ""), + auth_settings.access_key_id, + auth_settings.secret_access_key, + auth_settings.server_side_encryption_customer_key_base64, std::move(sse_kms_config), - std::move(headers), - S3::CredentialsConfiguration - { - config.getBool(config_prefix + ".use_environment_credentials", config.getBool("s3.use_environment_credentials", true)), - config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)), - config.getUInt64(config_prefix + ".expiration_window_seconds", config.getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - config.getBool(config_prefix + ".no_sign_request", config.getBool("s3.no_sign_request", false)) - }); + auth_settings.headers, + credentials_configuration); } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 83bf7b179ef..194035365ea 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -22,9 +22,18 @@ namespace DB struct S3ObjectStorageSettings; -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); +std::unique_ptr getSettings( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); +std::unique_ptr getClient( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + const S3ObjectStorageSettings & settings, + bool for_disk_s3, + const S3::URI * url_ = nullptr); } diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 5039059f522..d33d5284240 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -157,8 +157,11 @@ void AuthSettings::updateFrom(const AuthSettings & from) if (!from.session_token.empty()) session_token = from.session_token; - headers = from.headers; - region = from.region; + if (!from.headers.empty()) + headers = from.headers; + if (!from.region.empty()) + region = from.region; + server_side_encryption_customer_key_base64 = from.server_side_encryption_customer_key_base64; server_side_encryption_kms_config = from.server_side_encryption_kms_config; diff --git a/src/Storages/Cache/SchemaCache.cpp b/src/Storages/Cache/SchemaCache.cpp index 299dd292772..35fb8d348ef 100644 --- a/src/Storages/Cache/SchemaCache.cpp +++ b/src/Storages/Cache/SchemaCache.cpp @@ -1,5 +1,6 @@ #include #include +#include #include namespace ProfileEvents @@ -109,6 +110,7 @@ std::optional SchemaCache::tryGetImpl(const Key & key, } ProfileEvents::increment(ProfileEvents::SchemaInferenceCacheHits); + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {}", StackTrace().toString()); auto & schema_info = it->second.schema_info; auto & queue_iterator = it->second.iterator; diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 109918dfc8b..9d21541e7e2 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -102,7 +102,7 @@ AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(Co return settings_ptr; } -ObjectStoragePtr StorageAzureBlobConfiguration::createOrUpdateObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +ObjectStoragePtr StorageAzureBlobConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { auto client = createClient(is_readonly); auto settings = createSettings(context); @@ -245,8 +245,6 @@ void StorageAzureBlobConfiguration::fromNamedCollection(const NamedCollection & compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); blobs_paths = {blob_path}; - if (format == "auto") - format = FormatFactory::instance().getFormatFromFileName(blob_path, true); } void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) @@ -367,9 +365,6 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte } blobs_paths = {blob_path}; - - if (format == "auto") - format = FormatFactory::instance().getFormatFromFileName(blob_path, true); } void StorageAzureBlobConfiguration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.h b/src/Storages/ObjectStorage/AzureBlob/Configuration.h index deeb365d012..3d701e72cb4 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.h +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.h @@ -31,7 +31,7 @@ public: String getNamespace() const override { return container; } void check(ContextPtr context) const override; - ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } void fromNamedCollection(const NamedCollection & collection) override; diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 95196cdd000..8a21fc1152f 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -39,7 +39,7 @@ public: std::optional format_settings_, bool attach) { - auto object_storage = base_configuration->createOrUpdateObjectStorage(context); + auto object_storage = base_configuration->createObjectStorage(context); DataLakeMetadataPtr metadata; NamesAndTypesList schema_from_metadata; ConfigurationPtr configuration = base_configuration->clone(); @@ -75,28 +75,22 @@ public: return ColumnsDescription(metadata->getTableSchema()); } - std::pair updateConfigurationAndGetCopy(ContextPtr local_context) override + void updateConfiguration(ContextPtr local_context) override { std::lock_guard lock(Storage::configuration_update_mutex); - auto new_object_storage = base_configuration->createOrUpdateObjectStorage(local_context); - bool updated = new_object_storage != nullptr; - if (updated) - Storage::object_storage = new_object_storage; + Storage::updateConfiguration(local_context); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); - if (!current_metadata || !(*current_metadata == *new_metadata)) - current_metadata = std::move(new_metadata); - else if (!updated) - return {Storage::configuration, Storage::object_storage}; + if (current_metadata && *current_metadata == *new_metadata) + return; + current_metadata = std::move(new_metadata); auto updated_configuration = base_configuration->clone(); /// If metadata wasn't changed, we won't list data files again. updated_configuration->getPaths() = current_metadata->getDataFiles(); Storage::configuration = updated_configuration; - - return {Storage::configuration, Storage::object_storage}; } template diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index c80237b3055..731b05f4621 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -27,7 +27,7 @@ void StorageHDFSConfiguration::check(ContextPtr context) const checkHDFSURL(url); } -ObjectStoragePtr StorageHDFSConfiguration::createOrUpdateObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { UNUSED(is_readonly); auto settings = std::make_unique(); @@ -42,16 +42,13 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr, bool /* with_str if (args.size() > 1) format_name = checkAndGetLiteralArgument(args[1], "format_name"); - if (format_name == "auto") - format_name = FormatFactory::instance().getFormatFromFileName(url, true); - String compression_method; if (args.size() == 3) compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); else compression_method = "auto"; - } + } #endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 03fb0824123..1013c2e00c2 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -26,7 +26,7 @@ public: String getDataSourceDescription() override { return url; } void check(ContextPtr context) const override; - ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } void fromNamedCollection(const NamedCollection &) override {} diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index a3e19b907bc..a0e719878ac 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int CANNOT_DETECT_FORMAT; } @@ -30,14 +31,15 @@ ReadBufferIterator::ReadBufferIterator( , query_settings(query_settings_) , schema_cache(schema_cache_) , read_keys(read_keys_) + , format(configuration->format.empty() || configuration->format == "auto" ? std::nullopt : std::optional(configuration->format)) , prev_read_keys_size(read_keys_.size()) { } -SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const String & path) const +SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const String & path, const String & format_name) const { auto source = fs::path(configuration->getDataSourceDescription()) / path; - return DB::getKeyForSchemaCache(source, configuration->format, format_settings, getContext()); + return DB::getKeyForSchemaCache(source, format_name, format_settings, getContext()); } SchemaCache::Keys ReadBufferIterator::getPathsForSchemaCache() const @@ -51,7 +53,7 @@ SchemaCache::Keys ReadBufferIterator::getPathsForSchemaCache() const { return fs::path(configuration->getDataSourceDescription()) / elem->relative_path; }); - return DB::getKeysForSchemaCache(sources, configuration->format, format_settings, getContext()); + return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); } std::optional ReadBufferIterator::tryGetColumnsFromCache( @@ -75,10 +77,29 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( } }; - auto cache_key = getKeyForSchemaCache(object_info->relative_path); - auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time); - if (columns) - return columns; + chassert(object_info); + if (format) + { + auto cache_key = getKeyForSchemaCache(object_info->relative_path, *format); + if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) + return columns; + } + else + { + /// If format is unknown, we can iterate through all possible input formats + /// and check if we have an entry with this format and this file in schema cache. + /// If we have such entry for some format, we can use this format to read the file. + for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) + { + auto cache_key = getKeyForSchemaCache(object_info->relative_path, format_name); + if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) + { + /// Now format is known. It should be the same for all files. + format = format_name; + return columns; + } + } + } } return std::nullopt; @@ -86,16 +107,18 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) { + chassert(current_object_info); if (query_settings.schema_inference_use_cache) - schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->relative_path), num_rows); + schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->relative_path, *format), num_rows); } void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) { + chassert(current_object_info); if (query_settings.schema_inference_use_cache && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) { - schema_cache.addColumns(getKeyForSchemaCache(current_object_info->relative_path), columns); + schema_cache.addColumns(getKeyForSchemaCache(current_object_info->relative_path, *format), columns); } } @@ -108,6 +131,11 @@ void ReadBufferIterator::setResultingSchema(const ColumnsDescription & columns) } } +void ReadBufferIterator::setFormatName(const String & format_name) +{ + format = format_name; +} + String ReadBufferIterator::getLastFileName() const { if (current_object_info) @@ -116,64 +144,128 @@ String ReadBufferIterator::getLastFileName() const return ""; } -std::pair, std::optional> ReadBufferIterator::next() +std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() { - /// For default mode check cached columns for currently read keys on first iteration. - if (first && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) - return {nullptr, cached_columns}; - } + chassert(current_object_info); - current_object_info = file_iterator->next(0); - if (!current_object_info || current_object_info->relative_path.empty()) + auto impl = object_storage->readObject( + StoredObject(current_object_info->relative_path), getContext()->getReadSettings()); + + int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); + return wrapReadBufferWithCompressionMethod( + std::move(impl), chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), + zstd_window_log_max); +} + +ReadBufferIterator::Data ReadBufferIterator::next() +{ + if (first) { - if (first) + /// If format is unknown we iterate through all currently read keys on first iteration and + /// try to determine format by file name. + if (!format) { - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "Cannot extract table structure from {} format file, " - "because there are no files with provided path. " - "You must specify table structure manually", - configuration->format); + for (const auto & object_info : read_keys) + { + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->relative_path)) + { + format = format_from_file_name; + break; + } + } + } + + /// For default mode check cached columns for currently read keys on first iteration. + if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + return {nullptr, cached_columns, format}; } - return {nullptr, std::nullopt}; } - first = false; - - /// File iterator could get new keys after new iteration, - /// check them in schema cache if schema inference mode is default. - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT - && read_keys.size() > prev_read_keys_size) + while (true) { - auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); - prev_read_keys_size = read_keys.size(); - if (columns_from_cache) - return {nullptr, columns_from_cache}; + current_object_info = file_iterator->next(0); + + if (!current_object_info || current_object_info->relative_path.empty()) + { + if (first) + { + if (format) + throw Exception( + ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, + "The table structure cannot be extracted from a {} format file, because there are no files with provided path " + "in S3 or all files are empty. You can specify table structure manually", + *format); + + throw Exception( + ErrorCodes::CANNOT_DETECT_FORMAT, + "The data format cannot be detected by the contents of the files, because there are no files with provided path " + "in S3 or all files are empty. You can specify the format manually"); + } + + return {nullptr, std::nullopt, format}; + } + + /// S3 file iterator could get new keys after new iteration + if (read_keys.size() > prev_read_keys_size) + { + /// If format is unknown we can try to determine it by new file names. + if (!format) + { + for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) + { + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->relative_path)) + { + format = format_from_file_name; + break; + } + } + } + + /// Check new files in schema cache if schema inference mode is default. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) + { + auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end()); + if (columns_from_cache) + return {nullptr, columns_from_cache, format}; + } + + prev_read_keys_size = read_keys.size(); + } + + if (getContext()->getSettingsRef().s3_skip_empty_files + && current_object_info->metadata && current_object_info->metadata->size_bytes == 0) + continue; + + /// In union mode, check cached columns only for current key. + if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) + { + ObjectInfos objects{current_object_info}; + if (auto columns_from_cache = tryGetColumnsFromCache(objects.begin(), objects.end())) + { + first = false; + return {nullptr, columns_from_cache, format}; + } + } + + std::unique_ptr read_buffer = object_storage->readObject( + StoredObject(current_object_info->relative_path), + getContext()->getReadSettings(), + {}, + current_object_info->metadata->size_bytes); + + if (!getContext()->getSettingsRef().s3_skip_empty_files || !read_buffer->eof()) + { + first = false; + + read_buffer = wrapReadBufferWithCompressionMethod( + std::move(read_buffer), + chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), + static_cast(getContext()->getSettingsRef().zstd_window_log_max)); + + return {std::move(read_buffer), std::nullopt, format}; + } } - else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - ObjectInfos paths = {current_object_info}; - if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end())) - return {nullptr, columns_from_cache}; - } - - first = false; - - chassert(current_object_info->metadata); - std::unique_ptr read_buffer = object_storage->readObject( - StoredObject(current_object_info->relative_path), - getContext()->getReadSettings(), - {}, - current_object_info->metadata->size_bytes); - - read_buffer = wrapReadBufferWithCompressionMethod( - std::move(read_buffer), - chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), - static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - - return {std::move(read_buffer), std::nullopt}; } - } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 4e9b8cfcfca..053bcbf894f 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -23,7 +24,7 @@ public: ObjectInfos & read_keys_, const ContextPtr & context_); - std::pair, std::optional> next() override; + Data next() override; void setNumRowsToLastFile(size_t num_rows) override; @@ -33,8 +34,14 @@ public: String getLastFileName() const override; + void setFormatName(const String & format_name) override; + + bool supportsLastReadBufferRecreation() const override { return true; } + + std::unique_ptr recreateLastReadBuffer() override; + private: - SchemaCache::Key getKeyForSchemaCache(const String & path) const; + SchemaCache::Key getKeyForSchemaCache(const String & path, const String & format_name) const; SchemaCache::Keys getPathsForSchemaCache() const; std::optional tryGetColumnsFromCache( const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); @@ -46,6 +53,7 @@ private: const StorageObjectStorageSettings query_settings; SchemaCache & schema_cache; ObjectInfos & read_keys; + std::optional format; size_t prev_read_keys_size; ObjectInfoPtr current_object_info; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index f057745d669..896131e74d7 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -58,106 +59,47 @@ void StorageS3Configuration::check(ContextPtr context) const StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) { url = other.url; - auth_settings = other.auth_settings; - request_settings = other.request_settings; static_configuration = other.static_configuration; headers_from_ast = other.headers_from_ast; keys = other.keys; - initialized = other.initialized; format = other.format; compression_method = other.compression_method; structure = other.structure; } -ObjectStoragePtr StorageS3Configuration::createOrUpdateObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT +ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT { - auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString()); - request_settings = s3_settings.request_settings; - request_settings.updateFromSettings(context->getSettings()); + const auto & config = context->getConfigRef(); + const std::string config_prefix = "s3."; - if (!initialized || (!static_configuration && auth_settings.hasUpdates(s3_settings.auth_settings))) + auto s3_settings = getSettings(config, config_prefix, context); + + auth_settings.updateFrom(s3_settings->auth_settings); + s3_settings->auth_settings = auth_settings; + s3_settings->request_settings = request_settings; + + if (!headers_from_ast.empty()) { - auth_settings.updateFrom(s3_settings.auth_settings); - keys[0] = url.key; - initialized = true; + s3_settings->auth_settings.headers.insert( + s3_settings->auth_settings.headers.end(), + headers_from_ast.begin(), headers_from_ast.end()); } - const auto & config = context->getConfigRef(); + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString())) + s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + + auto client = getClient(config, config_prefix, context, *s3_settings, false, &url); + auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); auto s3_capabilities = S3Capabilities { .support_batch_delete = config.getBool("s3.support_batch_delete", true), .support_proxy = config.getBool("s3.support_proxy", config.has("s3.proxy")), }; - auto s3_storage_settings = std::make_unique( - request_settings, - config.getUInt64("s3.min_bytes_for_seek", 1024 * 1024), - config.getInt("s3.list_object_keys_size", 1000), - config.getInt("s3.objects_chunk_size_to_delete", 1000), - config.getBool("s3.readonly", false)); - - auto key_generator = createObjectStorageKeysGeneratorAsIsWithPrefix(url.key); - auto client = createClient(context); - std::string disk_name = "StorageS3"; - return std::make_shared( - std::move(client), std::move(s3_storage_settings), url, s3_capabilities, key_generator, /*disk_name*/disk_name); -} - -std::unique_ptr StorageS3Configuration::createClient(ContextPtr context) -{ - const Settings & global_settings = context->getGlobalContext()->getSettingsRef(); - const Settings & local_settings = context->getSettingsRef(); - - auto client_configuration = S3::ClientFactory::instance().createClientConfiguration( - auth_settings.region, - context->getRemoteHostFilter(), - static_cast(global_settings.s3_max_redirects), - static_cast(global_settings.s3_retry_attempts), - global_settings.enable_s3_requests_logging, - /* for_disk_s3 = */ false, - request_settings.get_request_throttler, - request_settings.put_request_throttler, - url.uri.getScheme()); - - client_configuration.endpointOverride = url.endpoint; - client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.http_connection_pool_size = global_settings.s3_http_connection_pool_size; - - auto headers = auth_settings.headers; - if (!headers_from_ast.empty()) - headers.insert(headers.end(), headers_from_ast.begin(), headers_from_ast.end()); - - client_configuration.requestTimeoutMs = request_settings.request_timeout_ms; - - S3::ClientSettings client_settings{ - .use_virtual_addressing = url.is_virtual_hosted_style, - .disable_checksum = local_settings.s3_disable_checksum, - .gcs_issue_compose_request = context->getConfigRef().getBool("s3.gcs_issue_compose_request", false), - }; - - auto credentials = Aws::Auth::AWSCredentials(auth_settings.access_key_id, - auth_settings.secret_access_key, - auth_settings.session_token); - - auto credentials_configuration = S3::CredentialsConfiguration - { - auth_settings.use_environment_credentials.value_or(context->getConfigRef().getBool("s3.use_environment_credentials", true)), - auth_settings.use_insecure_imds_request.value_or(context->getConfigRef().getBool("s3.use_insecure_imds_request", false)), - auth_settings.expiration_window_seconds.value_or(context->getConfigRef().getUInt64("s3.expiration_window_seconds", S3::DEFAULT_EXPIRATION_WINDOW_SECONDS)), - auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), - }; - - return S3::ClientFactory::instance().create( - client_configuration, - client_settings, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - auth_settings.server_side_encryption_customer_key_base64, - auth_settings.server_side_encryption_kms_config, - std::move(headers), - credentials_configuration); + std::move(client), std::move(s3_settings), url, s3_capabilities, + key_generator, "StorageS3", false, headers_from_ast); } void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) @@ -185,10 +127,6 @@ void StorageS3Configuration::fromNamedCollection(const NamedCollection & collect static_configuration = !auth_settings.access_key_id.empty() || auth_settings.no_sign_request.has_value(); keys = {url.key}; - - //if (format == "auto" && get_format_from_file) - if (format == "auto") - format = FormatFactory::instance().getFormatFromFileName(url.key, true); } void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_structure) @@ -386,10 +324,6 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ auth_settings.no_sign_request = no_sign_request; keys = {url.key}; - - // if (format == "auto" && get_format_from_file) - if (format == "auto") - format = FormatFactory::instance().getFormatFromFileName(url.key, true); } void StorageS3Configuration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 037cf2eae87..88a084f29b3 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -27,27 +27,25 @@ public: String getDataSourceDescription() override; void check(ContextPtr context) const override; - ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } + bool isStaticConfiguration() const override { return static_configuration; } - void fromNamedCollection(const NamedCollection & collection) override; - void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT static void addStructureToArgs(ASTs & args, const String & structure, ContextPtr context); private: + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + S3::URI url; + std::vector keys; + S3::AuthSettings auth_settings; S3Settings::RequestSettings request_settings; + HTTPHeaderEntries headers_from_ast; /// Headers from ast is a part of static configuration. /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. bool static_configuration = true; - /// Headers from ast is a part of static configuration. - HTTPHeaderEntries headers_from_ast; - std::vector keys; - - std::unique_ptr createClient(ContextPtr context); - - bool initialized = false; }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 2e834da5529..7337a528a76 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -13,8 +14,9 @@ #include #include #include -#include #include +#include +#include namespace DB @@ -39,21 +41,24 @@ std::unique_ptr getStorageMetadata( const std::string & engine_name, const ContextPtr & context) { + using Storage = StorageObjectStorage; + auto storage_metadata = std::make_unique(); if (columns.empty()) { - auto fetched_columns = StorageObjectStorage::getTableStructureFromData( - object_storage, configuration, format_settings, context); + auto fetched_columns = Storage::getTableStructureFromData(object_storage, configuration, format_settings, context); storage_metadata->setColumns(fetched_columns); } + else if (!columns.hasOnlyOrdinary()) + { + /// We don't allow special columns. + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine {} doesn't support special columns " + "like MATERIALIZED, ALIAS or EPHEMERAL", engine_name); + } else { - /// We don't allow special columns. - if (!columns.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table engine {} doesn't support special columns " - "like MATERIALIZED, ALIAS or EPHEMERAL", - engine_name); + if (configuration->format == "auto") + Storage::setFormatFromData(object_storage, configuration, format_settings, context); storage_metadata->setColumns(columns); } @@ -120,14 +125,10 @@ bool StorageObjectStorage::parallelizeOutputAfterReading(Contex } template -std::pair -StorageObjectStorage::updateConfigurationAndGetCopy(ContextPtr local_context) +void StorageObjectStorage::updateConfiguration(ContextPtr context) { - std::lock_guard lock(configuration_update_mutex); - auto new_object_storage = configuration->createOrUpdateObjectStorage(local_context); - if (new_object_storage) - object_storage = new_object_storage; - return {configuration, object_storage}; + if (!configuration->isStaticConfiguration()) + object_storage->applyNewSettings(context->getConfigRef(), "s3.", context); } template @@ -151,8 +152,8 @@ void StorageObjectStorage::read( size_t max_block_size, size_t num_streams) { - auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); - if (partition_by && query_configuration->withWildcard()) + updateConfiguration(local_context); + if (partition_by && configuration->withWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned {} storage is not implemented yet", @@ -165,8 +166,8 @@ void StorageObjectStorage::read( && local_context->getSettingsRef().optimize_count_from_files; auto read_step = std::make_unique( - query_object_storage, - query_configuration, + object_storage, + configuration, getName(), virtual_columns, format_settings, @@ -192,10 +193,10 @@ SinkToStoragePtr StorageObjectStorage::write( ContextPtr local_context, bool /* async_insert */) { - auto [query_configuration, query_object_storage] = updateConfigurationAndGetCopy(local_context); + updateConfiguration(local_context); const auto sample_block = metadata_snapshot->getSampleBlock(); - if (query_configuration->withWildcard()) + if (configuration->withWildcard()) { ASTPtr partition_by_ast = nullptr; if (auto insert_query = std::dynamic_pointer_cast(query)) @@ -209,24 +210,28 @@ SinkToStoragePtr StorageObjectStorage::write( if (partition_by_ast) { return std::make_shared( - object_storage, query_configuration, format_settings, sample_block, local_context, partition_by_ast); + object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); } } - if (query_configuration->withGlobs()) + if (configuration->withGlobs()) { throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "{} key '{}' contains globs, so the table is in readonly mode", - getName(), query_configuration->getPath()); + getName(), configuration->getPath()); } const auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); + + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII: {}", object_storage->exists(StoredObject(configuration->getPath()))); + auto configuration_copy = configuration->clone(); if (!storage_settings.truncate_on_insert - && object_storage->exists(StoredObject(query_configuration->getPath()))) + && object_storage->exists(StoredObject(configuration->getPath()))) { + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII 2: {}", storage_settings.create_new_file_on_insert); if (storage_settings.create_new_file_on_insert) { - auto & paths = query_configuration->getPaths(); + auto & paths = configuration_copy->getPaths(); size_t index = paths.size(); const auto & first_key = paths[0]; auto pos = first_key.find_first_of('.'); @@ -243,6 +248,7 @@ SinkToStoragePtr StorageObjectStorage::write( while (object_storage->exists(StoredObject(new_key))); paths.push_back(new_key); + configuration->getPaths().push_back(new_key); } else { @@ -251,12 +257,13 @@ SinkToStoragePtr StorageObjectStorage::write( "Object in bucket {} with key {} already exists. " "If you want to overwrite it, enable setting [engine_name]_truncate_on_insert, if you " "want to create a new file on each insert, enable setting [engine_name]_create_new_file_on_insert", - query_configuration->getNamespace(), query_configuration->getPaths().back()); + configuration_copy->getNamespace(), configuration_copy->getPaths().back()); } } + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII 3: {}", configuration_copy->getPaths().size()); return std::make_shared( - object_storage, query_configuration, format_settings, sample_block, local_context); + object_storage, configuration_copy, format_settings, sample_block, local_context); } template @@ -279,25 +286,55 @@ void StorageObjectStorage::truncate( } template -ColumnsDescription StorageObjectStorage::getTableStructureFromData( - ObjectStoragePtr object_storage, +std::unique_ptr StorageObjectStorage::createReadBufferIterator( + const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, - ContextPtr context) + ObjectInfos & read_keys, + const ContextPtr & context) { - ObjectInfos read_keys; const auto settings = StorageSettings::create(context->getSettingsRef()); auto file_iterator = StorageObjectStorageSource::createFileIterator( configuration, object_storage, /* distributed_processing */false, context, /* predicate */{}, /* virtual_columns */{}, &read_keys, settings.list_object_keys_size, StorageSettings::ObjectStorageThreads(), StorageSettings::ObjectStorageThreadsActive(), StorageSettings::ObjectStorageThreadsScheduled()); - ReadBufferIterator read_buffer_iterator( + return std::make_unique( object_storage, configuration, file_iterator, format_settings, StorageSettings::create(context->getSettingsRef()), getSchemaCache(context), read_keys, context); +} - const bool retry = configuration->withGlobs(); - return readSchemaFromFormat(configuration->format, format_settings, read_buffer_iterator, retry, context); +template +ColumnsDescription StorageObjectStorage::getTableStructureFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto read_buffer_iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + if (configuration->format == "auto") + { + auto [columns, format] = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context); + configuration->format = format; + return columns; + } + else + { + return readSchemaFromFormat(configuration->format, format_settings, *read_buffer_iterator, context); + } +} + +template +void StorageObjectStorage::setFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto read_buffer_iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + configuration->format = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context).second; } template class StorageObjectStorage; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 6f18153c7af..64c4c74ab22 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -21,6 +21,7 @@ using ReadTaskCallback = std::function; class IOutputFormat; class IInputFormat; class SchemaCache; +class ReadBufferIterator; template @@ -89,13 +90,26 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & context); static ColumnsDescription getTableStructureFromData( - ObjectStoragePtr object_storage, + const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, - ContextPtr context); + const ContextPtr & context); + + static void setFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context); protected: - virtual std::pair updateConfigurationAndGetCopy(ContextPtr local_context); + virtual void updateConfiguration(ContextPtr local_context); + + static std::unique_ptr createReadBufferIterator( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + ObjectInfos & read_keys, + const ContextPtr & context); const std::string engine_name; const NamesAndTypesList virtual_columns; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index f0d9ea400c4..2bd2c022aa8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -33,12 +33,10 @@ StorageObjectStorageCluster::Storage const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_) + ContextPtr context_) : IStorageCluster(cluster_name_, table_id_, - getLogger(fmt::format("{}({})", engine_name_, table_id_.table_name)), - structure_argument_was_provided_) + getLogger(fmt::format("{}({})", engine_name_, table_id_.table_name))) , engine_name(engine_name_) , configuration{configuration_} , object_storage(object_storage_) @@ -48,13 +46,16 @@ StorageObjectStorageCluster::Storage if (columns_.empty()) { - /// `format_settings` is set to std::nullopt, because StorageObjectStorageCluster is used only as table function - auto columns = StorageObjectStorage::getTableStructureFromData( - object_storage, configuration, /*format_settings=*/std::nullopt, context_); + ColumnsDescription columns = Storage::getTableStructureFromData(object_storage, configuration, /*format_settings=*/std::nullopt, context_); storage_metadata.setColumns(columns); } else + { + if (configuration->format == "auto") + StorageS3::setFormatFromData(object_storage, configuration, /*format_settings=*/std::nullopt, context_); + storage_metadata.setColumns(columns_); + } storage_metadata.setConstraints(constraints_); setInMemoryMetadata(storage_metadata); @@ -64,9 +65,9 @@ StorageObjectStorageCluster::Storage } template -void StorageObjectStorageCluster::addColumnsStructureToQuery( +void StorageObjectStorageCluster::updateQueryToSendIfNeeded( ASTPtr & query, - const String & structure, + const DB::StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) { ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); @@ -76,13 +77,18 @@ void StorageObjectStorageCluster::ad "Expected SELECT query from table function {}, got '{}'", engine_name, queryToString(query)); } - using TableFunction = TableFunctionObjectStorageCluster; - TableFunction::addColumnsStructureToArguments(expression_list->children, structure, context); + + TableFunction::updateStructureAndFormatArgumentsIfNeeded( + expression_list->children, + storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), + configuration->format, + context); } template RemoteQueryExecutor::Extension -StorageObjectStorageCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & local_context) const +StorageObjectStorageCluster::getTaskIteratorExtension( + const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { const auto settings = StorageSettings::create(local_context->getSettingsRef()); auto iterator = std::make_shared( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 507de20e888..5d77d4ced60 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -21,6 +21,7 @@ class StorageObjectStorageCluster : public IStorageCluster { public: using Storage = StorageObjectStorage; + using TableFunction = TableFunctionObjectStorageCluster; StorageObjectStorageCluster( const String & cluster_name_, @@ -30,8 +31,7 @@ public: const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, - ContextPtr context_, - bool structure_argument_was_provided_); + ContextPtr context_); std::string getName() const override { return engine_name; } @@ -49,9 +49,9 @@ public: private: void updateBeforeRead(const ContextPtr & /* context */) override {} - void addColumnsStructureToQuery( + void updateQueryToSendIfNeeded( ASTPtr & query, - const String & structure, + const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; const String & engine_name; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 651f1d25ec1..a1c7d468fa6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -1,5 +1,5 @@ #include - +#include namespace DB { @@ -14,6 +14,10 @@ void StorageObjectStorageConfiguration::initialize( configuration.fromNamedCollection(*named_collection); else configuration.fromAST(engine_args, local_context, with_table_structure); + + // FIXME: it should be - if (format == "auto" && get_format_from_file) + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); } bool StorageObjectStorageConfiguration::withWildcard() const diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 04b2d8e8fd9..2da262eb55d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -39,8 +39,9 @@ public: std::string getPathWithoutGlob() const; virtual void check(ContextPtr context) const = 0; - virtual ObjectStoragePtr createOrUpdateObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT virtual StorageObjectStorageConfigurationPtr clone() = 0; + virtual bool isStaticConfiguration() const { return true; } String format = "auto"; String compression_method = "auto"; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 14e59312c8c..3b503fd4f0c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -5,10 +5,14 @@ #include #include #include +#include namespace DB { + +class SchemaCache; + class StorageObjectStorageSource : public SourceWithKeyCondition, WithContext { friend class StorageS3QueueSource; diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index e23457c04e9..3271b766f68 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -56,7 +56,7 @@ static std::shared_ptr> createStorageObjec return std::make_shared>( configuration, - configuration->createOrUpdateObjectStorage(context), + configuration->createObjectStorage(context), engine_name, args.getContext(), args.table_id, diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 2673aa94347..bd526ad687b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -134,7 +134,7 @@ StorageS3Queue::StorageS3Queue( checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef()); - object_storage = configuration->createOrUpdateObjectStorage(context_); + object_storage = configuration->createObjectStorage(context_); FormatFactory::instance().checkFormatName(configuration->format); configuration->check(context_); @@ -146,8 +146,10 @@ StorageS3Queue::StorageS3Queue( } else { - if (configuration.format == "auto") - configuration.format = StorageS3::getTableStructureAndFormatFromData(configuration, format_settings, context_).second; + if (configuration->format == "auto") + { + StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context_); + } storage_metadata.setColumns(columns_); } diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index b0c1160429a..8510a6e4bdd 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings) { - updateFromSettingsImpl(settings, false); + updateFromSettings(settings, false); validate(); } @@ -66,7 +66,7 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedC validate(); } -void S3Settings::RequestSettings::PartUploadSettings::updateFromSettingsImpl(const Settings & settings, bool if_changed) +void S3Settings::RequestSettings::PartUploadSettings::updateFromSettings(const Settings & settings, bool if_changed) { if (!if_changed || settings.s3_strict_upload_part_size.changed) strict_upload_part_size = settings.s3_strict_upload_part_size; @@ -263,13 +263,12 @@ void S3Settings::RequestSettings::updateFromSettingsImpl(const Settings & settin request_timeout_ms = settings.s3_request_timeout_ms; } -void S3Settings::RequestSettings::updateFromSettings(const Settings & settings) +void S3Settings::RequestSettings::updateFromSettingsIfChanged(const Settings & settings) { updateFromSettingsImpl(settings, true); - upload_settings.updateFromSettings(settings); + upload_settings.updateFromSettings(settings, true); } - void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings) { std::lock_guard lock(mutex); @@ -293,7 +292,7 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U } } -S3Settings StorageS3Settings::getSettings(const String & endpoint) const +std::optional StorageS3Settings::getSettings(const String & endpoint) const { std::lock_guard lock(mutex); auto next_prefix_setting = s3_settings.upper_bound(endpoint); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 0e152bb2d31..a4bc9f0b5cf 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -39,7 +39,7 @@ struct S3Settings size_t max_single_operation_copy_size = 5ULL * 1024 * 1024 * 1024; String storage_class_name; - void updateFromSettings(const Settings & settings) { updateFromSettingsImpl(settings, true); } + void updateFromSettings(const Settings & settings, bool if_changed); void validate(); private: @@ -52,8 +52,6 @@ struct S3Settings const Settings & settings, String setting_name_prefix = {}); - void updateFromSettingsImpl(const Settings & settings, bool if_changed); - friend struct RequestSettings; }; @@ -96,7 +94,7 @@ struct S3Settings const Settings & settings, String setting_name_prefix = {}); - void updateFromSettings(const Settings & settings); + void updateFromSettingsIfChanged(const Settings & settings); private: void updateFromSettingsImpl(const Settings & settings, bool if_changed); @@ -112,7 +110,7 @@ class StorageS3Settings public: void loadFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, const Settings & settings); - S3Settings getSettings(const String & endpoint) const; + std::optional getSettings(const String & endpoint) const; private: mutable std::mutex mutex; diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index a48c95469d0..b07b328eed9 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -31,7 +31,7 @@ ObjectStoragePtr TableFunctionObjectStorage< Definition, StorageSettings, Configuration>::getObjectStorage(const ContextPtr & context, bool create_readonly) const { if (!object_storage) - object_storage = configuration->createOrUpdateObjectStorage(context, create_readonly); + object_storage = configuration->createObjectStorage(context, create_readonly); return object_storage; } @@ -63,8 +63,8 @@ std::vector TableFunctionObjectStorage< } template -void TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context) +void TableFunctionObjectStorage::updateStructureAndFormatArgumentsIfNeeded( + ASTs & args, const String & structure, const String & /* format */, const ContextPtr & context) { Configuration::addStructureToArgs(args, structure, context); } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 5e180301862..9022f6e577f 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -110,7 +110,11 @@ public: virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); - static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context); + static void updateStructureAndFormatArgumentsIfNeeded( + ASTs & args, + const String & structure, + const String & format, + const ContextPtr & context); protected: using ConfigurationPtr = StorageObjectStorageConfigurationPtr; diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 5a29a693431..55b41cf6ca8 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -20,12 +20,10 @@ StoragePtr TableFunctionObjectStorageCluster; - auto configuration = Base::getConfiguration(); - bool structure_argument_was_provided = configuration->structure != "auto"; ColumnsDescription columns; - if (structure_argument_was_provided) + if (configuration->structure != "auto") columns = parseColumnsListFromString(configuration->structure, context); else if (!Base::structure_hint.empty()) columns = Base::structure_hint; @@ -58,8 +56,7 @@ StoragePtr TableFunctionObjectStorageClusterstartup(); From 2e9b6545b6f060e1fa92970276116734f483f417 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 19 Feb 2024 18:24:23 +0100 Subject: [PATCH 006/158] Fix --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 16 ++++++------- src/Storages/Cache/SchemaCache.cpp | 1 - .../ObjectStorage/StorageObjectStorage.cpp | 3 --- .../StorageObjectStorageCluster.cpp | 3 ++- .../StorageObjectStorageSource.cpp | 24 ++++++++++--------- .../TableFunctionObjectStorageCluster.cpp | 2 +- 6 files changed, 23 insertions(+), 26 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index cb2bb690292..43b1cffb3e6 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -27,12 +27,8 @@ std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { const Settings & settings = context->getSettingsRef(); - S3Settings::RequestSettings request_settings(config, config_prefix, settings, "s3_"); - /// TODO: add request settings prefix, becausse for StorageS3 it should be "s3." - - S3::AuthSettings auth_settings; - auth_settings.loadFromConfig(config_prefix, config); - + auto request_settings = S3Settings::RequestSettings(config, config_prefix, settings, "s3_"); + auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); return std::make_unique( request_settings, auth_settings, @@ -60,9 +56,9 @@ std::unique_ptr getClient( if (for_disk_s3) { String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - S3::URI uri(endpoint); - if (!uri.key.ends_with('/')) - uri.key.push_back('/'); + url = S3::URI(endpoint); + if (!url.key.ends_with('/')) + url.key.push_back('/'); } else { @@ -123,6 +119,8 @@ std::unique_ptr getClient( auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), }; + LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {} - {}", auth_settings.access_key_id, auth_settings.secret_access_key); + return S3::ClientFactory::instance().create( client_configuration, client_settings, diff --git a/src/Storages/Cache/SchemaCache.cpp b/src/Storages/Cache/SchemaCache.cpp index 35fb8d348ef..5dc39f04ae0 100644 --- a/src/Storages/Cache/SchemaCache.cpp +++ b/src/Storages/Cache/SchemaCache.cpp @@ -110,7 +110,6 @@ std::optional SchemaCache::tryGetImpl(const Key & key, } ProfileEvents::increment(ProfileEvents::SchemaInferenceCacheHits); - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {}", StackTrace().toString()); auto & schema_info = it->second.schema_info; auto & queue_iterator = it->second.iterator; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 7337a528a76..30f5c36879c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -223,12 +223,10 @@ SinkToStoragePtr StorageObjectStorage::write( const auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII: {}", object_storage->exists(StoredObject(configuration->getPath()))); auto configuration_copy = configuration->clone(); if (!storage_settings.truncate_on_insert && object_storage->exists(StoredObject(configuration->getPath()))) { - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII 2: {}", storage_settings.create_new_file_on_insert); if (storage_settings.create_new_file_on_insert) { auto & paths = configuration_copy->getPaths(); @@ -260,7 +258,6 @@ SinkToStoragePtr StorageObjectStorage::write( configuration_copy->getNamespace(), configuration_copy->getPaths().back()); } } - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII 3: {}", configuration_copy->getPaths().size()); return std::make_shared( object_storage, configuration_copy, format_settings, sample_block, local_context); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 2bd2c022aa8..9b98051086d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -92,7 +92,8 @@ StorageObjectStorageCluster::getTask { const auto settings = StorageSettings::create(local_context->getSettingsRef()); auto iterator = std::make_shared( - object_storage, configuration, predicate, virtual_columns, local_context, nullptr, settings.list_object_keys_size); + object_storage, configuration, predicate, virtual_columns, local_context, + nullptr, settings.list_object_keys_size, local_context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a8bde4cd56f..d91850bf99c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -362,9 +362,9 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } else { - const auto key_with_globs = configuration_->getPath(); - auto object_metadata = object_storage->getObjectMetadata(key_with_globs); - auto object_info = std::make_shared(key_with_globs, object_metadata); + const auto object_key = configuration_->getPath(); + auto object_metadata = object_storage->getObjectMetadata(object_key); + auto object_info = std::make_shared(object_key, object_metadata); object_infos.emplace_back(object_info); if (read_keys) @@ -381,12 +381,11 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor { std::lock_guard lock(next_mutex); - if (is_finished) + bool current_batch_processed = object_infos.empty() || index >= object_infos.size(); + if (is_finished && current_batch_processed) return {}; - bool need_new_batch = object_infos.empty() || index >= object_infos.size(); - - if (need_new_batch) + if (current_batch_processed) { ObjectInfos new_batch; while (new_batch.empty()) @@ -439,11 +438,10 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor } } - size_t current_index = index++; - if (current_index >= object_infos.size()) + if (index >= object_infos.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Index out of bound for blob metadata"); - return object_infos[current_index]; + return object_infos[index++]; } StorageObjectStorageSource::KeysIterator::KeysIterator( @@ -532,7 +530,11 @@ StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( pool.wait(); buffer.reserve(max_threads_count); for (auto & key_future : keys) - buffer.emplace_back(std::make_shared(key_future.get(), std::nullopt)); + { + auto key = key_future.get(); + if (!key.empty()) + buffer.emplace_back(std::make_shared(key, std::nullopt)); + } } ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::next(size_t) diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 55b41cf6ca8..4ec94cfaf7c 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -43,7 +43,7 @@ StoragePtr TableFunctionObjectStorageCluster Date: Mon, 19 Feb 2024 20:29:22 +0100 Subject: [PATCH 007/158] Fix style check --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 43b1cffb3e6..6fec4758456 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -22,6 +22,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) From d88f8646b180f0ca4fec7bab5c9c9c7cc7574c0c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Feb 2024 11:03:12 +0100 Subject: [PATCH 008/158] Fix after merge with master --- src/Coordination/Standalone/Context.cpp | 15 +++++++++++++++ src/Coordination/Standalone/Context.h | 3 +++ src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.cpp | 2 +- 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index 374610769c4..c16ecbfd5c3 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -77,6 +77,8 @@ struct ContextSharedPart : boost::noncopyable mutable ThrottlerPtr local_read_throttler; /// A server-wide throttler for local IO reads mutable ThrottlerPtr local_write_throttler; /// A server-wide throttler for local IO writes + + std::optional storage_s3_settings TSA_GUARDED_BY(mutex); /// Settings of S3 storage }; ContextData::ContextData() = default; @@ -382,4 +384,17 @@ std::shared_ptr Context::getZooKeeper() const throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Cannot connect to ZooKeeper from Keeper"); } +const StorageS3Settings & Context::getStorageS3Settings() const +{ + std::lock_guard lock(shared->mutex); + + if (!shared->storage_s3_settings) + { + const auto & config = shared->config ? *shared->config : Poco::Util::Application::instance().config(); + shared->storage_s3_settings.emplace().loadFromConfig("s3", config, getSettingsRef()); + } + + return *shared->storage_s3_settings; +} + } diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 49ad2b568fe..3346a865f0f 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -36,6 +36,7 @@ class FilesystemCacheLog; class FilesystemReadPrefetchesLog; class BlobStorageLog; class IOUringReader; +class StorageS3Settings; /// A small class which owns ContextShared. /// We don't use something like unique_ptr directly to allow ContextShared type to be incomplete. @@ -160,6 +161,8 @@ public: void updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config); zkutil::ZooKeeperPtr getZooKeeper() const; + + const StorageS3Settings & getStorageS3Settings() const; }; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a75a747f334..0869e2ebbd2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -552,7 +552,7 @@ void S3ObjectStorage::applyNewSettings( static_headers.begin(), static_headers.end()); } - if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString())) + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); auto current_s3_settings = s3_settings.get(); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 896131e74d7..47e7ebd53a6 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -86,7 +86,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, headers_from_ast.begin(), headers_from_ast.end()); } - if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString())) + if (auto endpoint_settings = context->getStorageS3Settings().getSettings(url.uri.toString(), context->getUserName())) s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); auto client = getClient(config, config_prefix, context, *s3_settings, false, &url); From 94c44cefc89fbb471505aedd803600bc8ace7a49 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Feb 2024 16:24:23 +0100 Subject: [PATCH 009/158] Fix clang tidy --- src/Storages/ObjectStorage/AzureBlob/Configuration.cpp | 5 +---- src/Storages/ObjectStorage/HDFS/Configuration.cpp | 4 +--- src/Storages/ObjectStorage/S3/Configuration.cpp | 5 +---- .../ObjectStorage/StorageObjectStorageConfiguration.cpp | 7 +++++++ .../ObjectStorage/StorageObjectStorageConfiguration.h | 1 + 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 9d21541e7e2..7a670441e72 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -78,6 +78,7 @@ void StorageAzureBlobConfiguration::check(ContextPtr context) const } StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other) + : StorageObjectStorageConfiguration(other) { connection_url = other.connection_url; is_connection_string = other.is_connection_string; @@ -86,10 +87,6 @@ StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureB container = other.container; blob_path = other.blob_path; blobs_paths = other.blobs_paths; - - format = other.format; - compression_method = other.compression_method; - structure = other.structure; } AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(ContextPtr context) diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 731b05f4621..2f2427edb24 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -12,13 +12,11 @@ namespace DB { StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) + : StorageObjectStorageConfiguration(other) { url = other.url; path = other.path; paths = other.paths; - format = other.format; - compression_method = other.compression_method; - structure = other.structure; } void StorageHDFSConfiguration::check(ContextPtr context) const diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 47e7ebd53a6..1e14ccc4b31 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -57,15 +57,12 @@ void StorageS3Configuration::check(ContextPtr context) const } StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) + : StorageObjectStorageConfiguration(other) { url = other.url; static_configuration = other.static_configuration; headers_from_ast = other.headers_from_ast; keys = other.keys; - - format = other.format; - compression_method = other.compression_method; - structure = other.structure; } ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index a1c7d468fa6..8a4dee2c31b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -20,6 +20,13 @@ void StorageObjectStorageConfiguration::initialize( configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); } +StorageObjectStorageConfiguration::StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other) +{ + format = other.format; + compression_method = other.compression_method; + structure = other.structure; +} + bool StorageObjectStorageConfiguration::withWildcard() const { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 2da262eb55d..47afbc5d0c6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -12,6 +12,7 @@ class StorageObjectStorageConfiguration { public: StorageObjectStorageConfiguration() = default; + StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other); virtual ~StorageObjectStorageConfiguration() = default; using Path = std::string; From 4e3f2aae408fc8559304fe4f7c4a21db3d9202a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Feb 2024 18:47:17 +0100 Subject: [PATCH 010/158] Fix keeper build --- src/Coordination/Standalone/Context.cpp | 1 + src/Coordination/Standalone/Context.h | 2 ++ 2 files changed, 3 insertions(+) diff --git a/src/Coordination/Standalone/Context.cpp b/src/Coordination/Standalone/Context.cpp index c16ecbfd5c3..7e8711c7910 100644 --- a/src/Coordination/Standalone/Context.cpp +++ b/src/Coordination/Standalone/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include diff --git a/src/Coordination/Standalone/Context.h b/src/Coordination/Standalone/Context.h index 3346a865f0f..943fcd106df 100644 --- a/src/Coordination/Standalone/Context.h +++ b/src/Coordination/Standalone/Context.h @@ -163,6 +163,8 @@ public: zkutil::ZooKeeperPtr getZooKeeper() const; const StorageS3Settings & getStorageS3Settings() const; + + const String & getUserName() const { static std::string user; return user; } }; } From 80eb0c37826de63d9e2b595c62c37abbbb9c16ab Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 21 Feb 2024 20:47:25 +0100 Subject: [PATCH 011/158] Fix for hdfs --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 28 +++++++++++++------ src/Storages/HDFS/WriteBufferFromHDFS.cpp | 7 +++-- .../ObjectStorage/HDFS/Configuration.cpp | 14 +++++++--- .../ObjectStorage/ReadBufferIterator.cpp | 12 ++++---- 4 files changed, 39 insertions(+), 22 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index fa5e227d853..360403b7f2d 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -36,10 +36,10 @@ ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & bool HDFSObjectStorage::exists(const StoredObject & object) const { - const auto & path = object.remote_path; - const size_t begin_of_path = path.find('/', path.find("//") + 2); - const String remote_fs_object_path = path.substr(begin_of_path); - return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str())); + // const auto & path = object.remote_path; + // const size_t begin_of_path = path.find('/', path.find("//") + 2); + // const String remote_fs_object_path = path.substr(begin_of_path); + return (0 == hdfsExists(hdfs_fs.get(), object.remote_path.c_str())); } std::unique_ptr HDFSObjectStorage::readObject( /// NOLINT @@ -86,9 +86,12 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); + auto path = object.remote_path.starts_with('/') ? object.remote_path.substr(1) : object.remote_path; + path = fs::path(hdfs_root_path) / path; + /// Single O_WRONLY in libhdfs adds O_TRUNC return std::make_unique( - object.remote_path, config, settings->replication, patchSettings(write_settings), buf_size, + path, config, settings->replication, patchSettings(write_settings), buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); } @@ -124,11 +127,18 @@ void HDFSObjectStorage::removeObjectsIfExist(const StoredObjects & objects) removeObjectIfExists(object); } -ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const +ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) const { - throw Exception( - ErrorCodes::UNSUPPORTED_METHOD, - "HDFS API doesn't support custom attributes/metadata for stored objects"); + auto * file_info = hdfsGetPathInfo(hdfs_fs.get(), path.data()); + if (!file_info) + throw Exception(ErrorCodes::HDFS_ERROR, "Cannot get file info for: {}. Error: {}", path, hdfsGetLastError()); + + ObjectMetadata metadata; + metadata.size_bytes = static_cast(file_info->mSize); + metadata.last_modified = file_info->mLastMod; + + hdfsFreeFileInfo(file_info, 1); + return metadata; } void HDFSObjectStorage::copyObject( /// NOLINT diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 173dd899ada..9d383aa8245 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -48,12 +48,13 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); const String path = hdfs_uri.substr(begin_of_path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); if (fout == nullptr) { - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} error: {}", - path, std::string(hdfsGetLastError())); + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "Unable to open HDFS file: {} ({}) error: {}", + path, hdfs_uri, std::string(hdfsGetLastError())); } } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 2f2427edb24..a64faafd53d 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -22,13 +22,14 @@ StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguratio void StorageHDFSConfiguration::check(ContextPtr context) const { context->getRemoteHostFilter().checkURL(Poco::URI(url)); - checkHDFSURL(url); + checkHDFSURL(fs::path(url) / path); } ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { UNUSED(is_readonly); auto settings = std::make_unique(); + chassert(!url.empty()); return std::make_shared(url, std::move(settings), context->getConfigRef()); } @@ -36,15 +37,20 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr, bool /* with_str { url = checkAndGetLiteralArgument(args[0], "url"); - String format_name = "auto"; if (args.size() > 1) - format_name = checkAndGetLiteralArgument(args[1], "format_name"); + format = checkAndGetLiteralArgument(args[1], "format_name"); + else + format = "auto"; - String compression_method; if (args.size() == 3) compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); else compression_method = "auto"; + + const size_t begin_of_path = url.find('/', url.find("//") + 2); + path = url.substr(begin_of_path + 1); + url = url.substr(0, begin_of_path); + paths = {path}; } } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index a0e719878ac..dd4bfe79b06 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -195,19 +195,19 @@ ReadBufferIterator::Data ReadBufferIterator::next() throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "The table structure cannot be extracted from a {} format file, because there are no files with provided path " - "in S3 or all files are empty. You can specify table structure manually", - *format); + "in {} or all files are empty. You can specify table structure manually", + *format, object_storage->getName()); throw Exception( ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files, because there are no files with provided path " - "in S3 or all files are empty. You can specify the format manually"); + "in {} or all files are empty. You can specify the format manually", object_storage->getName()); } return {nullptr, std::nullopt, format}; } - /// S3 file iterator could get new keys after new iteration + /// file iterator could get new keys after new iteration if (read_keys.size() > prev_read_keys_size) { /// If format is unknown we can try to determine it by new file names. @@ -234,7 +234,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() prev_read_keys_size = read_keys.size(); } - if (getContext()->getSettingsRef().s3_skip_empty_files + if (query_settings.skip_empty_files && current_object_info->metadata && current_object_info->metadata->size_bytes == 0) continue; @@ -255,7 +255,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() {}, current_object_info->metadata->size_bytes); - if (!getContext()->getSettingsRef().s3_skip_empty_files || !read_buffer->eof()) + if (!query_settings.skip_empty_files || !read_buffer->eof()) { first = false; From f23ddec69f51481b8a7c3b923ae5e9dbb3891b41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Feb 2024 11:50:36 +0100 Subject: [PATCH 012/158] Fix unit tests build --- src/IO/tests/gtest_writebuffer_s3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_writebuffer_s3.cpp b/src/IO/tests/gtest_writebuffer_s3.cpp index ae00bb2e9e2..7856f22ab1a 100644 --- a/src/IO/tests/gtest_writebuffer_s3.cpp +++ b/src/IO/tests/gtest_writebuffer_s3.cpp @@ -546,7 +546,7 @@ public: std::unique_ptr getWriteBuffer(String file_name = "file") { S3Settings::RequestSettings request_settings; - request_settings.updateFromSettings(settings); + request_settings.updateFromSettingsIfChanged(settings); client->resetCounters(); From b548ed976d11309f8fb3b643ab71d9fd7d26ab31 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 22 Feb 2024 14:45:29 +0100 Subject: [PATCH 013/158] Fxi --- src/Storages/ObjectStorage/StorageObjectStorageCluster.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 5d77d4ced60..d7940851b00 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -54,7 +54,7 @@ private: const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override; - const String & engine_name; + const String engine_name; const Storage::ConfigurationPtr configuration; const ObjectStoragePtr object_storage; NamesAndTypesList virtual_columns; From 70272d41744d9cc219d79c6dd5e3b6c9e523d447 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Mar 2024 10:55:01 +0100 Subject: [PATCH 014/158] Minor --- src/CMakeLists.txt | 2 +- src/Databases/DatabaseHDFS.cpp | 2 +- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 6 +- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 2 +- .../ObjectStorages/ObjectStorageFactory.cpp | 2 +- src/IO/examples/read_buffer_from_hdfs.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 1200 ----------------- src/Storages/Hive/HiveCommon.h | 2 +- src/Storages/Hive/HiveFile.h | 2 +- src/Storages/Hive/StorageHive.cpp | 4 +- src/Storages/Hive/StorageHive.h | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.cpp | 2 +- .../HDFS/AsynchronousReadBufferFromHDFS.h | 2 +- .../ObjectStorage/HDFS/Configuration.cpp | 2 +- .../{ => ObjectStorage}/HDFS/HDFSCommon.cpp | 2 +- .../{ => ObjectStorage}/HDFS/HDFSCommon.h | 0 .../HDFS/ReadBufferFromHDFS.cpp | 2 +- .../HDFS/ReadBufferFromHDFS.h | 0 .../HDFS/WriteBufferFromHDFS.cpp | 4 +- .../HDFS/WriteBufferFromHDFS.h | 0 .../examples/async_read_buffer_from_hdfs.cpp | 2 +- 21 files changed, 21 insertions(+), 1221 deletions(-) delete mode 100644 src/Storages/HDFS/StorageHDFS.cpp rename src/Storages/{ => ObjectStorage}/HDFS/AsynchronousReadBufferFromHDFS.cpp (99%) rename src/Storages/{ => ObjectStorage}/HDFS/AsynchronousReadBufferFromHDFS.h (96%) rename src/Storages/{ => ObjectStorage}/HDFS/HDFSCommon.cpp (99%) rename src/Storages/{ => ObjectStorage}/HDFS/HDFSCommon.h (100%) rename src/Storages/{ => ObjectStorage}/HDFS/ReadBufferFromHDFS.cpp (99%) rename src/Storages/{ => ObjectStorage}/HDFS/ReadBufferFromHDFS.h (100%) rename src/Storages/{ => ObjectStorage}/HDFS/WriteBufferFromHDFS.cpp (97%) rename src/Storages/{ => ObjectStorage}/HDFS/WriteBufferFromHDFS.h (100%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1cf0e4e2b98..3cb64b56c46 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -150,7 +150,7 @@ if (TARGET ch_contrib::azure_sdk) endif() if (TARGET ch_contrib::hdfs) - add_headers_and_sources(dbms Storages/HDFS) + add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Disks/ObjectStorages/HDFS) endif() diff --git a/src/Databases/DatabaseHDFS.cpp b/src/Databases/DatabaseHDFS.cpp index 3a1e6b16ccf..cda38a69c9a 100644 --- a/src/Databases/DatabaseHDFS.cpp +++ b/src/Databases/DatabaseHDFS.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 8bff687b915..2d03de60c3c 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -1,10 +1,10 @@ #include #include -#include -#include +#include +#include -#include +#include #include #include diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 66095eb9f8f..4072d21ed7c 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 02b6816d673..d1841c92a6b 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -7,7 +7,7 @@ #endif #if USE_HDFS && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) #include -#include +#include #endif #if USE_AZURE_BLOB_STORAGE && !defined(CLICKHOUSE_KEEPER_STANDALONE_BUILD) #include diff --git a/src/IO/examples/read_buffer_from_hdfs.cpp b/src/IO/examples/read_buffer_from_hdfs.cpp index 977dd2ae227..a5cf43b3e79 100644 --- a/src/IO/examples/read_buffer_from_hdfs.cpp +++ b/src/IO/examples/read_buffer_from_hdfs.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp deleted file mode 100644 index cd935fa3100..00000000000 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ /dev/null @@ -1,1200 +0,0 @@ -#include "config.h" - -#if USE_HDFS - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include - -#include - -namespace fs = std::filesystem; - -namespace ProfileEvents -{ - extern const Event EngineFileLikeReadFiles; -} - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ACCESS_DENIED; - extern const int DATABASE_ACCESS_DENIED; - extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int CANNOT_COMPILE_REGEXP; - extern const int CANNOT_DETECT_FORMAT; -} -namespace -{ - struct HDFSFileInfoDeleter - { - /// Can have only one entry (see hdfsGetPathInfo()) - void operator()(hdfsFileInfo * info) { hdfsFreeFileInfo(info, 1); } - }; - using HDFSFileInfoPtr = std::unique_ptr; - - /* Recursive directory listing with matched paths as a result. - * Have the same method in StorageFile. - */ - std::vector LSWithRegexpMatching( - const String & path_for_ls, - const HDFSFSPtr & fs, - const String & for_match) - { - std::vector result; - - const size_t first_glob_pos = for_match.find_first_of("*?{"); - - if (first_glob_pos == std::string::npos) - { - const String path = fs::path(path_for_ls + for_match.substr(1)).lexically_normal(); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path.c_str())); - if (hdfs_info) // NOLINT - { - result.push_back(StorageHDFS::PathWithInfo{ - String(path), - StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}}); - } - return result; - } - - const size_t end_of_path_without_globs = for_match.substr(0, first_glob_pos).rfind('/'); - const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - - const size_t next_slash_after_glob_pos = suffix_with_globs.find('/', 1); - - const std::string current_glob = suffix_with_globs.substr(0, next_slash_after_glob_pos); - - re2::RE2 matcher(makeRegexpPatternFromGlobs(current_glob)); - if (!matcher.ok()) - throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, - "Cannot compile regex from glob ({}): {}", for_match, matcher.error()); - - HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); - if (ls.file_info == nullptr && errno != ENOENT) // NOLINT - { - // ignore file not found exception, keep throw other exception, libhdfs3 doesn't have function to get exception type, so use errno. - throw Exception( - ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", prefix_without_globs, String(hdfsGetLastError())); - } - - if (!ls.file_info && ls.length > 0) - throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); - for (int i = 0; i < ls.length; ++i) - { - const String full_path = fs::path(ls.file_info[i].mName).lexically_normal(); - const size_t last_slash = full_path.rfind('/'); - const String file_name = full_path.substr(last_slash); - const bool looking_for_directory = next_slash_after_glob_pos != std::string::npos; - const bool is_directory = ls.file_info[i].mKind == 'D'; - /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if (!is_directory && !looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - result.push_back(StorageHDFS::PathWithInfo{ - String(full_path), - StorageHDFS::PathInfo{ls.file_info[i].mLastMod, static_cast(ls.file_info[i].mSize)}}); - } - else if (is_directory && looking_for_directory) - { - if (re2::RE2::FullMatch(file_name, matcher)) - { - std::vector result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, - suffix_with_globs.substr(next_slash_after_glob_pos)); - /// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check. - std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); - } - } - } - - return result; - } - - std::pair getPathFromUriAndUriWithoutPath(const String & uri) - { - auto pos = uri.find("//"); - if (pos != std::string::npos && pos + 2 < uri.length()) - { - pos = uri.find('/', pos + 2); - if (pos != std::string::npos) - return {uri.substr(pos), uri.substr(0, pos)}; - } - - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage HDFS requires valid URL to be set"); - } - - std::vector getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context) - { - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - Strings paths = expandSelectionGlob(path_from_uri); - - std::vector res; - - for (const auto & path : paths) - { - auto part_of_res = LSWithRegexpMatching("/", fs, path); - res.insert(res.end(), part_of_res.begin(), part_of_res.end()); - } - return res; - } -} - -StorageHDFS::StorageHDFS( - const String & uri_, - const StorageID & table_id_, - const String & format_name_, - const ColumnsDescription & columns_, - const ConstraintsDescription & constraints_, - const String & comment, - const ContextPtr & context_, - const String & compression_method_, - const bool distributed_processing_, - ASTPtr partition_by_) - : IStorage(table_id_) - , WithContext(context_) - , uris({uri_}) - , format_name(format_name_) - , compression_method(compression_method_) - , distributed_processing(distributed_processing_) - , partition_by(partition_by_) -{ - if (format_name != "auto") - FormatFactory::instance().checkFormatName(format_name); - context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); - checkHDFSURL(uri_); - - String path = uri_.substr(uri_.find('/', uri_.find("//") + 2)); - is_path_with_globs = path.find_first_of("*?{") != std::string::npos; - - StorageInMemoryMetadata storage_metadata; - - if (columns_.empty()) - { - ColumnsDescription columns; - if (format_name == "auto") - std::tie(columns, format_name) = getTableStructureAndFormatFromData(uri_, compression_method_, context_); - else - columns = getTableStructureFromData(format_name, uri_, compression_method, context_); - - storage_metadata.setColumns(columns); - } - else - { - if (format_name == "auto") - format_name = getTableStructureAndFormatFromData(uri_, compression_method_, context_).second; - - /// We don't allow special columns in HDFS storage. - if (!columns_.hasOnlyOrdinary()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine HDFS doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL"); - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); - - virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList()); -} - -namespace -{ - class ReadBufferIterator : public IReadBufferIterator, WithContext - { - public: - ReadBufferIterator( - const std::vector & paths_with_info_, - const String & uri_without_path_, - std::optional format_, - const String & compression_method_, - const ContextPtr & context_) - : WithContext(context_) - , paths_with_info(paths_with_info_) - , uri_without_path(uri_without_path_) - , format(std::move(format_)) - , compression_method(compression_method_) - { - } - - Data next() override - { - bool is_first = current_index == 0; - /// For default mode check cached columns for all paths on first iteration. - if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) - { - if (auto cached_columns = tryGetColumnsFromCache(paths_with_info)) - return {nullptr, cached_columns, format}; - } - - StorageHDFS::PathWithInfo path_with_info; - - while (true) - { - if (current_index == paths_with_info.size()) - { - if (is_first) - { - if (format) - throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because all files are empty. " - "You can specify table structure manually", *format); - - throw Exception( - ErrorCodes::CANNOT_DETECT_FORMAT, - "The data format cannot be detected by the contents of the files, because all files are empty. You can specify table structure manually"); - } - return {nullptr, std::nullopt, format}; - } - - path_with_info = paths_with_info[current_index++]; - if (getContext()->getSettingsRef().hdfs_skip_empty_files && path_with_info.info && path_with_info.info->size == 0) - continue; - - if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION) - { - std::vector paths = {path_with_info}; - if (auto cached_columns = tryGetColumnsFromCache(paths)) - return {nullptr, cached_columns, format}; - } - - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - if (!getContext()->getSettingsRef().hdfs_skip_empty_files || !impl->eof()) - { - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)), std::nullopt, format}; - } - } - } - - void setNumRowsToLastFile(size_t num_rows) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(key, num_rows); - } - - void setSchemaToLastFile(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION) - return; - - String source = uri_without_path + paths_with_info[current_index - 1].path; - auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addColumns(key, columns); - } - - void setResultingSchema(const ColumnsDescription & columns) override - { - if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs - || getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::DEFAULT) - return; - - Strings sources; - sources.reserve(paths_with_info.size()); - std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const StorageHDFS::PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; }); - auto cache_keys = getKeysForSchemaCache(sources, *format, {}, getContext()); - StorageHDFS::getSchemaCache(getContext()).addManyColumns(cache_keys, columns); - } - - void setFormatName(const String & format_name) override - { - format = format_name; - } - - String getLastFileName() const override - { - if (current_index != 0) - return paths_with_info[current_index - 1].path; - - return ""; - } - - bool supportsLastReadBufferRecreation() const override { return true; } - - std::unique_ptr recreateLastReadBuffer() override - { - chassert(current_index > 0 && current_index <= paths_with_info.size()); - auto path_with_info = paths_with_info[current_index - 1]; - auto compression = chooseCompressionMethod(path_with_info.path, compression_method); - auto impl = std::make_unique(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); - } - - private: - std::optional tryGetColumnsFromCache(const std::vector & paths_with_info_) - { - auto context = getContext(); - - if (!context->getSettingsRef().schema_inference_use_cache_for_hdfs) - return std::nullopt; - - auto & schema_cache = StorageHDFS::getSchemaCache(context); - for (const auto & path_with_info : paths_with_info_) - { - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - - auto builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str())); - if (hdfs_info) - return hdfs_info->mLastMod; - - return std::nullopt; - }; - - String url = uri_without_path + path_with_info.path; - if (format) - { - auto cache_key = getKeyForSchemaCache(url, *format, {}, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - return columns; - } - else - { - /// If format is unknown, we can iterate through all possible input formats - /// and check if we have an entry with this format and this file in schema cache. - /// If we have such entry for some format, we can use this format to read the file. - for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) - { - auto cache_key = getKeyForSchemaCache(url, format_name, {}, context); - if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) - { - /// Now format is known. It should be the same for all files. - format = format_name; - return columns; - } - } - } - } - - return std::nullopt; - } - - const std::vector & paths_with_info; - const String & uri_without_path; - std::optional format; - const String & compression_method; - size_t current_index = 0; - }; -} - -std::pair StorageHDFS::getTableStructureAndFormatFromDataImpl( - std::optional format, - const String & uri, - const String & compression_method, - const ContextPtr & ctx) -{ - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - auto paths_with_info = getPathsList(path_from_uri, uri, ctx); - - if (paths_with_info.empty() && (!format || !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(*format))) - { - if (format) - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because there are no files in HDFS with provided path." - " You can specify table structure manually", *format); - - throw Exception( - ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The data format cannot be detected by the contents of the files, because there are no files in HDFS with provided path." - " You can specify the format manually"); - } - - ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx); - if (format) - return {readSchemaFromFormat(*format, std::nullopt, read_buffer_iterator, ctx), *format}; - return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, ctx); -} - -std::pair StorageHDFS::getTableStructureAndFormatFromData(const String & uri, const String & compression_method, const ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(std::nullopt, uri, compression_method, ctx); -} - -ColumnsDescription StorageHDFS::getTableStructureFromData(const String & format, const String & uri, const String & compression_method, const DB::ContextPtr & ctx) -{ - return getTableStructureAndFormatFromDataImpl(format, uri, compression_method, ctx).first; -} - -class HDFSSource::DisclosedGlobIterator::Impl -{ -public: - Impl(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - { - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri); - uris = getPathsList(path_from_uri, uri_without_path, context); - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & path_with_info : uris) - paths.push_back(path_with_info.path); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, context); - } - auto file_progress_callback = context->getFileProgressCallback(); - - for (auto & elem : uris) - { - elem.path = uri_without_path + elem.path; - if (file_progress_callback && elem.info) - file_progress_callback(FileProgress(0, elem.info->size)); - } - uris_iter = uris.begin(); - } - - StorageHDFS::PathWithInfo next() - { - std::lock_guard lock(mutex); - if (uris_iter != uris.end()) - { - auto answer = *uris_iter; - ++uris_iter; - return answer; - } - return {}; - } -private: - std::mutex mutex; - std::vector uris; - std::vector::iterator uris_iter; -}; - -class HDFSSource::URISIterator::Impl : WithContext -{ -public: - explicit Impl(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context_) - : WithContext(context_), uris(uris_), file_progress_callback(context_->getFileProgressCallback()) - { - ActionsDAGPtr filter_dag; - if (!uris.empty()) - filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); - - if (filter_dag) - { - std::vector paths; - paths.reserve(uris.size()); - for (const auto & uri : uris) - paths.push_back(getPathFromUriAndUriWithoutPath(uri).first); - - VirtualColumnUtils::filterByPathOrFile(uris, paths, filter_dag, virtual_columns, getContext()); - } - - if (!uris.empty()) - { - auto path_and_uri = getPathFromUriAndUriWithoutPath(uris[0]); - builder = createHDFSBuilder(path_and_uri.second + "/", getContext()->getGlobalContext()->getConfigRef()); - fs = createHDFSFS(builder.get()); - } - } - - StorageHDFS::PathWithInfo next() - { - String uri; - HDFSFileInfoPtr hdfs_info; - do - { - size_t current_index = index.fetch_add(1); - if (current_index >= uris.size()) - return {"", {}}; - - uri = uris[current_index]; - auto path_and_uri = getPathFromUriAndUriWithoutPath(uri); - hdfs_info.reset(hdfsGetPathInfo(fs.get(), path_and_uri.first.c_str())); - } - /// Skip non-existed files. - while (!hdfs_info && String(hdfsGetLastError()).find("FileNotFoundException") != std::string::npos); - - std::optional info; - if (hdfs_info) - { - info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - if (file_progress_callback) - file_progress_callback(FileProgress(0, hdfs_info->mSize)); - } - - return {uri, info}; - } - -private: - std::atomic_size_t index = 0; - Strings uris; - HDFSBuilderWrapper builder; - HDFSFSPtr fs; - std::function file_progress_callback; -}; - -HDFSSource::DisclosedGlobIterator::DisclosedGlobIterator(const String & uri, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uri, predicate, virtual_columns, context)) {} - -StorageHDFS::PathWithInfo HDFSSource::DisclosedGlobIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::URISIterator::URISIterator(const std::vector & uris_, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, const ContextPtr & context) - : pimpl(std::make_shared(uris_, predicate, virtual_columns, context)) -{ -} - -StorageHDFS::PathWithInfo HDFSSource::URISIterator::next() -{ - return pimpl->next(); -} - -HDFSSource::HDFSSource( - const ReadFromFormatInfo & info, - StorageHDFSPtr storage_, - const ContextPtr & context_, - UInt64 max_block_size_, - std::shared_ptr file_iterator_, - bool need_only_count_) - : ISource(info.source_header, false) - , WithContext(context_) - , storage(std::move(storage_)) - , block_for_format(info.format_header) - , requested_columns(info.requested_columns) - , requested_virtual_columns(info.requested_virtual_columns) - , max_block_size(max_block_size_) - , file_iterator(file_iterator_) - , columns_description(info.columns_description) - , need_only_count(need_only_count_) -{ - initialize(); -} - -bool HDFSSource::initialize() -{ - bool skip_empty_files = getContext()->getSettingsRef().hdfs_skip_empty_files; - StorageHDFS::PathWithInfo path_with_info; - while (true) - { - path_with_info = (*file_iterator)(); - if (path_with_info.path.empty()) - return false; - - if (path_with_info.info && skip_empty_files && path_with_info.info->size == 0) - continue; - - current_path = path_with_info.path; - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); - - std::optional file_size; - if (!path_with_info.info) - { - auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_from_uri.c_str())); - if (hdfs_info) - path_with_info.info = StorageHDFS::PathInfo{hdfs_info->mLastMod, static_cast(hdfs_info->mSize)}; - } - - if (path_with_info.info) - file_size = path_with_info.info->size; - - auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); - auto impl = std::make_unique( - uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings(), 0, false, file_size); - if (!skip_empty_files || !impl->eof()) - { - impl->setProgressCallback(getContext()); - const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max; - read_buf = wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast(zstd_window_log_max)); - break; - } - } - - current_path = path_with_info.path; - current_file_size = path_with_info.info ? std::optional(path_with_info.info->size) : std::nullopt; - - QueryPipelineBuilder builder; - std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files ? tryGetNumRowsFromCache(path_with_info) : std::nullopt; - if (num_rows_from_cache) - { - /// We should not return single chunk with all number of rows, - /// because there is a chance that this chunk will be materialized later - /// (it can cause memory problems even with default values in columns or when virtual columns are requested). - /// Instead, we use a special ConstChunkGenerator that will generate chunks - /// with max_block_size rows until total number of rows is reached. - auto source = std::make_shared(block_for_format, *num_rows_from_cache, max_block_size); - builder.init(Pipe(source)); - } - else - { - std::optional max_parsing_threads; - if (need_only_count) - max_parsing_threads = 1; - - input_format = getContext()->getInputFormat(storage->format_name, *read_buf, block_for_format, max_block_size, std::nullopt, max_parsing_threads); - - if (need_only_count) - input_format->needOnlyCount(); - - builder.init(Pipe(input_format)); - if (columns_description.hasDefaults()) - { - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, columns_description, *input_format, getContext()); - }); - } - } - - /// Add ExtractColumnsTransform to extract requested columns/subcolumns - /// from the chunk read by IInputFormat. - builder.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, requested_columns); - }); - - pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); - reader = std::make_unique(*pipeline); - - ProfileEvents::increment(ProfileEvents::EngineFileLikeReadFiles); - return true; -} - -String HDFSSource::getName() const -{ - return "HDFSSource"; -} - -Chunk HDFSSource::generate() -{ - while (true) - { - if (isCancelled() || !reader) - { - if (reader) - reader->cancel(); - break; - } - - Chunk chunk; - if (reader->pull(chunk)) - { - UInt64 num_rows = chunk.getNumRows(); - total_rows_in_file += num_rows; - size_t chunk_size = 0; - if (input_format) - chunk_size = input_format->getApproxBytesReadForChunk(); - progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); - VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk(chunk, requested_virtual_columns, current_path, current_file_size); - return chunk; - } - - if (input_format && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(current_path, total_rows_in_file); - - total_rows_in_file = 0; - - reader.reset(); - pipeline.reset(); - input_format.reset(); - read_buf.reset(); - - if (!initialize()) - break; - } - return {}; -} - -void HDFSSource::addNumRowsToCache(const String & path, size_t num_rows) -{ - auto cache_key = getKeyForSchemaCache(path, storage->format_name, std::nullopt, getContext()); - StorageHDFS::getSchemaCache(getContext()).addNumRows(cache_key, num_rows); -} - -std::optional HDFSSource::tryGetNumRowsFromCache(const StorageHDFS::PathWithInfo & path_with_info) -{ - auto cache_key = getKeyForSchemaCache(path_with_info.path, storage->format_name, std::nullopt, getContext()); - auto get_last_mod_time = [&]() -> std::optional - { - if (path_with_info.info) - return path_with_info.info->last_mod_time; - return std::nullopt; - }; - - return StorageHDFS::getSchemaCache(getContext()).tryGetNumRows(cache_key, get_last_mod_time); -} - -class HDFSSink : public SinkToStorage -{ -public: - HDFSSink(const String & uri, - const String & format, - const Block & sample_block, - const ContextPtr & context, - const CompressionMethod compression_method) - : SinkToStorage(sample_block) - { - const auto & settings = context->getSettingsRef(); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique( - uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication, context->getWriteSettings()), - compression_method, - static_cast(settings.output_format_compression_level), - static_cast(settings.output_format_compression_zstd_window_log)); - writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); - } - - String getName() const override { return "HDFSSink"; } - - void consume(Chunk chunk) override - { - std::lock_guard lock(cancel_mutex); - if (cancelled) - return; - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); - } - - void onCancel() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - cancelled = true; - } - - void onException(std::exception_ptr exception) override - { - std::lock_guard lock(cancel_mutex); - try - { - std::rethrow_exception(exception); - } - catch (...) - { - /// An exception context is needed to proper delete write buffers without finalization - release(); - } - } - - void onFinish() override - { - std::lock_guard lock(cancel_mutex); - finalize(); - } - -private: - void finalize() - { - if (!writer) - return; - - try - { - writer->finalize(); - writer->flush(); - write_buf->sync(); - write_buf->finalize(); - } - catch (...) - { - /// Stop ParallelFormattingOutputFormat correctly. - release(); - throw; - } - } - - void release() - { - writer.reset(); - write_buf->finalize(); - } - - std::unique_ptr write_buf; - OutputFormatPtr writer; - std::mutex cancel_mutex; - bool cancelled = false; -}; - -class PartitionedHDFSSink : public PartitionedSink -{ -public: - PartitionedHDFSSink( - const ASTPtr & partition_by, - const String & uri_, - const String & format_, - const Block & sample_block_, - ContextPtr context_, - const CompressionMethod compression_method_) - : PartitionedSink(partition_by, context_, sample_block_) - , uri(uri_) - , format(format_) - , sample_block(sample_block_) - , context(context_) - , compression_method(compression_method_) - { - } - - SinkPtr createSinkForPartition(const String & partition_id) override - { - auto path = PartitionedSink::replaceWildcards(uri, partition_id); - PartitionedSink::validatePartitionKey(path, true); - return std::make_shared(path, format, sample_block, context, compression_method); - } - -private: - const String uri; - const String format; - const Block sample_block; - ContextPtr context; - const CompressionMethod compression_method; -}; - - -bool StorageHDFS::supportsSubsetOfColumns(const ContextPtr & context_) const -{ - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context_); -} - -class ReadFromHDFS : public SourceStepWithFilter -{ -public: - std::string getName() const override { return "ReadFromHDFS"; } - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - void applyFilters(ActionDAGNodes added_filter_nodes) override; - - ReadFromHDFS( - const Names & column_names_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const ContextPtr & context_, - Block sample_block, - ReadFromFormatInfo info_, - bool need_only_count_, - std::shared_ptr storage_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter( - DataStream{.header = std::move(sample_block)}, - column_names_, - query_info_, - storage_snapshot_, - context_) - , info(std::move(info_)) - , need_only_count(need_only_count_) - , storage(std::move(storage_)) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - { - } - -private: - ReadFromFormatInfo info; - const bool need_only_count; - std::shared_ptr storage; - - size_t max_block_size; - size_t num_streams; - - std::shared_ptr iterator_wrapper; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -void ReadFromHDFS::applyFilters(ActionDAGNodes added_filter_nodes) -{ - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void StorageHDFS::read( - 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) -{ - auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(context_), virtual_columns); - bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) - && context_->getSettingsRef().optimize_count_from_files; - - auto this_ptr = std::static_pointer_cast(shared_from_this()); - - auto reading = std::make_unique( - column_names, - query_info, - storage_snapshot, - context_, - read_from_format_info.source_header, - std::move(read_from_format_info), - need_only_count, - std::move(this_ptr), - max_block_size, - num_streams); - - query_plan.addStep(std::move(reading)); -} - -void ReadFromHDFS::createIterator(const ActionsDAG::Node * predicate) -{ - if (iterator_wrapper) - return; - - if (storage->distributed_processing) - { - iterator_wrapper = std::make_shared( - [callback = context->getReadTaskCallback()]() -> StorageHDFS::PathWithInfo { - return StorageHDFS::PathWithInfo{callback(), std::nullopt}; - }); - } - else if (storage->is_path_with_globs) - { - /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(storage->uris[0], predicate, storage->virtual_columns, context); - iterator_wrapper = std::make_shared([glob_iterator]() - { - return glob_iterator->next(); - }); - } - else - { - auto uris_iterator = std::make_shared(storage->uris, predicate, storage->virtual_columns, context); - iterator_wrapper = std::make_shared([uris_iterator]() - { - return uris_iterator->next(); - }); - } -} - -void ReadFromHDFS::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - - Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) - { - pipes.emplace_back(std::make_shared( - info, - storage, - context, - max_block_size, - iterator_wrapper, - need_only_count)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_, bool /*async_insert*/) -{ - String current_uri = uris.back(); - - bool has_wildcards = current_uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; - const auto * insert_query = dynamic_cast(query.get()); - auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; - bool is_partitioned_implementation = partition_by_ast && has_wildcards; - - if (is_partitioned_implementation) - { - return std::make_shared( - partition_by_ast, - current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } - else - { - if (is_path_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); - - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_uri); - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - bool truncate_on_insert = context_->getSettingsRef().hdfs_truncate_on_insert; - if (!truncate_on_insert && !hdfsExists(fs.get(), path_from_uri.c_str())) - { - if (context_->getSettingsRef().hdfs_create_new_file_on_insert) - { - auto pos = uris[0].find_first_of('.', uris[0].find_last_of('/')); - size_t index = uris.size(); - String new_uri; - do - { - new_uri = uris[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : uris[0].substr(pos)); - ++index; - } - while (!hdfsExists(fs.get(), new_uri.c_str())); - uris.push_back(new_uri); - current_uri = new_uri; - } - else - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " - "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", - path_from_uri); - } - - return std::make_shared(current_uri, - format_name, - metadata_snapshot->getSampleBlock(), - context_, - chooseCompressionMethod(current_uri, compression_method)); - } -} - -void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) -{ - const size_t begin_of_path = uris[0].find('/', uris[0].find("//") + 2); - const String url = uris[0].substr(0, begin_of_path); - - HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", local_context->getGlobalContext()->getConfigRef()); - auto fs = createHDFSFS(builder.get()); - - for (const auto & uri : uris) - { - const String path = uri.substr(begin_of_path); - int ret = hdfsDelete(fs.get(), path.data(), 0); - if (ret) - throw Exception(ErrorCodes::ACCESS_DENIED, "Unable to truncate hdfs table: {}", std::string(hdfsGetLastError())); - } -} - - -void registerStorageHDFS(StorageFactory & factory) -{ - factory.registerStorage("HDFS", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - - if (engine_args.empty() || engine_args.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Storage HDFS requires 1, 2 or 3 arguments: " - "url, name of used format (taken from file extension by default) and optional compression method."); - - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.getLocalContext()); - - String url = checkAndGetLiteralArgument(engine_args[0], "url"); - - String format_name = "auto"; - if (engine_args.size() > 1) - { - engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.getLocalContext()); - format_name = checkAndGetLiteralArgument(engine_args[1], "format_name"); - } - - if (format_name == "auto") - format_name = FormatFactory::instance().tryGetFormatFromFileName(url).value_or("auto"); - - String compression_method; - if (engine_args.size() == 3) - { - engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.getLocalContext()); - compression_method = checkAndGetLiteralArgument(engine_args[2], "compression_method"); - } else compression_method = "auto"; - - ASTPtr partition_by; - if (args.storage_def->partition_by) - partition_by = args.storage_def->partition_by->clone(); - - return std::make_shared( - url, args.table_id, format_name, args.columns, args.constraints, args.comment, args.getContext(), compression_method, false, partition_by); - }, - { - .supports_sort_order = true, // for partition by - .supports_schema_inference = true, - .source_access_type = AccessType::HDFS, - }); -} - -NamesAndTypesList StorageHDFS::getVirtuals() const -{ - return virtual_columns; -} - -Names StorageHDFS::getVirtualColumnNames() -{ - return VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage({}).getNames(); -} - -SchemaCache & StorageHDFS::getSchemaCache(const ContextPtr & ctx) -{ - static SchemaCache schema_cache(ctx->getConfigRef().getUInt("schema_inference_cache_max_elements_for_hdfs", DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - -} - -#endif diff --git a/src/Storages/Hive/HiveCommon.h b/src/Storages/Hive/HiveCommon.h index 0f9d3364ffd..81c167165d3 100644 --- a/src/Storages/Hive/HiveCommon.h +++ b/src/Storages/Hive/HiveCommon.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include diff --git a/src/Storages/Hive/HiveFile.h b/src/Storages/Hive/HiveFile.h index 1f5e31f1d54..affb72fe09b 100644 --- a/src/Storages/Hive/HiveFile.h +++ b/src/Storages/Hive/HiveFile.h @@ -14,7 +14,7 @@ #include #include #include -#include +#include namespace orc { diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index 183a4532281..a76cef2d45d 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -38,8 +38,8 @@ #include #include #include -#include -#include +#include +#include #include #include #include diff --git a/src/Storages/Hive/StorageHive.h b/src/Storages/Hive/StorageHive.h index 07440097f7a..43a22a886a8 100644 --- a/src/Storages/Hive/StorageHive.h +++ b/src/Storages/Hive/StorageHive.h @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp similarity index 99% rename from src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp index 6b6151f5474..21df7e35284 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.cpp @@ -1,9 +1,9 @@ #include "AsynchronousReadBufferFromHDFS.h" #if USE_HDFS +#include "ReadBufferFromHDFS.h" #include #include -#include #include #include diff --git a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h similarity index 96% rename from src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h index 10e2749fd4a..5aef92315a4 100644 --- a/src/Storages/HDFS/AsynchronousReadBufferFromHDFS.h +++ b/src/Storages/ObjectStorage/HDFS/AsynchronousReadBufferFromHDFS.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include namespace DB diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index a64faafd53d..6c7fe1cef7e 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -1,7 +1,7 @@ #include #if USE_HDFS -#include +#include #include #include #include diff --git a/src/Storages/HDFS/HDFSCommon.cpp b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp similarity index 99% rename from src/Storages/HDFS/HDFSCommon.cpp rename to src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp index f9a55a1285a..5d14cec14bd 100644 --- a/src/Storages/HDFS/HDFSCommon.cpp +++ b/src/Storages/ObjectStorage/HDFS/HDFSCommon.cpp @@ -1,4 +1,4 @@ -#include +#include "HDFSCommon.h" #include #include #include diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/ObjectStorage/HDFS/HDFSCommon.h similarity index 100% rename from src/Storages/HDFS/HDFSCommon.h rename to src/Storages/ObjectStorage/HDFS/HDFSCommon.h diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp similarity index 99% rename from src/Storages/HDFS/ReadBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index 4df05d47003..18b22805dfc 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -1,7 +1,7 @@ #include "ReadBufferFromHDFS.h" #if USE_HDFS -#include +#include "HDFSCommon.h" #include #include #include diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h similarity index 100% rename from src/Storages/HDFS/ReadBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp similarity index 97% rename from src/Storages/HDFS/WriteBufferFromHDFS.cpp rename to src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 9d383aa8245..2c14b38ce01 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -2,8 +2,8 @@ #if USE_HDFS -#include -#include +#include "WriteBufferFromHDFS.h" +#include "HDFSCommon.h" #include #include #include diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h similarity index 100% rename from src/Storages/HDFS/WriteBufferFromHDFS.h rename to src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h diff --git a/src/Storages/examples/async_read_buffer_from_hdfs.cpp b/src/Storages/examples/async_read_buffer_from_hdfs.cpp index 4f6aed8ef65..1c47a07ba58 100644 --- a/src/Storages/examples/async_read_buffer_from_hdfs.cpp +++ b/src/Storages/examples/async_read_buffer_from_hdfs.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include int main() { From e019b3a391bb8e3bbfa991e083e65e76438a2a9e Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Mar 2024 16:12:39 +0100 Subject: [PATCH 015/158] Fix build after merge --- src/Backups/BackupIO_AzureBlobStorage.cpp | 2 +- .../IO/WriteBufferFromAzureBlobStorage.h | 2 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 12 +-- .../ObjectStorage/AzureBlob/Configuration.cpp | 73 +++++++++++-------- .../ObjectStorage/AzureBlob/Configuration.h | 2 +- .../DataLakes/DeltaLakeMetadata.cpp | 1 + .../ObjectStorage/HDFS/Configuration.cpp | 8 +- .../ObjectStorage/S3/Configuration.cpp | 2 + .../StorageObjectStorageConfiguration.cpp | 10 +++ .../StorageObjectStorageConfiguration.h | 4 + src/Storages/S3Queue/S3QueueTableMetadata.cpp | 1 - src/Storages/S3Queue/S3QueueTableMetadata.h | 1 + 12 files changed, 77 insertions(+), 41 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index dc4a825189f..8a3ff1c3b5e 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -36,7 +36,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = configuration.createClient(/* is_read_only */ false); + auto client_ptr = configuration.createClient(/* is_read_only */ false, /* attempt_to_create_container */true); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), configuration.createSettings(context_), diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h index 6e10c07b255..dbf0b2a3052 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.h +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include namespace Poco { diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 13d4c2a551b..872f7eec07b 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -69,10 +69,6 @@ std::unique_ptr getClient( { String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - if (S3::isS3ExpressEndpoint(endpoint) && !config.has(config_prefix + ".region")) - throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets ({})", config_prefix); - url = S3::URI(endpoint); if (!url.key.ends_with('/')) url.key.push_back('/'); @@ -83,6 +79,12 @@ std::unique_ptr getClient( throw Exception(ErrorCodes::LOGICAL_ERROR, "URL not passed"); url = *url_; } + const bool is_s3_express_bucket = S3::isS3ExpressEndpoint(url.endpoint); + if (is_s3_express_bucket && !config.has(config_prefix + ".region")) + { + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Region should be explicitly specified for directory buckets ({})", config_prefix); + } S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( auth_settings.region, @@ -130,7 +132,7 @@ std::unique_ptr getClient( .use_virtual_addressing = url.is_virtual_hosted_style, .disable_checksum = local_settings.s3_disable_checksum, .gcs_issue_compose_request = config.getBool("s3.gcs_issue_compose_request", false), - .is_s3express_bucket = S3::isS3ExpressEndpoint(endpoint), + .is_s3express_bucket = is_s3_express_bucket, }; auto credentials_configuration = S3::CredentialsConfiguration diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 7a670441e72..018cec51e7c 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -47,7 +48,8 @@ namespace return !candidate.starts_with("http"); } - bool containerExists(Azure::Storage::Blobs::BlobServiceClient & blob_service_client, std::string container_name) + template + bool containerExists(T & blob_service_client, const std::string & container_name) { Azure::Storage::Blobs::ListBlobContainersOptions options; options.Prefix = container_name; @@ -101,12 +103,13 @@ AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(Co ObjectStoragePtr StorageAzureBlobConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { - auto client = createClient(is_readonly); + assertInitialized(); + auto client = createClient(is_readonly, /* attempt_to_create_container */true); auto settings = createSettings(context); return std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), container); } -AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) +AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only, bool attempt_to_create_container) { using namespace Azure::Storage::Blobs; @@ -114,28 +117,32 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) if (is_connection_string) { - auto blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); + std::shared_ptr managed_identity_credential = std::make_shared(); + std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_url, container)); - bool container_exists = containerExists(*blob_service_client, container); - if (!container_exists) + if (attempt_to_create_container) { - if (is_read_only) - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "AzureBlobStorage container does not exist '{}'", - container); + bool container_exists = containerExists(*blob_service_client, container); + if (!container_exists) + { + if (is_read_only) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "AzureBlobStorage container does not exist '{}'", + container); - try - { - result->CreateIfNotExists(); - } - catch (const Azure::Storage::StorageException & e) - { - if (e.StatusCode != Azure::Core::Http::HttpStatusCode::Conflict - || e.ReasonPhrase != "The specified container already exists.") + try { - throw; + result->CreateIfNotExists(); + } + catch (const Azure::Storage::StorageException & e) + { + if (!(e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict + && e.ReasonPhrase == "The specified container already exists.")) + { + throw; + } } } } @@ -145,22 +152,22 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) std::shared_ptr storage_shared_key_credential; if (account_name.has_value() && account_key.has_value()) { - storage_shared_key_credential = - std::make_shared(*account_name, *account_key); + storage_shared_key_credential + = std::make_shared(*account_name, *account_key); } std::unique_ptr blob_service_client; + std::shared_ptr managed_identity_credential; if (storage_shared_key_credential) { blob_service_client = std::make_unique(connection_url, storage_shared_key_credential); } else { - blob_service_client = std::make_unique(connection_url); + managed_identity_credential = std::make_shared(); + blob_service_client = std::make_unique(connection_url, managed_identity_credential); } - bool container_exists = containerExists(*blob_service_client, container); - std::string final_url; size_t pos = connection_url.find('?'); if (pos != std::string::npos) @@ -173,12 +180,21 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) final_url = connection_url + (connection_url.back() == '/' ? "" : "/") + container; + if (!attempt_to_create_container) + { + if (storage_shared_key_credential) + return std::make_unique(final_url, storage_shared_key_credential); + else + return std::make_unique(final_url, managed_identity_credential); + } + + bool container_exists = containerExists(*blob_service_client, container); if (container_exists) { if (storage_shared_key_credential) result = std::make_unique(final_url, storage_shared_key_credential); else - result = std::make_unique(final_url); + result = std::make_unique(final_url, managed_identity_credential); } else { @@ -190,8 +206,7 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) try { result = std::make_unique(blob_service_client->CreateBlobContainer(container).Value); - } - catch (const Azure::Storage::StorageException & e) + } catch (const Azure::Storage::StorageException & e) { if (e.StatusCode == Azure::Core::Http::HttpStatusCode::Conflict && e.ReasonPhrase == "The specified container already exists.") @@ -199,7 +214,7 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only) if (storage_shared_key_credential) result = std::make_unique(final_url, storage_shared_key_credential); else - result = std::make_unique(final_url); + result = std::make_unique(final_url, managed_identity_credential); } else { diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.h b/src/Storages/ObjectStorage/AzureBlob/Configuration.h index 3d701e72cb4..8040d433d99 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.h +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.h @@ -52,7 +52,7 @@ protected: std::string blob_path; std::vector blobs_paths; - AzureClientPtr createClient(bool is_read_only); + AzureClientPtr createClient(bool is_read_only, bool attempt_to_create_container); AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context); }; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 903558b73ab..1caa2c000d6 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 848fdb292e8..03a0a1a5e69 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -29,12 +29,14 @@ void StorageHDFSConfiguration::check(ContextPtr context) const checkHDFSURL(fs::path(url) / path); } -ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT { - UNUSED(is_readonly); - auto settings = std::make_unique(); + assertInitialized(); + if (!url.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS url is empty"); + + auto settings = std::make_unique(); return std::make_shared(url, std::move(settings), context->getConfigRef()); } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 0c05f77541b..4e6d8980aa7 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -66,6 +66,8 @@ StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & ot ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT { + assertInitialized(); + const auto & config = context->getConfigRef(); const std::string config_prefix = "s3."; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 8a4dee2c31b..6172f8934af 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -18,6 +18,8 @@ void StorageObjectStorageConfiguration::initialize( // FIXME: it should be - if (format == "auto" && get_format_from_file) if (configuration.format == "auto") configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); + + configuration.initialized = true; } StorageObjectStorageConfiguration::StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other) @@ -48,4 +50,12 @@ std::string StorageObjectStorageConfiguration::getPathWithoutGlob() const return getPath().substr(0, getPath().find_first_of("*?{")); } +void StorageObjectStorageConfiguration::assertInitialized() const +{ + if (!initialized) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); + } +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 8134bd07806..66fe6a68d76 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -54,6 +54,10 @@ public: protected: virtual void fromNamedCollection(const NamedCollection & collection) = 0; virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; + + void assertInitialized() const; + + bool initialized = false; }; using StorageObjectStorageConfigurationPtr = std::shared_ptr; diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index e1978259230..8354e6aa2ae 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index a649f211abc..2158b189070 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -3,6 +3,7 @@ #if USE_AWS_S3 #include +#include #include #include From f5982fdb1ff30280dfebd89afb9274fca33c56b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Mar 2024 19:19:54 +0100 Subject: [PATCH 016/158] Fix some tests --- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 16 ++----- .../ObjectStorages/ObjectStorageFactory.cpp | 3 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 2 - .../ObjectStorage/HDFS/Configuration.cpp | 48 +++++++++++++------ .../ObjectStorage/HDFS/Configuration.h | 6 +-- .../ObjectStorage/ReadBufferIterator.cpp | 4 +- .../ObjectStorage/S3/Configuration.cpp | 6 +++ src/Storages/ObjectStorage/S3/Configuration.h | 2 + .../ObjectStorage/StorageObjectStorage.cpp | 2 + .../ObjectStorage/StorageObjectStorage.h | 2 + .../StorageObjectStorageConfiguration.cpp | 4 ++ .../StorageObjectStorageConfiguration.h | 2 + .../StorageObjectStorageSink.cpp | 40 ++++++++++++++-- .../ObjectStorage/StorageObjectStorageSink.h | 3 ++ src/Storages/StorageS3Settings.cpp | 2 +- .../queries/0_stateless/02114_hdfs_bad_url.sh | 1 - .../0_stateless/02700_s3_part_INT_MAX.sh | 2 +- ...ed_url_and_url_with_special_characters.sql | 3 +- 18 files changed, 104 insertions(+), 44 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 4072d21ed7c..f92e160fd4d 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -16,21 +16,13 @@ namespace DB struct HDFSObjectStorageSettings { - - HDFSObjectStorageSettings() = default; - - size_t min_bytes_for_seek; - int objects_chunk_size_to_delete; - int replication; - - HDFSObjectStorageSettings( - int min_bytes_for_seek_, - int objects_chunk_size_to_delete_, - int replication_) + HDFSObjectStorageSettings(int min_bytes_for_seek_, int replication_) : min_bytes_for_seek(min_bytes_for_seek_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) , replication(replication_) {} + + size_t min_bytes_for_seek; + int replication; }; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index f30a552f8dd..67e38d6389a 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -227,9 +227,8 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) if (uri.back() != '/') throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri); - std::unique_ptr settings = std::make_unique( + auto settings = std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), context->getSettingsRef().hdfs_replication ); diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 872f7eec07b..1aecb590526 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -143,8 +143,6 @@ std::unique_ptr getClient( auth_settings.no_sign_request.value_or(context->getConfigRef().getBool("s3.no_sign_request", false)), }; - LOG_TEST(&Poco::Logger::get("kssenii"), "KSSENII: {} - {}", auth_settings.access_key_id, auth_settings.secret_access_key); - return S3::ClientFactory::instance().create( client_configuration, client_settings, diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 03a0a1a5e69..5edc660d717 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include namespace DB @@ -13,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) @@ -29,37 +31,53 @@ void StorageHDFSConfiguration::check(ContextPtr context) const checkHDFSURL(fs::path(url) / path); } -ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT +ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT + ContextPtr context, + bool /* is_readonly */) { assertInitialized(); - - if (!url.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS url is empty"); - - auto settings = std::make_unique(); - return std::make_shared(url, std::move(settings), context->getConfigRef()); + const auto & settings = context->getSettingsRef(); + auto hdfs_settings = std::make_unique( + settings.remote_read_min_bytes_for_seek, + settings.hdfs_replication + ); + return std::make_shared(url, std::move(hdfs_settings), context->getConfigRef()); } -void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr, bool /* with_structure */) +void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool /* with_structure */) { url = checkAndGetLiteralArgument(args[0], "url"); if (args.size() > 1) + { + args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], context); format = checkAndGetLiteralArgument(args[1], "format_name"); - else - format = "auto"; + } if (args.size() == 3) + { + args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context); compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); - else - compression_method = "auto"; + } - const size_t begin_of_path = url.find('/', url.find("//") + 2); - path = url.substr(begin_of_path + 1); - url = url.substr(0, begin_of_path); + auto pos = url.find("//"); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid url: {}", url); + + pos = url.find('/', pos + 2); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid url: {}", url); + + path = url.substr(pos + 1); + url = url.substr(0, pos); paths = {path}; } +void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fromNamedColection() is not implemented"); +} + } #endif diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 1013c2e00c2..5765edbf36c 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -29,12 +29,12 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } - void fromNamedCollection(const NamedCollection &) override {} - void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; - static void addStructureToArgs(ASTs &, const String &, ContextPtr) {} private: + void fromNamedCollection(const NamedCollection &) override; + void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; + String url; String path; std::vector paths; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index dd4bfe79b06..0b6e34fb831 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -31,7 +31,7 @@ ReadBufferIterator::ReadBufferIterator( , query_settings(query_settings_) , schema_cache(schema_cache_) , read_keys(read_keys_) - , format(configuration->format.empty() || configuration->format == "auto" ? std::nullopt : std::optional(configuration->format)) + , format(configuration->format == "auto" ? std::nullopt : std::optional(configuration->format)) , prev_read_keys_size(read_keys_.size()) { } @@ -191,7 +191,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() { if (first) { - if (format) + if (format.has_value()) throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "The table structure cannot be extracted from a {} format file, because there are no files with provided path " diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 4e6d8980aa7..132a5045d8a 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -51,10 +51,16 @@ String StorageS3Configuration::getDataSourceDescription() void StorageS3Configuration::check(ContextPtr context) const { + validateNamespace(url.bucket); context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); } +void StorageS3Configuration::validateNamespace(const String & name) const +{ + S3::URI::validateBucket(name, {}); +} + StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) : StorageObjectStorageConfiguration(other) { diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 88a084f29b3..f9614da4b95 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -27,6 +27,8 @@ public: String getDataSourceDescription() override; void check(ContextPtr context) const override; + void validateNamespace(const String & name) const override; + StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } bool isStaticConfiguration() const override { return static_configuration; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f1d3635514f..3a894af3e01 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -87,6 +87,7 @@ StorageObjectStorage::StorageObjectStorage( , format_settings(format_settings_) , partition_by(partition_by_) , distributed_processing(distributed_processing_) + , log(getLogger("Storage" + engine_name_)) , object_storage(object_storage_) , configuration(configuration_) { @@ -204,6 +205,7 @@ SinkToStoragePtr StorageObjectStorage::write( if (partition_by_ast) { + LOG_TEST(log, "Using PartitionedSink for {}", configuration->getPath()); return std::make_shared( object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 743b725a88a..ebaf504f532 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -113,6 +114,7 @@ protected: const ASTPtr partition_by; const bool distributed_processing; + LoggerPtr log; ObjectStoragePtr object_storage; ConfigurationPtr configuration; std::mutex configuration_update_mutex; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 6172f8934af..9a8b8191907 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -18,7 +19,10 @@ void StorageObjectStorageConfiguration::initialize( // FIXME: it should be - if (format == "auto" && get_format_from_file) if (configuration.format == "auto") configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); + else + FormatFactory::instance().checkFormatName(configuration.format); + configuration.check(local_context); configuration.initialized = true; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 66fe6a68d76..0beed91b128 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -43,6 +43,8 @@ public: std::string getPathWithoutGlob() const; virtual void check(ContextPtr context) const = 0; + virtual void validateNamespace(const String & /* name */) const {} + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT virtual StorageObjectStorageConfigurationPtr clone() = 0; virtual bool isStaticConfiguration() const { return true; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 37f93a2b82f..2dd8516ebe8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -1,9 +1,14 @@ #include "StorageObjectStorageSink.h" #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_TEXT; +} StorageObjectStorageSink::StorageObjectStorageSink( ObjectStoragePtr object_storage, @@ -93,6 +98,7 @@ void StorageObjectStorageSink::release() write_buf->finalize(); } + PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, @@ -111,9 +117,12 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String & partition_id) { - auto blob = configuration->getPaths().back(); - auto partition_key = replaceWildcards(blob, partition_id); - validatePartitionKey(partition_key, true); + auto partition_bucket = replaceWildcards(configuration->getNamespace(), partition_id); + validateNamespace(partition_bucket); + + auto partition_key = replaceWildcards(configuration->getPath(), partition_id); + validateKey(partition_key); + return std::make_shared( object_storage, configuration, @@ -124,4 +133,29 @@ SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String ); } +void PartitionedStorageObjectStorageSink::validateKey(const String & str) +{ + /// See: + /// - https://docs.aws.amazon.com/AmazonS3/latest/userguide/object-keys.html + /// - https://cloud.ibm.com/apidocs/cos/cos-compatibility#putobject + + if (str.empty() || str.size() > 1024) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incorrect key length (not empty, max 1023 characters), got: {}", str.size()); + + if (!UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in key"); + + validatePartitionKey(str, true); +} + +void PartitionedStorageObjectStorageSink::validateNamespace(const String & str) +{ + configuration->validateNamespace(str); + + if (!UTF8::isValidUTF8(reinterpret_cast(str.data()), str.size())) + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Incorrect non-UTF8 sequence in bucket name"); + + validatePartitionKey(str, false); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 14298376d0e..a352e2c66a3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -54,6 +54,9 @@ public: SinkPtr createSinkForPartition(const String & partition_id) override; private: + void validateKey(const String & str); + void validateNamespace(const String & str); + ObjectStoragePtr object_storage; StorageObjectStorageConfigurationPtr configuration; const std::optional format_settings; diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 3eff6e0f6c9..e8f32388b1b 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -108,7 +108,7 @@ void S3Settings::RequestSettings::PartUploadSettings::validate() if (max_upload_part_size > max_upload_part_size_limit) throw Exception( ErrorCodes::INVALID_SETTING_VALUE, - "Setting max_upload_part_size has invalid value {} which is grater than the s3 API limit {}", + "Setting max_upload_part_size has invalid value {} which is greater than the s3 API limit {}", ReadableSize(max_upload_part_size), ReadableSize(max_upload_part_size_limit)); if (max_single_part_upload_size > max_upload_part_size_limit) diff --git a/tests/queries/0_stateless/02114_hdfs_bad_url.sh b/tests/queries/0_stateless/02114_hdfs_bad_url.sh index 22975dddf6f..5bd5610a9f0 100755 --- a/tests/queries/0_stateless/02114_hdfs_bad_url.sh +++ b/tests/queries/0_stateless/02114_hdfs_bad_url.sh @@ -23,4 +23,3 @@ $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs1:9000/data', 'CSV', 'x UInt32')" $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs://hdfs1/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "HDFS_ERROR" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('http://hdfs1:9000/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "BAD_ARGUMENTS" && echo 'OK' || echo 'FAIL'; $CLICKHOUSE_CLIENT -q "SELECT * FROM hdfs('hdfs://hdfs1@nameservice/abcd/data', 'CSV', 'x UInt32')" 2>&1 | grep -F -q "HDFS_ERROR" && echo 'OK' || echo 'FAIL'; - diff --git a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh index d831c7d9806..a34a480a078 100755 --- a/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh +++ b/tests/queries/0_stateless/02700_s3_part_INT_MAX.sh @@ -13,7 +13,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -nm -q " INSERT INTO FUNCTION s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv', '', '', 'TSV') SELECT repeat('a', 1024) FROM numbers((pow(2, 30) * 2) / 1024) - SETTINGS s3_max_single_part_upload_size = '10Gi'; + SETTINGS s3_max_single_part_upload_size = '5Gi'; SELECT count() FROM s3('http://localhost:11111/test/$CLICKHOUSE_DATABASE/test_INT_MAX.tsv'); " diff --git a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql index da76a5cb88f..1e99eb8b83d 100644 --- a/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql +++ b/tests/queries/0_stateless/02873_s3_presigned_url_and_url_with_special_characters.sql @@ -2,5 +2,4 @@ select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/BU%20-%20UNIT%20-%201/*.parquet'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } -select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_EXTRACT_TABLE_STRUCTURE } - +select * from s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/MyPrefix/*.parquet?some_tocken=ABCD'); -- { serverError CANNOT_DETECT_FORMAT } From cb97f8dab52aeaf492530d66a8553c422ffbcebd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 25 Mar 2024 19:22:20 +0100 Subject: [PATCH 017/158] Fix style check --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - .../ObjectStorage/StorageObjectStorageConfiguration.cpp | 4 ++++ src/Storages/ObjectStorage/StorageObjectStorageSink.cpp | 1 + 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 3a894af3e01..8d85224cff0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -27,7 +27,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int DATABASE_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; - } template diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 9a8b8191907..1d5c0cd3a39 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -4,6 +4,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} void StorageObjectStorageConfiguration::initialize( StorageObjectStorageConfiguration & configuration, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 2dd8516ebe8..cf1c583ca62 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -8,6 +8,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_PARSE_TEXT; + extern const int BAD_ARGUMENTS; } StorageObjectStorageSink::StorageObjectStorageSink( From 7a991de488567a255086a14faa830e1ba1610924 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Mar 2024 19:06:19 +0100 Subject: [PATCH 018/158] Fix tests --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 68 ++++++++++++++-- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 2 + .../ObjectStorages/ObjectStorageIterator.h | 24 +++--- .../ObjectStorageIteratorAsync.cpp | 12 +++ .../ObjectStorageIteratorAsync.h | 15 ++-- .../ObjectStorages/S3/S3ObjectStorage.cpp | 23 ++++-- .../ObjectStorage/HDFS/Configuration.cpp | 81 +++++++++++++++---- .../ObjectStorage/HDFS/Configuration.h | 3 + .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 17 ++-- .../ReadFromStorageObjectStorage.cpp | 4 +- .../ObjectStorage/StorageObjectStorage.cpp | 4 +- .../StorageObjectStorageCluster.cpp | 3 +- .../StorageObjectStorageConfiguration.cpp | 3 +- .../StorageObjectStorageConfiguration.h | 2 +- .../StorageObjectStorageQuerySettings.h | 4 + .../StorageObjectStorageSource.cpp | 56 ++++++++++--- .../StorageObjectStorageSource.h | 31 ++++--- src/Storages/S3Queue/S3QueueSource.cpp | 5 +- src/Storages/S3Queue/S3QueueSource.h | 2 +- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 4 +- 21 files changed, 279 insertions(+), 86 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 2d03de60c3c..db79ff365aa 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #if USE_HDFS @@ -18,6 +19,7 @@ namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; extern const int HDFS_ERROR; + extern const int ACCESS_DENIED; } void HDFSObjectStorage::shutdown() @@ -48,7 +50,7 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLIN std::optional, std::optional) const { - return std::make_unique(object.remote_path, object.remote_path, config, patchSettings(read_settings)); + return std::make_unique(hdfs_root_path, object.remote_path, config, patchSettings(read_settings)); } std::unique_ptr HDFSObjectStorage::readObjects( /// NOLINT @@ -62,12 +64,12 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI [this, disk_read_settings] (bool /* restricted_seek */, const std::string & path) -> std::unique_ptr { - size_t begin_of_path = path.find('/', path.find("//") + 2); - auto hdfs_path = path.substr(begin_of_path); - auto hdfs_uri = path.substr(0, begin_of_path); + // size_t begin_of_path = path.find('/', path.find("//") + 2); + // auto hdfs_path = path.substr(begin_of_path); + // auto hdfs_uri = path.substr(0, begin_of_path); return std::make_unique( - hdfs_uri, hdfs_path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); + hdfs_root_path, path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; return std::make_unique( @@ -131,7 +133,8 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co { auto * file_info = hdfsGetPathInfo(hdfs_fs.get(), path.data()); if (!file_info) - throw Exception(ErrorCodes::HDFS_ERROR, "Cannot get file info for: {}. Error: {}", path, hdfsGetLastError()); + throw Exception(ErrorCodes::HDFS_ERROR, + "Cannot get file info for: {}. Error: {}", path, hdfsGetLastError()); ObjectMetadata metadata; metadata.size_bytes = static_cast(file_info->mSize); @@ -141,6 +144,54 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co return metadata; } +void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const +{ + auto * log = &Poco::Logger::get("HDFSObjectStorage"); + LOG_TRACE(log, "Trying to list files for {}", path); + + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(hdfs_fs.get(), path.data(), &ls.length); + + if (ls.file_info == nullptr && errno != ENOENT) // NOLINT + { + // ignore file not found exception, keep throw other exception, + // libhdfs3 doesn't have function to get exception type, so use errno. + throw Exception(ErrorCodes::ACCESS_DENIED, "Cannot list directory {}: {}", + path, String(hdfsGetLastError())); + } + + if (!ls.file_info && ls.length > 0) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); + } + + LOG_TRACE(log, "Listed {} files for {}", ls.length, path); + + for (int i = 0; i < ls.length; ++i) + { + const String file_path = fs::path(ls.file_info[i].mName).lexically_normal(); + const size_t last_slash = file_path.rfind('/'); + const String file_name = file_path.substr(last_slash); + + const bool is_directory = ls.file_info[i].mKind == 'D'; + if (is_directory) + { + listObjects(fs::path(file_path) / "", children, max_keys); + } + else + { + LOG_TEST(log, "Found file: {}", file_path); + + children.emplace_back(std::make_shared( + String(file_path), + ObjectMetadata{ + static_cast(ls.file_info[i].mSize), + Poco::Timestamp::fromEpochTime(ls.file_info[i].mLastMod), + {}})); + } + } +} + void HDFSObjectStorage::copyObject( /// NOLINT const StoredObject & object_from, const StoredObject & object_to, @@ -160,7 +211,10 @@ void HDFSObjectStorage::copyObject( /// NOLINT } -std::unique_ptr HDFSObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr) +std::unique_ptr HDFSObjectStorage::cloneObjectStorage( + const std::string &, + const Poco::Util::AbstractConfiguration &, + const std::string &, ContextPtr) { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS object storage doesn't support cloning"); } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index f92e160fd4d..24642ec635a 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -92,6 +92,8 @@ public: const WriteSettings & write_settings, std::optional object_to_attributes = {}) override; + void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; + void shutdown() override; void startup() override; diff --git a/src/Disks/ObjectStorages/ObjectStorageIterator.h b/src/Disks/ObjectStorages/ObjectStorageIterator.h index e934fc2056d..26c3c690ba5 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIterator.h +++ b/src/Disks/ObjectStorages/ObjectStorageIterator.h @@ -27,9 +27,7 @@ class ObjectStorageIteratorFromList : public IObjectStorageIterator public: explicit ObjectStorageIteratorFromList(RelativePathsWithMetadata && batch_) : batch(std::move(batch_)) - , batch_iterator(batch.begin()) - { - } + , batch_iterator(batch.begin()) {} void next() override { @@ -37,21 +35,23 @@ public: ++batch_iterator; } - void nextBatch() override - { - batch_iterator = batch.end(); - } + void nextBatch() override { batch_iterator = batch.end(); } - bool isValid() override - { - return batch_iterator != batch.end(); - } + bool isValid() override { return batch_iterator != batch.end(); } RelativePathWithMetadataPtr current() override; RelativePathsWithMetadata currentBatch() override { return batch; } - std::optional getCurrentBatchAndScheduleNext() override { return std::nullopt; } + std::optional getCurrentBatchAndScheduleNext() override + { + if (batch.empty()) + return {}; + + auto current_batch = std::move(batch); + batch = {}; + return current_batch; + } size_t getAccumulatedSize() const override { return batch.size(); } diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index f441b18d59d..94a0751dcc8 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -21,6 +21,18 @@ IObjectStorageIteratorAsync::IObjectStorageIteratorAsync( { } +IObjectStorageIteratorAsync::~IObjectStorageIteratorAsync() +{ + if (!deactivated) + deactivate(); +} + +void IObjectStorageIteratorAsync::deactivate() +{ + list_objects_pool.wait(); + deactivated = true; +} + void IObjectStorageIteratorAsync::nextBatch() { std::lock_guard lock(mutex); diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h index c4bde91f415..3e3269fb550 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.h @@ -19,18 +19,20 @@ public: CurrentMetrics::Metric threads_scheduled_metric, const std::string & thread_name); - void next() override; - void nextBatch() override; + ~IObjectStorageIteratorAsync() override; + bool isValid() override; + RelativePathWithMetadataPtr current() override; RelativePathsWithMetadata currentBatch() override; + + void next() override; + void nextBatch() override; + size_t getAccumulatedSize() const override; std::optional getCurrentBatchAndScheduleNext() override; - ~IObjectStorageIteratorAsync() override - { - list_objects_pool.wait(); - } + void deactivate(); protected: @@ -46,6 +48,7 @@ protected: bool is_initialized{false}; bool is_finished{false}; + bool deactivated{false}; mutable std::recursive_mutex mutex; ThreadPool list_objects_pool; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 33c0afda4c1..d902a33ae4a 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -110,10 +110,19 @@ public: CurrentMetrics::ObjectStorageS3ThreadsScheduled, "ListObjectS3") , client(client_) + , request(std::make_unique()) { - request.SetBucket(bucket_); - request.SetPrefix(path_prefix); - request.SetMaxKeys(static_cast(max_list_size)); + request->SetBucket(bucket_); + request->SetPrefix(path_prefix); + request->SetMaxKeys(static_cast(max_list_size)); + } + + ~S3IteratorAsync() override + { + /// Deactivate background threads before resetting the request to avoid data race. + deactivate(); + request.reset(); + client.reset(); } private: @@ -121,12 +130,12 @@ private: { ProfileEvents::increment(ProfileEvents::S3ListObjects); - auto outcome = client->ListObjectsV2(request); + auto outcome = client->ListObjectsV2(*request); /// Outcome failure will be handled on the caller side. if (outcome.IsSuccess()) { - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + request->SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); auto objects = outcome.GetResult().GetContents(); for (const auto & object : objects) @@ -141,12 +150,12 @@ private: throw S3Exception(outcome.GetError().GetErrorType(), "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}", - quoteString(request.GetBucket()), quoteString(request.GetPrefix()), + quoteString(request->GetBucket()), quoteString(request->GetPrefix()), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); } std::shared_ptr client; - S3::ListObjectsV2Request request; + std::unique_ptr request; }; } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 5edc660d717..50e8918a12e 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -8,6 +8,8 @@ #include #include #include +#include + namespace DB { @@ -28,7 +30,7 @@ StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguratio void StorageHDFSConfiguration::check(ContextPtr context) const { context->getRemoteHostFilter().checkURL(Poco::URI(url)); - checkHDFSURL(fs::path(url) / path); + checkHDFSURL(fs::path(url) / path.substr(1)); } ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT @@ -44,9 +46,22 @@ ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT return std::make_shared(url, std::move(hdfs_settings), context->getConfigRef()); } -void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool /* with_structure */) +std::string StorageHDFSConfiguration::getPathWithoutGlob() const { - url = checkAndGetLiteralArgument(args[0], "url"); + /// Unlike s3 and azure, which are object storages, + /// hdfs is a filesystem, so it cannot list files by partual prefix, + /// only by directory. + auto first_glob_pos = path.find_first_of("*?{"); + auto end_of_path_without_globs = path.substr(0, first_glob_pos).rfind('/'); + if (end_of_path_without_globs == std::string::npos || end_of_path_without_globs == 0) + return "/"; + return path.substr(0, end_of_path_without_globs); +} + +void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) +{ + std::string url_str; + url_str = checkAndGetLiteralArgument(args[0], "url"); if (args.size() > 1) { @@ -54,28 +69,60 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool /* format = checkAndGetLiteralArgument(args[1], "format_name"); } - if (args.size() == 3) + if (with_structure) + { + if (args.size() > 2) + { + structure = checkAndGetLiteralArgument(args[2], "structure"); + } + if (args.size() > 3) + { + args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(args[3], context); + compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); + } + } + else if (args.size() > 2) { args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context); compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } - auto pos = url.find("//"); - if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid url: {}", url); - - pos = url.find('/', pos + 2); - if (pos == std::string::npos) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid url: {}", url); - - path = url.substr(pos + 1); - url = url.substr(0, pos); - paths = {path}; + setURL(url_str); } -void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection &) +void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & collection) { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method fromNamedColection() is not implemented"); + std::string url_str; + + auto filename = collection.getOrDefault("filename", ""); + if (!filename.empty()) + url_str = std::filesystem::path(collection.get("url")) / filename; + 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"); + + setURL(url_str); +} + +void StorageHDFSConfiguration::setURL(const std::string url_) +{ + auto pos = url_.find("//"); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + + pos = url_.find('/', pos + 2); + if (pos == std::string::npos) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); + + path = url_.substr(pos + 1); + url = url_.substr(0, pos); + path = '/' + path; + paths = {path}; + + LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); } } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 5765edbf36c..8506c7c9700 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -31,9 +31,12 @@ public: static void addStructureToArgs(ASTs &, const String &, ContextPtr) {} + std::string getPathWithoutGlob() const override; + private: void fromNamedCollection(const NamedCollection &) override; void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; + void setURL(const std::string url_); String url; String path; diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index 18b22805dfc..c29189804e6 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -55,10 +56,10 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(use_external_buffer_ ? 0 : read_settings_.remote_fs_buffer_size) , hdfs_uri(hdfs_uri_) , hdfs_file_path(hdfs_file_path_) - , builder(createHDFSBuilder(hdfs_uri_, config_)) , read_settings(read_settings_) , read_until_position(read_until_position_) { + builder = createHDFSBuilder(hdfs_uri_, config_); fs = createHDFSFS(builder.get()); fin = hdfsOpenFile(fs.get(), hdfs_file_path.c_str(), O_RDONLY, 0, 0, 0); @@ -96,11 +97,14 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory {})", file_offset, read_until_position - 1); @@ -111,10 +115,11 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory= file_size) - { - return false; - } + // if (file_size != 0 && file_offset >= file_size) + // { + // LOG_TEST(log, "KSSENII 1 2"); + // return false; + // } ResourceGuard rlock(read_settings.resource_link, num_bytes_to_read); int bytes_read; @@ -145,6 +150,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemoryadd(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); + + LOG_TEST(log, "KSSENII SIZE: {}", bytes_read); return true; } diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp index ce157972161..f2595299430 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp @@ -49,8 +49,8 @@ void ReadFromStorageObejctStorage::createIterator(const ActionsDAG::Node * predi { auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, distributed_processing, context, predicate, - virtual_columns, nullptr, query_settings.list_object_keys_size, metric_threads_count, + configuration, object_storage, query_settings, distributed_processing, + context, predicate, virtual_columns, nullptr, metric_threads_count, metric_threads_active, metric_threads_scheduled, context->getFileProgressCallback()); } } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8d85224cff0..0276ff62778 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -288,8 +288,8 @@ std::unique_ptr StorageObjectStorage::creat { const auto settings = StorageSettings::create(context->getSettingsRef()); auto file_iterator = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, /* distributed_processing */false, - context, /* predicate */{}, /* virtual_columns */{}, &read_keys, settings.list_object_keys_size, + configuration, object_storage, settings, /* distributed_processing */false, + context, /* predicate */{}, /* virtual_columns */{}, &read_keys, StorageSettings::ObjectStorageThreads(), StorageSettings::ObjectStorageThreadsActive(), StorageSettings::ObjectStorageThreadsScheduled()); return std::make_unique( diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index c5421f1d319..f023bb068d4 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -92,7 +92,8 @@ StorageObjectStorageCluster::getTask const auto settings = StorageSettings::create(local_context->getSettingsRef()); auto iterator = std::make_shared( object_storage, configuration, predicate, virtual_columns, local_context, - nullptr, settings.list_object_keys_size, local_context->getFileProgressCallback()); + nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, + local_context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 1d5c0cd3a39..61e569cee05 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -40,7 +40,8 @@ StorageObjectStorageConfiguration::StorageObjectStorageConfiguration(const Stora bool StorageObjectStorageConfiguration::withWildcard() const { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return getPath().find(PARTITION_ID_WILDCARD) != String::npos; + return getPath().find(PARTITION_ID_WILDCARD) != String::npos + || getNamespace().find(PARTITION_ID_WILDCARD) != String::npos; } bool StorageObjectStorageConfiguration::isPathWithGlobs() const diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 0beed91b128..48825c6a012 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -40,7 +40,7 @@ public: bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } bool isPathWithGlobs() const; bool isNamespaceWithGlobs() const; - std::string getPathWithoutGlob() const; + virtual std::string getPathWithoutGlob() const; virtual void check(ContextPtr context) const = 0; virtual void validateNamespace(const String & /* name */) const {} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h index 454da7c355f..8bcc2ad3b37 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h @@ -25,6 +25,7 @@ struct StorageObjectStorageSettings SchemaInferenceMode schema_inference_mode; bool skip_empty_files; size_t list_object_keys_size; + bool throw_on_zero_files_match; }; struct S3StorageSettings @@ -38,6 +39,7 @@ struct S3StorageSettings .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.s3_skip_empty_files, .list_object_keys_size = settings.s3_list_object_keys_size, + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, }; } @@ -59,6 +61,7 @@ struct AzureStorageSettings .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure .list_object_keys_size = settings.azure_list_object_keys_size, + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, }; } @@ -80,6 +83,7 @@ struct HDFSStorageSettings .schema_inference_mode = settings.schema_inference_mode, .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for hdfs .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 3c8484194c9..5a88f1436c1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -28,6 +28,7 @@ namespace ErrorCodes extern const int CANNOT_COMPILE_REGEXP; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; + extern const int FILE_DOESNT_EXIST; } StorageObjectStorageSource::StorageObjectStorageSource( @@ -75,12 +76,12 @@ StorageObjectStorageSource::~StorageObjectStorageSource() std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, + const StorageObjectStorageSettings & settings, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - size_t list_object_keys_size, CurrentMetrics::Metric metric_threads_, CurrentMetrics::Metric metric_threads_active_, CurrentMetrics::Metric metric_threads_scheduled_, @@ -99,12 +100,14 @@ std::shared_ptr StorageObjectStorageSourc { /// Iterate through disclosed globs and make a source for each file return std::make_shared( - object_storage, configuration, predicate, virtual_columns, local_context, read_keys, list_object_keys_size, file_progress_callback); + object_storage, configuration, predicate, virtual_columns, local_context, + read_keys, settings.list_object_keys_size, settings.throw_on_zero_files_match, file_progress_callback); } else { return std::make_shared( - object_storage, configuration, virtual_columns, read_keys, file_progress_callback); + object_storage, configuration, virtual_columns, read_keys, + settings.throw_on_zero_files_match, file_progress_callback); } } @@ -209,6 +212,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade do { object_info = file_iterator->next(processor); + if (!object_info || object_info->relative_path.empty()) return {}; @@ -226,8 +230,11 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade ? tryGetNumRowsFromCache(object_info) : std::nullopt; + LOG_TRACE(&Poco::Logger::get("kssenii"), "HAS NUM ROWS FROM CACHE: {}", num_rows_from_cache.has_value()); if (num_rows_from_cache) { + LOG_TRACE(&Poco::Logger::get("kssenii"), "NUM ROWS FROM CACHE: {}", num_rows_from_cache.value()); + /// We should not return single chunk with all number of rows, /// because there is a chance that this chunk will be materialized later /// (it can cause memory problems even with default values in columns or when virtual columns are requested). @@ -324,6 +331,29 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const S } } +StorageObjectStorageSource::IIterator::IIterator(bool throw_on_zero_files_match_, const std::string & logger_name_) + : throw_on_zero_files_match(throw_on_zero_files_match_) + , logger(getLogger(logger_name_)) +{ +} + +ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) +{ + auto object_info = nextImpl(processor); + + if (object_info) + { + first_iteration = false; + LOG_TEST(&Poco::Logger::get("KeysIterator"), "Next key: {}", object_info->relative_path); + } + else if (first_iteration && throw_on_zero_files_match) + { + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files"); + } + + return object_info; +} + StorageObjectStorageSource::GlobIterator::GlobIterator( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, @@ -332,8 +362,10 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( ContextPtr context_, ObjectInfos * read_keys_, size_t list_object_keys_size, + bool throw_on_zero_files_match_, std::function file_progress_callback_) - : WithContext(context_) + : IIterator(throw_on_zero_files_match_, "GlobIterator") + , WithContext(context_) , object_storage(object_storage_) , configuration(configuration_) , virtual_columns(virtual_columns_) @@ -380,7 +412,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } } -ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor */) +ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t /* processor */) { std::lock_guard lock(next_mutex); @@ -401,9 +433,10 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::next(size_t /* processor } new_batch = std::move(result.value()); + LOG_TEST(logger, "Batch size: {}", new_batch.size()); + for (auto it = new_batch.begin(); it != new_batch.end();) { - chassert(*it); if (!recursive && !re2::RE2::FullMatch((*it)->relative_path, *matcher)) it = new_batch.erase(it); else @@ -452,8 +485,10 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, + bool throw_on_zero_files_match_, std::function file_progress_callback_) - : object_storage(object_storage_) + : IIterator(throw_on_zero_files_match_, "KeysIterator") + , object_storage(object_storage_) , configuration(configuration_) , virtual_columns(virtual_columns_) , file_progress_callback(file_progress_callback_) @@ -470,7 +505,7 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( } } -ObjectInfoPtr StorageObjectStorageSource::KeysIterator::next(size_t /* processor */) +ObjectInfoPtr StorageObjectStorageSource::KeysIterator::nextImpl(size_t /* processor */) { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= keys.size()) @@ -520,7 +555,8 @@ StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( CurrentMetrics::Metric metric_threads_, CurrentMetrics::Metric metric_threads_active_, CurrentMetrics::Metric metric_threads_scheduled_) - : callback(callback_) + : IIterator(false, "ReadTaskIterator") + , callback(callback_) { ThreadPool pool(metric_threads_, metric_threads_active_, metric_threads_scheduled_, max_threads_count); auto pool_scheduler = threadPoolCallbackRunner(pool, "ReadTaskIter"); @@ -540,7 +576,7 @@ StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( } } -ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::next(size_t) +ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::nextImpl(size_t) { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= buffer.size()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index d02cb4a3a90..7c5497a6eaa 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -53,12 +53,12 @@ public: static std::shared_ptr createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, + const StorageObjectStorageSettings & settings, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - size_t list_object_keys_size, CurrentMetrics::Metric metric_threads_, CurrentMetrics::Metric metric_threads_active_, CurrentMetrics::Metric metric_threads_scheduled_, @@ -133,10 +133,21 @@ protected: class StorageObjectStorageSource::IIterator { public: + IIterator(bool throw_on_zero_files_match_, const std::string & logger_name_); + virtual ~IIterator() = default; virtual size_t estimatedKeysCount() = 0; - virtual ObjectInfoPtr next(size_t processor) = 0; + + ObjectInfoPtr next(size_t processor); + +protected: + virtual ObjectInfoPtr nextImpl(size_t processor) = 0; + +protected: + const bool throw_on_zero_files_match; + bool first_iteration = true; + LoggerPtr logger; }; class StorageObjectStorageSource::ReadTaskIterator : public IIterator @@ -151,9 +162,9 @@ public: size_t estimatedKeysCount() override { return buffer.size(); } - ObjectInfoPtr next(size_t) override; - private: + ObjectInfoPtr nextImpl(size_t) override; + ReadTaskCallback callback; ObjectInfos buffer; std::atomic_size_t index = 0; @@ -170,15 +181,17 @@ public: ContextPtr context_, ObjectInfos * read_keys_, size_t list_object_keys_size, + bool throw_on_zero_files_match_, std::function file_progress_callback_ = {}); ~GlobIterator() override = default; size_t estimatedKeysCount() override { return object_infos.size(); } - ObjectInfoPtr next(size_t processor) override; - private: + ObjectInfoPtr nextImpl(size_t processor) override; + void createFilterAST(const String & any_key); + ObjectStoragePtr object_storage; ConfigurationPtr configuration; ActionsDAGPtr filter_dag; @@ -193,7 +206,6 @@ private: std::unique_ptr matcher; - void createFilterAST(const String & any_key); bool is_finished = false; std::mutex next_mutex; @@ -208,15 +220,16 @@ public: ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, + bool throw_on_zero_files_match_, std::function file_progress_callback = {}); ~KeysIterator() override = default; size_t estimatedKeysCount() override { return keys.size(); } - ObjectInfoPtr next(size_t processor) override; - private: + ObjectInfoPtr nextImpl(size_t processor) override; + const ObjectStoragePtr object_storage; const ConfigurationPtr configuration; const NamesAndTypesList virtual_columns; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 42cd210018a..ee3071ea71f 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -45,7 +45,8 @@ StorageS3QueueSource::FileIterator::FileIterator( std::unique_ptr glob_iterator_, size_t current_shard_, std::atomic & shutdown_called_) - : metadata(metadata_) + : StorageObjectStorageSource::IIterator(false, "S3QueueIterator") + , metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) , shutdown_called(shutdown_called_) , log(&Poco::Logger::get("StorageS3QueueSource")) @@ -59,7 +60,7 @@ StorageS3QueueSource::FileIterator::FileIterator( } } -StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::next(size_t processor) +StorageS3QueueSource::ObjectInfoPtr StorageS3QueueSource::FileIterator::nextImpl(size_t processor) { while (!shutdown_called) { diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 2bdac7f2311..8c785e683c2 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -56,7 +56,7 @@ public: /// Note: /// List results in s3 are always returned in UTF-8 binary order. /// (https://docs.aws.amazon.com/AmazonS3/latest/userguide/ListingKeysUsingAPIs.html) - ObjectInfoPtr next(size_t processor) override; + ObjectInfoPtr nextImpl(size_t processor) override; size_t estimatedKeysCount() override; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index aafcdc39f9e..c5799d23abd 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -598,7 +598,7 @@ std::shared_ptr StorageS3Queue::createFileIterator { auto settings = S3StorageSettings::create(local_context->getSettingsRef()); auto glob_iterator = std::make_unique( - object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size); + object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); return std::make_shared(files_metadata, std::move(glob_iterator), s3queue_settings->s3queue_current_shard_num, shutdown_called); } diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 9dec1954406..5632c7ae060 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -61,7 +61,7 @@ def test_read_write_storage_with_globs(started_cluster): hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" - assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n" + assert node1.query("select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1") == "3\n" assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n" assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n" @@ -159,7 +159,7 @@ def test_bad_hdfs_uri(started_cluster): ) except Exception as ex: print(ex) - assert "Unable to create builder to connect to HDFS" in str(ex) + assert "Unable to connect to HDFS" in str(ex) try: node1.query( From 480251e5932f2d15891a403887b5afc96f40ee89 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Mar 2024 19:28:11 +0100 Subject: [PATCH 019/158] Fix style check --- tests/integration/test_storage_hdfs/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 5632c7ae060..f6e486d6594 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -61,7 +61,12 @@ def test_read_write_storage_with_globs(started_cluster): hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" - assert node1.query("select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1") == "3\n" + assert ( + node1.query( + "select count(*) from HDFSStorageWithRange settings s3_throw_on_zero_files_match=1" + ) + == "3\n" + ) assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == "3\n" assert node1.query("select count(*) from HDFSStorageWithAsterisk") == "3\n" From a2e210462d7d78212c32408ea3d276ef366b57c4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 27 Mar 2024 22:31:22 +0100 Subject: [PATCH 020/158] Fix style check --- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 1 + src/Storages/ObjectStorage/HDFS/Configuration.cpp | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index db79ff365aa..9bc75b740e5 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -20,6 +20,7 @@ namespace ErrorCodes extern const int UNSUPPORTED_METHOD; extern const int HDFS_ERROR; extern const int ACCESS_DENIED; + extern const int LOGICAL_ERROR; } void HDFSObjectStorage::shutdown() diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 50e8918a12e..3828afc0bea 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -16,7 +16,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int NOT_IMPLEMENTED; } StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) From 5c63d09c5bb91f7dc159befeb505a74e4c0257a5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 14:15:14 +0100 Subject: [PATCH 021/158] More tests fixes --- src/Core/Settings.h | 3 + .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 4 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 8 +- .../ObjectStorage/HDFS/Configuration.cpp | 18 ++++- .../ObjectStorage/HDFS/Configuration.h | 2 +- .../StorageObjectStorageQuerySettings.h | 4 + .../StorageObjectStorageSource.cpp | 76 ++++++++++++------- .../StorageObjectStorageSource.h | 20 ++--- src/Storages/S3Queue/S3QueueSource.cpp | 2 +- tests/integration/test_storage_hdfs/test.py | 12 +-- .../0_stateless/02725_database_hdfs.sh | 3 +- 11 files changed, 98 insertions(+), 54 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f8f3595094c..2fae390c35b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -113,6 +113,9 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, s3_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageS3", 0) \ + M(Bool, hdfs_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageHDFS", 0) \ + M(Bool, azure_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageAzure", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 85d3e921f22..8bfba6fcfad 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -103,10 +103,10 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL void HDFSObjectStorage::removeObject(const StoredObject & object) { const auto & path = object.remote_path; - const size_t begin_of_path = path.find('/', path.find("//") + 2); + // const size_t begin_of_path = path.find('/', path.find("//") + 2); /// Add path from root to file name - int res = hdfsDelete(hdfs_fs.get(), path.substr(begin_of_path).c_str(), 0); + int res = hdfsDelete(hdfs_fs.get(), path.c_str(), 0); if (res == -1) throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: {}", path); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index b9995620c0f..9085fddfd08 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -432,7 +432,9 @@ void S3ObjectStorage::removeObjectsIfExist(const StoredObjects & objects) std::optional S3ObjectStorage::tryGetObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); + auto object_info = S3::getObjectInfo( + *client.get(), uri.bucket, path, {}, settings_ptr->request_settings, + /* with_metadata= */ true, /* for_disk_s3= */ true, /* throw_on_error= */ false); if (object_info.size == 0 && object_info.last_modification_time == 0 && object_info.metadata.empty()) return {}; @@ -448,7 +450,9 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true, /* for_disk_s3= */ true); + auto object_info = S3::getObjectInfo( + *client.get(), uri.bucket, path, {}, settings_ptr->request_settings, + /* with_metadata= */ true, /* for_disk_s3= */ true); ObjectMetadata result; result.size_bytes = object_info.size; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 3828afc0bea..594f0b89454 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) @@ -62,6 +63,13 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit std::string url_str; url_str = checkAndGetLiteralArgument(args[0], "url"); + const size_t max_args_num = with_structure ? 4 : 3; + if (args.size() > max_args_num) + { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Expected not more than {} arguments", max_args_num); + } + if (args.size() > 1) { args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], context); @@ -72,6 +80,7 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit { if (args.size() > 2) { + args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context); structure = checkAndGetLiteralArgument(args[2], "structure"); } if (args.size() > 3) @@ -100,13 +109,14 @@ void StorageHDFSConfiguration::fromNamedCollection(const NamedCollection & colle url_str = collection.get("url"); format = collection.getOrDefault("format", "auto"); - compression_method = collection.getOrDefault("compression_method", collection.getOrDefault("compression", "auto")); + compression_method = collection.getOrDefault("compression_method", + collection.getOrDefault("compression", "auto")); structure = collection.getOrDefault("structure", "auto"); setURL(url_str); } -void StorageHDFSConfiguration::setURL(const std::string url_) +void StorageHDFSConfiguration::setURL(const std::string & url_) { auto pos = url_.find("//"); if (pos == std::string::npos) @@ -117,8 +127,10 @@ void StorageHDFSConfiguration::setURL(const std::string url_) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Bad hdfs url: {}", url_); path = url_.substr(pos + 1); + if (!path.starts_with('/')) + path = '/' + path; + url = url_.substr(0, pos); - path = '/' + path; paths = {path}; LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 8506c7c9700..7154f790665 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -36,7 +36,7 @@ public: private: void fromNamedCollection(const NamedCollection &) override; void fromAST(ASTs & args, ContextPtr, bool /* with_structure */) override; - void setURL(const std::string url_); + void setURL(const std::string & url_); String url; String path; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h index 8bcc2ad3b37..f0687776aa7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h @@ -26,6 +26,7 @@ struct StorageObjectStorageSettings bool skip_empty_files; size_t list_object_keys_size; bool throw_on_zero_files_match; + bool ignore_non_existent_file; }; struct S3StorageSettings @@ -40,6 +41,7 @@ struct S3StorageSettings .skip_empty_files = settings.s3_skip_empty_files, .list_object_keys_size = settings.s3_list_object_keys_size, .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.s3_ignore_file_doesnt_exist, }; } @@ -62,6 +64,7 @@ struct AzureStorageSettings .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure .list_object_keys_size = settings.azure_list_object_keys_size, .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist, }; } @@ -84,6 +87,7 @@ struct HDFSStorageSettings .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for hdfs .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, }; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 5a88f1436c1..80aa0c210e9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -100,14 +100,15 @@ std::shared_ptr StorageObjectStorageSourc { /// Iterate through disclosed globs and make a source for each file return std::make_shared( - object_storage, configuration, predicate, virtual_columns, local_context, - read_keys, settings.list_object_keys_size, settings.throw_on_zero_files_match, file_progress_callback); + object_storage, configuration, predicate, virtual_columns, + local_context, read_keys, settings.list_object_keys_size, + settings.throw_on_zero_files_match, file_progress_callback); } else { return std::make_shared( object_storage, configuration, virtual_columns, read_keys, - settings.throw_on_zero_files_match, file_progress_callback); + settings.ignore_non_existent_file, file_progress_callback); } } @@ -331,9 +332,8 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const S } } -StorageObjectStorageSource::IIterator::IIterator(bool throw_on_zero_files_match_, const std::string & logger_name_) - : throw_on_zero_files_match(throw_on_zero_files_match_) - , logger(getLogger(logger_name_)) +StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name_) + : logger(getLogger(logger_name_)) { } @@ -343,13 +343,8 @@ ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) if (object_info) { - first_iteration = false; LOG_TEST(&Poco::Logger::get("KeysIterator"), "Next key: {}", object_info->relative_path); } - else if (first_iteration && throw_on_zero_files_match) - { - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files"); - } return object_info; } @@ -364,11 +359,12 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( size_t list_object_keys_size, bool throw_on_zero_files_match_, std::function file_progress_callback_) - : IIterator(throw_on_zero_files_match_, "GlobIterator") + : IIterator("GlobIterator") , WithContext(context_) , object_storage(object_storage_) , configuration(configuration_) , virtual_columns(virtual_columns_) + , throw_on_zero_files_match(throw_on_zero_files_match_) , read_keys(read_keys_) , file_progress_callback(file_progress_callback_) { @@ -412,10 +408,24 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } } -ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t /* processor */) +ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processor) { std::lock_guard lock(next_mutex); + auto object_info = nextImplUnlocked(processor); + if (object_info) + { + if (first_iteration) + first_iteration = false; + } + else if (first_iteration && throw_on_zero_files_match) + { + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files"); + } + return object_info; +} +ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImplUnlocked(size_t /* processor */) +{ bool current_batch_processed = object_infos.empty() || index >= object_infos.size(); if (is_finished && current_batch_processed) return {}; @@ -485,14 +495,15 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, - bool throw_on_zero_files_match_, + bool ignore_non_existent_files_, std::function file_progress_callback_) - : IIterator(throw_on_zero_files_match_, "KeysIterator") + : IIterator("KeysIterator") , object_storage(object_storage_) , configuration(configuration_) , virtual_columns(virtual_columns_) , file_progress_callback(file_progress_callback_) , keys(configuration->getPaths()) + , ignore_non_existent_files(ignore_non_existent_files_) { if (read_keys_) { @@ -507,20 +518,29 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( ObjectInfoPtr StorageObjectStorageSource::KeysIterator::nextImpl(size_t /* processor */) { - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= keys.size()) - return {}; - - auto key = keys[current_index]; - - ObjectMetadata metadata{}; - if (file_progress_callback) + while (true) { - metadata = object_storage->getObjectMetadata(key); - file_progress_callback(FileProgress(0, metadata.size_bytes)); - } + size_t current_index = index.fetch_add(1, std::memory_order_relaxed); + if (current_index >= keys.size()) + return {}; - return std::make_shared(key, metadata); + auto key = keys[current_index]; + + ObjectMetadata object_metadata{}; + if (ignore_non_existent_files) + { + auto metadata = object_storage->tryGetObjectMetadata(key); + if (!metadata) + continue; + } + else + object_metadata = object_storage->getObjectMetadata(key); + + if (file_progress_callback) + file_progress_callback(FileProgress(0, object_metadata.size_bytes)); + + return std::make_shared(key, object_metadata); + } } StorageObjectStorageSource::ReaderHolder::ReaderHolder( @@ -555,7 +575,7 @@ StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( CurrentMetrics::Metric metric_threads_, CurrentMetrics::Metric metric_threads_active_, CurrentMetrics::Metric metric_threads_scheduled_) - : IIterator(false, "ReadTaskIterator") + : IIterator("ReadTaskIterator") , callback(callback_) { ThreadPool pool(metric_threads_, metric_threads_active_, metric_threads_scheduled_, max_threads_count); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 7c5497a6eaa..3d4cc4fbd20 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -133,7 +133,7 @@ protected: class StorageObjectStorageSource::IIterator { public: - IIterator(bool throw_on_zero_files_match_, const std::string & logger_name_); + explicit IIterator(const std::string & logger_name_); virtual ~IIterator() = default; @@ -143,10 +143,6 @@ public: protected: virtual ObjectInfoPtr nextImpl(size_t processor) = 0; - -protected: - const bool throw_on_zero_files_match; - bool first_iteration = true; LoggerPtr logger; }; @@ -190,23 +186,26 @@ public: private: ObjectInfoPtr nextImpl(size_t processor) override; + ObjectInfoPtr nextImplUnlocked(size_t processor); void createFilterAST(const String & any_key); - ObjectStoragePtr object_storage; - ConfigurationPtr configuration; - ActionsDAGPtr filter_dag; - NamesAndTypesList virtual_columns; + const ObjectStoragePtr object_storage; + const ConfigurationPtr configuration; + const NamesAndTypesList virtual_columns; + const bool throw_on_zero_files_match; size_t index = 0; ObjectInfos object_infos; ObjectInfos * read_keys; + ActionsDAGPtr filter_dag; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; std::unique_ptr matcher; bool is_finished = false; + bool first_iteration = true; std::mutex next_mutex; std::function file_progress_callback; @@ -220,7 +219,7 @@ public: ConfigurationPtr configuration_, const NamesAndTypesList & virtual_columns_, ObjectInfos * read_keys_, - bool throw_on_zero_files_match_, + bool ignore_non_existent_files_, std::function file_progress_callback = {}); ~KeysIterator() override = default; @@ -236,5 +235,6 @@ private: const std::function file_progress_callback; const std::vector keys; std::atomic index = 0; + bool ignore_non_existent_files; }; } diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index ee3071ea71f..8e7155205c4 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -45,7 +45,7 @@ StorageS3QueueSource::FileIterator::FileIterator( std::unique_ptr glob_iterator_, size_t current_shard_, std::atomic & shutdown_called_) - : StorageObjectStorageSource::IIterator(false, "S3QueueIterator") + : StorageObjectStorageSource::IIterator("S3QueueIterator") , metadata(metadata_) , glob_iterator(std::move(glob_iterator_)) , shutdown_called(shutdown_called_) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index f6e486d6594..fbf97adcee0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -326,7 +326,7 @@ def test_virtual_columns(started_cluster): hdfs_api.write_data("/file1", "1\n") hdfs_api.write_data("/file2", "2\n") hdfs_api.write_data("/file3", "3\n") - expected = "1\tfile1\thdfs://hdfs1:9000/file1\n2\tfile2\thdfs://hdfs1:9000/file2\n3\tfile3\thdfs://hdfs1:9000/file3\n" + expected = "1\tfile1\t/file1\n2\tfile2\t/file2\n3\tfile3\t/file3\n" assert ( node1.query( "select id, _file as file_name, _path as file_path from virtual_cols order by id" @@ -365,7 +365,7 @@ def test_truncate_table(started_cluster): assert hdfs_api.read_data("/tr") == "1\tMark\t72.53\n" assert node1.query("select * from test_truncate") == "1\tMark\t72.53\n" node1.query("truncate table test_truncate") - assert node1.query("select * from test_truncate") == "" + assert node1.query("select * from test_truncate settings hdfs_ignore_file_doesnt_exist=1") == "" node1.query("drop table test_truncate") @@ -488,13 +488,13 @@ def test_hdfsCluster(started_cluster): actual = node1.query( "select id, _file as file_name, _path as file_path from hdfs('hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + expected = "1\tfile1\t/test_hdfsCluster/file1\n2\tfile2\t/test_hdfsCluster/file2\n3\tfile3\t/test_hdfsCluster/file3\n" assert actual == expected actual = node1.query( "select id, _file as file_name, _path as file_path from hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\thdfs://hdfs1:9000/test_hdfsCluster/file1\n2\tfile2\thdfs://hdfs1:9000/test_hdfsCluster/file2\n3\tfile3\thdfs://hdfs1:9000/test_hdfsCluster/file3\n" + expected = "1\tfile1\t/test_hdfsCluster/file1\n2\tfile2\t/test_hdfsCluster/file2\n3\tfile3\t/test_hdfsCluster/file3\n" assert actual == expected fs.delete(dir, recursive=True) @@ -502,7 +502,7 @@ def test_hdfsCluster(started_cluster): def test_hdfs_directory_not_exist(started_cluster): ddl = "create table HDFSStorageWithNotExistDir (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/data/not_eixst', 'TSV')" node1.query(ddl) - assert "" == node1.query("select * from HDFSStorageWithNotExistDir") + assert "" == node1.query("select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1") def test_overwrite(started_cluster): @@ -658,7 +658,7 @@ def test_virtual_columns_2(started_cluster): node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") - assert result.strip() == "hdfs://hdfs1:9000/parquet_2" + assert result.strip() == "/parquet_2" table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index d62f928e947..623af707542 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -60,7 +60,8 @@ SELECT * FROM \"abacaba/file.tsv\" ${CLICKHOUSE_CLIENT} -q "SELECT * FROM test_hdfs_4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: - +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: From 961704ba173bef199735c52e5296b371a5168f15 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 15:00:49 +0100 Subject: [PATCH 022/158] Style check --- tests/integration/test_storage_hdfs/test.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index fbf97adcee0..77a55ced5c8 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -365,7 +365,12 @@ def test_truncate_table(started_cluster): assert hdfs_api.read_data("/tr") == "1\tMark\t72.53\n" assert node1.query("select * from test_truncate") == "1\tMark\t72.53\n" node1.query("truncate table test_truncate") - assert node1.query("select * from test_truncate settings hdfs_ignore_file_doesnt_exist=1") == "" + assert ( + node1.query( + "select * from test_truncate settings hdfs_ignore_file_doesnt_exist=1" + ) + == "" + ) node1.query("drop table test_truncate") @@ -502,7 +507,9 @@ def test_hdfsCluster(started_cluster): def test_hdfs_directory_not_exist(started_cluster): ddl = "create table HDFSStorageWithNotExistDir (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/data/not_eixst', 'TSV')" node1.query(ddl) - assert "" == node1.query("select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1") + assert "" == node1.query( + "select * from HDFSStorageWithNotExistDir settings hdfs_ignore_file_doesnt_exist=1" + ) def test_overwrite(started_cluster): From 34a87666ebe932fbedef68ac7fef05f2a6e5880a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Mar 2024 16:55:39 +0100 Subject: [PATCH 023/158] Update settings changes history --- src/Core/SettingsChangesHistory.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index db6fb2f1c0e..8cde00fcc14 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -124,6 +124,9 @@ static std::map sett {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, + {"hdfs_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageHDFS"}, + {"azure_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageAzureBlob"}, + {"s3_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageS3"}, }}, {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, From b24a2afd5fb6c44fd1ecd2435963f3433c61f2af Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 13:21:22 +0200 Subject: [PATCH 024/158] A few more test fixes --- src/TableFunctions/TableFunctionObjectStorageCluster.cpp | 5 +++-- src/TableFunctions/TableFunctionObjectStorageCluster.h | 8 ++++++++ tests/queries/0_stateless/02725_database_hdfs.sh | 6 ++---- 3 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 4ec94cfaf7c..909ace788eb 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -17,9 +17,8 @@ namespace DB template StoragePtr TableFunctionObjectStorageCluster::executeImpl( const ASTPtr & /*function*/, ContextPtr context, - const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const + const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const { - using Base = TableFunctionObjectStorage; auto configuration = Base::getConfiguration(); ColumnsDescription columns; @@ -27,6 +26,8 @@ StoragePtr TableFunctionObjectStorageClusterstructure, context); else if (!Base::structure_hint.empty()) columns = Base::structure_hint; + else if (!cached_columns.empty()) + columns = cached_columns; auto object_storage = Base::getObjectStorage(context, !is_insert_query); StoragePtr storage; diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index 461456e37df..21c2f8995dc 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -67,6 +67,8 @@ public: String getSignature() const override { return signature; } protected: + using Base = TableFunctionObjectStorage; + StoragePtr executeImpl( const ASTPtr & ast_function, ContextPtr context, @@ -75,6 +77,12 @@ protected: bool is_insert_query) const override; const char * getStorageTypeName() const override { return Definition::storage_type_name; } + + bool hasStaticStructure() const override { return Base::getConfiguration()->structure != "auto"; } + + bool needStructureHint() const override { return Base::getConfiguration()->structure == "auto"; } + + void setStructureHint(const ColumnsDescription & structure_hint_) override { Base::structure_hint = structure_hint_; } }; #if USE_AWS_S3 diff --git a/tests/queries/0_stateless/02725_database_hdfs.sh b/tests/queries/0_stateless/02725_database_hdfs.sh index 623af707542..1eb22976b84 100755 --- a/tests/queries/0_stateless/02725_database_hdfs.sh +++ b/tests/queries/0_stateless/02725_database_hdfs.sh @@ -58,10 +58,8 @@ SELECT * FROM \"abacaba/file.tsv\" """ 2>&1 | tr '\n' ' ' | grep -oF "CANNOT_EXTRACT_TABLE_STRUCTURE" ${CLICKHOUSE_CLIENT} -q "SELECT * FROM test_hdfs_4.\`http://localhost:11111/test/a.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "CANNOT_EXTRACT_TABLE_STRUCTURE" > /dev/null && echo "OK" || echo 'FAIL' ||: -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" -${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/file.myext\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "The data format cannot be detected" > /dev/null && echo "OK" || echo 'FAIL' ||: +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222/test_02725_3.tsv\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "The table structure cannot be extracted" > /dev/null && echo "OK" || echo 'FAIL' ||: ${CLICKHOUSE_CLIENT} --query "SELECT * FROM test_hdfs_4.\`hdfs://localhost:12222\`" 2>&1 | tr '\n' ' ' | grep -oF -e "UNKNOWN_TABLE" -e "BAD_ARGUMENTS" > /dev/null && echo "OK" || echo 'FAIL' ||: From aa804e744b1f1c233ef7158431feb4c016d0026c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Apr 2024 14:05:50 +0200 Subject: [PATCH 025/158] Fix style check --- src/Storages/ObjectStorage/HDFS/Configuration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 5a4fb322692..0a49ba5e251 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -153,7 +153,7 @@ void StorageHDFSConfiguration::addStructureToArgs(ASTs & args, const String & st { size_t count = args.size(); if (count == 0 || count > 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected 1 to 3 arguments in table function, got {}", count); auto structure_literal = std::make_shared(structure_); From e5ffe3cf8d7362335ef6150e7864d5deb74c9479 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Apr 2024 16:15:11 +0200 Subject: [PATCH 026/158] More tests fixes --- src/Storages/MergeTree/KeyCondition.cpp | 7 +++++ .../ObjectStorage/AzureBlob/Configuration.cpp | 3 +- .../ObjectStorage/AzureBlob/Configuration.h | 4 ++- .../ObjectStorage/HDFS/Configuration.cpp | 28 +++++++++++++------ .../ObjectStorage/HDFS/Configuration.h | 4 ++- .../ReadFromStorageObjectStorage.cpp | 3 +- .../ReadFromStorageObjectStorage.h | 1 + .../ObjectStorage/S3/Configuration.cpp | 4 ++- src/Storages/ObjectStorage/S3/Configuration.h | 4 ++- .../ObjectStorage/StorageObjectStorage.cpp | 3 ++ .../StorageObjectStorageConfiguration.h | 1 + .../StorageObjectStorageSource.cpp | 16 ++++++++++- .../StorageObjectStorageSource.h | 2 +- .../TableFunctionObjectStorage.cpp | 4 +-- 14 files changed, 65 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2d57ea40c9c..a720e243fdb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2661,6 +2661,13 @@ BoolMask KeyCondition::checkInHyperrectangle( else if (element.function == RPNElement::FUNCTION_IN_RANGE || element.function == RPNElement::FUNCTION_NOT_IN_RANGE) { + if (element.key_column >= hyperrectangle.size()) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Hyperrectangle size is {}, but requested element at posittion {} ({})", + hyperrectangle.size(), element.key_column, element.toString()); + } + const Range * key_range = &hyperrectangle[element.key_column]; /// The case when the column is wrapped in a chain of possibly monotonic functions. diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 018cec51e7c..fe01251e58a 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -379,7 +379,8 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte blobs_paths = {blob_path}; } -void StorageAzureBlobConfiguration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) +void StorageAzureBlobConfiguration::addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & /* format */, ContextPtr context) { if (tryGetNamedCollectionWithOverrides(args, context)) { diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.h b/src/Storages/ObjectStorage/AzureBlob/Configuration.h index 8040d433d99..c12ff81197d 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.h +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.h @@ -26,6 +26,7 @@ public: const Paths & getPaths() const override { return blobs_paths; } Paths & getPaths() override { return blobs_paths; } + void setPaths(const Paths & paths) override { blobs_paths = paths; } String getDataSourceDescription() override { return fs::path(connection_url) / container; } String getNamespace() const override { return container; } @@ -36,7 +37,8 @@ public: void fromNamedCollection(const NamedCollection & collection) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; - static void addStructureToArgs(ASTs & args, const String & structure, ContextPtr context); + static void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context); protected: using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 0a49ba5e251..220857fead6 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -139,7 +139,11 @@ void StorageHDFSConfiguration::setURL(const std::string & url_) LOG_TRACE(getLogger("StorageHDFSConfiguration"), "Using url: {}, path: {}", url, path); } -void StorageHDFSConfiguration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) +void StorageHDFSConfiguration::addStructureAndFormatToArgs( + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) { if (tryGetNamedCollectionWithOverrides(args, context)) { @@ -152,10 +156,13 @@ void StorageHDFSConfiguration::addStructureToArgs(ASTs & args, const String & st else { size_t count = args.size(); - if (count == 0 || count > 3) + if (count == 0 || count > 4) + { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Expected 1 to 3 arguments in table function, got {}", count); + "Expected 1 to 4 arguments in table function, got {}", count); + } + auto format_literal = std::make_shared(format_); auto structure_literal = std::make_shared(structure_); /// hdfs(url) @@ -168,15 +175,18 @@ void StorageHDFSConfiguration::addStructureToArgs(ASTs & args, const String & st /// hdfs(url, format) else if (count == 2) { + if (checkAndGetLiteralArgument(args[1], "format") == "auto") + args.back() = format_literal; args.push_back(structure_literal); } - /// hdfs(url, format, compression_method) - else if (count == 3) + /// hdfs(url, format, structure) + /// hdfs(url, format, structure, compression_method) + else if (count >= 3) { - auto compression_method = args.back(); - args.pop_back(); - args.push_back(structure_literal); - args.push_back(compression_method); + if (checkAndGetLiteralArgument(args[1], "format") == "auto") + args[1] = format_literal; + if (checkAndGetLiteralArgument(args[2], "structure") == "auto") + args[2] = structure_literal; } } } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 7dc1f8073c1..23a7e8e4549 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -21,6 +21,7 @@ public: const Paths & getPaths() const override { return paths; } Paths & getPaths() override { return paths; } + void setPaths(const Paths & paths_) override { paths = paths_; } String getNamespace() const override { return ""; } String getDataSourceDescription() override { return url; } @@ -29,7 +30,8 @@ public: ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } - static void addStructureToArgs(ASTs &, const String &, ContextPtr); + static void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context); std::string getPathWithoutGlob() const override; diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp index f2595299430..89d33191f41 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp @@ -9,6 +9,7 @@ ReadFromStorageObejctStorage::ReadFromStorageObejctStorage( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, const String & name_, + const Names & columns_to_read, const NamesAndTypesList & virtual_columns_, const SelectQueryInfo & query_info_, const StorageSnapshotPtr & storage_snapshot_, @@ -24,7 +25,7 @@ ReadFromStorageObejctStorage::ReadFromStorageObejctStorage( CurrentMetrics::Metric metric_threads_count_, CurrentMetrics::Metric metric_threads_active_, CurrentMetrics::Metric metric_threads_scheduled_) - : SourceStepWithFilter(DataStream{.header = info_.source_header}, info_.requested_columns.getNames(), query_info_, storage_snapshot_, context_) + : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) , object_storage(object_storage_) , configuration(configuration_) , info(std::move(info_)) diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h index 44b992f8c12..c0dd02d75f8 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h +++ b/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h @@ -15,6 +15,7 @@ public: ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, const String & name_, + const Names & columns_to_read, const NamesAndTypesList & virtual_columns_, const SelectQueryInfo & query_info_, const StorageSnapshotPtr & storage_snapshot_, diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 132a5045d8a..f532af24017 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -330,7 +330,8 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_ keys = {url.key}; } -void StorageS3Configuration::addStructureToArgs(ASTs & args, const String & structure_, ContextPtr context) +void StorageS3Configuration::addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) { if (tryGetNamedCollectionWithOverrides(args, context)) { @@ -348,6 +349,7 @@ void StorageS3Configuration::addStructureToArgs(ASTs & args, const String & stru if (count == 0 || count > 6) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to 6 arguments in table function, got {}", count); + auto format_literal = std::make_shared(format_); auto structure_literal = std::make_shared(structure_); /// s3(s3_url) diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index f9614da4b95..ff5e8680e66 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -22,6 +22,7 @@ public: const Paths & getPaths() const override { return keys; } Paths & getPaths() override { return keys; } + void setPaths(const Paths & paths) override { keys = paths; } String getNamespace() const override { return url.bucket; } String getDataSourceDescription() override; @@ -33,7 +34,8 @@ public: bool isStaticConfiguration() const override { return static_configuration; } ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT - static void addStructureToArgs(ASTs & args, const String & structure, ContextPtr context); + static void addStructureAndFormatToArgs( + ASTs & args, const String & structure, const String & format, ContextPtr context); private: void fromNamedCollection(const NamedCollection & collection) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 84810c117c9..8fc3de4de1b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -158,10 +158,13 @@ void StorageObjectStorage::read( const bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII SOURCE HEADER: {}", read_from_format_info.source_header.dumpStructure()); + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII FORMAT HEADER: {}", read_from_format_info.format_header.dumpStructure()); auto read_step = std::make_unique( object_storage, configuration, getName(), + column_names, getVirtualsList(), query_info, storage_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 48825c6a012..647575aaa90 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -32,6 +32,7 @@ public: virtual const Paths & getPaths() const = 0; virtual Paths & getPaths() = 0; + virtual void setPaths(const Paths & paths) = 0; virtual String getDataSourceDescription() = 0; virtual String getNamespace() const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index fd3ac58b1a2..30316af987c 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -106,8 +106,21 @@ std::shared_ptr StorageObjectStorageSourc } else { + ConfigurationPtr copy_configuration = configuration->clone(); + auto filter_dag = VirtualColumnUtils::createPathAndFileFilterDAG(predicate, virtual_columns); + if (filter_dag) + { + auto keys = configuration->getPaths(); + std::vector paths; + paths.reserve(keys.size()); + for (const auto & key : keys) + paths.push_back(fs::path(configuration->getNamespace()) / key); + VirtualColumnUtils::filterByPathOrFile(keys, paths, filter_dag, virtual_columns, local_context); + copy_configuration->setPaths(keys); + } + return std::make_shared( - object_storage, configuration, virtual_columns, read_keys, + object_storage, copy_configuration, virtual_columns, read_keys, settings.ignore_non_existent_file, file_progress_callback); } } @@ -247,6 +260,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade const auto max_parsing_threads = need_only_count ? std::optional(1) : std::nullopt; read_buf = createReadBuffer(object_info->relative_path, object_info->metadata->size_bytes); + LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII HEADER: {}", read_from_format_info.format_header.dumpStructure()); auto input_format = FormatFactory::instance().getInput( configuration->format, *read_buf, read_from_format_info.format_header, getContext(), max_block_size, format_settings, max_parsing_threads, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 3d4cc4fbd20..28962aadecd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -45,7 +45,7 @@ public: void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override { - setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.source_header); + setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); } Chunk generate() override; diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index d407017d5f7..9223642a7e6 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -65,9 +65,9 @@ std::vector TableFunctionObjectStorage< template void TableFunctionObjectStorage::updateStructureAndFormatArgumentsIfNeeded( - ASTs & args, const String & structure, const String & /* format */, const ContextPtr & context) + ASTs & args, const String & structure, const String & format, const ContextPtr & context) { - Configuration::addStructureToArgs(args, structure, context); + Configuration::addStructureAndFormatToArgs(args, structure, format, context); } template From 14c461338b12719daa1dc044148f914fd6a5fac6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 10 Apr 2024 12:56:29 +0200 Subject: [PATCH 027/158] Replay ZK logs using keeper-bench --- src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +- src/Common/ZooKeeper/ZooKeeperImpl.h | 3 +- utils/keeper-bench/CMakeLists.txt | 3 +- utils/keeper-bench/Generator.cpp | 194 +----- utils/keeper-bench/Generator.h | 18 - utils/keeper-bench/Runner.cpp | 821 ++++++++++++++++++++++++- utils/keeper-bench/Runner.h | 77 ++- utils/keeper-bench/main.cpp | 24 +- 8 files changed, 875 insertions(+), 269 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2185d32e47a..ed7498b1ac9 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1259,11 +1259,13 @@ void ZooKeeper::initFeatureFlags() void ZooKeeper::executeGenericRequest( const ZooKeeperRequestPtr & request, - ResponseCallback callback) + ResponseCallback callback, + WatchCallbackPtr watch) { RequestInfo request_info; request_info.request = request; request_info.callback = callback; + request_info.watch = watch; pushRequest(std::move(request_info)); } diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index cf331a03d06..8fdf0f97d9d 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -139,7 +139,8 @@ public: void executeGenericRequest( const ZooKeeperRequestPtr & request, - ResponseCallback callback); + ResponseCallback callback, + WatchCallbackPtr watch = nullptr); /// See the documentation about semantics of these methods in IKeeper class. diff --git a/utils/keeper-bench/CMakeLists.txt b/utils/keeper-bench/CMakeLists.txt index 5514c34f4ef..4fe0d852fd2 100644 --- a/utils/keeper-bench/CMakeLists.txt +++ b/utils/keeper-bench/CMakeLists.txt @@ -4,5 +4,4 @@ if (NOT TARGET ch_contrib::rapidjson) endif () clickhouse_add_executable(keeper-bench Generator.cpp Runner.cpp Stats.cpp main.cpp) -target_link_libraries(keeper-bench PRIVATE dbms) -target_link_libraries(keeper-bench PRIVATE ch_contrib::rapidjson) +target_link_libraries(keeper-bench PRIVATE dbms clickhouse_functions ch_contrib::rapidjson) diff --git a/utils/keeper-bench/Generator.cpp b/utils/keeper-bench/Generator.cpp index 2212f7158ae..cbf1bcdae23 100644 --- a/utils/keeper-bench/Generator.cpp +++ b/utils/keeper-bench/Generator.cpp @@ -40,54 +40,6 @@ std::string generateRandomString(size_t length) } } -void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & path) -{ - namespace fs = std::filesystem; - - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - Strings children; - auto list_callback = [promise, &children] (const ListResponse & response) - { - children = response.names; - - promise->set_value(); - }; - zookeeper.list(path, ListRequestType::ALL, list_callback, nullptr); - future.get(); - - while (!children.empty()) - { - Coordination::Requests ops; - for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) - { - removeRecursive(zookeeper, fs::path(path) / children.back()); - ops.emplace_back(makeRemoveRequest(fs::path(path) / children.back(), -1)); - children.pop_back(); - } - auto multi_promise = std::make_shared>(); - auto multi_future = multi_promise->get_future(); - - auto multi_callback = [multi_promise] (const MultiResponse &) - { - multi_promise->set_value(); - }; - zookeeper.multi(ops, multi_callback); - multi_future.get(); - } - auto remove_promise = std::make_shared>(); - auto remove_future = remove_promise->get_future(); - - auto remove_callback = [remove_promise] (const RemoveResponse &) - { - remove_promise->set_value(); - }; - - zookeeper.remove(path, -1, remove_callback); - remove_future.get(); -} - NumberGetter NumberGetter::fromConfig(const std::string & key, const Poco::Util::AbstractConfiguration & config, std::optional default_value) { @@ -603,148 +555,16 @@ Generator::Generator(const Poco::Util::AbstractConfiguration & config) acl.id = "anyone"; default_acls.emplace_back(std::move(acl)); - static const std::string generator_key = "generator"; - - std::cerr << "---- Parsing setup ---- " << std::endl; - static const std::string setup_key = generator_key + ".setup"; - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(setup_key, keys); - for (const auto & key : keys) - { - if (key.starts_with("node")) - { - auto node_key = setup_key + "." + key; - auto parsed_root_node = parseNode(node_key, config); - const auto node = root_nodes.emplace_back(parsed_root_node); - - if (config.has(node_key + ".repeat")) - { - if (!node->name.isRandom()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key); - - auto repeat_count = config.getUInt64(node_key + ".repeat"); - node->repeat_count = repeat_count; - for (size_t i = 1; i < repeat_count; ++i) - root_nodes.emplace_back(node->clone()); - } - - std::cerr << "Tree to create:" << std::endl; - - node->dumpTree(); - std::cerr << std::endl; - } - } - std::cerr << "---- Done parsing data setup ----\n" << std::endl; - std::cerr << "---- Collecting request generators ----" << std::endl; - static const std::string requests_key = generator_key + ".requests"; + static const std::string requests_key = "generator.requests"; request_getter = RequestGetter::fromConfig(requests_key, config); std::cerr << request_getter.description() << std::endl; std::cerr << "---- Done collecting request generators ----\n" << std::endl; } -std::shared_ptr Generator::parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config) -{ - auto node = std::make_shared(); - node->name = StringGetter::fromConfig(key + ".name", config); - - if (config.has(key + ".data")) - node->data = StringGetter::fromConfig(key + ".data", config); - - Poco::Util::AbstractConfiguration::Keys node_keys; - config.keys(key, node_keys); - - for (const auto & node_key : node_keys) - { - if (!node_key.starts_with("node")) - continue; - - const auto node_key_string = key + "." + node_key; - auto child_node = parseNode(node_key_string, config); - node->children.push_back(child_node); - - if (config.has(node_key_string + ".repeat")) - { - if (!child_node->name.isRandom()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key_string); - - auto repeat_count = config.getUInt64(node_key_string + ".repeat"); - child_node->repeat_count = repeat_count; - for (size_t i = 1; i < repeat_count; ++i) - node->children.push_back(child_node); - } - } - - return node; -} - -void Generator::Node::dumpTree(int level) const -{ - std::string data_string - = data.has_value() ? fmt::format("{}", data->description()) : "no data"; - - std::string repeat_count_string = repeat_count != 0 ? fmt::format(", repeated {} times", repeat_count) : ""; - - std::cerr << fmt::format("{}name: {}, data: {}{}", std::string(level, '\t'), name.description(), data_string, repeat_count_string) << std::endl; - - for (auto it = children.begin(); it != children.end();) - { - const auto & child = *it; - child->dumpTree(level + 1); - std::advance(it, child->repeat_count != 0 ? child->repeat_count : 1); - } -} - -std::shared_ptr Generator::Node::clone() const -{ - auto new_node = std::make_shared(); - new_node->name = name; - new_node->data = data; - new_node->repeat_count = repeat_count; - - // don't do deep copy of children because we will do clone only for root nodes - new_node->children = children; - - return new_node; -} - -void Generator::Node::createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const -{ - auto path = std::filesystem::path(parent_path) / name.getString(); - auto promise = std::make_shared>(); - auto future = promise->get_future(); - auto create_callback = [promise] (const CreateResponse & response) - { - if (response.error != Coordination::Error::ZOK) - promise->set_exception(std::make_exception_ptr(zkutil::KeeperException(response.error))); - else - promise->set_value(); - }; - zookeeper.create(path, data ? data->getString() : "", false, false, acls, create_callback); - future.get(); - - for (const auto & child : children) - child->createNode(zookeeper, path, acls); -} - void Generator::startup(Coordination::ZooKeeper & zookeeper) { - std::cerr << "---- Creating test data ----" << std::endl; - for (const auto & node : root_nodes) - { - auto node_name = node->name.getString(); - node->name.setString(node_name); - - std::string root_path = std::filesystem::path("/") / node_name; - std::cerr << "Cleaning up " << root_path << std::endl; - removeRecursive(zookeeper, root_path); - - node->createNode(zookeeper, "/", default_acls); - } - std::cerr << "---- Created test data ----\n" << std::endl; - std::cerr << "---- Initializing generators ----" << std::endl; - request_getter.startup(zookeeper); } @@ -752,15 +572,3 @@ Coordination::ZooKeeperRequestPtr Generator::generate() { return request_getter.getRequestGenerator()->generate(default_acls); } - -void Generator::cleanup(Coordination::ZooKeeper & zookeeper) -{ - std::cerr << "---- Cleaning up test data ----" << std::endl; - for (const auto & node : root_nodes) - { - auto node_name = node->name.getString(); - std::string root_path = std::filesystem::path("/") / node_name; - std::cerr << "Cleaning up " << root_path << std::endl; - removeRecursive(zookeeper, root_path); - } -} diff --git a/utils/keeper-bench/Generator.h b/utils/keeper-bench/Generator.h index 5b4c05b2d8b..35dce1a95d9 100644 --- a/utils/keeper-bench/Generator.h +++ b/utils/keeper-bench/Generator.h @@ -173,27 +173,9 @@ public: void startup(Coordination::ZooKeeper & zookeeper); Coordination::ZooKeeperRequestPtr generate(); - void cleanup(Coordination::ZooKeeper & zookeeper); private: - struct Node - { - StringGetter name; - std::optional data; - std::vector> children; - size_t repeat_count = 0; - - std::shared_ptr clone() const; - - void createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const; - void dumpTree(int level = 0) const; - }; - - static std::shared_ptr parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config); std::uniform_int_distribution request_picker; - std::vector> root_nodes; RequestGetter request_getter; Coordination::ACLs default_acls; }; - -std::optional getGenerator(const std::string & name); diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index a4b579f1f7b..8b111f5adb9 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1,14 +1,28 @@ #include "Runner.h" +#include +#include #include +#include "Common/ConcurrentBoundedQueue.h" +#include "Common/ZooKeeper/IKeeper.h" +#include "Common/ZooKeeper/ZooKeeperArgs.h" #include "Common/ZooKeeper/ZooKeeperCommon.h" #include "Common/ZooKeeper/ZooKeeperConstants.h" #include #include -#include "IO/ReadBufferFromString.h" +#include "Core/ColumnWithTypeAndName.h" +#include "Core/ColumnsWithTypeAndName.h" +#include "IO/ReadBuffer.h" +#include "IO/ReadBufferFromFile.h" +#include "base/Decimal.h" +#include "base/types.h" +#include #include #include #include +#include +#include +#include namespace CurrentMetrics @@ -22,23 +36,41 @@ namespace DB::ErrorCodes { extern const int CANNOT_BLOCK_SIGNAL; extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } Runner::Runner( std::optional concurrency_, const std::string & config_path, + const std::string & input_request_log_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, std::optional continue_on_error_, std::optional max_iterations_) - : info(std::make_shared()) + : input_request_log(input_request_log_) + , info(std::make_shared()) { DB::ConfigProcessor config_processor(config_path, true, false); - auto config = config_processor.loadConfig().configuration; + DB::ConfigurationPtr config = nullptr; + + if (!config_path.empty()) + { + config = config_processor.loadConfig().configuration; + + if (config->has("generator")) + generator.emplace(*config); + } + else + { + if (input_request_log.empty()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Both --config and --input_request_log cannot be empty"); + + if (!std::filesystem::exists(input_request_log)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "File on path {} does not exist", input_request_log); + } - generator.emplace(*config); if (!hosts_strings_.empty()) { @@ -57,6 +89,8 @@ Runner::Runner( static constexpr uint64_t DEFAULT_CONCURRENCY = 1; if (concurrency_) concurrency = *concurrency_; + else if (!config) + concurrency = DEFAULT_CONCURRENCY; else concurrency = config->getUInt64("concurrency", DEFAULT_CONCURRENCY); std::cerr << "Concurrency: " << concurrency << std::endl; @@ -64,6 +98,8 @@ Runner::Runner( static constexpr uint64_t DEFAULT_ITERATIONS = 0; if (max_iterations_) max_iterations = *max_iterations_; + else if (!config) + max_iterations = DEFAULT_ITERATIONS; else max_iterations = config->getUInt64("iterations", DEFAULT_ITERATIONS); std::cerr << "Iterations: " << max_iterations << std::endl; @@ -71,6 +107,8 @@ Runner::Runner( static constexpr double DEFAULT_DELAY = 1.0; if (delay_) delay = *delay_; + else if (!config) + delay = DEFAULT_DELAY; else delay = config->getDouble("report_delay", DEFAULT_DELAY); std::cerr << "Report delay: " << delay << std::endl; @@ -78,44 +116,48 @@ Runner::Runner( static constexpr double DEFAULT_TIME_LIMIT = 0.0; if (max_time_) max_time = *max_time_; + else if (!config) + max_time = DEFAULT_TIME_LIMIT; else max_time = config->getDouble("timelimit", DEFAULT_TIME_LIMIT); std::cerr << "Time limit: " << max_time << std::endl; if (continue_on_error_) continue_on_error = *continue_on_error_; + else if (!config) + continue_on_error_ = false; else continue_on_error = config->getBool("continue_on_error", false); std::cerr << "Continue on error: " << continue_on_error << std::endl; - static const std::string output_key = "output"; - print_to_stdout = config->getBool(output_key + ".stdout", false); - std::cerr << "Printing output to stdout: " << print_to_stdout << std::endl; - - static const std::string output_file_key = output_key + ".file"; - if (config->has(output_file_key)) + if (config) { - if (config->has(output_file_key + ".path")) - { - file_output = config->getString(output_file_key + ".path"); - output_file_with_timestamp = config->getBool(output_file_key + ".with_timestamp"); - } - else - file_output = config->getString(output_file_key); + benchmark_context.initializeFromConfig(*config); - std::cerr << "Result file path: " << file_output->string() << std::endl; + static const std::string output_key = "output"; + print_to_stdout = config->getBool(output_key + ".stdout", false); + std::cerr << "Printing output to stdout: " << print_to_stdout << std::endl; + + static const std::string output_file_key = output_key + ".file"; + if (config->has(output_file_key)) + { + if (config->has(output_file_key + ".path")) + { + file_output = config->getString(output_file_key + ".path"); + output_file_with_timestamp = config->getBool(output_file_key + ".with_timestamp"); + } + else + file_output = config->getString(output_file_key); + + std::cerr << "Result file path: " << file_output->string() << std::endl; + } } std::cerr << "---- Run options ----\n" << std::endl; - - pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); - queue.emplace(concurrency); } void Runner::parseHostsFromConfig(const Poco::Util::AbstractConfiguration & config) { - ConnectionInfo default_connection_info; - const auto fill_connection_details = [&](const std::string & key, auto & connection_info) { if (config.has(key + ".secure")) @@ -328,9 +370,519 @@ bool Runner::tryPushRequestInteractively(Coordination::ZooKeeperRequestPtr && re void Runner::runBenchmark() { + if (generator) + runBenchmarkWithGenerator(); + else + runBenchmarkFromLog(); +} + + +struct ZooKeeperRequestBlock +{ + explicit ZooKeeperRequestBlock(DB::Block block_) + : block(std::move(block_)) + , hostname_idx(block.getPositionByName("hostname")) // + , request_event_time_idx(block.getPositionByName("request_event_time")) // + , thread_id_idx(block.getPositionByName("thread_id")) // + , session_id_idx(block.getPositionByName("session_id")) // + , xid_idx(block.getPositionByName("xid")) // + , has_watch_idx(block.getPositionByName("has_watch")) + , op_num_idx(block.getPositionByName("op_num")) + , path_idx(block.getPositionByName("path")) + , data_idx(block.getPositionByName("data")) + , is_ephemeral_idx(block.getPositionByName("is_ephemeral")) + , is_sequential_idx(block.getPositionByName("is_sequential")) + , response_event_time_idx(block.getPositionByName("response_event_time")) // + , error_idx(block.getPositionByName("error")) + , requests_size_idx(block.getPositionByName("requests_size")) + , version_idx(block.getPositionByName("version")) + {} + + size_t rows() const + { + return block.rows(); + } + + UInt64 getExecutorId(size_t row) const + { + return getSessionId(row); + } + + std::string getHostname(size_t row) const + { + return getField(hostname_idx, row).safeGet(); + } + + UInt64 getThreadId(size_t row) const + { + return getField(thread_id_idx, row).safeGet(); + } + + DB::DateTime64 getRequestEventTime(size_t row) const + { + return getField(request_event_time_idx, row).safeGet(); + } + + DB::DateTime64 getResponseEventTime(size_t row) const + { + return getField(response_event_time_idx, row).safeGet(); + } + + Int64 getSessionId(size_t row) const + { + return getField(session_id_idx, row).safeGet(); + } + + Int64 getXid(size_t row) const + { + return getField(xid_idx, row).safeGet(); + } + + bool hasWatch(size_t row) const + { + return getField(has_watch_idx, row).safeGet(); + } + + Coordination::OpNum getOpNum(size_t row) const + { + return static_cast(getField(op_num_idx, row).safeGet()); + } + + bool isEphemeral(size_t row) const + { + return getField(is_ephemeral_idx, row).safeGet(); + } + + bool isSequential(size_t row) const + { + return getField(is_sequential_idx, row).safeGet(); + } + + std::string getPath(size_t row) const + { + return getField(path_idx, row).safeGet(); + } + + std::string getData(size_t row) const + { + return getField(data_idx, row).safeGet(); + } + + UInt64 getRequestsSize(size_t row) const + { + return getField(requests_size_idx, row).safeGet(); + } + + std::optional getVersion(size_t row) const + { + auto field = getField(version_idx, row); + if (field.isNull()) + return std::nullopt; + return static_cast(field.safeGet()); + } + + std::optional getError(size_t row) const + { + auto field = getField(error_idx, row); + if (field.isNull()) + return std::nullopt; + + return static_cast(field.safeGet()); + } +private: + DB::Field getField(size_t position, size_t row) const + { + DB::Field field; + block.getByPosition(position).column->get(row, field); + return field; + } + + DB::Block block; + size_t hostname_idx = 0; + size_t request_event_time_idx = 0; + size_t thread_id_idx = 0; + size_t session_id_idx = 0; + size_t xid_idx = 0; + size_t has_watch_idx = 0; + size_t op_num_idx = 0; + size_t path_idx = 0; + size_t data_idx = 0; + size_t is_ephemeral_idx = 0; + size_t is_sequential_idx = 0; + size_t response_event_time_idx = 0; + size_t error_idx = 0; + size_t requests_size_idx = 0; + size_t version_idx = 0; +}; + +struct RequestFromLog +{ + Coordination::ZooKeeperRequestPtr request; + std::optional expected_result; + int64_t session_id = 0; + size_t executor_id = 0; + bool has_watch = false; + DB::DateTime64 request_event_time; + DB::DateTime64 response_event_time; + std::shared_ptr connection; +}; + +struct ZooKeeperRequestFromLogReader +{ + ZooKeeperRequestFromLogReader(const std::string & input_request_log, DB::ContextPtr context) + { + std::optional format_settings; + + file_read_buf = std::make_unique(input_request_log); + auto compression_method = DB::chooseCompressionMethod(input_request_log, ""); + file_read_buf = DB::wrapReadBufferWithCompressionMethod(std::move(file_read_buf), compression_method); + + DB::SingleReadBufferIterator read_buffer_iterator(std::move(file_read_buf)); + auto [columns_description, format] = DB::detectFormatAndReadSchema(format_settings, read_buffer_iterator, context); + + DB::ColumnsWithTypeAndName columns; + columns.reserve(columns_description.size()); + + for (const auto & column_description : columns_description) + columns.push_back(DB::ColumnWithTypeAndName{column_description.type, column_description.name}); + + header_block = std::move(columns); + + file_read_buf + = DB::wrapReadBufferWithCompressionMethod(std::make_unique(input_request_log), compression_method); + + input_format = DB::FormatFactory::instance().getInput( + format, + *file_read_buf, + header_block, + context, + context->getSettingsRef().max_block_size, + format_settings, + 1, + std::nullopt, + /*is_remote_fs*/ false, + DB::CompressionMethod::None, + false); + + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + default_acls.emplace_back(std::move(acl)); + } + + std::optional getNextRequest(bool for_multi = false) + { + RequestFromLog request_from_log; + + if (!current_block) + { + auto chunk = input_format->generate(); + + if (chunk.empty()) + return std::nullopt; + + current_block.emplace(header_block.cloneWithColumns(chunk.detachColumns())); + idx_in_block = 0; + } + + + request_from_log.expected_result = current_block->getError(idx_in_block); + request_from_log.session_id = current_block->getSessionId(idx_in_block); + request_from_log.has_watch = current_block->hasWatch(idx_in_block); + request_from_log.executor_id = current_block->getExecutorId(idx_in_block); + request_from_log.request_event_time = current_block->getRequestEventTime(idx_in_block); + request_from_log.response_event_time = current_block->getResponseEventTime(idx_in_block); + + const auto move_row_iterator = [&] + { + if (idx_in_block == current_block->rows() - 1) + current_block.reset(); + else + ++idx_in_block; + }; + + auto op_num = current_block->getOpNum(idx_in_block); + switch (op_num) + { + case Coordination::OpNum::Create: + { + auto create_request = std::make_shared(); + create_request->path = current_block->getPath(idx_in_block); + create_request->data = current_block->getData(idx_in_block); + create_request->is_ephemeral = current_block->isEphemeral(idx_in_block); + create_request->is_sequential = current_block->isSequential(idx_in_block); + request_from_log.request = create_request; + break; + } + case Coordination::OpNum::Set: + { + auto set_request = std::make_shared(); + set_request->path = current_block->getPath(idx_in_block); + set_request->data = current_block->getData(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + set_request->version = *version; + request_from_log.request = set_request; + break; + } + case Coordination::OpNum::Remove: + { + auto remove_request = std::make_shared(); + remove_request->path = current_block->getPath(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + remove_request->version = *version; + request_from_log.request = remove_request; + break; + } + case Coordination::OpNum::Check: + { + auto check_request = std::make_shared(); + check_request->path = current_block->getPath(idx_in_block); + if (auto version = current_block->getVersion(idx_in_block)) + check_request->version = *version; + request_from_log.request = check_request; + break; + } + case Coordination::OpNum::Sync: + { + auto sync_request = std::make_shared(); + sync_request->path = current_block->getPath(idx_in_block); + request_from_log.request = sync_request; + break; + } + case Coordination::OpNum::Get: + { + auto get_request = std::make_shared(); + get_request->path = current_block->getPath(idx_in_block); + request_from_log.request = get_request; + break; + } + case Coordination::OpNum::SimpleList: + case Coordination::OpNum::FilteredList: + { + auto list_request = std::make_shared(); + list_request->path = current_block->getPath(idx_in_block); + request_from_log.request = list_request; + break; + } + case Coordination::OpNum::Exists: + { + auto exists_request = std::make_shared(); + exists_request->path = current_block->getPath(idx_in_block); + request_from_log.request = exists_request; + break; + } + case Coordination::OpNum::Multi: + case Coordination::OpNum::MultiRead: + { + if (for_multi) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Nested multi requests are not allowed"); + + auto requests_size = current_block->getRequestsSize(idx_in_block); + + Coordination::Requests requests; + requests.reserve(requests_size); + move_row_iterator(); + + for (size_t i = 0; i < requests_size; ++i) + { + auto subrequest_from_log = getNextRequest(/*for_multi=*/true); + if (!subrequest_from_log) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Failed to fetch subrequest for {}, subrequest index {}", op_num, i); + + requests.push_back(std::move(subrequest_from_log->request)); + + if (subrequest_from_log->session_id != request_from_log.session_id) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Session id mismatch for subrequest in {}, subrequest index {}", op_num, i); + + if (subrequest_from_log->executor_id != request_from_log.executor_id) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Executor id mismatch for subrequest in {}, subrequest index {}", op_num, i); + } + + request_from_log.request = std::make_shared(requests, default_acls); + + return request_from_log; + } + default: + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported operation {} ({})", op_num, static_cast(op_num)); + } + + move_row_iterator(); + + return request_from_log; + } + +private: + DB::Block header_block; + + std::unique_ptr file_read_buf; + DB::InputFormatPtr input_format; + + std::optional current_block; + size_t idx_in_block = 0; + + Coordination::ACLs default_acls; +}; + + +namespace +{ + + +struct RequestFromLogStats +{ + struct Stats + { + std::atomic total = 0; + std::atomic unexpected_results = 0; + }; + + Stats write_requests; + Stats read_requests; +}; + +void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_for_type) +{ + std::cerr << fmt::format( + "{} requests: {} total, {} with unexpected results ({:.4}%)", + type, + stats_for_type.total, + stats_for_type.unexpected_results, + static_cast(stats_for_type.unexpected_results) / stats_for_type.total * 100) + << std::endl; +}; + +void requestFromLogExecutor(std::shared_ptr> queue, RequestFromLogStats & request_stats) +{ + RequestFromLog request_from_log; + std::optional> last_request; + while (queue->pop(request_from_log)) + { + auto request_promise = std::make_shared>(); + last_request = request_promise->get_future(); + Coordination::ResponseCallback callback + = [&, request_promise, request = request_from_log.request, expected_result = request_from_log.expected_result]( + const Coordination::Response & response) mutable + { + auto & stats = request->isReadRequest() ? request_stats.read_requests : request_stats.write_requests; + + stats.total.fetch_add(1, std::memory_order_relaxed); + + if (*expected_result != response.error) + stats.unexpected_results.fetch_add(1, std::memory_order_relaxed); + + //if (!expected_result) + // return; + + //if (*expected_result != response.error) + // std::cerr << fmt::format( + // "Unexpected result for {}, got {}, expected {}", request->getOpNum(), response.error, *expected_result) + // << std::endl; + + request_promise->set_value(); + }; + + Coordination::WatchCallbackPtr watch; + if (request_from_log.has_watch) + watch = std::make_shared([](const Coordination::WatchResponse &) {}); + + request_from_log.connection->executeGenericRequest(request_from_log.request, callback, watch); + } + + if (last_request) + last_request->wait(); +} + +} + +void Runner::runBenchmarkFromLog() +{ + std::cerr << fmt::format("Running benchmark using requests from {}", input_request_log) << std::endl; + + pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); + + shared_context = DB::Context::createShared(); + global_context = DB::Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + DB::registerFormats(); + + /// Randomly choosing connection index + pcg64 rng(randomSeed()); + std::uniform_int_distribution connection_distribution(0, connection_infos.size() - 1); + + std::unordered_map> zookeeper_connections; + auto get_zookeeper_connection = [&](int64_t session_id) + { + if (auto it = zookeeper_connections.find(session_id); it != zookeeper_connections.end() && !it->second->isExpired()) + return it->second; + + auto connection_idx = connection_distribution(rng); + auto zk_connection = getConnection(connection_infos[connection_idx], connection_idx); + zookeeper_connections.insert_or_assign(session_id, zk_connection); + return zk_connection; + }; + + RequestFromLogStats stats; + + + std::unordered_map>> executor_id_to_queue; + + SCOPE_EXIT({ + for (const auto & [executor_id, executor_queue] : executor_id_to_queue) + executor_queue->finish(); + + pool->wait(); + + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + }); + + auto push_request = [&](RequestFromLog request) + { + if (auto it = executor_id_to_queue.find(request.executor_id); it != executor_id_to_queue.end()) + { + auto success = it->second->push(std::move(request)); + if (!success) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to push to the executor's queue"); + return; + } + + auto executor_queue = std::make_shared>(std::numeric_limits().max()); + executor_id_to_queue.emplace(request.executor_id, executor_queue); + auto scheduled = pool->trySchedule([&, executor_queue]() mutable + { + requestFromLogExecutor(std::move(executor_queue), stats); + }); + + if (!scheduled) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Failed to schedule worker, try to increase concurrency parameter"); + + auto success = executor_queue->push(std::move(request)); + if (!success) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to push to the executor's queue"); + }; + + { + auto setup_connection = getConnection(connection_infos[0], 0); + benchmark_context.startup(*setup_connection); + } + + ZooKeeperRequestFromLogReader request_reader(input_request_log, global_context); + while (auto request_from_log = request_reader.getNextRequest()) + { + request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); + push_request(std::move(*request_from_log)); + } +} + +void Runner::runBenchmarkWithGenerator() +{ + pool.emplace(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, concurrency); + queue.emplace(concurrency); createConnections(); std::cerr << "Preparing to run\n"; + benchmark_context.startup(*connections[0]); generator->startup(*connections[0]); std::cerr << "Prepared\n"; @@ -458,8 +1010,225 @@ std::vector> Runner::refreshConnections Runner::~Runner() { - queue->clearAndFinish(); + if (queue) + queue->clearAndFinish(); shutdown = true; - pool->wait(); - generator->cleanup(*connections[0]); + + if (pool) + pool->wait(); + + auto connection = getConnection(connection_infos[0], 0); + benchmark_context.cleanup(*connection); +} + +namespace +{ + +void removeRecursive(Coordination::ZooKeeper & zookeeper, const std::string & path) +{ + namespace fs = std::filesystem; + + auto promise = std::make_shared>(); + auto future = promise->get_future(); + + Strings children; + auto list_callback = [promise, &children] (const Coordination::ListResponse & response) + { + children = response.names; + promise->set_value(); + }; + zookeeper.list(path, Coordination::ListRequestType::ALL, list_callback, nullptr); + future.get(); + + std::span children_span(children); + while (!children_span.empty()) + { + Coordination::Requests ops; + for (size_t i = 0; i < 1000 && !children.empty(); ++i) + { + removeRecursive(zookeeper, fs::path(path) / children.back()); + ops.emplace_back(zkutil::makeRemoveRequest(fs::path(path) / children_span.back(), -1)); + children_span = children_span.subspan(0, children_span.size() - 1); + } + auto multi_promise = std::make_shared>(); + auto multi_future = multi_promise->get_future(); + + auto multi_callback = [multi_promise] (const Coordination::MultiResponse &) + { + multi_promise->set_value(); + }; + zookeeper.multi(ops, multi_callback); + multi_future.get(); + } + auto remove_promise = std::make_shared>(); + auto remove_future = remove_promise->get_future(); + + auto remove_callback = [remove_promise] (const Coordination::RemoveResponse &) + { + remove_promise->set_value(); + }; + + zookeeper.remove(path, -1, remove_callback); + remove_future.get(); +} + +} + +void BenchmarkContext::initializeFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + default_acls.emplace_back(std::move(acl)); + + std::cerr << "---- Parsing setup ---- " << std::endl; + static const std::string setup_key = "setup"; + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(setup_key, keys); + for (const auto & key : keys) + { + if (key.starts_with("node")) + { + auto node_key = setup_key + "." + key; + auto parsed_root_node = parseNode(node_key, config); + const auto node = root_nodes.emplace_back(parsed_root_node); + + if (config.has(node_key + ".repeat")) + { + if (!node->name.isRandom()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key); + + auto repeat_count = config.getUInt64(node_key + ".repeat"); + node->repeat_count = repeat_count; + for (size_t i = 1; i < repeat_count; ++i) + root_nodes.emplace_back(node->clone()); + } + + std::cerr << "Tree to create:" << std::endl; + + node->dumpTree(); + std::cerr << std::endl; + } + } + std::cerr << "---- Done parsing data setup ----\n" << std::endl; +} + +std::shared_ptr BenchmarkContext::parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config) +{ + auto node = std::make_shared(); + node->name = StringGetter::fromConfig(key + ".name", config); + + if (config.has(key + ".data")) + node->data = StringGetter::fromConfig(key + ".data", config); + + Poco::Util::AbstractConfiguration::Keys node_keys; + config.keys(key, node_keys); + + for (const auto & node_key : node_keys) + { + if (!node_key.starts_with("node")) + continue; + + const auto node_key_string = key + "." + node_key; + auto child_node = parseNode(node_key_string, config); + node->children.push_back(child_node); + + if (config.has(node_key_string + ".repeat")) + { + if (!child_node->name.isRandom()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Repeating node creation for key {}, but name is not randomly generated", node_key_string); + + auto repeat_count = config.getUInt64(node_key_string + ".repeat"); + child_node->repeat_count = repeat_count; + for (size_t i = 1; i < repeat_count; ++i) + node->children.push_back(child_node); + } + } + + return node; +} + +void BenchmarkContext::Node::dumpTree(int level) const +{ + std::string data_string + = data.has_value() ? fmt::format("{}", data->description()) : "no data"; + + std::string repeat_count_string = repeat_count != 0 ? fmt::format(", repeated {} times", repeat_count) : ""; + + std::cerr << fmt::format("{}name: {}, data: {}{}", std::string(level, '\t'), name.description(), data_string, repeat_count_string) << std::endl; + + for (auto it = children.begin(); it != children.end();) + { + const auto & child = *it; + child->dumpTree(level + 1); + std::advance(it, child->repeat_count != 0 ? child->repeat_count : 1); + } +} + +std::shared_ptr BenchmarkContext::Node::clone() const +{ + auto new_node = std::make_shared(); + new_node->name = name; + new_node->data = data; + new_node->repeat_count = repeat_count; + + // don't do deep copy of children because we will do clone only for root nodes + new_node->children = children; + + return new_node; +} + +void BenchmarkContext::Node::createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const +{ + auto path = std::filesystem::path(parent_path) / name.getString(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); + auto create_callback = [promise] (const Coordination::CreateResponse & response) + { + if (response.error != Coordination::Error::ZOK) + promise->set_exception(std::make_exception_ptr(zkutil::KeeperException(response.error))); + else + promise->set_value(); + }; + zookeeper.create(path, data ? data->getString() : "", false, false, acls, create_callback); + future.get(); + + for (const auto & child : children) + child->createNode(zookeeper, path, acls); +} + +void BenchmarkContext::startup(Coordination::ZooKeeper & zookeeper) +{ + if (root_nodes.empty()) + return; + + std::cerr << "---- Creating test data ----" << std::endl; + for (const auto & node : root_nodes) + { + auto node_name = node->name.getString(); + node->name.setString(node_name); + + std::string root_path = std::filesystem::path("/") / node_name; + std::cerr << "Cleaning up " << root_path << std::endl; + removeRecursive(zookeeper, root_path); + + node->createNode(zookeeper, "/", default_acls); + } + std::cerr << "---- Created test data ----\n" << std::endl; +} + +void BenchmarkContext::cleanup(Coordination::ZooKeeper & zookeeper) +{ + if (root_nodes.empty()) + return; + + std::cerr << "---- Cleaning up test data ----" << std::endl; + for (const auto & node : root_nodes) + { + auto node_name = node->name.getString(); + std::string root_path = std::filesystem::path("/") / node_name; + std::cerr << "Cleaning up " << root_path << std::endl; + removeRecursive(zookeeper, root_path); + } } diff --git a/utils/keeper-bench/Runner.h b/utils/keeper-bench/Runner.h index 4f4a75e6ecf..0c646eb2166 100644 --- a/utils/keeper-bench/Runner.h +++ b/utils/keeper-bench/Runner.h @@ -1,5 +1,5 @@ #pragma once -#include "Common/ZooKeeper/ZooKeeperConstants.h" +#include "Common/ZooKeeper/ZooKeeperArgs.h" #include #include "Generator.h" #include @@ -12,6 +12,7 @@ #include #include +#include "Interpreters/Context.h" #include "Stats.h" #include @@ -19,12 +20,40 @@ using Ports = std::vector; using Strings = std::vector; +struct BenchmarkContext +{ +public: + void initializeFromConfig(const Poco::Util::AbstractConfiguration & config); + + void startup(Coordination::ZooKeeper & zookeeper); + void cleanup(Coordination::ZooKeeper & zookeeper); +private: + struct Node + { + StringGetter name; + std::optional data; + std::vector> children; + size_t repeat_count = 0; + + std::shared_ptr clone() const; + + void createNode(Coordination::ZooKeeper & zookeeper, const std::string & parent_path, const Coordination::ACLs & acls) const; + void dumpTree(int level = 0) const; + }; + + static std::shared_ptr parseNode(const std::string & key, const Poco::Util::AbstractConfiguration & config); + + std::vector> root_nodes; + Coordination::ACLs default_acls; +}; + class Runner { public: Runner( std::optional concurrency_, const std::string & config_path, + const std::string & input_request_log_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, @@ -44,8 +73,30 @@ public: ~Runner(); private: + struct ConnectionInfo + { + std::string host; + + bool secure = false; + int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; + int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; + int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; + bool use_compression = false; + + size_t sessions = 1; + }; + void parseHostsFromConfig(const Poco::Util::AbstractConfiguration & config); + void runBenchmarkWithGenerator(); + void runBenchmarkFromLog(); + + void createConnections(); + std::vector> refreshConnections(); + std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); + + std::string input_request_log; + size_t concurrency = 1; std::optional pool; @@ -54,7 +105,8 @@ private: double max_time = 0; double delay = 1; bool continue_on_error = false; - std::atomic max_iterations = 0; + size_t max_iterations = 0; + std::atomic requests_executed = 0; std::atomic shutdown = false; @@ -71,25 +123,14 @@ private: using Queue = ConcurrentBoundedQueue; std::optional queue; - struct ConnectionInfo - { - std::string host; - - bool secure = false; - int32_t session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; - int32_t connection_timeout_ms = Coordination::DEFAULT_CONNECTION_TIMEOUT_MS; - int32_t operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; - bool use_compression = false; - - size_t sessions = 1; - }; - std::mutex connection_mutex; + ConnectionInfo default_connection_info; std::vector connection_infos; std::vector> connections; std::unordered_map connections_to_info_map; - void createConnections(); - std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); - std::vector> refreshConnections(); + DB::SharedContextHolder shared_context; + DB::ContextMutablePtr global_context; + + BenchmarkContext benchmark_context; }; diff --git a/utils/keeper-bench/main.cpp b/utils/keeper-bench/main.cpp index 0753d66850f..45fc28f3bca 100644 --- a/utils/keeper-bench/main.cpp +++ b/utils/keeper-bench/main.cpp @@ -1,8 +1,6 @@ #include #include #include "Runner.h" -#include "Stats.h" -#include "Generator.h" #include "Common/Exception.h" #include #include @@ -27,6 +25,10 @@ int main(int argc, char *argv[]) bool print_stacktrace = true; + //Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + //Poco::Logger::root().setChannel(channel); + //Poco::Logger::root().setLevel("trace"); + try { using boost::program_options::value; @@ -34,12 +36,13 @@ int main(int argc, char *argv[]) boost::program_options::options_description desc = createOptionsDescription("Allowed options", getTerminalWidth()); desc.add_options() ("help", "produce help message") - ("config", value()->default_value(""), "yaml/xml file containing configuration") - ("concurrency,c", value(), "number of parallel queries") - ("report-delay,d", value(), "delay between intermediate reports in seconds (set 0 to disable reports)") - ("iterations,i", value(), "amount of queries to be executed") - ("time-limit,t", value(), "stop launch of queries after specified time limit") - ("hosts,h", value()->multitoken()->default_value(Strings{}, ""), "") + ("config", value()->default_value(""), "yaml/xml file containing configuration") + ("input-request-log", value()->default_value(""), "log of requests that will be replayed") + ("concurrency,c", value(), "number of parallel queries") + ("report-delay,d", value(), "delay between intermediate reports in seconds (set 0 to disable reports)") + ("iterations,i", value(), "amount of queries to be executed") + ("time-limit,t", value(), "stop launch of queries after specified time limit") + ("hosts,h", value()->multitoken()->default_value(Strings{}, ""), "") ("continue_on_errors", "continue testing even if a query fails") ; @@ -56,6 +59,7 @@ int main(int argc, char *argv[]) Runner runner(valueToOptional(options["concurrency"]), options["config"].as(), + options["input-request-log"].as(), options["hosts"].as(), valueToOptional(options["time-limit"]), valueToOptional(options["report-delay"]), @@ -66,9 +70,9 @@ int main(int argc, char *argv[]) { runner.runBenchmark(); } - catch (const DB::Exception & e) + catch (...) { - std::cout << "Got exception while trying to run benchmark: " << e.message() << std::endl; + std::cout << "Got exception while trying to run benchmark: " << DB::getCurrentExceptionMessage(true) << std::endl; } return 0; From 652796acd6a10515e862260d18e002bae27f3c85 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 15 Apr 2024 16:37:38 +0100 Subject: [PATCH 028/158] Fix MergeTree with HDFS --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 38 +++++++++++++++---- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 16 ++++++-- 2 files changed, 42 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 8bfba6fcfad..82c9a6c6c21 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -34,15 +34,21 @@ void HDFSObjectStorage::startup() ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { /// what ever data_source_description.description value is, consider that key as relative key - return ObjectStorageKey::createAsRelative(hdfs_root_path, getRandomASCIIString(32)); + chassert(data_directory.starts_with("/")); + return ObjectStorageKey::createAsRelative( + fs::path(url_without_path) / data_directory.substr(1), getRandomASCIIString(32)); } bool HDFSObjectStorage::exists(const StoredObject & object) const { + std::string path = object.remote_path; + if (path.starts_with(url_without_path)) + path = path.substr(url_without_path.size()); + // const auto & path = object.remote_path; // const size_t begin_of_path = path.find('/', path.find("//") + 2); // const String remote_fs_object_path = path.substr(begin_of_path); - return (0 == hdfsExists(hdfs_fs.get(), object.remote_path.c_str())); + return (0 == hdfsExists(hdfs_fs.get(), path.c_str())); } std::unique_ptr HDFSObjectStorage::readObject( /// NOLINT @@ -51,7 +57,14 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLIN std::optional, std::optional) const { - return std::make_unique(hdfs_root_path, object.remote_path, config, patchSettings(read_settings)); + std::string path = object.remote_path; + if (path.starts_with(url)) + path = path.substr(url.size()); + if (path.starts_with("/")) + path.substr(1); + + return std::make_unique( + fs::path(url_without_path) / "", fs::path(data_directory) / path, config, patchSettings(read_settings)); } std::unique_ptr HDFSObjectStorage::readObjects( /// NOLINT @@ -69,8 +82,13 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI // auto hdfs_path = path.substr(begin_of_path); // auto hdfs_uri = path.substr(0, begin_of_path); + std::string path = object_.remote_path; + if (path.starts_with(url)) + path = path.substr(url.size()); + if (path.starts_with("/")) + path.substr(1); return std::make_unique( - hdfs_root_path, object_.remote_path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); + fs::path(url_without_path) / "", fs::path(data_directory) / path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; return std::make_unique( @@ -89,8 +107,11 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects"); - auto path = object.remote_path.starts_with('/') ? object.remote_path.substr(1) : object.remote_path; - path = fs::path(hdfs_root_path) / path; + std::string path = object.remote_path; + if (path.starts_with("/")) + path = path.substr(1); + if (!path.starts_with(url)) + path = fs::path(url) / path; /// Single O_WRONLY in libhdfs adds O_TRUNC return std::make_unique( @@ -102,8 +123,9 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL /// Remove file. Throws exception if file doesn't exists or it's a directory. void HDFSObjectStorage::removeObject(const StoredObject & object) { - const auto & path = object.remote_path; - // const size_t begin_of_path = path.find('/', path.find("//") + 2); + auto path = object.remote_path; + if (path.starts_with(url_without_path)) + path = path.substr(url_without_path.size()); /// Add path from root to file name int res = hdfsDelete(hdfs_fs.get(), path.c_str(), 0); diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 24642ec635a..8987fa5eaf1 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -40,15 +40,21 @@ public: , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) - , hdfs_root_path(hdfs_root_path_) { + const size_t begin_of_path = hdfs_root_path_.find('/', hdfs_root_path_.find("//") + 2); + url = hdfs_root_path_; + url_without_path = url.substr(0, begin_of_path); + if (begin_of_path < url.size()) + data_directory = url.substr(begin_of_path); + else + data_directory = "/"; } std::string getName() const override { return "HDFSObjectStorage"; } - std::string getCommonKeyPrefix() const override { return hdfs_root_path; } + std::string getCommonKeyPrefix() const override { return url; } - std::string getDescription() const override { return hdfs_root_path; } + std::string getDescription() const override { return url; } ObjectStorageType getType() const override { return ObjectStorageType::HDFS; } @@ -116,7 +122,9 @@ private: HDFSBuilderWrapper hdfs_builder; HDFSFSPtr hdfs_fs; SettingsPtr settings; - const std::string hdfs_root_path; + std::string url; + std::string url_without_path; + std::string data_directory; }; } From ccee2d668793370c3f947a4be24d1edbabba1724 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 15 Apr 2024 23:28:14 +0100 Subject: [PATCH 029/158] Fix parsing --- src/Storages/ObjectStorage/HDFS/Configuration.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 220857fead6..e12c2f15b28 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -63,9 +63,6 @@ std::string StorageHDFSConfiguration::getPathWithoutGlob() const void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) { - std::string url_str; - url_str = checkAndGetLiteralArgument(args[0], "url"); - const size_t max_args_num = with_structure ? 4 : 3; if (!args.size() || args.size() > max_args_num) { @@ -73,6 +70,9 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit "Expected not more than {} arguments", max_args_num); } + std::string url_str; + url_str = checkAndGetLiteralArgument(args[0], "url"); + if (args.size() > 1) { args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], context); From 11be538ac870d231a13a2648038ea1b469f73a08 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 16 Apr 2024 10:20:56 +0100 Subject: [PATCH 030/158] Fix several tests --- src/Disks/ObjectStorages/S3/diskSettings.cpp | 8 +++++-- src/Disks/ObjectStorages/S3/diskSettings.h | 3 ++- .../ObjectStorage/AzureBlob/Configuration.cpp | 7 +++--- .../ObjectStorage/HDFS/Configuration.cpp | 2 +- .../ObjectStorage/S3/Configuration.cpp | 2 +- .../StorageObjectStorageSink.cpp | 3 +-- src/Storages/S3Queue/S3QueueSource.cpp | 14 ++++++++++++ src/Storages/S3Queue/S3QueueSource.h | 1 + src/Storages/StorageS3Settings.cpp | 22 +++++++++++-------- src/Storages/StorageS3Settings.h | 10 +++++---- 10 files changed, 49 insertions(+), 23 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 9bd4bf699e8..2bca7df7db9 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -36,11 +36,15 @@ extern const int NO_ELEMENTS_IN_CONFIG; } std::unique_ptr getSettings( - const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + bool validate_settings) { const Settings & settings = context->getSettingsRef(); - auto request_settings = S3Settings::RequestSettings(config, config_prefix, settings, "s3_"); + auto request_settings = S3Settings::RequestSettings(config, config_prefix, settings, "s3_", validate_settings); auto auth_settings = S3::AuthSettings::loadFromConfig(config_prefix, config); + return std::make_unique( request_settings, auth_settings, diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 5b655f35508..11ac64ce913 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -17,7 +17,8 @@ struct S3ObjectStorageSettings; std::unique_ptr getSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, - ContextPtr context); + ContextPtr context, + bool validate_settings = true); std::unique_ptr getClient( const Poco::Util::AbstractConfiguration & config, diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index fe01251e58a..44ace9c3b65 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -282,12 +282,11 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte auto is_format_arg = [] (const std::string & s) -> bool { - return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); + return s == "auto" || FormatFactory::instance().getAllFormats().contains(Poco::toLower(s)); }; if (engine_args.size() == 4) { - //'c1 UInt64, c2 UInt64 auto fourth_arg = checkAndGetLiteralArgument(engine_args[3], "format/account_name"); if (is_format_arg(fourth_arg)) { @@ -298,7 +297,9 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte if (with_structure) structure = fourth_arg; else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown format or account name specified without account key"); + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unknown format or account name specified without account key: {}", fourth_arg); } } else if (engine_args.size() == 5) diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index e12c2f15b28..af191070329 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -64,7 +64,7 @@ std::string StorageHDFSConfiguration::getPathWithoutGlob() const void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) { const size_t max_args_num = with_structure ? 4 : 3; - if (!args.size() || args.size() > max_args_num) + if (args.empty() || args.size() > max_args_num) { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Expected not more than {} arguments", max_args_num); diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index f532af24017..46be0a01862 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -77,7 +77,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, const auto & config = context->getConfigRef(); const std::string config_prefix = "s3."; - auto s3_settings = getSettings(config, config_prefix, context); + auto s3_settings = getSettings(config, config_prefix, context, false); /// FIXME: add a setting auth_settings.updateFrom(s3_settings->auth_settings); s3_settings->auth_settings = auth_settings; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index cf1c583ca62..8381737a4f5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -96,10 +96,9 @@ void StorageObjectStorageSink::finalize() void StorageObjectStorageSink::release() { writer.reset(); - write_buf->finalize(); + write_buf.reset(); } - PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 8e7155205c4..7c6d952d181 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -197,8 +197,22 @@ String StorageS3QueueSource::getName() const return name; } +void StorageS3QueueSource::lazyInitialize() +{ + if (initialized) + return; + + internal_source->lazyInitialize(processing_id); + reader = std::move(internal_source->reader); + if (reader) + reader_future = std::move(internal_source->reader_future); + initialized = true; +} + Chunk StorageS3QueueSource::generate() { + lazyInitialize(); + while (true) { if (!reader) diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index 8c785e683c2..c1b45108b36 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -117,6 +117,7 @@ private: void applyActionAfterProcessing(const String & path); void appendLogElement(const std::string & filename, S3QueueFilesMetadata::FileStatus & file_status_, size_t processed_rows, bool processed); + void lazyInitialize(); }; } diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 2780249e3fd..b767805f637 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -18,18 +18,20 @@ namespace ErrorCodes extern const int INVALID_SETTING_VALUE; } -S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings) +S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const Settings & settings, bool validate_settings) { updateFromSettings(settings, false); - validate(); + if (validate_settings) + validate(); } S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix) - : PartUploadSettings(settings) + String setting_name_prefix, + bool validate_settings) + : PartUploadSettings(settings, validate_settings) { String key = config_prefix + "." + setting_name_prefix; strict_upload_part_size = config.getUInt64(key + "strict_upload_part_size", strict_upload_part_size); @@ -46,7 +48,8 @@ S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings( storage_class_name = config.getString(config_prefix + ".s3_storage_class", storage_class_name); storage_class_name = Poco::toUpperInPlace(storage_class_name); - validate(); + if (validate_settings) + validate(); } S3Settings::RequestSettings::PartUploadSettings::PartUploadSettings(const NamedCollection & collection) @@ -170,8 +173,8 @@ void S3Settings::RequestSettings::PartUploadSettings::validate() } -S3Settings::RequestSettings::RequestSettings(const Settings & settings) - : upload_settings(settings) +S3Settings::RequestSettings::RequestSettings(const Settings & settings, bool validate_settings) + : upload_settings(settings, validate_settings) { updateFromSettingsImpl(settings, false); } @@ -190,8 +193,9 @@ S3Settings::RequestSettings::RequestSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix) - : upload_settings(config, config_prefix, settings, setting_name_prefix) + String setting_name_prefix, + bool validate_settings) + : upload_settings(config, config_prefix, settings, setting_name_prefix, validate_settings) { String key = config_prefix + "." + setting_name_prefix; max_single_read_retries = config.getUInt64(key + "max_single_read_retries", settings.s3_max_single_read_retries); diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index e09be8654e7..c3bc8aa6ed6 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -44,13 +44,14 @@ struct S3Settings private: PartUploadSettings() = default; - explicit PartUploadSettings(const Settings & settings); + explicit PartUploadSettings(const Settings & settings, bool validate_settings = true); explicit PartUploadSettings(const NamedCollection & collection); PartUploadSettings( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix = {}); + String setting_name_prefix = {}, + bool validate_settings = true); friend struct RequestSettings; }; @@ -78,7 +79,7 @@ struct S3Settings void setStorageClassName(const String & storage_class_name) { upload_settings.storage_class_name = storage_class_name; } RequestSettings() = default; - explicit RequestSettings(const Settings & settings); + explicit RequestSettings(const Settings & settings, bool validate_settings = true); explicit RequestSettings(const NamedCollection & collection); /// What's the setting_name_prefix, and why do we need it? @@ -92,7 +93,8 @@ struct S3Settings const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings, - String setting_name_prefix = {}); + String setting_name_prefix = {}, + bool validate_settings = true); void updateFromSettingsIfChanged(const Settings & settings); From 4e1005bc43fabce6baf28f5f91b8a6db0315cc7d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Apr 2024 14:13:21 +0100 Subject: [PATCH 031/158] Fix s3 throttler --- src/Storages/ObjectStorage/S3/Configuration.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 46be0a01862..4c9e49d0705 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -79,7 +79,9 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, auto s3_settings = getSettings(config, config_prefix, context, false); /// FIXME: add a setting + request_settings.updateFromSettingsIfChanged(context->getSettingsRef()); auth_settings.updateFrom(s3_settings->auth_settings); + s3_settings->auth_settings = auth_settings; s3_settings->request_settings = request_settings; From 51c8dd133888964b50c2fa3db5cf6069ccca0252 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 17 Apr 2024 16:17:57 +0100 Subject: [PATCH 032/158] Fix delta lake tests --- .../DataLakes/IStorageDataLake.h | 24 +++++++++++++++---- src/TableFunctions/ITableFunctionDataLake.h | 6 +++-- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 0e83bb70a2f..21ebc32c8ae 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -57,8 +57,8 @@ public: } return std::make_shared>( - base_configuration, std::move(metadata), configuration, object_storage, engine_name_, context, - table_id_, + base_configuration, std::move(metadata), configuration, object_storage, + engine_name_, context, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment_, format_settings_); } @@ -68,11 +68,23 @@ public: static ColumnsDescription getTableStructureFromData( ObjectStoragePtr object_storage_, ConfigurationPtr base_configuration, - const std::optional &, + const std::optional & format_settings_, ContextPtr local_context) { auto metadata = DataLakeMetadata::create(object_storage_, base_configuration, local_context); - return ColumnsDescription(metadata->getTableSchema()); + + auto schema_from_metadata = metadata->getTableSchema(); + if (schema_from_metadata != NamesAndTypesList{}) + { + return ColumnsDescription(std::move(schema_from_metadata)); + } + else + { + ConfigurationPtr configuration = base_configuration->clone(); + configuration->getPaths() = metadata->getDataFiles(); + return Storage::getTableStructureFromData( + object_storage_, configuration, format_settings_, local_context); + } } void updateConfiguration(ContextPtr local_context) override @@ -102,6 +114,10 @@ public: , base_configuration(base_configuration_) , current_metadata(std::move(metadata_)) { + if (base_configuration->format == "auto") + { + base_configuration->format = Storage::configuration->format; + } } private: diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index c86970307c0..8cbd855bb96 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -57,8 +57,10 @@ protected: auto object_storage = TableFunction::getObjectStorage(context, !is_insert_query); return Storage::getTableStructureFromData(object_storage, configuration, std::nullopt, context); } - - return parseColumnsListFromString(configuration->structure, context); + else + { + return parseColumnsListFromString(configuration->structure, context); + } } void parseArguments(const ASTPtr & ast_function, ContextPtr context) override From c8915a16a51719e6ba569806b377f01859971e87 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Apr 2024 17:22:51 +0100 Subject: [PATCH 033/158] Fix a few mote tests --- src/Backups/BackupIO_AzureBlobStorage.cpp | 3 ++- .../registerBackupEngineAzureBlobStorage.cpp | 6 ++++-- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 ++++- src/Disks/ObjectStorages/S3/diskSettings.cpp | 5 ++--- src/Storages/ObjectStorage/DataLakes/Common.cpp | 2 +- .../ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 12 ++++++------ .../ObjectStorage/DataLakes/DeltaLakeMetadata.h | 6 ++++-- .../ObjectStorage/DataLakes/HudiMetadata.h | 4 +++- .../ObjectStorage/DataLakes/IStorageDataLake.h | 14 +++++++++++--- 9 files changed, 37 insertions(+), 20 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 4dd54712e5e..673930b5976 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -193,7 +193,8 @@ void BackupWriterAzureBlobStorage::copyDataToFile( { copyDataToAzureBlobStorageFile( create_read_buffer, start_pos, length, client, configuration.container, - path_in_backup, settings, threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); + fs::path(configuration.blob_path) / path_in_backup, settings, + threadPoolCallbackRunnerUnsafe(getBackupsIOThreadPool().get(), "BackupWRAzure")); } BackupWriterAzureBlobStorage::~BackupWriterAzureBlobStorage() = default; diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 700c8cb222f..049a4b1a338 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -117,8 +117,10 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Using archives with backups on clusters is disabled"); auto path = configuration.getPath(); - configuration.setPath(removeFileNameFromURL(path)); - archive_params.archive_name = configuration.getPath(); + auto filename = removeFileNameFromURL(path); + configuration.setPath(path); + + archive_params.archive_name = filename; archive_params.compression_method = params.compression_method; archive_params.compression_level = params.compression_level; archive_params.password = params.password; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index f97d6f937ef..a2522212f90 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -60,7 +60,10 @@ void throwIfError(const Aws::Utils::Outcome & response) if (!response.IsSuccess()) { const auto & err = response.GetError(); - throw S3Exception(fmt::format("{} (Code: {})", err.GetMessage(), static_cast(err.GetErrorType())), err.GetErrorType()); + throw S3Exception( + fmt::format("{} (Code: {}, s3 exception: {})", + err.GetMessage(), static_cast(err.GetErrorType()), err.GetExceptionName()), + err.GetErrorType()); } } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 2bca7df7db9..66731e85d41 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -72,7 +72,6 @@ std::unique_ptr getClient( if (for_disk_s3) { String endpoint = context->getMacros()->expand(config.getString(config_prefix + ".endpoint")); - url = S3::URI(endpoint); if (!url.key.ends_with('/')) url.key.push_back('/'); @@ -103,8 +102,8 @@ std::unique_ptr getClient( client_configuration.endpointOverride = url.endpoint; client_configuration.maxConnections = static_cast(request_settings.max_connections); - client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS); - client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS); + client_configuration.connectTimeoutMs = config.getUInt64(config_prefix + ".connect_timeout_ms", local_settings.s3_connect_timeout_ms.value); + client_configuration.requestTimeoutMs = config.getUInt64(config_prefix + ".request_timeout_ms", local_settings.s3_request_timeout_ms.value); client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT); client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp index 5f0138078d4..0c9237127b9 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -21,7 +21,7 @@ std::vector listFiles( if (filename.ends_with(suffix)) res.push_back(filename); } - LOG_TRACE(getLogger("DataLakeCommon"), "Listed {} files", res.size()); + LOG_TRACE(getLogger("DataLakeCommon"), "Listed {} files ({})", res.size(), fmt::join(res, ", ")); return res; } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 123c63439b0..d0f203b32bd 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -27,10 +27,11 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct DeltaLakeMetadata::Impl final : private WithContext +struct DeltaLakeMetadata::Impl { ObjectStoragePtr object_storage; ConfigurationPtr configuration; + ContextPtr context; /** * Useful links: @@ -39,9 +40,9 @@ struct DeltaLakeMetadata::Impl final : private WithContext Impl(ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, ContextPtr context_) - : WithContext(context_) - , object_storage(object_storage_) + : object_storage(object_storage_) , configuration(configuration_) + , context(context_) { } @@ -137,7 +138,7 @@ struct DeltaLakeMetadata::Impl final : private WithContext */ void processMetadataFile(const String & key, std::set & result) { - auto read_settings = getContext()->getReadSettings(); + auto read_settings = context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(key), read_settings); char c; @@ -190,7 +191,7 @@ struct DeltaLakeMetadata::Impl final : private WithContext return 0; String json_str; - auto read_settings = getContext()->getReadSettings(); + auto read_settings = context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(last_checkpoint_file), read_settings); readJSONObjectPossiblyInvalid(json_str, *buf); @@ -252,7 +253,6 @@ struct DeltaLakeMetadata::Impl final : private WithContext LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); - auto context = getContext(); auto read_settings = context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(checkpoint_path), read_settings); auto format_settings = getFormatSettings(context); diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index 1a5bb85586a..5050b88d809 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -9,7 +9,7 @@ namespace DB { -class DeltaLakeMetadata final : public IDataLakeMetadata, private WithContext +class DeltaLakeMetadata final : public IDataLakeMetadata { public: using ConfigurationPtr = StorageObjectStorageConfigurationPtr; @@ -28,7 +28,9 @@ public: bool operator ==(const IDataLakeMetadata & other) const override { const auto * deltalake_metadata = dynamic_cast(&other); - return deltalake_metadata && getDataFiles() == deltalake_metadata->getDataFiles(); + return deltalake_metadata + && !data_files.empty() && !deltalake_metadata->data_files.empty() + && data_files == deltalake_metadata->data_files; } static DataLakeMetadataPtr create( diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h index ee8b1ea4978..6054c3f15d6 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -29,7 +29,9 @@ public: bool operator ==(const IDataLakeMetadata & other) const override { const auto * hudi_metadata = dynamic_cast(&other); - return hudi_metadata && getDataFiles() == hudi_metadata->getDataFiles(); + return hudi_metadata + && !data_files.empty() && !hudi_metadata->data_files.empty() + && data_files == hudi_metadata->data_files; } static DataLakeMetadataPtr create( diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 21ebc32c8ae..64228e880f8 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -42,17 +42,25 @@ public: auto object_storage = base_configuration->createObjectStorage(context); DataLakeMetadataPtr metadata; NamesAndTypesList schema_from_metadata; + + if (base_configuration->format == "auto") + base_configuration->format = "Parquet"; + ConfigurationPtr configuration = base_configuration->clone(); + try { metadata = DataLakeMetadata::create(object_storage, base_configuration, context); schema_from_metadata = metadata->getTableSchema(); - configuration->getPaths() = metadata->getDataFiles(); + configuration->setPaths(metadata->getDataFiles()); } catch (...) { if (mode <= LoadingStrictnessLevel::CREATE) throw; + + metadata.reset(); + configuration->setPaths({}); tryLogCurrentException(__PRETTY_FUNCTION__); } @@ -100,8 +108,8 @@ public: current_metadata = std::move(new_metadata); auto updated_configuration = base_configuration->clone(); - /// If metadata wasn't changed, we won't list data files again. - updated_configuration->getPaths() = current_metadata->getDataFiles(); + updated_configuration->setPaths(current_metadata->getDataFiles()); + Storage::configuration = updated_configuration; } From e2e6071063b4ce09530746c9ef49d12a36ccec37 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 Apr 2024 13:43:43 +0100 Subject: [PATCH 034/158] Fix a few more tests --- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 21 ++++ .../ObjectStorages/HDFS/HDFSObjectStorage.h | 8 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 3 +- .../ObjectStorage/AzureBlob/Configuration.cpp | 97 +++++++++++++++---- .../ObjectStorage/HDFS/Configuration.cpp | 10 +- .../ObjectStorage/StorageObjectStorage.cpp | 50 +++------- .../StorageObjectStorageQuerySettings.h | 2 +- .../StorageObjectStorageSink.cpp | 9 ++ .../ObjectStorage/StorageObjectStorageSink.h | 3 + src/Storages/ObjectStorage/Utils.cpp | 43 ++++++++ src/Storages/ObjectStorage/Utils.h | 17 ++++ tests/integration/test_storage_hdfs/test.py | 8 +- .../test_storage_kerberized_hdfs/test.py | 2 +- 13 files changed, 204 insertions(+), 69 deletions(-) create mode 100644 src/Storages/ObjectStorage/Utils.cpp create mode 100644 src/Storages/ObjectStorage/Utils.h diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 82c9a6c6c21..fc7d49324c7 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -31,8 +31,18 @@ void HDFSObjectStorage::startup() { } +void HDFSObjectStorage::initializeHDFS() const +{ + if (hdfs_fs) + return; + + hdfs_builder = createHDFSBuilder(url, config); + hdfs_fs = createHDFSFS(hdfs_builder.get()); +} + ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { + initializeHDFS(); /// what ever data_source_description.description value is, consider that key as relative key chassert(data_directory.starts_with("/")); return ObjectStorageKey::createAsRelative( @@ -41,6 +51,7 @@ ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & bool HDFSObjectStorage::exists(const StoredObject & object) const { + initializeHDFS(); std::string path = object.remote_path; if (path.starts_with(url_without_path)) path = path.substr(url_without_path.size()); @@ -57,6 +68,7 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLIN std::optional, std::optional) const { + initializeHDFS(); std::string path = object.remote_path; if (path.starts_with(url)) path = path.substr(url.size()); @@ -73,6 +85,7 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI std::optional, std::optional) const { + initializeHDFS(); auto disk_read_settings = patchSettings(read_settings); auto read_buffer_creator = [this, disk_read_settings] @@ -102,6 +115,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL size_t buf_size, const WriteSettings & write_settings) { + initializeHDFS(); if (attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, @@ -123,6 +137,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL /// Remove file. Throws exception if file doesn't exists or it's a directory. void HDFSObjectStorage::removeObject(const StoredObject & object) { + initializeHDFS(); auto path = object.remote_path; if (path.starts_with(url_without_path)) path = path.substr(url_without_path.size()); @@ -136,24 +151,28 @@ void HDFSObjectStorage::removeObject(const StoredObject & object) void HDFSObjectStorage::removeObjects(const StoredObjects & objects) { + initializeHDFS(); for (const auto & object : objects) removeObject(object); } void HDFSObjectStorage::removeObjectIfExists(const StoredObject & object) { + initializeHDFS(); if (exists(object)) removeObject(object); } void HDFSObjectStorage::removeObjectsIfExist(const StoredObjects & objects) { + initializeHDFS(); for (const auto & object : objects) removeObjectIfExists(object); } ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) const { + initializeHDFS(); auto * file_info = hdfsGetPathInfo(hdfs_fs.get(), path.data()); if (!file_info) throw Exception(ErrorCodes::HDFS_ERROR, @@ -169,6 +188,7 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { + initializeHDFS(); auto * log = &Poco::Logger::get("HDFSObjectStorage"); LOG_TRACE(log, "Trying to list files for {}", path); @@ -222,6 +242,7 @@ void HDFSObjectStorage::copyObject( /// NOLINT const WriteSettings & write_settings, std::optional object_to_attributes) { + initializeHDFS(); if (object_to_attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 8987fa5eaf1..f57b7e1fda8 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -37,8 +37,6 @@ public: SettingsPtr settings_, const Poco::Util::AbstractConfiguration & config_) : config(config_) - , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) - , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) { const size_t begin_of_path = hdfs_root_path_.find('/', hdfs_root_path_.find("//") + 2); @@ -117,10 +115,12 @@ public: bool isRemote() const override { return true; } private: + void initializeHDFS() const; + const Poco::Util::AbstractConfiguration & config; - HDFSBuilderWrapper hdfs_builder; - HDFSFSPtr hdfs_fs; + mutable HDFSBuilderWrapper hdfs_builder; + mutable HDFSFSPtr hdfs_fs; SettingsPtr settings; std::string url; std::string url_without_path; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 66731e85d41..49300a9cd89 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -157,7 +157,8 @@ std::unique_ptr getClient( auth_settings.server_side_encryption_customer_key_base64, std::move(sse_kms_config), auth_settings.headers, - credentials_configuration); + credentials_configuration, + auth_settings.session_token); } } diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 44ace9c3b65..4b826a0c721 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -381,7 +381,7 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte } void StorageAzureBlobConfiguration::addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & /* format */, ContextPtr context) + ASTs & args, const String & structure_, const String & format_, ContextPtr context) { if (tryGetNamedCollectionWithOverrides(args, context)) { @@ -397,66 +397,129 @@ void StorageAzureBlobConfiguration::addStructureAndFormatToArgs( { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage Azure requires 3 to 7 arguments: " - "StorageObjectStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])"); + "StorageObjectStorage(connection_string|storage_account_url, container_name, " + "blobpath, [account_name, account_key, format, compression, structure])"); } + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + auto structure_literal = std::make_shared(structure_); + auto format_literal = std::make_shared(format_); auto is_format_arg = [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); }; + /// (connection_string, container_name, blobpath) if (args.size() == 3) { - /// Add format=auto & compression=auto before structure argument. - args.push_back(std::make_shared("auto")); + args.push_back(format_literal); + /// Add compression = "auto" before structure argument. args.push_back(std::make_shared("auto")); args.push_back(structure_literal); } + /// (connection_string, container_name, blobpath, structure) or + /// (connection_string, container_name, blobpath, format) + /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. else if (args.size() == 4) { auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name/structure"); + /// (..., format) -> (..., format, compression, structure) if (is_format_arg(fourth_arg)) { + if (fourth_arg == "auto") + args[3] = format_literal; /// Add compression=auto before structure argument. args.push_back(std::make_shared("auto")); args.push_back(structure_literal); } + /// (..., structure) -> (..., format, compression, structure) else { - args.back() = structure_literal; + auto structure_arg = args.back(); + args[3] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + if (fourth_arg == "auto") + args.push_back(structure_literal); + else + args.push_back(structure_arg); } } + /// (connection_string, container_name, blobpath, format, compression) or + /// (storage_account_url, container_name, blobpath, account_name, account_key) + /// We can distinguish them by looking at the 4-th argument: check if it's format name or not. else if (args.size() == 5) { auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - if (!is_format_arg(fourth_arg)) + /// (..., format, compression) -> (..., format, compression, structure) + if (is_format_arg(fourth_arg)) { - /// Add format=auto & compression=auto before structure argument. - args.push_back(std::make_shared("auto")); - args.push_back(std::make_shared("auto")); + if (fourth_arg == "auto") + args[3] = format_literal; + args.push_back(structure_literal); } - args.push_back(structure_literal); - } - else if (args.size() == 6) - { - auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); - if (!is_format_arg(fourth_arg)) + /// (..., account_name, account_key) -> (..., account_name, account_key, format, compression, structure) + else { + args.push_back(format_literal); /// Add compression=auto before structure argument. args.push_back(std::make_shared("auto")); args.push_back(structure_literal); } + } + /// (connection_string, container_name, blobpath, format, compression, structure) or + /// (storage_account_url, container_name, blobpath, account_name, account_key, structure) or + /// (storage_account_url, container_name, blobpath, account_name, account_key, format) + else if (args.size() == 6) + { + auto fourth_arg = checkAndGetLiteralArgument(args[3], "format/account_name"); + auto sixth_arg = checkAndGetLiteralArgument(args[5], "format/structure"); + + /// (..., format, compression, structure) + if (is_format_arg(fourth_arg)) + { + if (fourth_arg == "auto") + args[3] = format_literal; + if (checkAndGetLiteralArgument(args[5], "structure") == "auto") + args[5] = structure_literal; + } + /// (..., account_name, account_key, format) -> (..., account_name, account_key, format, compression, structure) + else if (is_format_arg(sixth_arg)) + { + if (sixth_arg == "auto") + args[5] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + args.push_back(structure_literal); + } + /// (..., account_name, account_key, structure) -> (..., account_name, account_key, format, compression, structure) else { - args.back() = structure_literal; + auto structure_arg = args.back(); + args[5] = format_literal; + /// Add compression=auto before structure argument. + args.push_back(std::make_shared("auto")); + if (sixth_arg == "auto") + args.push_back(structure_literal); + else + args.push_back(structure_arg); } } + /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression) else if (args.size() == 7) { + /// (..., format, compression) -> (..., format, compression, structure) + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = format_literal; args.push_back(structure_literal); } + /// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure) else if (args.size() == 8) { - args.back() = structure_literal; + if (checkAndGetLiteralArgument(args[5], "format") == "auto") + args[5] = format_literal; + if (checkAndGetLiteralArgument(args[7], "structure") == "auto") + args[7] = structure_literal; } } } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index af191070329..84f0a7bfe9f 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -73,9 +73,11 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit std::string url_str; url_str = checkAndGetLiteralArgument(args[0], "url"); + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + if (args.size() > 1) { - args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], context); format = checkAndGetLiteralArgument(args[1], "format_name"); } @@ -83,18 +85,15 @@ void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool wit { if (args.size() > 2) { - args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context); structure = checkAndGetLiteralArgument(args[2], "structure"); } if (args.size() > 3) { - args[3] = evaluateConstantExpressionOrIdentifierAsLiteral(args[3], context); compression_method = checkAndGetLiteralArgument(args[3], "compression_method"); } } else if (args.size() > 2) { - args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], context); compression_method = checkAndGetLiteralArgument(args[2], "compression_method"); } @@ -165,6 +164,9 @@ void StorageHDFSConfiguration::addStructureAndFormatToArgs( auto format_literal = std::make_shared(format_); auto structure_literal = std::make_shared(structure_); + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + /// hdfs(url) if (count == 1) { diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 8fc3de4de1b..13f3557d927 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -193,6 +194,7 @@ SinkToStoragePtr StorageObjectStorage::write( { updateConfiguration(local_context); const auto sample_block = metadata_snapshot->getSampleBlock(); + const auto & query_settings = StorageSettings::create(local_context->getSettingsRef()); if (configuration->withWildcard()) { @@ -209,7 +211,8 @@ SinkToStoragePtr StorageObjectStorage::write( { LOG_TEST(log, "Using PartitionedSink for {}", configuration->getPath()); return std::make_shared( - object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); + object_storage, configuration, query_settings, + format_settings, sample_block, local_context, partition_by_ast); } } @@ -220,46 +223,19 @@ SinkToStoragePtr StorageObjectStorage::write( getName(), configuration->getPath()); } - const auto storage_settings = StorageSettings::create(local_context->getSettingsRef()); - - auto configuration_copy = configuration->clone(); - if (!storage_settings.truncate_on_insert - && object_storage->exists(StoredObject(configuration->getPath()))) + auto & paths = configuration->getPaths(); + if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( + *object_storage, *configuration, query_settings, paths.front(), paths.size())) { - if (storage_settings.create_new_file_on_insert) - { - auto & paths = configuration_copy->getPaths(); - size_t index = paths.size(); - const auto & first_key = paths[0]; - auto pos = first_key.find_first_of('.'); - String new_key; - - do - { - new_key = first_key.substr(0, pos) - + "." - + std::to_string(index) - + (pos == std::string::npos ? "" : first_key.substr(pos)); - ++index; - } - while (object_storage->exists(StoredObject(new_key))); - - paths.push_back(new_key); - configuration->getPaths().push_back(new_key); - } - else - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting [engine_name]_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting [engine_name]_create_new_file_on_insert", - configuration_copy->getNamespace(), configuration_copy->getPaths().back()); - } + paths.push_back(*new_key); } return std::make_shared( - object_storage, configuration_copy, format_settings, sample_block, local_context); + object_storage, + configuration->clone(), + format_settings, + sample_block, + local_context); } template diff --git a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h index f0687776aa7..606456011c3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h @@ -84,7 +84,7 @@ struct HDFSStorageSettings .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for hdfs + .skip_empty_files = settings.hdfs_skip_empty_files, /// TODO: add setting for hdfs .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 8381737a4f5..42371764920 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -102,6 +103,7 @@ void StorageObjectStorageSink::release() PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, + const StorageObjectStorageSettings & query_settings_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, @@ -109,6 +111,7 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( : PartitionedSink(partition_by, context_, sample_block_) , object_storage(object_storage_) , configuration(configuration_) + , query_settings(query_settings_) , format_settings(format_settings_) , sample_block(sample_block_) , context(context_) @@ -123,6 +126,12 @@ SinkPtr PartitionedStorageObjectStorageSink::createSinkForPartition(const String auto partition_key = replaceWildcards(configuration->getPath(), partition_id); validateKey(partition_key); + if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( + *object_storage, *configuration, query_settings, partition_key, /* sequence_number */1)) + { + partition_key = *new_key; + } + return std::make_shared( object_storage, configuration, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index a352e2c66a3..38805332a35 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include #include @@ -46,6 +47,7 @@ public: PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, + const StorageObjectStorageSettings & query_settings_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, @@ -59,6 +61,7 @@ private: ObjectStoragePtr object_storage; StorageObjectStorageConfigurationPtr configuration; + const StorageObjectStorageSettings query_settings; const std::optional format_settings; const Block sample_block; const ContextPtr context; diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp new file mode 100644 index 00000000000..6cc3962209f --- /dev/null +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -0,0 +1,43 @@ +#include +#include +#include +#include + + +namespace DB +{ + +std::optional checkAndGetNewFileOnInsertIfNeeded( + const IObjectStorage & object_storage, + const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorageSettings & query_settings, + const String & key, + size_t sequence_number) +{ + if (query_settings.truncate_on_insert + || !object_storage.exists(StoredObject(key))) + return std::nullopt; + + if (query_settings.create_new_file_on_insert) + { + auto pos = key.find_first_of('.'); + String new_key; + do + { + new_key = key.substr(0, pos) + "." + std::to_string(sequence_number) + (pos == std::string::npos ? "" : key.substr(pos)); + ++sequence_number; + } + while (object_storage.exists(StoredObject(new_key))); + + return new_key; + } + + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. " + "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", + configuration.getNamespace(), key); +} + +} diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h new file mode 100644 index 00000000000..9291bb72615 --- /dev/null +++ b/src/Storages/ObjectStorage/Utils.h @@ -0,0 +1,17 @@ +#include + +namespace DB +{ + +class IObjectStorage; +class StorageObjectStorageConfiguration; +struct StorageObjectStorageSettings; + +std::optional checkAndGetNewFileOnInsertIfNeeded( + const IObjectStorage & object_storage, + const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorageSettings & query_settings, + const std::string & key, + size_t sequence_number); + +} diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index d8dab85ee6a..dc375b9ec36 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -980,7 +980,7 @@ def test_read_subcolumns(started_cluster): assert ( res - == "2\thdfs://hdfs1:9000/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + == "2\t/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" ) res = node.query( @@ -989,7 +989,7 @@ def test_read_subcolumns(started_cluster): assert ( res - == "2\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + == "2\t/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" ) res = node.query( @@ -998,7 +998,7 @@ def test_read_subcolumns(started_cluster): assert ( res - == "0\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + == "0\t/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" ) res = node.query( @@ -1007,7 +1007,7 @@ def test_read_subcolumns(started_cluster): assert ( res - == "42\thdfs://hdfs1:9000/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + == "42\t/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" ) diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index c72152fa376..ddfc1f6483d 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -130,7 +130,7 @@ def test_prohibited(started_cluster): assert False, "Exception have to be thrown" except Exception as ex: assert ( - "Unable to open HDFS file: /storage_user_two_prohibited error: Permission denied: user=specuser, access=WRITE" + "Unable to open HDFS file: /storage_user_two_prohibited (hdfs://suser@kerberizedhdfs1:9010/storage_user_two_prohibited) error: Permission denied: user=specuser, access=WRITE" in str(ex) ) From 191937c0c6c5e5a31c6045269026ca1a1e5171c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Apr 2024 10:19:55 +0100 Subject: [PATCH 035/158] Fix style check --- tests/integration/test_storage_hdfs/test.py | 20 ++++---------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index dc375b9ec36..820e3db6eb1 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -978,37 +978,25 @@ def test_read_subcolumns(started_cluster): f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "2\t/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" - ) + assert res == "2\t/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "2\t/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" - ) + assert res == "2\t/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert ( - res - == "0\t/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" - ) + assert res == "0\t/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert ( - res - == "42\t/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" - ) + assert res == "42\t/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" def test_union_schema_inference_mode(started_cluster): From c7f0cfc4c2df850cf97c81febd61b3411c4e7869 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Apr 2024 11:47:41 +0100 Subject: [PATCH 036/158] Fix style check --- src/Storages/ObjectStorage/Utils.cpp | 5 +++++ src/Storages/ObjectStorage/Utils.h | 1 + 2 files changed, 6 insertions(+) diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 6cc3962209f..9caab709081 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -7,6 +7,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, const StorageObjectStorageConfiguration & configuration, diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 9291bb72615..afc0f31a33f 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -1,3 +1,4 @@ +#pragma once #include namespace DB From a4daf2b454c44e1891a61eaddf3a2fd965e5f880 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Apr 2024 14:46:32 +0100 Subject: [PATCH 037/158] Fix hdfs race --- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 7 ++++++- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 4 ++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index fc7d49324c7..ed63795cb05 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -33,11 +33,16 @@ void HDFSObjectStorage::startup() void HDFSObjectStorage::initializeHDFS() const { - if (hdfs_fs) + if (initialized) + return; + + std::lock_guard lock(init_mutex); + if (initialized) return; hdfs_builder = createHDFSBuilder(url, config); hdfs_fs = createHDFSFS(hdfs_builder.get()); + initialized = true; } ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index f57b7e1fda8..b626d3dc779 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -121,6 +121,10 @@ private: mutable HDFSBuilderWrapper hdfs_builder; mutable HDFSFSPtr hdfs_fs; + + mutable std::mutex init_mutex; + mutable std::atomic_bool initialized{false}; + SettingsPtr settings; std::string url; std::string url_without_path; From 399414bb40e517b315ab396669875af8e365ece0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Apr 2024 17:27:54 +0100 Subject: [PATCH 038/158] Better --- src/Common/CurrentMetrics.cpp | 3 + src/Interpreters/InterpreterSystemQuery.cpp | 9 +- .../ObjectStorage/AzureBlob/Configuration.cpp | 15 ++ .../ObjectStorage/AzureBlob/Configuration.h | 11 +- .../DataLakes/DeltaLakeMetadata.cpp | 2 +- .../DataLakes/IStorageDataLake.h | 17 +- .../DataLakes/registerDataLakeStorages.cpp | 7 +- .../ObjectStorage/HDFS/Configuration.cpp | 14 ++ .../ObjectStorage/HDFS/Configuration.h | 11 +- .../ObjectStorage/ReadBufferIterator.cpp | 4 +- .../ObjectStorage/ReadBufferIterator.h | 4 +- ...rage.cpp => ReadFromObjectStorageStep.cpp} | 32 +--- ...tStorage.h => ReadFromObjectStorageStep.h} | 18 +- .../ObjectStorage/S3/Configuration.cpp | 15 ++ src/Storages/ObjectStorage/S3/Configuration.h | 11 +- .../ObjectStorage/StorageObjectStorage.cpp | 181 +++++++----------- .../ObjectStorage/StorageObjectStorage.h | 69 +++---- .../StorageObjectStorageCluster.cpp | 89 ++++----- .../StorageObjectStorageCluster.h | 28 +-- .../StorageObjectStorageConfiguration.h | 7 + .../StorageObjectStorageQuerySettings.h | 102 ---------- .../StorageObjectStorageSink.cpp | 3 +- .../ObjectStorage/StorageObjectStorageSink.h | 4 +- .../StorageObjectStorageSource.cpp | 49 ++--- .../StorageObjectStorageSource.h | 26 +-- src/Storages/ObjectStorage/Utils.cpp | 42 +++- src/Storages/ObjectStorage/Utils.h | 14 +- .../registerStorageObjectStorage.cpp | 20 +- src/Storages/S3Queue/S3QueueSource.h | 3 +- src/Storages/S3Queue/StorageS3Queue.cpp | 25 +-- src/Storages/S3Queue/StorageS3Queue.h | 3 +- .../StorageSystemSchemaInferenceCache.cpp | 9 +- src/TableFunctions/ITableFunctionDataLake.h | 2 +- .../TableFunctionObjectStorage.cpp | 78 ++++---- .../TableFunctionObjectStorage.h | 8 +- .../TableFunctionObjectStorageCluster.cpp | 16 +- .../TableFunctionObjectStorageCluster.h | 12 +- 37 files changed, 427 insertions(+), 536 deletions(-) rename src/Storages/ObjectStorage/{ReadFromStorageObjectStorage.cpp => ReadFromObjectStorageStep.cpp} (62%) rename src/Storages/ObjectStorage/{ReadFromStorageObjectStorage.h => ReadFromObjectStorageStep.h} (70%) delete mode 100644 src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 0f25397a961..983e737991c 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -168,6 +168,9 @@ M(ObjectStorageS3Threads, "Number of threads in the S3ObjectStorage thread pool.") \ M(ObjectStorageS3ThreadsActive, "Number of threads in the S3ObjectStorage thread pool running a task.") \ M(ObjectStorageS3ThreadsScheduled, "Number of queued or active jobs in the S3ObjectStorage thread pool.") \ + M(StorageObjectStorageThreads, "Number of threads in the remote table engines thread pools.") \ + M(StorageObjectStorageThreadsActive, "Number of threads in the remote table engines thread pool running a task.") \ + M(StorageObjectStorageThreadsScheduled, "Number of queued or active jobs in remote table engines thread pool.") \ M(ObjectStorageAzureThreads, "Number of threads in the AzureObjectStorage thread pool.") \ M(ObjectStorageAzureThreadsActive, "Number of threads in the AzureObjectStorage thread pool running a task.") \ M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \ diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 27b2a9460b7..af9dc08e8c7 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -53,6 +53,9 @@ #include #include #include +#include +#include +#include #include #include #include @@ -489,17 +492,17 @@ BlockIO InterpreterSystemQuery::execute() StorageFile::getSchemaCache(getContext()).clear(); #if USE_AWS_S3 if (caches_to_drop.contains("S3")) - StorageS3::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageS3Configuration::type_name).clear(); #endif #if USE_HDFS if (caches_to_drop.contains("HDFS")) - StorageHDFS::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageHDFSConfiguration::type_name).clear(); #endif if (caches_to_drop.contains("URL")) StorageURL::getSchemaCache(getContext()).clear(); #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) - StorageAzureBlob::getSchemaCache(getContext()).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageAzureBlobConfiguration::type_name).clear(); #endif break; } diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index 4b826a0c721..c9bc59d62aa 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -101,6 +101,21 @@ AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(Co return settings_ptr; } +StorageObjectStorage::QuerySettings StorageAzureBlobConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.azure_truncate_on_insert, + .create_new_file_on_insert = settings.azure_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure + .list_object_keys_size = settings.azure_list_object_keys_size, + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist, + }; +} + ObjectStoragePtr StorageAzureBlobConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { assertInitialized(); diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.h b/src/Storages/ObjectStorage/AzureBlob/Configuration.h index c12ff81197d..7e105ea82b5 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.h +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.h @@ -18,9 +18,15 @@ class StorageAzureBlobConfiguration : public StorageObjectStorageConfiguration friend void registerBackupEngineAzureBlobStorage(BackupFactory & factory); public: + static constexpr auto type_name = "azure"; + static constexpr auto engine_name = "Azure"; + StorageAzureBlobConfiguration() = default; StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other); + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return engine_name; } + Path getPath() const override { return blob_path; } void setPath(const Path & path) override { blob_path = path; } @@ -30,6 +36,7 @@ public: String getDataSourceDescription() override { return fs::path(connection_url) / container; } String getNamespace() const override { return container; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT @@ -37,8 +44,8 @@ public: void fromNamedCollection(const NamedCollection & collection) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; - static void addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & format_, ContextPtr context); + void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) override; protected: using AzureClient = Azure::Storage::Blobs::BlobContainerClient; diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index d0f203b32bd..c6590ba8d43 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -184,7 +184,7 @@ struct DeltaLakeMetadata::Impl * * We need to get "version", which is the version of the checkpoint we need to read. */ - size_t readLastCheckpointIfExists() + size_t readLastCheckpointIfExists() const { const auto last_checkpoint_file = fs::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; if (!object_storage->exists(StoredObject(last_checkpoint_file))) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 64228e880f8..e1851775925 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -21,17 +21,16 @@ namespace DB /// Storage for read-only integration with Apache Iceberg tables in Amazon S3 (see https://iceberg.apache.org/) /// Right now it's implemented on top of StorageS3 and right now it doesn't support /// many Iceberg features like schema evolution, partitioning, positional and equality deletes. -template -class IStorageDataLake final : public StorageObjectStorage +template +class IStorageDataLake final : public StorageObjectStorage { public: - using Storage = StorageObjectStorage; + using Storage = StorageObjectStorage; using ConfigurationPtr = Storage::ConfigurationPtr; static StoragePtr create( ConfigurationPtr base_configuration, ContextPtr context, - const String & engine_name_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -64,9 +63,9 @@ public: tryLogCurrentException(__PRETTY_FUNCTION__); } - return std::make_shared>( + return std::make_shared>( base_configuration, std::move(metadata), configuration, object_storage, - engine_name_, context, table_id_, + context, table_id_, columns_.empty() ? ColumnsDescription(schema_from_metadata) : columns_, constraints_, comment_, format_settings_); } @@ -133,9 +132,9 @@ private: DataLakeMetadataPtr current_metadata; }; -using StorageIceberg = IStorageDataLake; -using StorageDeltaLake = IStorageDataLake; -using StorageHudi = IStorageDataLake; +using StorageIceberg = IStorageDataLake; +using StorageDeltaLake = IStorageDataLake; +using StorageHudi = IStorageDataLake; } diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index d11dd1ca836..a5170e5ed6b 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -24,7 +23,7 @@ void registerStorageIceberg(StorageFactory & factory) StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageIceberg::create( - configuration, args.getContext(), "Iceberg", args.table_id, args.columns, + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); }, { @@ -47,7 +46,7 @@ void registerStorageDeltaLake(StorageFactory & factory) StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageDeltaLake::create( - configuration, args.getContext(), "DeltaLake", args.table_id, args.columns, + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); }, { @@ -68,7 +67,7 @@ void registerStorageHudi(StorageFactory & factory) StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageHudi::create( - configuration, args.getContext(), "Hudi", args.table_id, args.columns, + configuration, args.getContext(), args.table_id, args.columns, args.constraints, args.comment, std::nullopt, args.mode); }, { diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 84f0a7bfe9f..0062ac969ac 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -60,6 +60,20 @@ std::string StorageHDFSConfiguration::getPathWithoutGlob() const return "/"; return path.substr(0, end_of_path_without_globs); } +StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.hdfs_truncate_on_insert, + .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.hdfs_skip_empty_files, /// TODO: add setting for hdfs + .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, + }; +} void StorageHDFSConfiguration::fromAST(ASTs & args, ContextPtr context, bool with_structure) { diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 23a7e8e4549..0a502857153 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -13,9 +13,15 @@ namespace DB class StorageHDFSConfiguration : public StorageObjectStorageConfiguration { public: + static constexpr auto type_name = "hdfs"; + static constexpr auto engine_name = "HDFS"; + StorageHDFSConfiguration() = default; StorageHDFSConfiguration(const StorageHDFSConfiguration & other); + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return engine_name; } + Path getPath() const override { return path; } void setPath(const Path & path_) override { path = path_; } @@ -25,13 +31,14 @@ public: String getNamespace() const override { return ""; } String getDataSourceDescription() override { return url; } + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } - static void addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & format_, ContextPtr context); + void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) override; std::string getPathWithoutGlob() const override; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 0b6e34fb831..f8ce90a2b1f 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -19,7 +18,6 @@ ReadBufferIterator::ReadBufferIterator( ConfigurationPtr configuration_, const FileIterator & file_iterator_, const std::optional & format_settings_, - const StorageObjectStorageSettings & query_settings_, SchemaCache & schema_cache_, ObjectInfos & read_keys_, const ContextPtr & context_) @@ -28,7 +26,7 @@ ReadBufferIterator::ReadBufferIterator( , configuration(configuration_) , file_iterator(file_iterator_) , format_settings(format_settings_) - , query_settings(query_settings_) + , query_settings(configuration->getQuerySettings(context_)) , schema_cache(schema_cache_) , read_keys(read_keys_) , format(configuration->format == "auto" ? std::nullopt : std::optional(configuration->format)) diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 053bcbf894f..2d58e1c789e 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -2,7 +2,6 @@ #include #include #include -#include #include @@ -19,7 +18,6 @@ public: ConfigurationPtr configuration_, const FileIterator & file_iterator_, const std::optional & format_settings_, - const StorageObjectStorageSettings & query_settings_, SchemaCache & schema_cache_, ObjectInfos & read_keys_, const ContextPtr & context_); @@ -50,7 +48,7 @@ private: const ConfigurationPtr configuration; const FileIterator file_iterator; const std::optional & format_settings; - const StorageObjectStorageSettings query_settings; + const StorageObjectStorage::QuerySettings query_settings; SchemaCache & schema_cache; ObjectInfos & read_keys; std::optional format; diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp similarity index 62% rename from src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp rename to src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp index 89d33191f41..f19e01cdc3e 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp @@ -1,11 +1,11 @@ -#include +#include #include #include namespace DB { -ReadFromStorageObejctStorage::ReadFromStorageObejctStorage( +ReadFromObjectStorageStep::ReadFromObjectStorageStep( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, const String & name_, @@ -14,49 +14,41 @@ ReadFromStorageObejctStorage::ReadFromStorageObejctStorage( const SelectQueryInfo & query_info_, const StorageSnapshotPtr & storage_snapshot_, const std::optional & format_settings_, - const StorageObjectStorageSettings & query_settings_, bool distributed_processing_, ReadFromFormatInfo info_, SchemaCache & schema_cache_, const bool need_only_count_, ContextPtr context_, size_t max_block_size_, - size_t num_streams_, - CurrentMetrics::Metric metric_threads_count_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_) + size_t num_streams_) : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) , object_storage(object_storage_) , configuration(configuration_) , info(std::move(info_)) , virtual_columns(virtual_columns_) , format_settings(format_settings_) - , query_settings(query_settings_) + , query_settings(configuration->getQuerySettings(context_)) , schema_cache(schema_cache_) , name(name_ + "Source") , need_only_count(need_only_count_) , max_block_size(max_block_size_) , num_streams(num_streams_) , distributed_processing(distributed_processing_) - , metric_threads_count(metric_threads_count_) - , metric_threads_active(metric_threads_active_) - , metric_threads_scheduled(metric_threads_scheduled_) { } -void ReadFromStorageObejctStorage::createIterator(const ActionsDAG::Node * predicate) +void ReadFromObjectStorageStep::createIterator(const ActionsDAG::Node * predicate) { if (!iterator_wrapper) { auto context = getContext(); iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, query_settings, distributed_processing, - context, predicate, virtual_columns, nullptr, metric_threads_count, - metric_threads_active, metric_threads_scheduled, context->getFileProgressCallback()); + configuration, object_storage, distributed_processing, + context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); } } -void ReadFromStorageObejctStorage::applyFilters(ActionDAGNodes added_filter_nodes) +void ReadFromObjectStorageStep::applyFilters(ActionDAGNodes added_filter_nodes) { filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); const ActionsDAG::Node * predicate = nullptr; @@ -66,7 +58,7 @@ void ReadFromStorageObejctStorage::applyFilters(ActionDAGNodes added_filter_node createIterator(predicate); } -void ReadFromStorageObejctStorage::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +void ReadFromObjectStorageStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { createIterator(nullptr); auto context = getContext(); @@ -74,13 +66,9 @@ void ReadFromStorageObejctStorage::initializePipeline(QueryPipelineBuilder & pip Pipes pipes; for (size_t i = 0; i < num_streams; ++i) { - auto threadpool = std::make_shared( - metric_threads_count, metric_threads_active, metric_threads_scheduled, /* max_threads */1); - auto source = std::make_shared( getName(), object_storage, configuration, info, format_settings, query_settings, - context, max_block_size, iterator_wrapper, need_only_count, schema_cache, - std::move(threadpool), metric_threads_count, metric_threads_active, metric_threads_scheduled); + context, max_block_size, iterator_wrapper, need_only_count, schema_cache); source->setKeyCondition(filter_actions_dag, context); pipes.emplace_back(std::move(source)); diff --git a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.h similarity index 70% rename from src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h rename to src/Storages/ObjectStorage/ReadFromObjectStorageStep.h index c0dd02d75f8..d98ebfef1f2 100644 --- a/src/Storages/ObjectStorage/ReadFromStorageObjectStorage.h +++ b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.h @@ -1,17 +1,16 @@ #pragma once -#include -#include #include +#include namespace DB { -class ReadFromStorageObejctStorage : public SourceStepWithFilter +class ReadFromObjectStorageStep : public SourceStepWithFilter { public: using ConfigurationPtr = StorageObjectStorageConfigurationPtr; - ReadFromStorageObejctStorage( + ReadFromObjectStorageStep( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, const String & name_, @@ -20,17 +19,13 @@ public: const SelectQueryInfo & query_info_, const StorageSnapshotPtr & storage_snapshot_, const std::optional & format_settings_, - const StorageObjectStorageSettings & query_settings_, bool distributed_processing_, ReadFromFormatInfo info_, SchemaCache & schema_cache_, bool need_only_count_, ContextPtr context_, size_t max_block_size_, - size_t num_streams_, - CurrentMetrics::Metric metric_threads_count_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_); + size_t num_streams_); std::string getName() const override { return name; } @@ -46,16 +41,13 @@ private: const ReadFromFormatInfo info; const NamesAndTypesList virtual_columns; const std::optional format_settings; - const StorageObjectStorageSettings query_settings; + const StorageObjectStorage::QuerySettings query_settings; SchemaCache & schema_cache; const String name; const bool need_only_count; const size_t max_block_size; const size_t num_streams; const bool distributed_processing; - const CurrentMetrics::Metric metric_threads_count; - const CurrentMetrics::Metric metric_threads_active; - const CurrentMetrics::Metric metric_threads_scheduled; void createIterator(const ActionsDAG::Node * predicate); }; diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 4c9e49d0705..139d9004f8e 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -70,6 +70,21 @@ StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & ot keys = other.keys; } +StorageObjectStorage::QuerySettings StorageS3Configuration::getQuerySettings(const ContextPtr & context) const +{ + const auto & settings = context->getSettingsRef(); + return StorageObjectStorage::QuerySettings{ + .truncate_on_insert = settings.s3_truncate_on_insert, + .create_new_file_on_insert = settings.s3_create_new_file_on_insert, + .schema_inference_use_cache = settings.schema_inference_use_cache_for_s3, + .schema_inference_mode = settings.schema_inference_mode, + .skip_empty_files = settings.s3_skip_empty_files, + .list_object_keys_size = settings.s3_list_object_keys_size, + .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .ignore_non_existent_file = settings.s3_ignore_file_doesnt_exist, + }; +} + ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, bool /* is_readonly */) /// NOLINT { assertInitialized(); diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index ff5e8680e66..de4a6d17579 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -14,9 +15,14 @@ namespace DB class StorageS3Configuration : public StorageObjectStorageConfiguration { public: + static constexpr auto type_name = "s3"; + StorageS3Configuration() = default; StorageS3Configuration(const StorageS3Configuration & other); + std::string getTypeName() const override { return type_name; } + std::string getEngineName() const override { return url.storage_name; } + Path getPath() const override { return url.key; } void setPath(const Path & path) override { url.key = path; } @@ -26,6 +32,7 @@ public: String getNamespace() const override { return url.bucket; } String getDataSourceDescription() override; + StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; void validateNamespace(const String & name) const override; @@ -34,8 +41,8 @@ public: bool isStaticConfiguration() const override { return static_configuration; } ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT - static void addStructureAndFormatToArgs( - ASTs & args, const String & structure, const String & format, ContextPtr context); + void addStructureAndFormatToArgs( + ASTs & args, const String & structure, const String & format, ContextPtr context) override; private: void fromNamedCollection(const NamedCollection & collection) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 13f3557d927..441639629a3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -11,10 +11,9 @@ #include #include #include -#include #include #include -#include +#include #include #include #include @@ -25,53 +24,13 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int DATABASE_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; } -template -std::unique_ptr getStorageMetadata( - ObjectStoragePtr object_storage, - const StorageObjectStorageConfigurationPtr & configuration, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints, - std::optional format_settings, - const String & comment, - const std::string & engine_name, - const ContextPtr & context) -{ - using Storage = StorageObjectStorage; - - auto storage_metadata = std::make_unique(); - if (columns.empty()) - { - auto fetched_columns = Storage::getTableStructureFromData(object_storage, configuration, format_settings, context); - storage_metadata->setColumns(fetched_columns); - } - else if (!columns.hasOnlyOrdinary()) - { - /// We don't allow special columns. - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine {} doesn't support special columns " - "like MATERIALIZED, ALIAS or EPHEMERAL", engine_name); - } - else - { - if (configuration->format == "auto") - Storage::setFormatFromData(object_storage, configuration, format_settings, context); - - storage_metadata->setColumns(columns); - } - storage_metadata->setConstraints(constraints); - storage_metadata->setComment(comment); - return storage_metadata; -} - -template -StorageObjectStorage::StorageObjectStorage( +StorageObjectStorage::StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, - const String & engine_name_, ContextPtr context, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -80,16 +39,13 @@ StorageObjectStorage::StorageObjectStorage( std::optional format_settings_, bool distributed_processing_, ASTPtr partition_by_) - : IStorage(table_id_, getStorageMetadata( - object_storage_, configuration_, columns_, constraints_, format_settings_, - comment, engine_name, context)) - , engine_name(engine_name_) + : IStorage(table_id_) + , configuration(configuration_) + , object_storage(object_storage_) , format_settings(format_settings_) , partition_by(partition_by_) , distributed_processing(distributed_processing_) - , log(getLogger("Storage" + engine_name_)) - , object_storage(object_storage_) - , configuration(configuration_) + , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { FormatFactory::instance().checkFormatName(configuration->format); configuration->check(context); @@ -98,46 +54,41 @@ StorageObjectStorage::StorageObjectStorage( for (const auto & key : configuration->getPaths()) objects.emplace_back(key); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(getInMemoryMetadataPtr()->getColumns())); + auto metadata = getStorageMetadata( + object_storage_, configuration_, columns_, + constraints_, format_settings_, comment, context); + + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setInMemoryMetadata(std::move(metadata)); } -template -bool StorageObjectStorage::prefersLargeBlocks() const +String StorageObjectStorage::getName() const +{ + return configuration->getEngineName(); +} + +bool StorageObjectStorage::prefersLargeBlocks() const { return FormatFactory::instance().checkIfOutputFormatPrefersLargeBlocks(configuration->format); } -template -bool StorageObjectStorage::parallelizeOutputAfterReading(ContextPtr context) const +bool StorageObjectStorage::parallelizeOutputAfterReading(ContextPtr context) const { return FormatFactory::instance().checkParallelizeOutputAfterReading(configuration->format, context); } -template -bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) const +bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) const { return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context, format_settings); } -template -void StorageObjectStorage::updateConfiguration(ContextPtr context) +void StorageObjectStorage::updateConfiguration(ContextPtr context) { if (!configuration->isStaticConfiguration()) object_storage->applyNewSettings(context->getConfigRef(), "s3.", context); } -template -SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) -{ - static SchemaCache schema_cache( - context->getConfigRef().getUInt( - StorageSettings::SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING, - DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; -} - -template -void StorageObjectStorage::read( +void StorageObjectStorage::read( QueryPlan & query_plan, const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -155,13 +106,12 @@ void StorageObjectStorage::read( getName()); } - const auto read_from_format_info = prepareReadingFromFormat(column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); + const auto read_from_format_info = prepareReadingFromFormat( + column_names, storage_snapshot, supportsSubsetOfColumns(local_context)); const bool need_only_count = (query_info.optimize_trivial_count || read_from_format_info.requested_columns.empty()) && local_context->getSettingsRef().optimize_count_from_files; - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII SOURCE HEADER: {}", read_from_format_info.source_header.dumpStructure()); - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII FORMAT HEADER: {}", read_from_format_info.format_header.dumpStructure()); - auto read_step = std::make_unique( + auto read_step = std::make_unique( object_storage, configuration, getName(), @@ -170,23 +120,18 @@ void StorageObjectStorage::read( query_info, storage_snapshot, format_settings, - StorageSettings::create(local_context->getSettingsRef()), distributed_processing, std::move(read_from_format_info), getSchemaCache(local_context), need_only_count, local_context, max_block_size, - num_streams, - StorageSettings::ObjectStorageThreads(), - StorageSettings::ObjectStorageThreadsActive(), - StorageSettings::ObjectStorageThreadsScheduled()); + num_streams); query_plan.addStep(std::move(read_step)); } -template -SinkToStoragePtr StorageObjectStorage::write( +SinkToStoragePtr StorageObjectStorage::write( const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context, @@ -194,7 +139,7 @@ SinkToStoragePtr StorageObjectStorage::write( { updateConfiguration(local_context); const auto sample_block = metadata_snapshot->getSampleBlock(); - const auto & query_settings = StorageSettings::create(local_context->getSettingsRef()); + const auto & settings = configuration->getQuerySettings(local_context); if (configuration->withWildcard()) { @@ -209,23 +154,22 @@ SinkToStoragePtr StorageObjectStorage::write( if (partition_by_ast) { - LOG_TEST(log, "Using PartitionedSink for {}", configuration->getPath()); return std::make_shared( - object_storage, configuration, query_settings, - format_settings, sample_block, local_context, partition_by_ast); + object_storage, configuration, format_settings, sample_block, local_context, partition_by_ast); } } if (configuration->withGlobs()) { - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, - "{} key '{}' contains globs, so the table is in readonly mode", - getName(), configuration->getPath()); + throw Exception( + ErrorCodes::DATABASE_ACCESS_DENIED, + "{} key '{}' contains globs, so the table is in readonly mode", + getName(), configuration->getPath()); } auto & paths = configuration->getPaths(); if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( - *object_storage, *configuration, query_settings, paths.front(), paths.size())) + *object_storage, *configuration, settings, paths.front(), paths.size())) { paths.push_back(*new_key); } @@ -238,9 +182,11 @@ SinkToStoragePtr StorageObjectStorage::write( local_context); } -template -void StorageObjectStorage::truncate( - const ASTPtr &, const StorageMetadataPtr &, ContextPtr, TableExclusiveLockHolder &) +void StorageObjectStorage::truncate( + const ASTPtr &, + const StorageMetadataPtr &, + ContextPtr, + TableExclusiveLockHolder &) { if (configuration->withGlobs()) { @@ -257,34 +203,37 @@ void StorageObjectStorage::truncate( object_storage->removeObjectsIfExist(objects); } -template -std::unique_ptr StorageObjectStorage::createReadBufferIterator( +std::unique_ptr StorageObjectStorage::createReadBufferIterator( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, ObjectInfos & read_keys, const ContextPtr & context) { - const auto settings = StorageSettings::create(context->getSettingsRef()); auto file_iterator = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, settings, /* distributed_processing */false, - context, /* predicate */{}, /* virtual_columns */{}, &read_keys, - StorageSettings::ObjectStorageThreads(), StorageSettings::ObjectStorageThreadsActive(), StorageSettings::ObjectStorageThreadsScheduled()); + configuration, + object_storage, + false/* distributed_processing */, + context, + {}/* predicate */, + {}/* virtual_columns */, + &read_keys); return std::make_unique( object_storage, configuration, file_iterator, - format_settings, StorageSettings::create(context->getSettingsRef()), getSchemaCache(context), read_keys, context); + format_settings, getSchemaCache(context, configuration->getTypeName()), read_keys, context); } -template -ColumnsDescription StorageObjectStorage::getTableStructureFromData( +ColumnsDescription StorageObjectStorage::getTableStructureFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, const ContextPtr & context) { ObjectInfos read_keys; - auto read_buffer_iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto read_buffer_iterator = createReadBufferIterator( + object_storage, configuration, format_settings, read_keys, context); + if (configuration->format == "auto") { auto [columns, format] = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context); @@ -297,20 +246,34 @@ ColumnsDescription StorageObjectStorage::getTableStructureFromD } } -template -void StorageObjectStorage::setFormatFromData( +void StorageObjectStorage::setFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, const ContextPtr & context) { ObjectInfos read_keys; - auto read_buffer_iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto read_buffer_iterator = createReadBufferIterator( + object_storage, configuration, format_settings, read_keys, context); configuration->format = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context).second; } -template class StorageObjectStorage; -template class StorageObjectStorage; -template class StorageObjectStorage; +SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) +{ + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_" + configuration->getTypeName(), + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; +} + +SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name) +{ + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_" + storage_type_name, + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; +} } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index a2112f7ed01..3dbe010e406 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -1,31 +1,22 @@ #pragma once - -#include #include #include -#include #include #include #include - namespace DB { -struct SelectQueryInfo; class StorageObjectStorageConfiguration; -struct S3StorageSettings; -struct HDFSStorageSettings; -struct AzureStorageSettings; -class PullingPipelineExecutor; -using ReadTaskCallback = std::function; -class IOutputFormat; -class IInputFormat; -class SchemaCache; class ReadBufferIterator; +class SchemaCache; - -template +/** + * A general class containing implementation for external table engines + * such as StorageS3, StorageAzure, StorageHDFS. + * Works with an object of IObjectStorage class. + */ class StorageObjectStorage : public IStorage { public: @@ -35,10 +26,26 @@ public: using ObjectInfoPtr = std::shared_ptr; using ObjectInfos = std::vector; + struct QuerySettings + { + /// Insert settings: + bool truncate_on_insert; + bool create_new_file_on_insert; + + /// Schema inference settings: + bool schema_inference_use_cache; + SchemaInferenceMode schema_inference_mode; + + /// List settings: + bool skip_empty_files; + size_t list_object_keys_size; + bool throw_on_zero_files_match; + bool ignore_non_existent_file; + }; + StorageObjectStorage( ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, - const String & engine_name_, ContextPtr context_, const StorageID & table_id_, const ColumnsDescription & columns_, @@ -48,17 +55,17 @@ public: bool distributed_processing_ = false, ASTPtr partition_by_ = nullptr); - String getName() const override { return engine_name; } + String getName() const override; void read( QueryPlan & query_plan, - const Names &, - const StorageSnapshotPtr &, - SelectQueryInfo &, - ContextPtr, - QueryProcessingStage::Enum, - size_t, - size_t) override; + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr local_context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; SinkToStoragePtr write( const ASTPtr & query, @@ -84,7 +91,9 @@ public: bool parallelizeOutputAfterReading(ContextPtr context) const override; - static SchemaCache & getSchemaCache(const ContextPtr & context); + SchemaCache & getSchemaCache(const ContextPtr & context); + + static SchemaCache & getSchemaCache(const ContextPtr & context, const std::string & storage_type_name); static ColumnsDescription getTableStructureFromData( const ObjectStoragePtr & object_storage, @@ -108,19 +117,15 @@ protected: ObjectInfos & read_keys, const ContextPtr & context); + ConfigurationPtr configuration; + const ObjectStoragePtr object_storage; const std::string engine_name; - std::optional format_settings; + const std::optional format_settings; const ASTPtr partition_by; const bool distributed_processing; LoggerPtr log; - ObjectStoragePtr object_storage; - ConfigurationPtr configuration; std::mutex configuration_update_mutex; }; -using StorageS3 = StorageObjectStorage; -using StorageAzureBlob = StorageObjectStorage; -using StorageHDFS = StorageObjectStorage; - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index f023bb068d4..72a35ae33eb 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB { @@ -24,47 +25,34 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -template -StorageObjectStorageCluster::StorageObjectStorageCluster( +StorageObjectStorageCluster::StorageObjectStorageCluster( const String & cluster_name_, - const Storage::ConfigurationPtr & configuration_, + ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, - const String & engine_name_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_) - : IStorageCluster(cluster_name_, - table_id_, - getLogger(fmt::format("{}({})", engine_name_, table_id_.table_name))) - , engine_name(engine_name_) + : IStorageCluster( + cluster_name_, table_id_, getLogger(fmt::format("{}({})", configuration_->getEngineName(), table_id_.table_name))) , configuration{configuration_} , object_storage(object_storage_) { configuration->check(context_); - StorageInMemoryMetadata storage_metadata; + auto metadata = getStorageMetadata( + object_storage, configuration, columns_, constraints_, + {}/* format_settings */, ""/* comment */, context_); - if (columns_.empty()) - { - ColumnsDescription columns = Storage::getTableStructureFromData(object_storage, configuration, /*format_settings=*/std::nullopt, context_); - storage_metadata.setColumns(columns); - } - else - { - if (configuration->format == "auto") - StorageS3::setFormatFromData(object_storage, configuration, /*format_settings=*/std::nullopt, context_); - - storage_metadata.setColumns(columns_); - } - - storage_metadata.setConstraints(constraints_); - setInMemoryMetadata(storage_metadata); - - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); + setInMemoryMetadata(std::move(metadata)); } -template -void StorageObjectStorageCluster::updateQueryToSendIfNeeded( +std::string StorageObjectStorageCluster::getName() const +{ + return configuration->getEngineName(); +} + +void StorageObjectStorageCluster::updateQueryToSendIfNeeded( ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) @@ -72,24 +60,32 @@ void StorageObjectStorageCluster::up ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); if (!expression_list) { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected SELECT query from table function {}, got '{}'", - engine_name, queryToString(query)); + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Expected SELECT query from table function {}, got '{}'", + configuration->getEngineName(), queryToString(query)); } - TableFunction::updateStructureAndFormatArgumentsIfNeeded( - expression_list->children, - storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), - configuration->format, - context); + ASTs & args = expression_list->children; + const auto & structure = storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(); + if (args.empty()) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Unexpected empty list of arguments for {}Cluster table function", + configuration->getEngineName()); + } + + ASTPtr cluster_name_arg = args.front(); + args.erase(args.begin()); + configuration->addStructureAndFormatToArgs(args, structure, configuration->format, context); + args.insert(args.begin(), cluster_name_arg); } -template -RemoteQueryExecutor::Extension -StorageObjectStorageCluster::getTaskIteratorExtension( +RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExtension( const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { - const auto settings = StorageSettings::create(local_context->getSettingsRef()); + const auto settings = configuration->getQuerySettings(local_context); auto iterator = std::make_shared( object_storage, configuration, predicate, virtual_columns, local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, @@ -106,17 +102,4 @@ StorageObjectStorageCluster::getTask return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) }; } - -#if USE_AWS_S3 -template class StorageObjectStorageCluster; -#endif - -#if USE_AZURE_BLOB_STORAGE -template class StorageObjectStorageCluster; -#endif - -#if USE_HDFS -template class StorageObjectStorageCluster; -#endif - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index ac894e14f24..2db8f5c352e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -11,32 +11,25 @@ namespace DB { -class StorageS3Settings; -class StorageAzureBlobSettings; - class Context; -template class StorageObjectStorageCluster : public IStorageCluster { public: - using Storage = StorageObjectStorage; - using TableFunction = TableFunctionObjectStorageCluster; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; StorageObjectStorageCluster( const String & cluster_name_, - const Storage::ConfigurationPtr & configuration_, + ConfigurationPtr configuration_, ObjectStoragePtr object_storage_, - const String & engine_name_, const StorageID & table_id_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, ContextPtr context_); - std::string getName() const override { return engine_name; } + std::string getName() const override; - RemoteQueryExecutor::Extension - getTaskIteratorExtension( + RemoteQueryExecutor::Extension getTaskIteratorExtension( const ActionsDAG::Node * predicate, const ContextPtr & context) const override; @@ -53,20 +46,9 @@ private: const ContextPtr & context) override; const String engine_name; - const Storage::ConfigurationPtr configuration; + const StorageObjectStorage::ConfigurationPtr configuration; const ObjectStoragePtr object_storage; NamesAndTypesList virtual_columns; }; - -#if USE_AWS_S3 -using StorageS3Cluster = StorageObjectStorageCluster; -#endif -#if USE_AZURE_BLOB_STORAGE -using StorageAzureBlobCluster = StorageObjectStorageCluster; -#endif -#if USE_HDFS -using StorageHDFSCluster = StorageObjectStorageCluster; -#endif - } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 647575aaa90..34965174bf9 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include "StorageObjectStorage.h" #include namespace fs = std::filesystem; @@ -27,6 +28,9 @@ public: ContextPtr local_context, bool with_table_structure); + virtual std::string getTypeName() const = 0; + virtual std::string getEngineName() const = 0; + virtual Path getPath() const = 0; virtual void setPath(const Path & path) = 0; @@ -36,6 +40,9 @@ public: virtual String getDataSourceDescription() = 0; virtual String getNamespace() const = 0; + virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; + virtual void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; bool withWildcard() const; bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h b/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h deleted file mode 100644 index 606456011c3..00000000000 --- a/src/Storages/ObjectStorage/StorageObjectStorageQuerySettings.h +++ /dev/null @@ -1,102 +0,0 @@ -#pragma once -#include -#include -#include - -namespace CurrentMetrics -{ - extern const Metric ObjectStorageAzureThreads; - extern const Metric ObjectStorageAzureThreadsActive; - extern const Metric ObjectStorageAzureThreadsScheduled; - - extern const Metric ObjectStorageS3Threads; - extern const Metric ObjectStorageS3ThreadsActive; - extern const Metric ObjectStorageS3ThreadsScheduled; -} - -namespace DB -{ - -struct StorageObjectStorageSettings -{ - bool truncate_on_insert; - bool create_new_file_on_insert; - bool schema_inference_use_cache; - SchemaInferenceMode schema_inference_mode; - bool skip_empty_files; - size_t list_object_keys_size; - bool throw_on_zero_files_match; - bool ignore_non_existent_file; -}; - -struct S3StorageSettings -{ - static StorageObjectStorageSettings create(const Settings & settings) - { - return StorageObjectStorageSettings{ - .truncate_on_insert = settings.s3_truncate_on_insert, - .create_new_file_on_insert = settings.s3_create_new_file_on_insert, - .schema_inference_use_cache = settings.schema_inference_use_cache_for_s3, - .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.s3_skip_empty_files, - .list_object_keys_size = settings.s3_list_object_keys_size, - .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, - .ignore_non_existent_file = settings.s3_ignore_file_doesnt_exist, - }; - } - - static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_s3"; - - static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageS3Threads; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageS3ThreadsActive; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageS3ThreadsScheduled; } /// NOLINT -}; - -struct AzureStorageSettings -{ - static StorageObjectStorageSettings create(const Settings & settings) - { - return StorageObjectStorageSettings{ - .truncate_on_insert = settings.azure_truncate_on_insert, - .create_new_file_on_insert = settings.azure_create_new_file_on_insert, - .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, - .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure - .list_object_keys_size = settings.azure_list_object_keys_size, - .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, - .ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist, - }; - } - - static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_azure"; - - static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageAzureThreads; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageAzureThreadsActive; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageAzureThreadsScheduled; } /// NOLINT -}; - -struct HDFSStorageSettings -{ - static StorageObjectStorageSettings create(const Settings & settings) - { - return StorageObjectStorageSettings{ - .truncate_on_insert = settings.hdfs_truncate_on_insert, - .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, - .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, - .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.hdfs_skip_empty_files, /// TODO: add setting for hdfs - .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs - .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, - .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, - }; - } - - static constexpr auto SCHEMA_CACHE_MAX_ELEMENTS_CONFIG_SETTING = "schema_inference_cache_max_elements_for_hdfs"; - - /// TODO: s3 -> hdfs - static CurrentMetrics::Metric ObjectStorageThreads() { return CurrentMetrics::ObjectStorageS3Threads; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsActive() { return CurrentMetrics::ObjectStorageS3ThreadsActive; } /// NOLINT - static CurrentMetrics::Metric ObjectStorageThreadsScheduled() { return CurrentMetrics::ObjectStorageS3ThreadsScheduled; } /// NOLINT -}; - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 42371764920..62367a6b933 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -103,7 +103,6 @@ void StorageObjectStorageSink::release() PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, - const StorageObjectStorageSettings & query_settings_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, @@ -111,7 +110,7 @@ PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( : PartitionedSink(partition_by, context_, sample_block_) , object_storage(object_storage_) , configuration(configuration_) - , query_settings(query_settings_) + , query_settings(configuration_->getQuerySettings(context_)) , format_settings(format_settings_) , sample_block(sample_block_) , context(context_) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 38805332a35..6c2f73e40e3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -1,7 +1,6 @@ #pragma once #include #include -#include #include #include @@ -47,7 +46,6 @@ public: PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, StorageObjectStorageConfigurationPtr configuration_, - const StorageObjectStorageSettings & query_settings_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, @@ -61,7 +59,7 @@ private: ObjectStoragePtr object_storage; StorageObjectStorageConfigurationPtr configuration; - const StorageObjectStorageSettings query_settings; + const StorageObjectStorage::QuerySettings query_settings; const std::optional format_settings; const Block sample_block; const ContextPtr context; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 82824b0e7f7..3101a7ebf51 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -20,6 +19,13 @@ namespace ProfileEvents extern const Event EngineFileLikeReadFiles; } +namespace CurrentMetrics +{ + extern const Metric StorageObjectStorageThreads; + extern const Metric StorageObjectStorageThreadsActive; + extern const Metric StorageObjectStorageThreadsScheduled; +} + namespace DB { @@ -37,16 +43,12 @@ StorageObjectStorageSource::StorageObjectStorageSource( ConfigurationPtr configuration_, const ReadFromFormatInfo & info, std::optional format_settings_, - const StorageObjectStorageSettings & query_settings_, + const StorageObjectStorage::QuerySettings & query_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, bool need_only_count_, - SchemaCache & schema_cache_, - std::shared_ptr reader_pool_, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_) + SchemaCache & schema_cache_) : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) , name(std::move(name_)) @@ -57,13 +59,14 @@ StorageObjectStorageSource::StorageObjectStorageSource( , max_block_size(max_block_size_) , need_only_count(need_only_count_) , read_from_format_info(info) - , create_reader_pool(reader_pool_) + , create_reader_pool(std::make_shared( + CurrentMetrics::StorageObjectStorageThreads, + CurrentMetrics::StorageObjectStorageThreadsActive, + CurrentMetrics::StorageObjectStorageThreadsScheduled, + 1/* max_threads */)) , columns_desc(info.columns_description) , file_iterator(file_iterator_) , schema_cache(schema_cache_) - , metric_threads(metric_threads_) - , metric_threads_active(metric_threads_active_) - , metric_threads_scheduled(metric_threads_scheduled_) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { } @@ -76,26 +79,23 @@ StorageObjectStorageSource::~StorageObjectStorageSource() std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, - const StorageObjectStorageSettings & settings, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_, std::function file_progress_callback) { if (distributed_processing) return std::make_shared( local_context->getReadTaskCallback(), - local_context->getSettingsRef().max_threads, - metric_threads_, metric_threads_active_, metric_threads_scheduled_); + local_context->getSettingsRef().max_threads); if (configuration->isNamespaceWithGlobs()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); + auto settings = configuration->getQuerySettings(local_context); + if (configuration->isPathWithGlobs()) { /// Iterate through disclosed globs and make a source for each file @@ -568,7 +568,8 @@ StorageObjectStorageSource::ReaderHolder::ReaderHolder( { } -StorageObjectStorageSource::ReaderHolder & StorageObjectStorageSource::ReaderHolder::operator=(ReaderHolder && other) noexcept +StorageObjectStorageSource::ReaderHolder & +StorageObjectStorageSource::ReaderHolder::operator=(ReaderHolder && other) noexcept { /// The order of destruction is important. /// reader uses pipeline, pipeline uses read_buf. @@ -581,15 +582,15 @@ StorageObjectStorageSource::ReaderHolder & StorageObjectStorageSource::ReaderHol } StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( - const ReadTaskCallback & callback_, - size_t max_threads_count, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_) + const ReadTaskCallback & callback_, size_t max_threads_count) : IIterator("ReadTaskIterator") , callback(callback_) { - ThreadPool pool(metric_threads_, metric_threads_active_, metric_threads_scheduled_, max_threads_count); + ThreadPool pool( + CurrentMetrics::StorageObjectStorageThreads, + CurrentMetrics::StorageObjectStorageThreadsActive, + CurrentMetrics::StorageObjectStorageThreadsScheduled, max_threads_count); + auto pool_scheduler = threadPoolCallbackRunnerUnsafe(pool, "ReadTaskIter"); std::vector> keys; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index f75bfc390bb..3c2cc3f80cd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -28,16 +27,12 @@ public: ConfigurationPtr configuration, const ReadFromFormatInfo & info, std::optional format_settings_, - const StorageObjectStorageSettings & query_settings_, + const StorageObjectStorage::QuerySettings & query_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, bool need_only_count_, - SchemaCache & schema_cache_, - std::shared_ptr reader_pool_, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_); + SchemaCache & schema_cache_); ~StorageObjectStorageSource() override; @@ -53,15 +48,11 @@ public: static std::shared_ptr createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, - const StorageObjectStorageSettings & settings, bool distributed_processing, const ContextPtr & local_context, const ActionsDAG::Node * predicate, const NamesAndTypesList & virtual_columns, ObjectInfos * read_keys, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_, std::function file_progress_callback = {}); protected: @@ -69,7 +60,7 @@ protected: ObjectStoragePtr object_storage; const ConfigurationPtr configuration; const std::optional format_settings; - const StorageObjectStorageSettings query_settings; + const StorageObjectStorage::QuerySettings query_settings; const UInt64 max_block_size; const bool need_only_count; const ReadFromFormatInfo read_from_format_info; @@ -79,10 +70,6 @@ protected: SchemaCache & schema_cache; bool initialized = false; - const CurrentMetrics::Metric metric_threads; - const CurrentMetrics::Metric metric_threads_active; - const CurrentMetrics::Metric metric_threads_scheduled; - size_t total_rows_in_file = 0; LoggerPtr log = getLogger("StorageObjectStorageSource"); @@ -149,12 +136,7 @@ protected: class StorageObjectStorageSource::ReadTaskIterator : public IIterator { public: - ReadTaskIterator( - const ReadTaskCallback & callback_, - size_t max_threads_count, - CurrentMetrics::Metric metric_threads_, - CurrentMetrics::Metric metric_threads_active_, - CurrentMetrics::Metric metric_threads_scheduled_); + ReadTaskIterator(const ReadTaskCallback & callback_, size_t max_threads_count); size_t estimatedKeysCount() override { return buffer.size(); } diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 9caab709081..94d6dadee3b 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -1,8 +1,6 @@ #include #include #include -#include - namespace DB { @@ -15,15 +13,15 @@ namespace ErrorCodes std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, const StorageObjectStorageConfiguration & configuration, - const StorageObjectStorageSettings & query_settings, + const StorageObjectStorage::QuerySettings & settings, const String & key, size_t sequence_number) { - if (query_settings.truncate_on_insert + if (settings.truncate_on_insert || !object_storage.exists(StoredObject(key))) return std::nullopt; - if (query_settings.create_new_file_on_insert) + if (settings.create_new_file_on_insert) { auto pos = key.find_first_of('.'); String new_key; @@ -45,4 +43,38 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( configuration.getNamespace(), key); } +StorageInMemoryMetadata getStorageMetadata( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfigurationPtr & configuration, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + std::optional format_settings, + const String & comment, + const ContextPtr & context) +{ + StorageInMemoryMetadata storage_metadata; + if (columns.empty()) + { + auto fetched_columns = StorageObjectStorage::getTableStructureFromData(object_storage, configuration, format_settings, context); + storage_metadata.setColumns(fetched_columns); + } + else if (!columns.hasOnlyOrdinary()) + { + /// We don't allow special columns. + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Special columns are not supported for {} storage" + "like MATERIALIZED, ALIAS or EPHEMERAL", configuration->getTypeName()); + } + else + { + if (configuration->format == "auto") + StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context); + + storage_metadata.setColumns(columns); + } + storage_metadata.setConstraints(constraints); + storage_metadata.setComment(comment); + return storage_metadata; +} + } diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index afc0f31a33f..37bd49a77c0 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -1,18 +1,30 @@ #pragma once #include +#include "StorageObjectStorage.h" namespace DB { class IObjectStorage; class StorageObjectStorageConfiguration; +using StorageObjectStorageConfigurationPtr = std::shared_ptr; struct StorageObjectStorageSettings; std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, const StorageObjectStorageConfiguration & configuration, - const StorageObjectStorageSettings & query_settings, + const StorageObjectStorage::QuerySettings & settings, const std::string & key, size_t sequence_number); + +StorageInMemoryMetadata getStorageMetadata( + ObjectStoragePtr object_storage, + const StorageObjectStorageConfigurationPtr & configuration, + const ColumnsDescription & columns, + const ConstraintsDescription & constraints, + std::optional format_settings, + const String & comment, + const ContextPtr & context); + } diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 3271b766f68..06b8aefb716 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -2,22 +2,23 @@ #include #include #include +#include #include #include namespace DB { +#if USE_AWS_S3 || USE_AZURE_BLOB_STORAGE || USE_HDFS + namespace ErrorCodes { extern const int BAD_ARGUMENTS; } -template -static std::shared_ptr> createStorageObjectStorage( +static std::shared_ptr createStorageObjectStorage( const StorageFactory::Arguments & args, - typename StorageObjectStorage::ConfigurationPtr configuration, - const String & engine_name, + typename StorageObjectStorage::ConfigurationPtr configuration, ContextPtr context) { auto & engine_args = args.engine_args; @@ -54,10 +55,9 @@ static std::shared_ptr> createStorageObjec if (args.storage_def->partition_by) partition_by = args.storage_def->partition_by->clone(); - return std::make_shared>( + return std::make_shared( configuration, configuration->createObjectStorage(context), - engine_name, args.getContext(), args.table_id, args.columns, @@ -68,6 +68,8 @@ static std::shared_ptr> createStorageObjec partition_by); } +#endif + #if USE_AZURE_BLOB_STORAGE void registerStorageAzure(StorageFactory & factory) { @@ -76,7 +78,7 @@ void registerStorageAzure(StorageFactory & factory) auto context = args.getLocalContext(); auto configuration = std::make_shared(); StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, "Azure", context); + return createStorageObjectStorage(args, configuration, context); }, { .supports_settings = true, @@ -95,7 +97,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) auto context = args.getLocalContext(); auto configuration = std::make_shared(); StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, name, context); + return createStorageObjectStorage(args, configuration, context); }, { .supports_settings = true, @@ -130,7 +132,7 @@ void registerStorageHDFS(StorageFactory & factory) auto context = args.getLocalContext(); auto configuration = std::make_shared(); StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, "HDFS", context); + return createStorageObjectStorage(args, configuration, context); }, { .supports_settings = true, diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index c1b45108b36..5a1f0f6dd04 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -7,7 +7,6 @@ #include #include #include -#include #include @@ -21,7 +20,7 @@ struct ObjectMetadata; class StorageS3QueueSource : public ISource, WithContext { public: - using Storage = StorageObjectStorage; + using Storage = StorageObjectStorage; using ConfigurationPtr = Storage::ConfigurationPtr; using GlobIterator = StorageObjectStorageSource::GlobIterator; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index c5799d23abd..6b504b0d986 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -37,13 +37,6 @@ namespace ProfileEvents extern const Event S3ListObjects; } -namespace CurrentMetrics -{ - extern const Metric ObjectStorageS3Threads; - extern const Metric ObjectStorageS3ThreadsActive; - extern const Metric ObjectStorageS3ThreadsScheduled; -} - namespace DB { @@ -151,14 +144,14 @@ StorageS3Queue::StorageS3Queue( StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = Storage::getTableStructureFromData(object_storage, configuration, format_settings, context_); + auto columns = StorageObjectStorage::getTableStructureFromData(object_storage, configuration, format_settings, context_); storage_metadata.setColumns(columns); } else { if (configuration->format == "auto") { - StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context_); + StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context_); } storage_metadata.setColumns(columns_); } @@ -370,26 +363,18 @@ std::shared_ptr StorageS3Queue::createSource( size_t max_block_size, ContextPtr local_context) { - auto threadpool = std::make_shared(CurrentMetrics::ObjectStorageS3Threads, - CurrentMetrics::ObjectStorageS3ThreadsActive, - CurrentMetrics::ObjectStorageS3ThreadsScheduled, - /* max_threads */1); auto internal_source = std::make_unique( getName(), object_storage, configuration, info, format_settings, - S3StorageSettings::create(local_context->getSettingsRef()), + configuration->getQuerySettings(local_context), local_context, max_block_size, file_iterator, false, - Storage::getSchemaCache(local_context), - threadpool, - CurrentMetrics::ObjectStorageS3Threads, - CurrentMetrics::ObjectStorageS3ThreadsActive, - CurrentMetrics::ObjectStorageS3ThreadsScheduled); + StorageObjectStorage::getSchemaCache(local_context, configuration->getTypeName())); auto file_deleter = [=, this](const std::string & path) mutable { @@ -596,7 +581,7 @@ void StorageS3Queue::checkTableStructure(const String & zookeeper_prefix, const std::shared_ptr StorageS3Queue::createFileIterator(ContextPtr local_context, const ActionsDAG::Node * predicate) { - auto settings = S3StorageSettings::create(local_context->getSettingsRef()); + auto settings = configuration->getQuerySettings(local_context); auto glob_iterator = std::make_unique( object_storage, configuration, predicate, getVirtualsList(), local_context, nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match); diff --git a/src/Storages/S3Queue/StorageS3Queue.h b/src/Storages/S3Queue/StorageS3Queue.h index 72c41a6a694..1464e15ebf2 100644 --- a/src/Storages/S3Queue/StorageS3Queue.h +++ b/src/Storages/S3Queue/StorageS3Queue.h @@ -21,8 +21,7 @@ class S3QueueFilesMetadata; class StorageS3Queue : public IStorage, WithContext { public: - using Storage = StorageObjectStorage; - using ConfigurationPtr = Storage::ConfigurationPtr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; StorageS3Queue( std::unique_ptr s3queue_settings_, diff --git a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index 9ef64f2b90d..a2d3f342a63 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -9,6 +9,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -74,14 +77,14 @@ void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, C { fillDataImpl(res_columns, StorageFile::getSchemaCache(context), "File"); #if USE_AWS_S3 - fillDataImpl(res_columns, StorageS3::getSchemaCache(context), "S3"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageS3Configuration::type_name), "S3"); #endif #if USE_HDFS - fillDataImpl(res_columns, StorageHDFS::getSchemaCache(context), "HDFS"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageHDFSConfiguration::type_name), "HDFS"); #endif fillDataImpl(res_columns, StorageURL::getSchemaCache(context), "URL"); #if USE_AZURE_BLOB_STORAGE - fillDataImpl(res_columns, StorageAzureBlob::getSchemaCache(context), "Azure"); /// FIXME + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageAzureBlobConfiguration::type_name), "Azure"); #endif } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 8cbd855bb96..02c8c623e61 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -39,7 +39,7 @@ protected: columns = cached_columns; StoragePtr storage = Storage::create( - configuration, context, "", StorageID(TableFunction::getDatabaseName(), table_name), + configuration, context, StorageID(TableFunction::getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, std::nullopt, LoadingStrictnessLevel::CREATE); storage->startup(); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 9223642a7e6..2b5c774ff78 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -27,27 +27,27 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -template +template ObjectStoragePtr TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::getObjectStorage(const ContextPtr & context, bool create_readonly) const + Definition, Configuration>::getObjectStorage(const ContextPtr & context, bool create_readonly) const { if (!object_storage) object_storage = configuration->createObjectStorage(context, create_readonly); return object_storage; } -template +template StorageObjectStorageConfigurationPtr TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::getConfiguration() const + Definition, Configuration>::getConfiguration() const { if (!configuration) configuration = std::make_shared(); return configuration; } -template +template std::vector TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const + Definition, Configuration>::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const { auto & table_function_node = query_node_table_function->as(); auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); @@ -63,22 +63,21 @@ std::vector TableFunctionObjectStorage< return result; } -template -void TableFunctionObjectStorage::updateStructureAndFormatArgumentsIfNeeded( +template +void TableFunctionObjectStorage::updateStructureAndFormatArgumentsIfNeeded( ASTs & args, const String & structure, const String & format, const ContextPtr & context) { - Configuration::addStructureAndFormatToArgs(args, structure, format, context); + Configuration().addStructureAndFormatToArgs(args, structure, format, context); } -template -void TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) +template +void TableFunctionObjectStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) { StorageObjectStorageConfiguration::initialize(*getConfiguration(), engine_args, local_context, true); } -template -void TableFunctionObjectStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) +template +void TableFunctionObjectStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Clone ast function, because we can modify its arguments like removing headers. auto ast_copy = ast_function->clone(); @@ -90,38 +89,38 @@ void TableFunctionObjectStorage::par parseArgumentsImpl(args, context); } -template +template ColumnsDescription TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const + Definition, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const { chassert(configuration); if (configuration->structure == "auto") { context->checkAccess(getSourceAccessType()); auto storage = getObjectStorage(context, !is_insert_query); - return StorageObjectStorage::getTableStructureFromData(storage, configuration, std::nullopt, context); + return StorageObjectStorage::getTableStructureFromData(storage, configuration, std::nullopt, context); } return parseColumnsListFromString(configuration->structure, context); } -template +template bool TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::supportsReadingSubsetOfColumns(const ContextPtr & context) + Definition, Configuration>::supportsReadingSubsetOfColumns(const ContextPtr & context) { chassert(configuration); return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); } -template +template std::unordered_set TableFunctionObjectStorage< - Definition, StorageSettings, Configuration>::getVirtualsToCheckBeforeUsingStructureHint() const + Definition, Configuration>::getVirtualsToCheckBeforeUsingStructureHint() const { return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); } -template -StoragePtr TableFunctionObjectStorage::executeImpl( +template +StoragePtr TableFunctionObjectStorage::executeImpl( const ASTPtr & /* ast_function */, ContextPtr context, const std::string & table_name, @@ -137,10 +136,9 @@ StoragePtr TableFunctionObjectStorage>( + StoragePtr storage = std::make_shared( configuration, getObjectStorage(context, !is_insert_query), - Definition::storage_type_name, context, StorageID(getDatabaseName(), table_name), columns, @@ -159,7 +157,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) { UNUSED(factory); #if USE_AWS_S3 - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -170,7 +168,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -181,7 +179,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .allow_readonly = false }); - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -191,7 +189,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) .categories{"DataLake"}}, .allow_readonly = false }); - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -204,7 +202,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) #endif #if USE_AZURE_BLOB_STORAGE - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -220,7 +218,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) }); #endif #if USE_HDFS - factory.registerFunction>( + factory.registerFunction>( { .allow_readonly = false }); @@ -228,21 +226,21 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) } #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 } diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index 9022f6e577f..bd43cae3697 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -85,7 +85,7 @@ struct HDFSDefinition " - uri, format, structure, compression_method\n"; }; -template +template class TableFunctionObjectStorage : public ITableFunction { public: @@ -142,14 +142,14 @@ protected: }; #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 } diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index 909ace788eb..ce78076dd21 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -14,8 +14,8 @@ namespace DB { -template -StoragePtr TableFunctionObjectStorageCluster::executeImpl( +template +StoragePtr TableFunctionObjectStorageCluster::executeImpl( const ASTPtr & /*function*/, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns, bool is_insert_query) const { @@ -34,10 +34,9 @@ StoragePtr TableFunctionObjectStorageClustergetClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY) { /// On worker node this filename won't contains globs - storage = std::make_shared>( + storage = std::make_shared( configuration, object_storage, - Definition::storage_type_name, context, StorageID(Base::getDatabaseName(), table_name), columns, @@ -49,11 +48,10 @@ StoragePtr TableFunctionObjectStorageCluster>( + storage = std::make_shared( ITableFunctionCluster::cluster_name, configuration, object_storage, - Definition::storage_type_name, StorageID(Base::getDatabaseName(), table_name), columns, ConstraintsDescription{}, @@ -107,14 +105,14 @@ void registerTableFunctionObjectStorageCluster(TableFunctionFactory & factory) } #if USE_AWS_S3 -template class TableFunctionObjectStorageCluster; +template class TableFunctionObjectStorageCluster; #endif #if USE_AZURE_BLOB_STORAGE -template class TableFunctionObjectStorageCluster; +template class TableFunctionObjectStorageCluster; #endif #if USE_HDFS -template class TableFunctionObjectStorageCluster; +template class TableFunctionObjectStorageCluster; #endif } diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index 21c2f8995dc..a8bc11b5e40 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -56,8 +56,8 @@ struct HDFSClusterDefinition " - cluster_name, uri, format, structure, compression_method\n"; }; -template -class TableFunctionObjectStorageCluster : public ITableFunctionCluster> +template +class TableFunctionObjectStorageCluster : public ITableFunctionCluster> { public: static constexpr auto name = Definition::name; @@ -67,7 +67,7 @@ public: String getSignature() const override { return signature; } protected: - using Base = TableFunctionObjectStorage; + using Base = TableFunctionObjectStorage; StoragePtr executeImpl( const ASTPtr & ast_function, @@ -86,14 +86,14 @@ protected: }; #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 } From 9eb9a76592dada103c40baa2c4acf5a3918b8e95 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 22 Apr 2024 14:18:46 +0100 Subject: [PATCH 039/158] Fix --- .../ObjectStorage/AzureBlob/Configuration.cpp | 1 + .../DataLakes/IStorageDataLake.h | 2 +- .../ObjectStorage/HDFS/Configuration.cpp | 1 + .../ObjectStorage/S3/Configuration.cpp | 1 + .../ObjectStorage/StorageObjectStorage.cpp | 47 +++++++++++-------- .../ObjectStorage/StorageObjectStorage.h | 10 +++- .../StorageObjectStorageCluster.cpp | 9 ++-- .../StorageObjectStorageConfiguration.cpp | 5 ++ .../StorageObjectStorageConfiguration.h | 2 +- src/Storages/ObjectStorage/Utils.cpp | 33 ++++++------- src/Storages/ObjectStorage/Utils.h | 10 ++-- src/Storages/S3Queue/StorageS3Queue.cpp | 21 +++------ .../TableFunctionObjectStorage.cpp | 5 +- 13 files changed, 80 insertions(+), 67 deletions(-) diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp index c9bc59d62aa..f268b812c03 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp @@ -77,6 +77,7 @@ void StorageAzureBlobConfiguration::check(ContextPtr context) const url_to_check = Poco::URI(connection_url); context->getGlobalContext()->getRemoteHostFilter().checkURL(url_to_check); + StorageObjectStorageConfiguration::check(context); } StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other) diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index e1851775925..144cc16939c 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -89,7 +89,7 @@ public: { ConfigurationPtr configuration = base_configuration->clone(); configuration->getPaths() = metadata->getDataFiles(); - return Storage::getTableStructureFromData( + return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 0062ac969ac..12e3f3adb12 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -34,6 +34,7 @@ void StorageHDFSConfiguration::check(ContextPtr context) const { context->getRemoteHostFilter().checkURL(Poco::URI(url)); checkHDFSURL(fs::path(url) / path.substr(1)); + StorageObjectStorageConfiguration::check(context); } ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 139d9004f8e..bfd61c647f8 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -54,6 +54,7 @@ void StorageS3Configuration::check(ContextPtr context) const validateNamespace(url.bucket); context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); + StorageObjectStorageConfiguration::check(context); } void StorageS3Configuration::validateNamespace(const String & name) const diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 441639629a3..36a8beba41a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -47,17 +47,19 @@ StorageObjectStorage::StorageObjectStorage( , distributed_processing(distributed_processing_) , log(getLogger(fmt::format("Storage{}({})", configuration->getEngineName(), table_id_.getFullTableName()))) { - FormatFactory::instance().checkFormatName(configuration->format); + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context); configuration->check(context); + StorageInMemoryMetadata metadata; + metadata.setColumns(columns); + metadata.setConstraints(constraints_); + metadata.setComment(comment); + StoredObjects objects; for (const auto & key : configuration->getPaths()) objects.emplace_back(key); - auto metadata = getStorageMetadata( - object_storage_, configuration_, columns_, - constraints_, format_settings_, comment, context); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); setInMemoryMetadata(std::move(metadata)); } @@ -224,7 +226,7 @@ std::unique_ptr StorageObjectStorage::createReadBufferIterat format_settings, getSchemaCache(context, configuration->getTypeName()), read_keys, context); } -ColumnsDescription StorageObjectStorage::getTableStructureFromData( +ColumnsDescription StorageObjectStorage::resolveSchemaFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, @@ -233,20 +235,11 @@ ColumnsDescription StorageObjectStorage::getTableStructureFromData( ObjectInfos read_keys; auto read_buffer_iterator = createReadBufferIterator( object_storage, configuration, format_settings, read_keys, context); - - if (configuration->format == "auto") - { - auto [columns, format] = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context); - configuration->format = format; - return columns; - } - else - { - return readSchemaFromFormat(configuration->format, format_settings, *read_buffer_iterator, context); - } + return readSchemaFromFormat( + configuration->format, format_settings, *read_buffer_iterator, context); } -void StorageObjectStorage::setFormatFromData( +std::string StorageObjectStorage::resolveFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, @@ -255,7 +248,23 @@ void StorageObjectStorage::setFormatFromData( ObjectInfos read_keys; auto read_buffer_iterator = createReadBufferIterator( object_storage, configuration, format_settings, read_keys, context); - configuration->format = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context).second; + return detectFormatAndReadSchema( + format_settings, *read_buffer_iterator, context).second; +} + +std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context) +{ + ObjectInfos read_keys; + auto read_buffer_iterator = createReadBufferIterator( + object_storage, configuration, format_settings, read_keys, context); + + auto [columns, format] = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context); + configuration->format = format; + return std::pair(columns, format); } SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3dbe010e406..d46a875bf42 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -95,13 +95,19 @@ public: static SchemaCache & getSchemaCache(const ContextPtr & context, const std::string & storage_type_name); - static ColumnsDescription getTableStructureFromData( + static ColumnsDescription resolveSchemaFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, const ContextPtr & context); - static void setFormatFromData( + static std::string resolveFormatFromData( + const ObjectStoragePtr & object_storage, + const ConfigurationPtr & configuration, + const std::optional & format_settings, + const ContextPtr & context); + + static std::pair resolveSchemaAndFormatFromData( const ObjectStoragePtr & object_storage, const ConfigurationPtr & configuration, const std::optional & format_settings, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 72a35ae33eb..f98fc32a3cc 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -38,10 +38,13 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( , configuration{configuration_} , object_storage(object_storage_) { + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, {}, context_); configuration->check(context_); - auto metadata = getStorageMetadata( - object_storage, configuration, columns_, constraints_, - {}/* format_settings */, ""/* comment */, context_); + + StorageInMemoryMetadata metadata; + metadata.setColumns(columns); + metadata.setConstraints(constraints_); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); setInMemoryMetadata(std::move(metadata)); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 61e569cee05..3635269db34 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -30,6 +30,11 @@ void StorageObjectStorageConfiguration::initialize( configuration.initialized = true; } +void StorageObjectStorageConfiguration::check(ContextPtr) const +{ + FormatFactory::instance().checkFormatName(format); +} + StorageObjectStorageConfiguration::StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other) { format = other.format; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h index 34965174bf9..c55362aa8bd 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h @@ -50,7 +50,7 @@ public: bool isNamespaceWithGlobs() const; virtual std::string getPathWithoutGlob() const; - virtual void check(ContextPtr context) const = 0; + virtual void check(ContextPtr context) const; virtual void validateNamespace(const String & /* name */) const {} virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 94d6dadee3b..2a7236ab196 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const StorageObjectStorage::QuerySettings & settings, const String & key, size_t sequence_number) @@ -43,38 +43,33 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( configuration.getNamespace(), key); } -StorageInMemoryMetadata getStorageMetadata( +void resolveSchemaAndFormat( + ColumnsDescription & columns, + std::string & format, ObjectStoragePtr object_storage, const StorageObjectStorageConfigurationPtr & configuration, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints, std::optional format_settings, - const String & comment, const ContextPtr & context) { - StorageInMemoryMetadata storage_metadata; if (columns.empty()) { - auto fetched_columns = StorageObjectStorage::getTableStructureFromData(object_storage, configuration, format_settings, context); - storage_metadata.setColumns(fetched_columns); + if (format == "auto") + std::tie(columns, format) = StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); + else + columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context); } - else if (!columns.hasOnlyOrdinary()) + else if (format == "auto") + { + format = StorageObjectStorage::resolveFormatFromData(object_storage, configuration, format_settings, context); + } + + if (!columns.hasOnlyOrdinary()) { /// We don't allow special columns. throw Exception(ErrorCodes::BAD_ARGUMENTS, "Special columns are not supported for {} storage" "like MATERIALIZED, ALIAS or EPHEMERAL", configuration->getTypeName()); } - else - { - if (configuration->format == "auto") - StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context); - - storage_metadata.setColumns(columns); - } - storage_metadata.setConstraints(constraints); - storage_metadata.setComment(comment); - return storage_metadata; } } diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 37bd49a77c0..3a752e6b8f0 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -12,19 +12,17 @@ struct StorageObjectStorageSettings; std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const StorageObjectStorage::QuerySettings & settings, const std::string & key, size_t sequence_number); - -StorageInMemoryMetadata getStorageMetadata( +void resolveSchemaAndFormat( + ColumnsDescription & columns, + std::string & format, ObjectStoragePtr object_storage, const StorageObjectStorageConfigurationPtr & configuration, - const ColumnsDescription & columns, - const ConstraintsDescription & constraints, std::optional format_settings, - const String & comment, const ContextPtr & context); } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 6b504b0d986..229c40396c5 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -141,24 +142,14 @@ StorageS3Queue::StorageS3Queue( FormatFactory::instance().checkFormatName(configuration->format); configuration->check(context_); - StorageInMemoryMetadata storage_metadata; - if (columns_.empty()) - { - auto columns = StorageObjectStorage::getTableStructureFromData(object_storage, configuration, format_settings, context_); - storage_metadata.setColumns(columns); - } - else - { - if (configuration->format == "auto") - { - StorageObjectStorage::setFormatFromData(object_storage, configuration, format_settings, context_); - } - storage_metadata.setColumns(columns_); - } + ColumnsDescription columns{columns_}; + resolveSchemaAndFormat(columns, configuration->format, object_storage, configuration, format_settings, context_); + configuration->check(context_); + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); - setInMemoryMetadata(storage_metadata); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 2b5c774ff78..06676a8adfa 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -98,7 +99,9 @@ ColumnsDescription TableFunctionObjectStorage< { context->checkAccess(getSourceAccessType()); auto storage = getObjectStorage(context, !is_insert_query); - return StorageObjectStorage::getTableStructureFromData(storage, configuration, std::nullopt, context); + ColumnsDescription columns; + resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context); + return columns; } return parseColumnsListFromString(configuration->structure, context); From 16bc8aa0b1a68bd2422026ea7205a3746029e86b Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Apr 2024 16:08:13 +0200 Subject: [PATCH 040/158] Fxi --- src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp index 3635269db34..89c15085274 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp @@ -26,7 +26,6 @@ void StorageObjectStorageConfiguration::initialize( else FormatFactory::instance().checkFormatName(configuration.format); - configuration.check(local_context); configuration.initialized = true; } From 193ff63f87a2cef958983b2ef106a7c52f6db8be Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Apr 2024 22:44:12 +0200 Subject: [PATCH 041/158] Fix --- .../ObjectStorage/StorageObjectStorage.cpp | 37 ++++++++++++++----- src/Storages/S3Queue/StorageS3Queue.cpp | 1 + 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 36a8beba41a..f5bfb9d2a65 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -269,20 +269,37 @@ std::pair StorageObjectStorage::resolveSchemaAn SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) { - static SchemaCache schema_cache( - context->getConfigRef().getUInt( - "schema_inference_cache_max_elements_for_" + configuration->getTypeName(), - DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; + return getSchemaCache(context, configuration->getTypeName()); } SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name) { - static SchemaCache schema_cache( - context->getConfigRef().getUInt( - "schema_inference_cache_max_elements_for_" + storage_type_name, - DEFAULT_SCHEMA_CACHE_ELEMENTS)); - return schema_cache; + if (storage_type_name == "s3") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_s3", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else if (storage_type_name == "hdfs") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_hdfs", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else if (storage_type_name == "azure") + { + static SchemaCache schema_cache( + context->getConfigRef().getUInt( + "schema_inference_cache_max_elements_for_azure", + DEFAULT_SCHEMA_CACHE_ELEMENTS)); + return schema_cache; + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } } diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 229c40396c5..e84dabecf3b 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -151,6 +151,7 @@ StorageS3Queue::StorageS3Queue( storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(storage_metadata.getColumns())); + setInMemoryMetadata(storage_metadata); LOG_INFO(log, "Using zookeeper path: {}", zk_path.string()); task = getContext()->getSchedulePool().createTask("S3QueueStreamingTask", [this] { threadFunc(); }); From 18e4c0f1da79fc458707c5557b9e611a1fe916bd Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Apr 2024 13:35:18 +0200 Subject: [PATCH 042/158] Fix remaining integration test --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 4 ++-- src/IO/S3/getObjectInfo.cpp | 2 +- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 1 - .../ObjectStorage/ReadBufferIterator.cpp | 4 ++-- .../ObjectStorage/StorageObjectStorageSource.cpp | 16 +++++++++++----- .../ObjectStorage/StorageObjectStorageSource.h | 7 ++----- 6 files changed, 18 insertions(+), 16 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index a2522212f90..507e9dbafcb 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -447,7 +447,7 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata result; result.size_bytes = object_info.size; - result.last_modified = object_info.last_modification_time; + result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time); result.attributes = object_info.metadata; return result; @@ -462,7 +462,7 @@ ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) cons ObjectMetadata result; result.size_bytes = object_info.size; - result.last_modified = object_info.last_modification_time; + result.last_modified = Poco::Timestamp::fromEpochTime(object_info.last_modification_time); result.attributes = object_info.metadata; return result; diff --git a/src/IO/S3/getObjectInfo.cpp b/src/IO/S3/getObjectInfo.cpp index 88f79f8d8d5..c294e7905bd 100644 --- a/src/IO/S3/getObjectInfo.cpp +++ b/src/IO/S3/getObjectInfo.cpp @@ -53,7 +53,7 @@ namespace const auto & result = outcome.GetResult(); ObjectInfo object_info; object_info.size = static_cast(result.GetContentLength()); - object_info.last_modification_time = result.GetLastModified().Millis() / 1000; + object_info.last_modification_time = result.GetLastModified().Seconds(); if (with_metadata) object_info.metadata = result.GetMetadata(); diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index eeb553e0d62..b37b9de746b 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -116,7 +116,6 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory= file_size) // { - // LOG_TEST(log, "KSSENII 1 2"); // return false; // } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index f8ce90a2b1f..9c1d3f79c2b 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -67,11 +67,11 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( auto get_last_mod_time = [&] -> std::optional { if (object_info->metadata) - return object_info->metadata->last_modified.epochMicroseconds(); + return object_info->metadata->last_modified.epochTime(); else { object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - return object_info->metadata->last_modified.epochMicroseconds(); + return object_info->metadata->last_modified.epochTime(); } }; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 3101a7ebf51..4551c2df7c3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -76,6 +76,11 @@ StorageObjectStorageSource::~StorageObjectStorageSource() create_reader_pool->wait(); } +void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) +{ + setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); +} + std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, @@ -213,9 +218,11 @@ std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const O auto get_last_mod_time = [&]() -> std::optional { - return object_info->metadata - ? object_info->metadata->last_modified.epochMicroseconds() - : 0; + if (object_info->metadata) + { + return object_info->metadata->last_modified.epochTime(); + } + return std::nullopt; }; return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); } @@ -260,7 +267,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade const auto max_parsing_threads = need_only_count ? std::optional(1) : std::nullopt; read_buf = createReadBuffer(object_info->relative_path, object_info->metadata->size_bytes); - LOG_TEST(&Poco::Logger::get("KSSENII"), "KSSENII HEADER: {}", read_from_format_info.format_header.dumpStructure()); auto input_format = FormatFactory::instance().getInput( configuration->format, *read_buf, read_from_format_info.format_header, getContext(), max_block_size, format_settings, max_parsing_threads, @@ -354,7 +360,7 @@ ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) if (object_info) { - LOG_TEST(&Poco::Logger::get("KeysIterator"), "Next key: {}", object_info->relative_path); + LOG_TEST(logger, "Next key: {}", object_info->relative_path); } return object_info; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 3c2cc3f80cd..0afbf77db2b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -38,10 +38,7 @@ public: String getName() const override { return name; } - void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override - { - setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); - } + void setKeyCondition(const ActionsDAGPtr & filter_actions_dag, ContextPtr context_) override; Chunk generate() override; @@ -65,11 +62,11 @@ protected: const bool need_only_count; const ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; + ColumnsDescription columns_desc; std::shared_ptr file_iterator; SchemaCache & schema_cache; bool initialized = false; - size_t total_rows_in_file = 0; LoggerPtr log = getLogger("StorageObjectStorageSource"); From a4ed164074fcd96fc198000722563da70f6a31bf Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Apr 2024 13:38:38 +0200 Subject: [PATCH 043/158] Fix clang tidy --- src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp | 2 +- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 2 +- src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index c6590ba8d43..571e14325bb 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -136,7 +136,7 @@ struct DeltaLakeMetadata::Impl * \"nullCount\":{\"col-6c990940-59bb-4709-8f2e-17083a82c01a\":0,\"col-763cd7e2-7627-4d8e-9fb7-9e85d0c8845b\":0}}"}} * " */ - void processMetadataFile(const String & key, std::set & result) + void processMetadataFile(const String & key, std::set & result) const { auto read_settings = context->getReadSettings(); auto buf = object_storage->readObject(StoredObject(key), read_settings); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index f5bfb9d2a65..c5565d8b0e8 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -61,7 +61,7 @@ StorageObjectStorage::StorageObjectStorage( objects.emplace_back(key); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); - setInMemoryMetadata(std::move(metadata)); + setInMemoryMetadata(metadata); } String StorageObjectStorage::getName() const diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index f98fc32a3cc..1a1df399626 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -47,7 +47,7 @@ StorageObjectStorageCluster::StorageObjectStorageCluster( metadata.setConstraints(constraints_); setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); - setInMemoryMetadata(std::move(metadata)); + setInMemoryMetadata(metadata); } std::string StorageObjectStorageCluster::getName() const From 434d2d16f1056977dd80f47d0b687151ac9d16f2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Apr 2024 16:34:12 +0200 Subject: [PATCH 044/158] Cleanuo --- src/Backups/BackupIO_AzureBlobStorage.cpp | 4 +- src/Backups/BackupIO_AzureBlobStorage.h | 10 +- .../registerBackupEngineAzureBlobStorage.cpp | 4 +- src/CMakeLists.txt | 4 +- src/Core/Settings.h | 4 + src/Core/SettingsChangesHistory.h | 4 + .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 78 +++---- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 17 +- .../ObjectStorages/ObjectStorageFactory.cpp | 3 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 18 -- src/Disks/ObjectStorages/S3/diskSettings.cpp | 10 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- .../{AzureBlob => Azure}/Configuration.cpp | 33 +-- .../{AzureBlob => Azure}/Configuration.h | 16 +- .../ObjectStorage/DataLakes/Common.cpp | 4 +- src/Storages/ObjectStorage/DataLakes/Common.h | 4 +- .../DataLakes/DeltaLakeMetadata.cpp | 12 +- .../DataLakes/DeltaLakeMetadata.h | 5 +- .../ObjectStorage/DataLakes/HudiMetadata.h | 4 +- .../DataLakes/IStorageDataLake.h | 2 +- .../DataLakes/IcebergMetadata.cpp | 6 +- .../ObjectStorage/DataLakes/IcebergMetadata.h | 4 +- .../DataLakes/registerDataLakeStorages.cpp | 6 +- .../ObjectStorage/HDFS/Configuration.cpp | 32 +-- .../ObjectStorage/HDFS/Configuration.h | 12 +- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 8 +- .../ObjectStorage/ReadBufferIterator.cpp | 53 ++--- .../ObjectStorage/ReadBufferIterator.h | 8 +- .../ReadFromObjectStorageStep.cpp | 87 ------- .../ObjectStorage/ReadFromObjectStorageStep.h | 55 ----- .../ObjectStorage/S3/Configuration.cpp | 21 +- src/Storages/ObjectStorage/S3/Configuration.h | 11 +- .../ObjectStorage/StorageObjectStorage.cpp | 213 ++++++++++++++++-- .../ObjectStorage/StorageObjectStorage.h | 62 ++++- .../StorageObjectStorageCluster.cpp | 20 +- .../StorageObjectStorageCluster.h | 15 +- .../StorageObjectStorageConfiguration.cpp | 74 ------ .../StorageObjectStorageConfiguration.h | 75 ------ .../StorageObjectStorageSink.cpp | 7 +- .../ObjectStorage/StorageObjectStorageSink.h | 16 +- .../StorageObjectStorageSource.cpp | 23 +- .../StorageObjectStorageSource.h | 7 +- .../StorageObjectStorage_fwd_internal.h | 12 - src/Storages/ObjectStorage/Utils.cpp | 7 +- src/Storages/ObjectStorage/Utils.h | 6 +- .../registerStorageObjectStorage.cpp | 22 +- src/Storages/S3Queue/S3QueueTableMetadata.cpp | 3 +- src/Storages/S3Queue/S3QueueTableMetadata.h | 4 +- src/Storages/S3Queue/StorageS3Queue.cpp | 2 +- .../StorageSystemSchemaInferenceCache.cpp | 4 +- src/TableFunctions/ITableFunctionDataLake.h | 2 +- .../TableFunctionObjectStorage.cpp | 73 ++---- .../TableFunctionObjectStorage.h | 33 ++- .../TableFunctionObjectStorageCluster.cpp | 4 +- .../TableFunctionObjectStorageCluster.h | 4 +- src/TableFunctions/registerTableFunctions.cpp | 12 - .../configs/inf_s3_retries.xml | 1 + .../configs/s3_retries.xml | 1 + 58 files changed, 555 insertions(+), 690 deletions(-) rename src/Storages/ObjectStorage/{AzureBlob => Azure}/Configuration.cpp (93%) rename src/Storages/ObjectStorage/{AzureBlob => Azure}/Configuration.h (78%) delete mode 100644 src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp delete mode 100644 src/Storages/ObjectStorage/ReadFromObjectStorageStep.h delete mode 100644 src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp delete mode 100644 src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h delete mode 100644 src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index 673930b5976..f00da686c18 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes } BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( - const StorageAzureBlobConfiguration & configuration_, + const StorageAzureConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_) @@ -112,7 +112,7 @@ void BackupReaderAzureBlobStorage::copyFileToDisk(const String & path_in_backup, BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( - const StorageAzureBlobConfiguration & configuration_, + const StorageAzureConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, diff --git a/src/Backups/BackupIO_AzureBlobStorage.h b/src/Backups/BackupIO_AzureBlobStorage.h index 25c52f9b0d3..4643c103fd5 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.h +++ b/src/Backups/BackupIO_AzureBlobStorage.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB @@ -17,7 +17,7 @@ class BackupReaderAzureBlobStorage : public BackupReaderDefault { public: BackupReaderAzureBlobStorage( - const StorageAzureBlobConfiguration & configuration_, + const StorageAzureConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_); @@ -39,7 +39,7 @@ public: private: const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlobConfiguration configuration; + StorageAzureConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; @@ -48,7 +48,7 @@ class BackupWriterAzureBlobStorage : public BackupWriterDefault { public: BackupWriterAzureBlobStorage( - const StorageAzureBlobConfiguration & configuration_, + const StorageAzureConfiguration & configuration_, const ReadSettings & read_settings_, const WriteSettings & write_settings_, const ContextPtr & context_, @@ -85,7 +85,7 @@ private: const DataSourceDescription data_source_description; std::shared_ptr client; - StorageAzureBlobConfiguration configuration; + StorageAzureConfiguration configuration; std::unique_ptr object_storage; std::shared_ptr settings; }; diff --git a/src/Backups/registerBackupEngineAzureBlobStorage.cpp b/src/Backups/registerBackupEngineAzureBlobStorage.cpp index 049a4b1a338..1e3b3759257 100644 --- a/src/Backups/registerBackupEngineAzureBlobStorage.cpp +++ b/src/Backups/registerBackupEngineAzureBlobStorage.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #endif @@ -49,7 +49,7 @@ void registerBackupEngineAzureBlobStorage(BackupFactory & factory) const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; - StorageAzureBlobConfiguration configuration; + StorageAzureConfiguration configuration; if (!id_arg.empty()) { diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c26c40d4b87..d5d17f992dc 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -103,7 +103,6 @@ add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhous add_headers_and_sources(dbms Disks/IO) add_headers_and_sources(dbms Disks/ObjectStorages) -add_headers_and_sources(dbms Disks/ObjectStorages) if (TARGET ch_contrib::sqlite) add_headers_and_sources(dbms Databases/SQLite) endif() @@ -117,7 +116,7 @@ if (TARGET ch_contrib::nats_io) endif() add_headers_and_sources(dbms Storages/ObjectStorage) -add_headers_and_sources(dbms Storages/ObjectStorage/AzureBlob) +add_headers_and_sources(dbms Storages/ObjectStorage/Azure) add_headers_and_sources(dbms Storages/ObjectStorage/S3) add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Storages/ObjectStorage/DataLakes) @@ -148,7 +147,6 @@ if (TARGET ch_contrib::azure_sdk) endif() if (TARGET ch_contrib::hdfs) - add_headers_and_sources(dbms Storages/ObjectStorage/HDFS) add_headers_and_sources(dbms Disks/ObjectStorages/HDFS) endif() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ff7a9089327..bf558d7b1ba 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -113,9 +113,12 @@ class IColumn; M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ M(Bool, s3_allow_parallel_part_upload, true, "Use multiple threads for s3 multipart upload. It may lead to slightly higher memory usage", 0) \ M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ + M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, s3_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageS3", 0) \ M(Bool, hdfs_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageHDFS", 0) \ M(Bool, azure_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageAzure", 0) \ + M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ M(UInt64, s3_request_timeout_ms, 30000, "Idleness timeout for sending and receiving data to/from S3. Fail if a single TCP read or write call blocks for this long.", 0) \ @@ -128,6 +131,7 @@ class IColumn; M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ M(Bool, hdfs_skip_empty_files, false, "Allow to skip empty files in hdfs table engine", 0) \ + M(Bool, azure_skip_empty_files, false, "Allow to skip empty files in azure table engine", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cfe3c290d83..4954fa5d996 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -90,6 +90,10 @@ static std::map sett {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, + {"hdfs_throw_on_zero_files_match", false, false, "Throw an error, when ListObjects request cannot match any files"}, + {"azure_throw_on_zero_files_match", false, false, "Throw an error, when ListObjects request cannot match any files"}, + {"s3_validate_request_settings", true, true, "Validate S3 request settings"}, + {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, }}, {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index ed63795cb05..6c2f310a7d1 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -23,15 +23,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void HDFSObjectStorage::shutdown() -{ -} - -void HDFSObjectStorage::startup() -{ -} - -void HDFSObjectStorage::initializeHDFS() const +void HDFSObjectStorage::initializeHDFSFS() const { if (initialized) return; @@ -45,9 +37,25 @@ void HDFSObjectStorage::initializeHDFS() const initialized = true; } +std::string HDFSObjectStorage::extractObjectKeyFromURL(const StoredObject & object) const +{ + /// This is very unfortunate, but for disk HDFS we made a mistake + /// and now its behaviour is inconsistent with S3 and Azure disks. + /// The mistake is that for HDFS we write into metadata files whole URL + data directory + key, + /// while for S3 and Azure we write there only data_directory + key. + /// This leads us into ambiguity that for StorageHDFS we have just key in object.remote_path, + /// but for DiskHDFS we have there URL as well. + auto path = object.remote_path; + if (path.starts_with(url)) + path = path.substr(url.size()); + if (path.starts_with("/")) + path.substr(1); + return path; +} + ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { - initializeHDFS(); + initializeHDFSFS(); /// what ever data_source_description.description value is, consider that key as relative key chassert(data_directory.starts_with("/")); return ObjectStorageKey::createAsRelative( @@ -56,14 +64,11 @@ ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & bool HDFSObjectStorage::exists(const StoredObject & object) const { - initializeHDFS(); + initializeHDFSFS(); std::string path = object.remote_path; if (path.starts_with(url_without_path)) path = path.substr(url_without_path.size()); - // const auto & path = object.remote_path; - // const size_t begin_of_path = path.find('/', path.find("//") + 2); - // const String remote_fs_object_path = path.substr(begin_of_path); return (0 == hdfsExists(hdfs_fs.get(), path.c_str())); } @@ -73,13 +78,8 @@ std::unique_ptr HDFSObjectStorage::readObject( /// NOLIN std::optional, std::optional) const { - initializeHDFS(); - std::string path = object.remote_path; - if (path.starts_with(url)) - path = path.substr(url.size()); - if (path.starts_with("/")) - path.substr(1); - + initializeHDFSFS(); + auto path = extractObjectKeyFromURL(object); return std::make_unique( fs::path(url_without_path) / "", fs::path(data_directory) / path, config, patchSettings(read_settings)); } @@ -90,21 +90,13 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI std::optional, std::optional) const { - initializeHDFS(); + initializeHDFSFS(); auto disk_read_settings = patchSettings(read_settings); auto read_buffer_creator = [this, disk_read_settings] (bool /* restricted_seek */, const StoredObject & object_) -> std::unique_ptr { - // size_t begin_of_path = path.find('/', path.find("//") + 2); - // auto hdfs_path = path.substr(begin_of_path); - // auto hdfs_uri = path.substr(0, begin_of_path); - - std::string path = object_.remote_path; - if (path.starts_with(url)) - path = path.substr(url.size()); - if (path.starts_with("/")) - path.substr(1); + auto path = extractObjectKeyFromURL(object_); return std::make_unique( fs::path(url_without_path) / "", fs::path(data_directory) / path, config, disk_read_settings, /* read_until_position */0, /* use_external_buffer */true); }; @@ -120,7 +112,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL size_t buf_size, const WriteSettings & write_settings) { - initializeHDFS(); + initializeHDFSFS(); if (attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, @@ -142,7 +134,7 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL /// Remove file. Throws exception if file doesn't exists or it's a directory. void HDFSObjectStorage::removeObject(const StoredObject & object) { - initializeHDFS(); + initializeHDFSFS(); auto path = object.remote_path; if (path.starts_with(url_without_path)) path = path.substr(url_without_path.size()); @@ -156,28 +148,28 @@ void HDFSObjectStorage::removeObject(const StoredObject & object) void HDFSObjectStorage::removeObjects(const StoredObjects & objects) { - initializeHDFS(); + initializeHDFSFS(); for (const auto & object : objects) removeObject(object); } void HDFSObjectStorage::removeObjectIfExists(const StoredObject & object) { - initializeHDFS(); + initializeHDFSFS(); if (exists(object)) removeObject(object); } void HDFSObjectStorage::removeObjectsIfExist(const StoredObjects & objects) { - initializeHDFS(); + initializeHDFSFS(); for (const auto & object : objects) removeObjectIfExists(object); } ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) const { - initializeHDFS(); + initializeHDFSFS(); auto * file_info = hdfsGetPathInfo(hdfs_fs.get(), path.data()); if (!file_info) throw Exception(ErrorCodes::HDFS_ERROR, @@ -185,7 +177,7 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co ObjectMetadata metadata; metadata.size_bytes = static_cast(file_info->mSize); - metadata.last_modified = file_info->mLastMod; + metadata.last_modified = Poco::Timestamp::fromEpochTime(file_info->mLastMod); hdfsFreeFileInfo(file_info, 1); return metadata; @@ -193,9 +185,9 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { - initializeHDFS(); + initializeHDFSFS(); auto * log = &Poco::Logger::get("HDFSObjectStorage"); - LOG_TRACE(log, "Trying to list files for {}", path); + LOG_TEST(log, "Trying to list files for {}", path); HDFSFileInfo ls; ls.file_info = hdfsListDirectory(hdfs_fs.get(), path.data(), &ls.length); @@ -213,7 +205,7 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM throw Exception(ErrorCodes::LOGICAL_ERROR, "file_info shouldn't be null"); } - LOG_TRACE(log, "Listed {} files for {}", ls.length, path); + LOG_TEST(log, "Listed {} files for {}", ls.length, path); for (int i = 0; i < ls.length; ++i) { @@ -228,8 +220,6 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM } else { - LOG_TEST(log, "Found file: {}", file_path); - children.emplace_back(std::make_shared( String(file_path), ObjectMetadata{ @@ -247,7 +237,7 @@ void HDFSObjectStorage::copyObject( /// NOLINT const WriteSettings & write_settings, std::optional object_to_attributes) { - initializeHDFS(); + initializeHDFSFS(); if (object_to_attributes.has_value()) throw Exception( ErrorCodes::UNSUPPORTED_METHOD, diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index b626d3dc779..e747b283400 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -35,7 +35,8 @@ public: HDFSObjectStorage( const String & hdfs_root_path_, SettingsPtr settings_, - const Poco::Util::AbstractConfiguration & config_) + const Poco::Util::AbstractConfiguration & config_, + bool lazy_initialize) : config(config_) , settings(std::move(settings_)) { @@ -46,6 +47,9 @@ public: data_directory = url.substr(begin_of_path); else data_directory = "/"; + + if (!lazy_initialize) + initializeHDFSFS(); } std::string getName() const override { return "HDFSObjectStorage"; } @@ -98,10 +102,6 @@ public: void listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const override; - void shutdown() override; - - void startup() override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( @@ -114,8 +114,13 @@ public: bool isRemote() const override { return true; } + void startup() override { } + + void shutdown() override { } + private: - void initializeHDFS() const; + void initializeHDFSFS() const; + std::string extractObjectKeyFromURL(const StoredObject & object) const; const Poco::Util::AbstractConfiguration & config; diff --git a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp index 67e38d6389a..1a2ea0c2593 100644 --- a/src/Disks/ObjectStorages/ObjectStorageFactory.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageFactory.cpp @@ -232,7 +232,8 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory) context->getSettingsRef().hdfs_replication ); - return createObjectStorage(ObjectStorageType::HDFS, config, config_prefix, uri, std::move(settings), config); + return createObjectStorage( + ObjectStorageType::HDFS, config, config_prefix, uri, std::move(settings), config, /* lazy_initialize */false); }); } #endif diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 507e9dbafcb..0801a84ce13 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -582,27 +582,9 @@ void S3ObjectStorage::applyNewSettings( auto new_client = getClient(config, config_prefix, context, *new_s3_settings, for_disk_s3, &uri); client.set(std::move(new_client)); } - s3_settings.set(std::move(new_s3_settings)); } -// void S3ObjectStorage::applyNewSettings(ContextPtr context) -// { -// auto settings = s3_settings.get(); -// if (!endpoint_settings || !settings->auth_settings.hasUpdates(endpoint_settings->auth_settings)) -// return; -// -// const auto & config = context->getConfigRef(); -// auto new_s3_settings = getSettings(uri, config, "s3.", context); -// -// new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); -// -// auto new_client = getClient(config, "s3.", context, *new_s3_settings, false); -// -// s3_settings.set(std::move(new_s3_settings)); -// client.set(std::move(new_client)); -// } - std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 49300a9cd89..a38c0d3c85f 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -100,11 +100,9 @@ std::unique_ptr getClient( settings.request_settings.put_request_throttler, url.uri.getScheme()); - client_configuration.endpointOverride = url.endpoint; - client_configuration.maxConnections = static_cast(request_settings.max_connections); client_configuration.connectTimeoutMs = config.getUInt64(config_prefix + ".connect_timeout_ms", local_settings.s3_connect_timeout_ms.value); client_configuration.requestTimeoutMs = config.getUInt64(config_prefix + ".request_timeout_ms", local_settings.s3_request_timeout_ms.value); - client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS); + client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", static_cast(request_settings.max_connections)); client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT); client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS); @@ -112,12 +110,6 @@ std::unique_ptr getClient( client_configuration.s3_use_adaptive_timeouts = config.getBool( config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); - // client_configuration.http_keep_alive_timeout_ms = config.getUInt(config_prefix + ".http_keep_alive_timeout_ms", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT * 1000); - // client_configuration.http_connection_pool_size = config.getUInt( - // config_prefix + ".http_connection_pool_size", static_cast(global_settings.s3_http_connection_pool_size.value)); - // client_configuration.s3_use_adaptive_timeouts = config.getBool(config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts); - // client_configuration.wait_on_pool_size_limit = for_disk_s3; - if (for_disk_s3) { /* diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index af9dc08e8c7..56b2904363e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -55,7 +55,7 @@ #include #include #include -#include +#include #include #include #include @@ -502,7 +502,7 @@ BlockIO InterpreterSystemQuery::execute() StorageURL::getSchemaCache(getContext()).clear(); #if USE_AZURE_BLOB_STORAGE if (caches_to_drop.contains("AZURE")) - StorageObjectStorage::getSchemaCache(getContext(), StorageAzureBlobConfiguration::type_name).clear(); + StorageObjectStorage::getSchemaCache(getContext(), StorageAzureConfiguration::type_name).clear(); #endif break; } diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp similarity index 93% rename from src/Storages/ObjectStorage/AzureBlob/Configuration.cpp rename to src/Storages/ObjectStorage/Azure/Configuration.cpp index f268b812c03..43992a81eef 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -1,8 +1,9 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE #include +#include #include #include #include @@ -65,7 +66,7 @@ namespace } } -void StorageAzureBlobConfiguration::check(ContextPtr context) const +void StorageAzureConfiguration::check(ContextPtr context) const { Poco::URI url_to_check; if (is_connection_string) @@ -77,11 +78,11 @@ void StorageAzureBlobConfiguration::check(ContextPtr context) const url_to_check = Poco::URI(connection_url); context->getGlobalContext()->getRemoteHostFilter().checkURL(url_to_check); - StorageObjectStorageConfiguration::check(context); + Configuration::check(context); } -StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other) - : StorageObjectStorageConfiguration(other) +StorageAzureConfiguration::StorageAzureConfiguration(const StorageAzureConfiguration & other) + : Configuration(other) { connection_url = other.connection_url; is_connection_string = other.is_connection_string; @@ -92,7 +93,7 @@ StorageAzureBlobConfiguration::StorageAzureBlobConfiguration(const StorageAzureB blobs_paths = other.blobs_paths; } -AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(ContextPtr context) +AzureObjectStorage::SettingsPtr StorageAzureConfiguration::createSettings(ContextPtr context) { const auto & context_settings = context->getSettingsRef(); auto settings_ptr = std::make_unique(); @@ -102,7 +103,7 @@ AzureObjectStorage::SettingsPtr StorageAzureBlobConfiguration::createSettings(Co return settings_ptr; } -StorageObjectStorage::QuerySettings StorageAzureBlobConfiguration::getQuerySettings(const ContextPtr & context) const +StorageObjectStorage::QuerySettings StorageAzureConfiguration::getQuerySettings(const ContextPtr & context) const { const auto & settings = context->getSettingsRef(); return StorageObjectStorage::QuerySettings{ @@ -110,14 +111,14 @@ StorageObjectStorage::QuerySettings StorageAzureBlobConfiguration::getQuerySetti .create_new_file_on_insert = settings.azure_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_azure, .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.s3_skip_empty_files, /// TODO: add setting for azure + .skip_empty_files = settings.azure_skip_empty_files, .list_object_keys_size = settings.azure_list_object_keys_size, - .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .throw_on_zero_files_match = settings.azure_throw_on_zero_files_match, .ignore_non_existent_file = settings.azure_ignore_file_doesnt_exist, }; } -ObjectStoragePtr StorageAzureBlobConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT +ObjectStoragePtr StorageAzureConfiguration::createObjectStorage(ContextPtr context, bool is_readonly) /// NOLINT { assertInitialized(); auto client = createClient(is_readonly, /* attempt_to_create_container */true); @@ -125,7 +126,7 @@ ObjectStoragePtr StorageAzureBlobConfiguration::createObjectStorage(ContextPtr c return std::make_unique("AzureBlobStorage", std::move(client), std::move(settings), container); } -AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only, bool attempt_to_create_container) +AzureClientPtr StorageAzureConfiguration::createClient(bool is_read_only, bool attempt_to_create_container) { using namespace Azure::Storage::Blobs; @@ -133,8 +134,8 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only, bo if (is_connection_string) { - std::shared_ptr managed_identity_credential = std::make_shared(); - std::unique_ptr blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); + auto managed_identity_credential = std::make_shared(); + auto blob_service_client = std::make_unique(BlobServiceClient::CreateFromConnectionString(connection_url)); result = std::make_unique(BlobContainerClient::CreateFromConnectionString(connection_url, container)); if (attempt_to_create_container) @@ -243,7 +244,7 @@ AzureClientPtr StorageAzureBlobConfiguration::createClient(bool is_read_only, bo return result; } -void StorageAzureBlobConfiguration::fromNamedCollection(const NamedCollection & collection) +void StorageAzureConfiguration::fromNamedCollection(const NamedCollection & collection) { validateNamedCollection(collection, required_configuration_keys, optional_configuration_keys); @@ -275,7 +276,7 @@ void StorageAzureBlobConfiguration::fromNamedCollection(const NamedCollection & blobs_paths = {blob_path}; } -void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) +void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context, bool with_structure) { if (engine_args.size() < 3 || engine_args.size() > (with_structure ? 8 : 7)) { @@ -396,7 +397,7 @@ void StorageAzureBlobConfiguration::fromAST(ASTs & engine_args, ContextPtr conte blobs_paths = {blob_path}; } -void StorageAzureBlobConfiguration::addStructureAndFormatToArgs( +void StorageAzureConfiguration::addStructureAndFormatToArgs( ASTs & args, const String & structure_, const String & format_, ContextPtr context) { if (tryGetNamedCollectionWithOverrides(args, context)) diff --git a/src/Storages/ObjectStorage/AzureBlob/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h similarity index 78% rename from src/Storages/ObjectStorage/AzureBlob/Configuration.h rename to src/Storages/ObjectStorage/Azure/Configuration.h index 7e105ea82b5..91a9a0bbbd5 100644 --- a/src/Storages/ObjectStorage/AzureBlob/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -5,24 +5,27 @@ #if USE_AZURE_BLOB_STORAGE #include -#include +#include +#include namespace DB { class BackupFactory; -class StorageAzureBlobConfiguration : public StorageObjectStorageConfiguration +class StorageAzureConfiguration : public StorageObjectStorage::Configuration { friend class BackupReaderAzureBlobStorage; friend class BackupWriterAzureBlobStorage; friend void registerBackupEngineAzureBlobStorage(BackupFactory & factory); public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + static constexpr auto type_name = "azure"; static constexpr auto engine_name = "Azure"; - StorageAzureBlobConfiguration() = default; - StorageAzureBlobConfiguration(const StorageAzureBlobConfiguration & other); + StorageAzureConfiguration() = default; + StorageAzureConfiguration(const StorageAzureConfiguration & other); std::string getTypeName() const override { return type_name; } std::string getEngineName() const override { return engine_name; } @@ -31,16 +34,15 @@ public: void setPath(const Path & path) override { blob_path = path; } const Paths & getPaths() const override { return blobs_paths; } - Paths & getPaths() override { return blobs_paths; } void setPaths(const Paths & paths) override { blobs_paths = paths; } - String getDataSourceDescription() override { return fs::path(connection_url) / container; } + String getDataSourceDescription() override { return std::filesystem::path(connection_url) / container; } String getNamespace() const override { return container; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT - StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } + ConfigurationPtr clone() override { return std::make_shared(*this); } void fromNamedCollection(const NamedCollection & collection) override; void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; diff --git a/src/Storages/ObjectStorage/DataLakes/Common.cpp b/src/Storages/ObjectStorage/DataLakes/Common.cpp index 0c9237127b9..4830cc52a90 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.cpp +++ b/src/Storages/ObjectStorage/DataLakes/Common.cpp @@ -1,6 +1,6 @@ #include "Common.h" #include -#include +#include #include namespace DB @@ -8,7 +8,7 @@ namespace DB std::vector listFiles( const IObjectStorage & object_storage, - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const String & prefix, const String & suffix) { auto key = std::filesystem::path(configuration.getPath()) / prefix; diff --git a/src/Storages/ObjectStorage/DataLakes/Common.h b/src/Storages/ObjectStorage/DataLakes/Common.h index ae3767f2eec..db3afa9e4a6 100644 --- a/src/Storages/ObjectStorage/DataLakes/Common.h +++ b/src/Storages/ObjectStorage/DataLakes/Common.h @@ -1,15 +1,15 @@ #pragma once #include +#include namespace DB { class IObjectStorage; -class StorageObjectStorageConfiguration; std::vector listFiles( const IObjectStorage & object_storage, - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const String & prefix, const String & suffix); } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp index 571e14325bb..277d07d88ef 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.cpp @@ -85,7 +85,7 @@ struct DeltaLakeMetadata::Impl while (true) { const auto filename = withPadding(++current_version) + metadata_file_suffix; - const auto file_path = fs::path(configuration->getPath()) / deltalake_metadata_directory / filename; + const auto file_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / filename; if (!object_storage->exists(StoredObject(file_path))) break; @@ -161,12 +161,12 @@ struct DeltaLakeMetadata::Impl if (json.has("add")) { const auto path = json["add"]["path"].getString(); - result.insert(fs::path(configuration->getPath()) / path); + result.insert(std::filesystem::path(configuration->getPath()) / path); } else if (json.has("remove")) { const auto path = json["remove"]["path"].getString(); - result.erase(fs::path(configuration->getPath()) / path); + result.erase(std::filesystem::path(configuration->getPath()) / path); } } } @@ -186,7 +186,7 @@ struct DeltaLakeMetadata::Impl */ size_t readLastCheckpointIfExists() const { - const auto last_checkpoint_file = fs::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; + const auto last_checkpoint_file = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / "_last_checkpoint"; if (!object_storage->exists(StoredObject(last_checkpoint_file))) return 0; @@ -249,7 +249,7 @@ struct DeltaLakeMetadata::Impl return 0; const auto checkpoint_filename = withPadding(version) + ".checkpoint.parquet"; - const auto checkpoint_path = fs::path(configuration->getPath()) / deltalake_metadata_directory / checkpoint_filename; + const auto checkpoint_path = std::filesystem::path(configuration->getPath()) / deltalake_metadata_directory / checkpoint_filename; LOG_TRACE(log, "Using checkpoint file: {}", checkpoint_path.string()); @@ -311,7 +311,7 @@ struct DeltaLakeMetadata::Impl if (filename.empty()) continue; LOG_TEST(log, "Adding {}", filename); - const auto [_, inserted] = result.insert(fs::path(configuration->getPath()) / filename); + const auto [_, inserted] = result.insert(std::filesystem::path(configuration->getPath()) / filename); if (!inserted) throw Exception(ErrorCodes::INCORRECT_DATA, "File already exists {}", filename); } diff --git a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h index 5050b88d809..e527721b29e 100644 --- a/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/DeltaLakeMetadata.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -12,8 +12,7 @@ namespace DB class DeltaLakeMetadata final : public IDataLakeMetadata { public: - using ConfigurationPtr = StorageObjectStorageConfigurationPtr; - + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; static constexpr auto name = "DeltaLake"; DeltaLakeMetadata( diff --git a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h index 6054c3f15d6..3ab274b1fbf 100644 --- a/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/HudiMetadata.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -13,7 +13,7 @@ namespace DB class HudiMetadata final : public IDataLakeMetadata, private WithContext { public: - using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; static constexpr auto name = "Hudi"; diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 144cc16939c..3119b844aaf 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -88,7 +88,7 @@ public: else { ConfigurationPtr configuration = base_configuration->clone(); - configuration->getPaths() = metadata->getDataFiles(); + configuration->setPaths(metadata->getDataFiles()); return Storage::resolveSchemaFromData( object_storage_, configuration, format_settings_, local_context); } diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp index 8ee6f002ca6..591e5ef03f6 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.cpp @@ -45,7 +45,7 @@ namespace ErrorCodes IcebergMetadata::IcebergMetadata( ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + ConfigurationPtr configuration_, DB::ContextPtr context_, Int32 metadata_version_, Int32 format_version_, @@ -341,7 +341,7 @@ MutableColumns parseAvro( */ std::pair getMetadataFileAndVersion( ObjectStoragePtr object_storage, - const StorageObjectStorageConfiguration & configuration) + const StorageObjectStorage::Configuration & configuration) { const auto metadata_files = listFiles(*object_storage, configuration, "metadata", ".metadata.json"); if (metadata_files.empty()) @@ -378,7 +378,7 @@ std::pair getMetadataFileAndVersion( DataLakeMetadataPtr IcebergMetadata::create( ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, + ConfigurationPtr configuration, ContextPtr local_context) { const auto [metadata_version, metadata_file_path] = getMetadataFileAndVersion(object_storage, *configuration); diff --git a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h index f88e3eecc67..06dbd373bf9 100644 --- a/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h +++ b/src/Storages/ObjectStorage/DataLakes/IcebergMetadata.h @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include namespace DB @@ -61,7 +61,7 @@ namespace DB class IcebergMetadata : public IDataLakeMetadata, private WithContext { public: - using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; static constexpr auto name = "Iceberg"; diff --git a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp index a5170e5ed6b..0fa6402e892 100644 --- a/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp +++ b/src/Storages/ObjectStorage/DataLakes/registerDataLakeStorages.cpp @@ -20,7 +20,7 @@ void registerStorageIceberg(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageIceberg::create( configuration, args.getContext(), args.table_id, args.columns, @@ -43,7 +43,7 @@ void registerStorageDeltaLake(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageDeltaLake::create( configuration, args.getContext(), args.table_id, args.columns, @@ -64,7 +64,7 @@ void registerStorageHudi(StorageFactory & factory) [&](const StorageFactory::Arguments & args) { auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, args.getLocalContext(), false); return StorageHudi::create( configuration, args.getContext(), args.table_id, args.columns, diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.cpp b/src/Storages/ObjectStorage/HDFS/Configuration.cpp index 12e3f3adb12..a8a9ab5b557 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.cpp +++ b/src/Storages/ObjectStorage/HDFS/Configuration.cpp @@ -1,18 +1,21 @@ #include #if USE_HDFS -#include -#include -#include +#include #include -#include -#include #include +#include + +#include +#include + +#include +#include +#include + #include #include #include -#include - namespace DB { @@ -23,7 +26,7 @@ namespace ErrorCodes } StorageHDFSConfiguration::StorageHDFSConfiguration(const StorageHDFSConfiguration & other) - : StorageObjectStorageConfiguration(other) + : Configuration(other) { url = other.url; path = other.path; @@ -34,7 +37,7 @@ void StorageHDFSConfiguration::check(ContextPtr context) const { context->getRemoteHostFilter().checkURL(Poco::URI(url)); checkHDFSURL(fs::path(url) / path.substr(1)); - StorageObjectStorageConfiguration::check(context); + Configuration::check(context); } ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT @@ -47,10 +50,11 @@ ObjectStoragePtr StorageHDFSConfiguration::createObjectStorage( /// NOLINT settings.remote_read_min_bytes_for_seek, settings.hdfs_replication ); - return std::make_shared(url, std::move(hdfs_settings), context->getConfigRef()); + return std::make_shared( + url, std::move(hdfs_settings), context->getConfigRef(), /* lazy_initialize */true); } -std::string StorageHDFSConfiguration::getPathWithoutGlob() const +std::string StorageHDFSConfiguration::getPathWithoutGlobs() const { /// Unlike s3 and azure, which are object storages, /// hdfs is a filesystem, so it cannot list files by partual prefix, @@ -69,9 +73,9 @@ StorageObjectStorage::QuerySettings StorageHDFSConfiguration::getQuerySettings(c .create_new_file_on_insert = settings.hdfs_create_new_file_on_insert, .schema_inference_use_cache = settings.schema_inference_use_cache_for_hdfs, .schema_inference_mode = settings.schema_inference_mode, - .skip_empty_files = settings.hdfs_skip_empty_files, /// TODO: add setting for hdfs - .list_object_keys_size = settings.s3_list_object_keys_size, /// TODO: add a setting for hdfs - .throw_on_zero_files_match = settings.s3_throw_on_zero_files_match, + .skip_empty_files = settings.hdfs_skip_empty_files, + .list_object_keys_size = 0, /// HDFS does not support listing in batches. + .throw_on_zero_files_match = settings.hdfs_throw_on_zero_files_match, .ignore_non_existent_file = settings.hdfs_ignore_file_doesnt_exist, }; } diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index 0a502857153..cac09ee1d92 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -2,17 +2,18 @@ #include "config.h" #if USE_HDFS -#include +#include #include -#include #include namespace DB { -class StorageHDFSConfiguration : public StorageObjectStorageConfiguration +class StorageHDFSConfiguration : public StorageObjectStorage::Configuration { public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + static constexpr auto type_name = "hdfs"; static constexpr auto engine_name = "HDFS"; @@ -26,7 +27,6 @@ public: void setPath(const Path & path_) override { path = path_; } const Paths & getPaths() const override { return paths; } - Paths & getPaths() override { return paths; } void setPaths(const Paths & paths_) override { paths = paths_; } String getNamespace() const override { return ""; } @@ -35,12 +35,12 @@ public: void check(ContextPtr context) const override; ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT - StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } + ConfigurationPtr clone() override { return std::make_shared(*this); } void addStructureAndFormatToArgs( ASTs & args, const String & structure_, const String & format_, ContextPtr context) override; - std::string getPathWithoutGlob() const override; + std::string getPathWithoutGlobs() const override; private: void fromNamedCollection(const NamedCollection &) override; diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index b37b9de746b..be339d021dc 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -114,10 +114,10 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory= file_size) - // { - // return false; - // } + if (file_size != 0 && file_offset >= file_size) + { + return false; + } ResourceGuard rlock(read_settings.resource_link, num_bytes_to_read); int bytes_read; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 9c1d3f79c2b..3705725ffe1 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -10,7 +10,6 @@ namespace ErrorCodes { extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; extern const int CANNOT_DETECT_FORMAT; - } ReadBufferIterator::ReadBufferIterator( @@ -29,18 +28,19 @@ ReadBufferIterator::ReadBufferIterator( , query_settings(configuration->getQuerySettings(context_)) , schema_cache(schema_cache_) , read_keys(read_keys_) - , format(configuration->format == "auto" ? std::nullopt : std::optional(configuration->format)) , prev_read_keys_size(read_keys_.size()) { + if (configuration->format != "auto") + format = configuration->format; } SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const String & path, const String & format_name) const { - auto source = fs::path(configuration->getDataSourceDescription()) / path; + auto source = std::filesystem::path(configuration->getDataSourceDescription()) / path; return DB::getKeyForSchemaCache(source, format_name, format_settings, getContext()); } -SchemaCache::Keys ReadBufferIterator::getPathsForSchemaCache() const +SchemaCache::Keys ReadBufferIterator::getKeysForSchemaCache() const { Strings sources; sources.reserve(read_keys.size()); @@ -49,7 +49,7 @@ SchemaCache::Keys ReadBufferIterator::getPathsForSchemaCache() const std::back_inserter(sources), [&](const auto & elem) { - return fs::path(configuration->getDataSourceDescription()) / elem->relative_path; + return std::filesystem::path(configuration->getDataSourceDescription()) / elem->relative_path; }); return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); } @@ -66,16 +66,14 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( const auto & object_info = (*it); auto get_last_mod_time = [&] -> std::optional { - if (object_info->metadata) - return object_info->metadata->last_modified.epochTime(); - else - { - object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); - return object_info->metadata->last_modified.epochTime(); - } + if (!object_info->metadata) + object_info->metadata = object_storage->tryGetObjectMetadata(object_info->relative_path); + + return object_info->metadata + ? std::optional(object_info->metadata->last_modified.epochTime()) + : std::nullopt; }; - chassert(object_info); if (format) { auto cache_key = getKeyForSchemaCache(object_info->relative_path, *format); @@ -105,14 +103,12 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) { - chassert(current_object_info); if (query_settings.schema_inference_use_cache) schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->relative_path, *format), num_rows); } void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) { - chassert(current_object_info); if (query_settings.schema_inference_use_cache && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) { @@ -125,7 +121,7 @@ void ReadBufferIterator::setResultingSchema(const ColumnsDescription & columns) if (query_settings.schema_inference_use_cache && query_settings.schema_inference_mode == SchemaInferenceMode::DEFAULT) { - schema_cache.addManyColumns(getPathsForSchemaCache(), columns); + schema_cache.addManyColumns(getKeysForSchemaCache(), columns); } } @@ -144,15 +140,11 @@ String ReadBufferIterator::getLastFileName() const std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() { - chassert(current_object_info); - - auto impl = object_storage->readObject( - StoredObject(current_object_info->relative_path), getContext()->getReadSettings()); - - int zstd_window_log_max = static_cast(getContext()->getSettingsRef().zstd_window_log_max); - return wrapReadBufferWithCompressionMethod( - std::move(impl), chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), - zstd_window_log_max); + auto context = getContext(); + auto impl = object_storage->readObject(StoredObject(current_object_info->relative_path), context->getReadSettings()); + const auto compression_method = chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method); + const auto zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); + return wrapReadBufferWithCompressionMethod(std::move(impl), compression_method, zstd_window_log_max); } ReadBufferIterator::Data ReadBufferIterator::next() @@ -190,16 +182,21 @@ ReadBufferIterator::Data ReadBufferIterator::next() if (first) { if (format.has_value()) + { throw Exception( ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, - "The table structure cannot be extracted from a {} format file, because there are no files with provided path " + "The table structure cannot be extracted from a {} format file, " + "because there are no files with provided path " "in {} or all files are empty. You can specify table structure manually", *format, object_storage->getName()); + } throw Exception( ErrorCodes::CANNOT_DETECT_FORMAT, - "The data format cannot be detected by the contents of the files, because there are no files with provided path " - "in {} or all files are empty. You can specify the format manually", object_storage->getName()); + "The data format cannot be detected by the contents of the files, " + "because there are no files with provided path " + "in {} or all files are empty. You can specify the format manually", + object_storage->getName()); } return {nullptr, std::nullopt, format}; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 2d58e1c789e..287e316e243 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -1,8 +1,7 @@ #pragma once #include -#include -#include #include +#include namespace DB @@ -12,6 +11,9 @@ class ReadBufferIterator : public IReadBufferIterator, WithContext { public: using FileIterator = std::shared_ptr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ObjectInfoPtr = StorageObjectStorage::ObjectInfoPtr; + using ObjectInfos = StorageObjectStorage::ObjectInfos; ReadBufferIterator( ObjectStoragePtr object_storage_, @@ -40,7 +42,7 @@ public: private: SchemaCache::Key getKeyForSchemaCache(const String & path, const String & format_name) const; - SchemaCache::Keys getPathsForSchemaCache() const; + SchemaCache::Keys getKeysForSchemaCache() const; std::optional tryGetColumnsFromCache( const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); diff --git a/src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp deleted file mode 100644 index f19e01cdc3e..00000000000 --- a/src/Storages/ObjectStorage/ReadFromObjectStorageStep.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -ReadFromObjectStorageStep::ReadFromObjectStorageStep( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - const String & name_, - const Names & columns_to_read, - const NamesAndTypesList & virtual_columns_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const std::optional & format_settings_, - bool distributed_processing_, - ReadFromFormatInfo info_, - SchemaCache & schema_cache_, - const bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_) - : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) - , object_storage(object_storage_) - , configuration(configuration_) - , info(std::move(info_)) - , virtual_columns(virtual_columns_) - , format_settings(format_settings_) - , query_settings(configuration->getQuerySettings(context_)) - , schema_cache(schema_cache_) - , name(name_ + "Source") - , need_only_count(need_only_count_) - , max_block_size(max_block_size_) - , num_streams(num_streams_) - , distributed_processing(distributed_processing_) -{ -} - -void ReadFromObjectStorageStep::createIterator(const ActionsDAG::Node * predicate) -{ - if (!iterator_wrapper) - { - auto context = getContext(); - iterator_wrapper = StorageObjectStorageSource::createFileIterator( - configuration, object_storage, distributed_processing, - context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); - } -} - -void ReadFromObjectStorageStep::applyFilters(ActionDAGNodes added_filter_nodes) -{ - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); - const ActionsDAG::Node * predicate = nullptr; - if (filter_actions_dag) - predicate = filter_actions_dag->getOutputs().at(0); - - createIterator(predicate); -} - -void ReadFromObjectStorageStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) -{ - createIterator(nullptr); - auto context = getContext(); - - Pipes pipes; - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - getName(), object_storage, configuration, info, format_settings, query_settings, - context, max_block_size, iterator_wrapper, need_only_count, schema_cache); - - source->setKeyCondition(filter_actions_dag, context); - pipes.emplace_back(std::move(source)); - } - - auto pipe = Pipe::unitePipes(std::move(pipes)); - if (pipe.empty()) - pipe = Pipe(std::make_shared(info.source_header)); - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); -} - -} diff --git a/src/Storages/ObjectStorage/ReadFromObjectStorageStep.h b/src/Storages/ObjectStorage/ReadFromObjectStorageStep.h deleted file mode 100644 index d98ebfef1f2..00000000000 --- a/src/Storages/ObjectStorage/ReadFromObjectStorageStep.h +++ /dev/null @@ -1,55 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -class ReadFromObjectStorageStep : public SourceStepWithFilter -{ -public: - using ConfigurationPtr = StorageObjectStorageConfigurationPtr; - - ReadFromObjectStorageStep( - ObjectStoragePtr object_storage_, - ConfigurationPtr configuration_, - const String & name_, - const Names & columns_to_read, - const NamesAndTypesList & virtual_columns_, - const SelectQueryInfo & query_info_, - const StorageSnapshotPtr & storage_snapshot_, - const std::optional & format_settings_, - bool distributed_processing_, - ReadFromFormatInfo info_, - SchemaCache & schema_cache_, - bool need_only_count_, - ContextPtr context_, - size_t max_block_size_, - size_t num_streams_); - - std::string getName() const override { return name; } - - void applyFilters(ActionDAGNodes added_filter_nodes) override; - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; - -private: - ObjectStoragePtr object_storage; - ConfigurationPtr configuration; - std::shared_ptr iterator_wrapper; - - const ReadFromFormatInfo info; - const NamesAndTypesList virtual_columns; - const std::optional format_settings; - const StorageObjectStorage::QuerySettings query_settings; - SchemaCache & schema_cache; - const String name; - const bool need_only_count; - const size_t max_block_size; - const size_t num_streams; - const bool distributed_processing; - - void createIterator(const ActionsDAG::Node * predicate); -}; - -} diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index bfd61c647f8..9fcbc6a6816 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -1,17 +1,23 @@ #include #if USE_AWS_S3 - #include +#include #include + +#include #include -#include + #include #include + #include #include #include +#include +#include + namespace DB { namespace ErrorCodes @@ -46,7 +52,7 @@ static const std::unordered_set optional_configuration_keys = String StorageS3Configuration::getDataSourceDescription() { - return fs::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; + return std::filesystem::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; } void StorageS3Configuration::check(ContextPtr context) const @@ -54,7 +60,7 @@ void StorageS3Configuration::check(ContextPtr context) const validateNamespace(url.bucket); context->getGlobalContext()->getRemoteHostFilter().checkURL(url.uri); context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(headers_from_ast); - StorageObjectStorageConfiguration::check(context); + Configuration::check(context); } void StorageS3Configuration::validateNamespace(const String & name) const @@ -63,7 +69,7 @@ void StorageS3Configuration::validateNamespace(const String & name) const } StorageS3Configuration::StorageS3Configuration(const StorageS3Configuration & other) - : StorageObjectStorageConfiguration(other) + : Configuration(other) { url = other.url; static_configuration = other.static_configuration; @@ -91,11 +97,12 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, assertInitialized(); const auto & config = context->getConfigRef(); + const auto & settings = context->getSettingsRef(); const std::string config_prefix = "s3."; - auto s3_settings = getSettings(config, config_prefix, context, false); /// FIXME: add a setting + auto s3_settings = getSettings(config, config_prefix, context, settings.s3_validate_request_settings); - request_settings.updateFromSettingsIfChanged(context->getSettingsRef()); + request_settings.updateFromSettingsIfChanged(settings); auth_settings.updateFrom(s3_settings->auth_settings); s3_settings->auth_settings = auth_settings; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index de4a6d17579..9eb724c4a64 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -4,17 +4,17 @@ #if USE_AWS_S3 -#include #include -#include -#include +#include namespace DB { -class StorageS3Configuration : public StorageObjectStorageConfiguration +class StorageS3Configuration : public StorageObjectStorage::Configuration { public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + static constexpr auto type_name = "s3"; StorageS3Configuration() = default; @@ -27,7 +27,6 @@ public: void setPath(const Path & path) override { url.key = path; } const Paths & getPaths() const override { return keys; } - Paths & getPaths() override { return keys; } void setPaths(const Paths & paths) override { keys = paths; } String getNamespace() const override { return url.bucket; } @@ -37,7 +36,7 @@ public: void check(ContextPtr context) const override; void validateNamespace(const String & name) const override; - StorageObjectStorageConfigurationPtr clone() override { return std::make_shared(*this); } + ConfigurationPtr clone() override { return std::make_shared(*this); } bool isStaticConfiguration() const override { return static_configuration; } ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c5565d8b0e8..2c9831f0d29 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -2,21 +2,25 @@ #include #include -#include #include +#include + +#include #include +#include +#include #include #include -#include + #include +#include #include -#include +#include +#include +#include #include #include -#include #include -#include -#include namespace DB @@ -26,6 +30,7 @@ namespace ErrorCodes { extern const int DATABASE_ACCESS_DENIED; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } StorageObjectStorage::StorageObjectStorage( @@ -90,6 +95,110 @@ void StorageObjectStorage::updateConfiguration(ContextPtr context) object_storage->applyNewSettings(context->getConfigRef(), "s3.", context); } +namespace +{ +class ReadFromObjectStorageStep : public SourceStepWithFilter +{ +public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + + ReadFromObjectStorageStep( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + const String & name_, + const Names & columns_to_read, + const NamesAndTypesList & virtual_columns_, + const SelectQueryInfo & query_info_, + const StorageSnapshotPtr & storage_snapshot_, + const std::optional & format_settings_, + bool distributed_processing_, + ReadFromFormatInfo info_, + SchemaCache & schema_cache_, + const bool need_only_count_, + ContextPtr context_, + size_t max_block_size_, + size_t num_streams_) + : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) + , object_storage(object_storage_) + , configuration(configuration_) + , schema_cache(schema_cache_) + , info(std::move(info_)) + , virtual_columns(virtual_columns_) + , format_settings(format_settings_) + , query_settings(configuration->getQuerySettings(context_)) + , name(name_ + "Source") + , need_only_count(need_only_count_) + , max_block_size(max_block_size_) + , num_streams(num_streams_) + , distributed_processing(distributed_processing_) + { + } + + std::string getName() const override { return name; } + + void applyFilters(ActionDAGNodes added_filter_nodes) override + { + filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + const ActionsDAG::Node * predicate = nullptr; + if (filter_actions_dag) + predicate = filter_actions_dag->getOutputs().at(0); + createIterator(predicate); + } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + createIterator(nullptr); + Pipes pipes; + auto context = getContext(); + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared( + getName(), object_storage, configuration, info, format_settings, query_settings, + context, max_block_size, iterator_wrapper, need_only_count, schema_cache); + + source->setKeyCondition(filter_actions_dag, context); + pipes.emplace_back(std::move(source)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + if (pipe.empty()) + pipe = Pipe(std::make_shared(info.source_header)); + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); + } + +private: + ObjectStoragePtr object_storage; + ConfigurationPtr configuration; + std::shared_ptr iterator_wrapper; + SchemaCache & schema_cache; + + const ReadFromFormatInfo info; + const NamesAndTypesList virtual_columns; + const std::optional format_settings; + const StorageObjectStorage::QuerySettings query_settings; + const String name; + const bool need_only_count; + const size_t max_block_size; + const size_t num_streams; + const bool distributed_processing; + + void createIterator(const ActionsDAG::Node * predicate) + { + if (iterator_wrapper) + return; + auto context = getContext(); + iterator_wrapper = StorageObjectStorageSource::createFileIterator( + configuration, object_storage, distributed_processing, + context, predicate, virtual_columns, nullptr, context->getFileProgressCallback()); + } +}; +} + void StorageObjectStorage::read( QueryPlan & query_plan, const Names & column_names, @@ -123,7 +232,7 @@ void StorageObjectStorage::read( storage_snapshot, format_settings, distributed_processing, - std::move(read_from_format_info), + read_from_format_info, getSchemaCache(local_context), need_only_count, local_context, @@ -169,12 +278,13 @@ SinkToStoragePtr StorageObjectStorage::write( getName(), configuration->getPath()); } - auto & paths = configuration->getPaths(); + auto paths = configuration->getPaths(); if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( *object_storage, *configuration, settings, paths.front(), paths.size())) { paths.push_back(*new_key); } + configuration->setPaths(paths); return std::make_shared( object_storage, @@ -185,10 +295,10 @@ SinkToStoragePtr StorageObjectStorage::write( } void StorageObjectStorage::truncate( - const ASTPtr &, - const StorageMetadataPtr &, - ContextPtr, - TableExclusiveLockHolder &) + const ASTPtr & /* query */, + const StorageMetadataPtr & /* metadata_snapshot */, + ContextPtr /* context */, + TableExclusiveLockHolder & /* table_holder */) { if (configuration->withGlobs()) { @@ -233,10 +343,8 @@ ColumnsDescription StorageObjectStorage::resolveSchemaFromData( const ContextPtr & context) { ObjectInfos read_keys; - auto read_buffer_iterator = createReadBufferIterator( - object_storage, configuration, format_settings, read_keys, context); - return readSchemaFromFormat( - configuration->format, format_settings, *read_buffer_iterator, context); + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + return readSchemaFromFormat(configuration->format, format_settings, *iterator, context); } std::string StorageObjectStorage::resolveFormatFromData( @@ -246,10 +354,8 @@ std::string StorageObjectStorage::resolveFormatFromData( const ContextPtr & context) { ObjectInfos read_keys; - auto read_buffer_iterator = createReadBufferIterator( - object_storage, configuration, format_settings, read_keys, context); - return detectFormatAndReadSchema( - format_settings, *read_buffer_iterator, context).second; + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + return detectFormatAndReadSchema(format_settings, *iterator, context).second; } std::pair StorageObjectStorage::resolveSchemaAndFormatFromData( @@ -259,10 +365,8 @@ std::pair StorageObjectStorage::resolveSchemaAn const ContextPtr & context) { ObjectInfos read_keys; - auto read_buffer_iterator = createReadBufferIterator( - object_storage, configuration, format_settings, read_keys, context); - - auto [columns, format] = detectFormatAndReadSchema(format_settings, *read_buffer_iterator, context); + auto iterator = createReadBufferIterator(object_storage, configuration, format_settings, read_keys, context); + auto [columns, format] = detectFormatAndReadSchema(format_settings, *iterator, context); configuration->format = format; return std::pair(columns, format); } @@ -302,4 +406,65 @@ SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, c throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported storage type: {}", storage_type_name); } +void StorageObjectStorage::Configuration::initialize( + Configuration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure) +{ + if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) + configuration.fromNamedCollection(*named_collection); + else + configuration.fromAST(engine_args, local_context, with_table_structure); + + // FIXME: it should be - if (format == "auto" && get_format_from_file) + if (configuration.format == "auto") + configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); + else + FormatFactory::instance().checkFormatName(configuration.format); + + configuration.initialized = true; +} + +void StorageObjectStorage::Configuration::check(ContextPtr) const +{ + FormatFactory::instance().checkFormatName(format); +} + +StorageObjectStorage::Configuration::Configuration(const Configuration & other) +{ + format = other.format; + compression_method = other.compression_method; + structure = other.structure; +} + +bool StorageObjectStorage::Configuration::withWildcard() const +{ + static const String PARTITION_ID_WILDCARD = "{_partition_id}"; + return getPath().find(PARTITION_ID_WILDCARD) != String::npos + || getNamespace().find(PARTITION_ID_WILDCARD) != String::npos; +} + +bool StorageObjectStorage::Configuration::isPathWithGlobs() const +{ + return getPath().find_first_of("*?{") != std::string::npos; +} + +bool StorageObjectStorage::Configuration::isNamespaceWithGlobs() const +{ + return getNamespace().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorage::Configuration::getPathWithoutGlobs() const +{ + return getPath().substr(0, getPath().find_first_of("*?{")); +} + +void StorageObjectStorage::Configuration::assertInitialized() const +{ + if (!initialized) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); + } +} } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index d46a875bf42..46d422b26c2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -2,15 +2,16 @@ #include #include #include +#include #include #include namespace DB { -class StorageObjectStorageConfiguration; class ReadBufferIterator; class SchemaCache; +class NamedCollection; /** * A general class containing implementation for external table engines @@ -20,7 +21,7 @@ class SchemaCache; class StorageObjectStorage : public IStorage { public: - using Configuration = StorageObjectStorageConfiguration; + class Configuration; using ConfigurationPtr = std::shared_ptr; using ObjectInfo = RelativePathWithMetadata; using ObjectInfoPtr = std::shared_ptr; @@ -134,4 +135,61 @@ protected: std::mutex configuration_update_mutex; }; +class StorageObjectStorage::Configuration +{ +public: + Configuration() = default; + Configuration(const Configuration & other); + virtual ~Configuration() = default; + + using Path = std::string; + using Paths = std::vector; + + static void initialize( + Configuration & configuration, + ASTs & engine_args, + ContextPtr local_context, + bool with_table_structure); + + virtual std::string getTypeName() const = 0; + virtual std::string getEngineName() const = 0; + + virtual Path getPath() const = 0; + virtual void setPath(const Path & path) = 0; + + virtual const Paths & getPaths() const = 0; + virtual void setPaths(const Paths & paths) = 0; + + virtual String getDataSourceDescription() = 0; + virtual String getNamespace() const = 0; + virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; + virtual void addStructureAndFormatToArgs( + ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; + + bool withWildcard() const; + bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } + bool isPathWithGlobs() const; + bool isNamespaceWithGlobs() const; + virtual std::string getPathWithoutGlobs() const; + + virtual void check(ContextPtr context) const; + virtual void validateNamespace(const String & /* name */) const {} + + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT + virtual ConfigurationPtr clone() = 0; + virtual bool isStaticConfiguration() const { return true; } + + String format = "auto"; + String compression_method = "auto"; + String structure = "auto"; + +protected: + virtual void fromNamedCollection(const NamedCollection & collection) = 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 1a1df399626..193894a1d44 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -1,21 +1,15 @@ #include "Storages/ObjectStorage/StorageObjectStorageCluster.h" -#include "config.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include #include +#include +#include + +#include #include +#include +#include + namespace DB { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 2db8f5c352e..b38eb722df5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -1,12 +1,10 @@ #pragma once -#include "config.h" - -#include +// #include #include #include #include -#include +// #include namespace DB { @@ -29,17 +27,14 @@ public: std::string getName() const override; - RemoteQueryExecutor::Extension getTaskIteratorExtension( - const ActionsDAG::Node * predicate, - const ContextPtr & context) const override; - bool supportsSubcolumns() const override { return true; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } -private: - void updateBeforeRead(const ContextPtr & /* context */) override {} + RemoteQueryExecutor::Extension getTaskIteratorExtension( + const ActionsDAG::Node * predicate, const ContextPtr & context) const override; +private: void updateQueryToSendIfNeeded( ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp deleted file mode 100644 index 89c15085274..00000000000 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.cpp +++ /dev/null @@ -1,74 +0,0 @@ -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -void StorageObjectStorageConfiguration::initialize( - StorageObjectStorageConfiguration & configuration, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure) -{ - if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args, local_context)) - configuration.fromNamedCollection(*named_collection); - else - configuration.fromAST(engine_args, local_context, with_table_structure); - - // FIXME: it should be - if (format == "auto" && get_format_from_file) - if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); - else - FormatFactory::instance().checkFormatName(configuration.format); - - configuration.initialized = true; -} - -void StorageObjectStorageConfiguration::check(ContextPtr) const -{ - FormatFactory::instance().checkFormatName(format); -} - -StorageObjectStorageConfiguration::StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other) -{ - format = other.format; - compression_method = other.compression_method; - structure = other.structure; -} - -bool StorageObjectStorageConfiguration::withWildcard() const -{ - static const String PARTITION_ID_WILDCARD = "{_partition_id}"; - return getPath().find(PARTITION_ID_WILDCARD) != String::npos - || getNamespace().find(PARTITION_ID_WILDCARD) != String::npos; -} - -bool StorageObjectStorageConfiguration::isPathWithGlobs() const -{ - return getPath().find_first_of("*?{") != std::string::npos; -} - -bool StorageObjectStorageConfiguration::isNamespaceWithGlobs() const -{ - return getNamespace().find_first_of("*?{") != std::string::npos; -} - -std::string StorageObjectStorageConfiguration::getPathWithoutGlob() const -{ - return getPath().substr(0, getPath().find_first_of("*?{")); -} - -void StorageObjectStorageConfiguration::assertInitialized() const -{ - if (!initialized) - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Configuration was not initialized before usage"); - } -} - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h b/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h deleted file mode 100644 index c55362aa8bd..00000000000 --- a/src/Storages/ObjectStorage/StorageObjectStorageConfiguration.h +++ /dev/null @@ -1,75 +0,0 @@ -#pragma once -#include -#include -#include "StorageObjectStorage.h" -#include - -namespace fs = std::filesystem; - -namespace DB -{ - -class StorageObjectStorageConfiguration; -using StorageObjectStorageConfigurationPtr = std::shared_ptr; - -class StorageObjectStorageConfiguration -{ -public: - StorageObjectStorageConfiguration() = default; - StorageObjectStorageConfiguration(const StorageObjectStorageConfiguration & other); - virtual ~StorageObjectStorageConfiguration() = default; - - using Path = std::string; - using Paths = std::vector; - - static void initialize( - StorageObjectStorageConfiguration & configuration, - ASTs & engine_args, - ContextPtr local_context, - bool with_table_structure); - - virtual std::string getTypeName() const = 0; - virtual std::string getEngineName() const = 0; - - virtual Path getPath() const = 0; - virtual void setPath(const Path & path) = 0; - - virtual const Paths & getPaths() const = 0; - virtual Paths & getPaths() = 0; - virtual void setPaths(const Paths & paths) = 0; - - virtual String getDataSourceDescription() = 0; - virtual String getNamespace() const = 0; - virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; - virtual void addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; - - bool withWildcard() const; - bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } - bool isPathWithGlobs() const; - bool isNamespaceWithGlobs() const; - virtual std::string getPathWithoutGlob() const; - - virtual void check(ContextPtr context) const; - virtual void validateNamespace(const String & /* name */) const {} - - virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT - virtual StorageObjectStorageConfigurationPtr clone() = 0; - virtual bool isStaticConfiguration() const { return true; } - - String format = "auto"; - String compression_method = "auto"; - String structure = "auto"; - -protected: - virtual void fromNamedCollection(const NamedCollection & collection) = 0; - virtual void fromAST(ASTs & args, ContextPtr context, bool with_structure) = 0; - - void assertInitialized() const; - - bool initialized = false; -}; - -using StorageObjectStorageConfigurationPtr = std::shared_ptr; - -} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 62367a6b933..81bdeaa43a3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -14,14 +14,13 @@ namespace ErrorCodes StorageObjectStorageSink::StorageObjectStorageSink( ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, + ConfigurationPtr configuration, std::optional format_settings_, const Block & sample_block_, ContextPtr context, const std::string & blob_path) : SinkToStorage(sample_block_) , sample_block(sample_block_) - , format_settings(format_settings_) { const auto & settings = context->getSettingsRef(); const auto path = blob_path.empty() ? configuration->getPaths().back() : blob_path; @@ -37,7 +36,7 @@ StorageObjectStorageSink::StorageObjectStorageSink( static_cast(settings.output_format_compression_zstd_window_log)); writer = FormatFactory::instance().getOutputFormatParallelIfPossible( - configuration->format, *write_buf, sample_block, context, format_settings); + configuration->format, *write_buf, sample_block, context, format_settings_); } void StorageObjectStorageSink::consume(Chunk chunk) @@ -102,7 +101,7 @@ void StorageObjectStorageSink::release() PartitionedStorageObjectStorageSink::PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + ConfigurationPtr configuration_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index 6c2f73e40e3..a3c8ef68cf0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -1,17 +1,18 @@ #pragma once #include -#include #include -#include +#include namespace DB { class StorageObjectStorageSink : public SinkToStorage { public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + StorageObjectStorageSink( ObjectStoragePtr object_storage, - StorageObjectStorageConfigurationPtr configuration, + ConfigurationPtr configuration, std::optional format_settings_, const Block & sample_block_, ContextPtr context, @@ -29,8 +30,6 @@ public: private: const Block sample_block; - const std::optional format_settings; - std::unique_ptr write_buf; OutputFormatPtr writer; bool cancelled = false; @@ -43,9 +42,11 @@ private: class PartitionedStorageObjectStorageSink : public PartitionedSink { public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + PartitionedStorageObjectStorageSink( ObjectStoragePtr object_storage_, - StorageObjectStorageConfigurationPtr configuration_, + ConfigurationPtr configuration_, std::optional format_settings_, const Block & sample_block_, ContextPtr context_, @@ -58,7 +59,8 @@ private: void validateNamespace(const String & str); ObjectStoragePtr object_storage; - StorageObjectStorageConfigurationPtr configuration; + ConfigurationPtr configuration; + const StorageObjectStorage::QuerySettings query_settings; const std::optional format_settings; const Block sample_block; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 4551c2df7c3..b224afb7a58 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -9,10 +9,11 @@ #include #include #include -#include +#include #include #include +namespace fs = std::filesystem; namespace ProfileEvents { @@ -218,11 +219,9 @@ std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const O auto get_last_mod_time = [&]() -> std::optional { - if (object_info->metadata) - { - return object_info->metadata->last_modified.epochTime(); - } - return std::nullopt; + return object_info->metadata + ? std::optional(object_info->metadata->last_modified.epochTime()) + : std::nullopt; }; return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); } @@ -354,7 +353,7 @@ StorageObjectStorageSource::IIterator::IIterator(const std::string & logger_name { } -ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::IIterator::next(size_t processor) { auto object_info = nextImpl(processor); @@ -392,7 +391,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( else if (configuration->isPathWithGlobs()) { const auto key_with_globs = configuration_->getPath(); - const auto key_prefix = configuration->getPathWithoutGlob(); + const auto key_prefix = configuration->getPathWithoutGlobs(); object_storage_iterator = object_storage->iterate(key_prefix, list_object_keys_size); matcher = std::make_unique(makeRegexpPatternFromGlobs(key_with_globs)); @@ -423,7 +422,7 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } } -ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processor) +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processor) { std::lock_guard lock(next_mutex); auto object_info = nextImplUnlocked(processor); @@ -439,7 +438,7 @@ ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processo return object_info; } -ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImplUnlocked(size_t /* processor */) +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImplUnlocked(size_t /* processor */) { bool current_batch_processed = object_infos.empty() || index >= object_infos.size(); if (is_finished && current_batch_processed) @@ -533,7 +532,7 @@ StorageObjectStorageSource::KeysIterator::KeysIterator( } } -ObjectInfoPtr StorageObjectStorageSource::KeysIterator::nextImpl(size_t /* processor */) +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::KeysIterator::nextImpl(size_t /* processor */) { while (true) { @@ -614,7 +613,7 @@ StorageObjectStorageSource::ReadTaskIterator::ReadTaskIterator( } } -ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::nextImpl(size_t) +StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator::nextImpl(size_t) { size_t current_index = index.fetch_add(1, std::memory_order_relaxed); if (current_index >= buffer.size()) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 0afbf77db2b..356478422bc 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include namespace DB @@ -16,6 +16,11 @@ class StorageObjectStorageSource : public SourceWithKeyCondition, WithContext { friend class StorageS3QueueSource; public: + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; + using ObjectInfo = StorageObjectStorage::ObjectInfo; + using ObjectInfos = StorageObjectStorage::ObjectInfos; + using ObjectInfoPtr = StorageObjectStorage::ObjectInfoPtr; + class IIterator; class ReadTaskIterator; class GlobIterator; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h b/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h deleted file mode 100644 index 241e2f20962..00000000000 --- a/src/Storages/ObjectStorage/StorageObjectStorage_fwd_internal.h +++ /dev/null @@ -1,12 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -using ConfigurationPtr = StorageObjectStorageConfigurationPtr; -using ObjectInfo = RelativePathWithMetadata; -using ObjectInfoPtr = std::shared_ptr; -using ObjectInfos = std::vector; - -} diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index 2a7236ab196..bde3cb7e1cb 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include namespace DB { @@ -47,14 +47,15 @@ void resolveSchemaAndFormat( ColumnsDescription & columns, std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorageConfigurationPtr & configuration, + const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, const ContextPtr & context) { if (columns.empty()) { if (format == "auto") - std::tie(columns, format) = StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); + std::tie(columns, format) = + StorageObjectStorage::resolveSchemaAndFormatFromData(object_storage, configuration, format_settings, context); else columns = StorageObjectStorage::resolveSchemaFromData(object_storage, configuration, format_settings, context); } diff --git a/src/Storages/ObjectStorage/Utils.h b/src/Storages/ObjectStorage/Utils.h index 3a752e6b8f0..2077999df41 100644 --- a/src/Storages/ObjectStorage/Utils.h +++ b/src/Storages/ObjectStorage/Utils.h @@ -1,14 +1,10 @@ #pragma once -#include #include "StorageObjectStorage.h" namespace DB { class IObjectStorage; -class StorageObjectStorageConfiguration; -using StorageObjectStorageConfigurationPtr = std::shared_ptr; -struct StorageObjectStorageSettings; std::optional checkAndGetNewFileOnInsertIfNeeded( const IObjectStorage & object_storage, @@ -21,7 +17,7 @@ void resolveSchemaAndFormat( ColumnsDescription & columns, std::string & format, ObjectStoragePtr object_storage, - const StorageObjectStorageConfigurationPtr & configuration, + const StorageObjectStorage::ConfigurationPtr & configuration, std::optional format_settings, const ContextPtr & context); diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 06b8aefb716..c23b180215e 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -1,8 +1,8 @@ -#include +#include #include #include #include -#include +#include #include #include @@ -18,13 +18,15 @@ namespace ErrorCodes static std::shared_ptr createStorageObjectStorage( const StorageFactory::Arguments & args, - typename StorageObjectStorage::ConfigurationPtr configuration, + StorageObjectStorage::ConfigurationPtr configuration, ContextPtr context) { auto & engine_args = args.engine_args; if (engine_args.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + StorageObjectStorage::Configuration::initialize(*configuration, args.engine_args, context, false); + // Use format settings from global server context + settings from // the SETTINGS clause of the create query. Settings from current // session and user are ignored. @@ -75,10 +77,8 @@ void registerStorageAzure(StorageFactory & factory) { factory.registerStorage("AzureBlobStorage", [](const StorageFactory::Arguments & args) { - auto context = args.getLocalContext(); - auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, context); + auto configuration = std::make_shared(); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); }, { .supports_settings = true, @@ -94,10 +94,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) { factory.registerStorage(name, [=](const StorageFactory::Arguments & args) { - auto context = args.getLocalContext(); auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, context); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); }, { .supports_settings = true, @@ -129,10 +127,8 @@ void registerStorageHDFS(StorageFactory & factory) { factory.registerStorage("HDFS", [=](const StorageFactory::Arguments & args) { - auto context = args.getLocalContext(); auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, context, false); - return createStorageObjectStorage(args, configuration, context); + return createStorageObjectStorage(args, configuration, args.getLocalContext()); }, { .supports_settings = true, diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.cpp b/src/Storages/S3Queue/S3QueueTableMetadata.cpp index 8354e6aa2ae..f0b7568ae7f 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.cpp +++ b/src/Storages/S3Queue/S3QueueTableMetadata.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -32,7 +33,7 @@ namespace S3QueueTableMetadata::S3QueueTableMetadata( - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata) { diff --git a/src/Storages/S3Queue/S3QueueTableMetadata.h b/src/Storages/S3Queue/S3QueueTableMetadata.h index 2158b189070..bb8f8ccf2c4 100644 --- a/src/Storages/S3Queue/S3QueueTableMetadata.h +++ b/src/Storages/S3Queue/S3QueueTableMetadata.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include namespace DB @@ -29,7 +29,7 @@ struct S3QueueTableMetadata S3QueueTableMetadata() = default; S3QueueTableMetadata( - const StorageObjectStorageConfiguration & configuration, + const StorageObjectStorage::Configuration & configuration, const S3QueueSettings & engine_settings, const StorageInMemoryMetadata & storage_metadata); diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index e84dabecf3b..38934a7895a 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -591,7 +591,7 @@ void registerStorageS3QueueImpl(const String & name, StorageFactory & factory) throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); auto configuration = std::make_shared(); - StorageObjectStorageConfiguration::initialize(*configuration, args.engine_args, args.getContext(), false); + StorageObjectStorage::Configuration::initialize(*configuration, 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/System/StorageSystemSchemaInferenceCache.cpp b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp index a2d3f342a63..b67a8b23e9d 100644 --- a/src/Storages/System/StorageSystemSchemaInferenceCache.cpp +++ b/src/Storages/System/StorageSystemSchemaInferenceCache.cpp @@ -11,7 +11,7 @@ #include #include #include -#include +#include namespace DB { @@ -84,7 +84,7 @@ void StorageSystemSchemaInferenceCache::fillData(MutableColumns & res_columns, C #endif fillDataImpl(res_columns, StorageURL::getSchemaCache(context), "URL"); #if USE_AZURE_BLOB_STORAGE - fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageAzureBlobConfiguration::type_name), "Azure"); + fillDataImpl(res_columns, StorageObjectStorage::getSchemaCache(context, StorageAzureConfiguration::type_name), "Azure"); #endif } diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 02c8c623e61..6ad8689a9b4 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 06676a8adfa..a997b34a75c 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -1,23 +1,23 @@ #include "config.h" +#include +#include +#include #include + #include +#include #include #include + #include -#include -#include -#include -#include -#include -#include + #include #include -#include -#include -#include -#include -#include "registerTableFunctions.h" +#include +#include +#include +#include namespace DB @@ -29,8 +29,7 @@ namespace ErrorCodes } template -ObjectStoragePtr TableFunctionObjectStorage< - Definition, Configuration>::getObjectStorage(const ContextPtr & context, bool create_readonly) const +ObjectStoragePtr TableFunctionObjectStorage::getObjectStorage(const ContextPtr & context, bool create_readonly) const { if (!object_storage) object_storage = configuration->createObjectStorage(context, create_readonly); @@ -38,8 +37,7 @@ ObjectStoragePtr TableFunctionObjectStorage< } template -StorageObjectStorageConfigurationPtr TableFunctionObjectStorage< - Definition, Configuration>::getConfiguration() const +StorageObjectStorage::ConfigurationPtr TableFunctionObjectStorage::getConfiguration() const { if (!configuration) configuration = std::make_shared(); @@ -47,8 +45,8 @@ StorageObjectStorageConfigurationPtr TableFunctionObjectStorage< } template -std::vector TableFunctionObjectStorage< - Definition, Configuration>::skipAnalysisForArguments(const QueryTreeNodePtr & query_node_table_function, ContextPtr) const +std::vector TableFunctionObjectStorage::skipAnalysisForArguments( + const QueryTreeNodePtr & query_node_table_function, ContextPtr) const { auto & table_function_node = query_node_table_function->as(); auto & table_function_arguments_nodes = table_function_node.getArguments().getNodes(); @@ -64,19 +62,6 @@ std::vector TableFunctionObjectStorage< return result; } -template -void TableFunctionObjectStorage::updateStructureAndFormatArgumentsIfNeeded( - ASTs & args, const String & structure, const String & format, const ContextPtr & context) -{ - Configuration().addStructureAndFormatToArgs(args, structure, format, context); -} - -template -void TableFunctionObjectStorage::parseArgumentsImpl(ASTs & engine_args, const ContextPtr & local_context) -{ - StorageObjectStorageConfiguration::initialize(*getConfiguration(), engine_args, local_context, true); -} - template void TableFunctionObjectStorage::parseArguments(const ASTPtr & ast_function, ContextPtr context) { @@ -94,32 +79,16 @@ template ColumnsDescription TableFunctionObjectStorage< Definition, Configuration>::getActualTableStructure(ContextPtr context, bool is_insert_query) const { - chassert(configuration); if (configuration->structure == "auto") { context->checkAccess(getSourceAccessType()); - auto storage = getObjectStorage(context, !is_insert_query); ColumnsDescription columns; + auto storage = getObjectStorage(context, !is_insert_query); resolveSchemaAndFormat(columns, configuration->format, storage, configuration, std::nullopt, context); return columns; } - - return parseColumnsListFromString(configuration->structure, context); -} - -template -bool TableFunctionObjectStorage< - Definition, Configuration>::supportsReadingSubsetOfColumns(const ContextPtr & context) -{ - chassert(configuration); - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); -} - -template -std::unordered_set TableFunctionObjectStorage< - Definition, Configuration>::getVirtualsToCheckBeforeUsingStructureHint() const -{ - return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); + else + return parseColumnsListFromString(configuration->structure, context); } template @@ -205,7 +174,7 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) #endif #if USE_AZURE_BLOB_STORAGE - factory.registerFunction>( + factory.registerFunction>( { .documentation = { @@ -229,8 +198,8 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) } #if USE_AZURE_BLOB_STORAGE -template class TableFunctionObjectStorage; -template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; +template class TableFunctionObjectStorage; #endif #if USE_AWS_S3 diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index bd43cae3697..bbc40cc6191 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -1,19 +1,18 @@ #pragma once #include "config.h" - #include -#include -#include +#include #include - +#include +#include namespace DB { class Context; class StorageS3Configuration; -class StorageAzureBlobConfiguration; +class StorageAzureConfiguration; class StorageHDFSConfiguration; struct S3StorageSettings; struct AzureStorageSettings; @@ -104,20 +103,32 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } - bool supportsReadingSubsetOfColumns(const ContextPtr & context) override; + bool supportsReadingSubsetOfColumns(const ContextPtr & context) override + { + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); + } - std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override; + std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override + { + return VirtualColumnUtils::getVirtualNamesForFileLikeStorage(); + } - virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context); + virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context) + { + StorageObjectStorage::Configuration::initialize(*getConfiguration(), args, context, true); + } static void updateStructureAndFormatArgumentsIfNeeded( ASTs & args, const String & structure, const String & format, - const ContextPtr & context); + const ContextPtr & context) + { + Configuration().addStructureAndFormatToArgs(args, structure, format, context); + } protected: - using ConfigurationPtr = StorageObjectStorageConfigurationPtr; + using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; StoragePtr executeImpl( const ASTPtr & ast_function, @@ -146,7 +157,7 @@ using TableFunctionS3 = TableFunctionObjectStorage; +using TableFunctionAzureBlob = TableFunctionObjectStorage; #endif #if USE_HDFS diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp index ce78076dd21..449bd2c8c49 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.cpp +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace DB @@ -109,7 +109,7 @@ template class TableFunctionObjectStorageCluster; +template class TableFunctionObjectStorageCluster; #endif #if USE_HDFS diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index a8bc11b5e40..76786fafe99 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -13,7 +13,7 @@ class Context; class StorageS3Settings; class StorageAzureBlobSettings; class StorageS3Configuration; -class StorageAzureBlobConfiguration; +class StorageAzureConfiguration; struct AzureClusterDefinition { @@ -90,7 +90,7 @@ using TableFunctionS3Cluster = TableFunctionObjectStorageCluster; +using TableFunctionAzureBlobCluster = TableFunctionObjectStorageCluster; #endif #if USE_HDFS diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 5e0bc3267d8..26b9a771416 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -29,18 +29,6 @@ void registerTableFunctions() registerTableFunctionFuzzJSON(factory); #endif -#if USE_AWS_S3 - // registerTableFunctionS3Cluster(factory); - // registerTableFunctionHudi(factory); -#if USE_PARQUET - // registerTableFunctionDeltaLake(factory); -#endif -#if USE_AVRO - // registerTableFunctionIceberg(factory); -#endif - -#endif - #if USE_HIVE registerTableFunctionHive(factory); #endif diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml index 4210c13b727..7df7b56b3b4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/inf_s3_retries.xml @@ -5,6 +5,7 @@ 1000000 1 + 0 diff --git a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml index 95a313ea4f2..c1ca258f6c4 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml +++ b/tests/integration/test_checking_s3_blobs_paranoid/configs/s3_retries.xml @@ -5,6 +5,7 @@ 5 0 + 0 From 671650bd2eaf2a07d5e6f517b40905c71ce798b6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 28 Apr 2024 12:18:24 +0200 Subject: [PATCH 045/158] Cleanup --- src/Backups/BackupIO_AzureBlobStorage.cpp | 4 ++-- src/Storages/ObjectStorage/Azure/Configuration.h | 16 ++++++++++------ .../ObjectStorage/DataLakes/IStorageDataLake.h | 4 +--- src/Storages/ObjectStorage/HDFS/Configuration.h | 11 +++++++---- src/Storages/ObjectStorage/S3/Configuration.h | 10 ++++++---- .../ObjectStorage/StorageObjectStorage.cpp | 16 +++------------- .../ObjectStorage/StorageObjectStorage.h | 5 +---- .../ObjectStorage/StorageObjectStorageSource.cpp | 9 ++++----- .../ObjectStorage/StorageObjectStorageSource.h | 5 +---- .../registerStorageObjectStorage.cpp | 3 +-- src/Storages/S3Queue/StorageS3Queue.cpp | 6 ++---- 11 files changed, 38 insertions(+), 51 deletions(-) diff --git a/src/Backups/BackupIO_AzureBlobStorage.cpp b/src/Backups/BackupIO_AzureBlobStorage.cpp index f00da686c18..3af66e5470f 100644 --- a/src/Backups/BackupIO_AzureBlobStorage.cpp +++ b/src/Backups/BackupIO_AzureBlobStorage.cpp @@ -36,7 +36,7 @@ BackupReaderAzureBlobStorage::BackupReaderAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = configuration.createClient(/* is_read_only */ false, /* attempt_to_create_container */true); + auto client_ptr = configuration.createClient(/* is_readonly */false, /* attempt_to_create_container */true); object_storage = std::make_unique("BackupReaderAzureBlobStorage", std::move(client_ptr), configuration.createSettings(context_), @@ -121,7 +121,7 @@ BackupWriterAzureBlobStorage::BackupWriterAzureBlobStorage( , data_source_description{DataSourceType::ObjectStorage, ObjectStorageType::Azure, MetadataStorageType::None, configuration_.container, false, false} , configuration(configuration_) { - auto client_ptr = configuration.createClient(/* is_read_only */ false, attempt_to_create_container); + auto client_ptr = configuration.createClient(/* is_readonly */false, attempt_to_create_container); object_storage = std::make_unique("BackupWriterAzureBlobStorage", std::move(client_ptr), configuration.createSettings(context_), diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 91a9a0bbbd5..1591cb42469 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -3,7 +3,6 @@ #include "config.h" #if USE_AZURE_BLOB_STORAGE - #include #include #include @@ -36,20 +35,25 @@ public: const Paths & getPaths() const override { return blobs_paths; } void setPaths(const Paths & paths) override { blobs_paths = paths; } - String getDataSourceDescription() override { return std::filesystem::path(connection_url) / container; } String getNamespace() const override { return container; } + String getDataSourceDescription() override { return std::filesystem::path(connection_url) / container; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; - ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT ConfigurationPtr clone() override { return std::make_shared(*this); } - void fromNamedCollection(const NamedCollection & collection) override; - void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; + void addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & format_, ContextPtr context) override; + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) override; protected: + void fromNamedCollection(const NamedCollection & collection) override; + void fromAST(ASTs & args, ContextPtr context, bool with_structure) override; + using AzureClient = Azure::Storage::Blobs::BlobContainerClient; using AzureClientPtr = std::unique_ptr; diff --git a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h index 3119b844aaf..83865c47eb8 100644 --- a/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h +++ b/src/Storages/ObjectStorage/DataLakes/IStorageDataLake.h @@ -38,7 +38,7 @@ public: std::optional format_settings_, LoadingStrictnessLevel mode) { - auto object_storage = base_configuration->createObjectStorage(context); + auto object_storage = base_configuration->createObjectStorage(context, /* is_readonly */true); DataLakeMetadataPtr metadata; NamesAndTypesList schema_from_metadata; @@ -96,8 +96,6 @@ public: void updateConfiguration(ContextPtr local_context) override { - std::lock_guard lock(Storage::configuration_update_mutex); - Storage::updateConfiguration(local_context); auto new_metadata = DataLakeMetadata::create(Storage::object_storage, base_configuration, local_context); diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index cac09ee1d92..dc06e754c44 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -28,19 +28,22 @@ public: const Paths & getPaths() const override { return paths; } void setPaths(const Paths & paths_) override { paths = paths_; } + std::string getPathWithoutGlobs() const override; String getNamespace() const override { return ""; } String getDataSourceDescription() override { return url; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; - ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT ConfigurationPtr clone() override { return std::make_shared(*this); } - void addStructureAndFormatToArgs( - ASTs & args, const String & structure_, const String & format_, ContextPtr context) override; + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; - std::string getPathWithoutGlobs() const override; + void addStructureAndFormatToArgs( + ASTs & args, + const String & structure_, + const String & format_, + ContextPtr context) override; private: void fromNamedCollection(const NamedCollection &) override; diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 9eb724c4a64..b28b1c226a7 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -3,7 +3,6 @@ #include "config.h" #if USE_AWS_S3 - #include #include @@ -35,13 +34,16 @@ public: void check(ContextPtr context) const override; void validateNamespace(const String & name) const override; - ConfigurationPtr clone() override { return std::make_shared(*this); } bool isStaticConfiguration() const override { return static_configuration; } - ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) override; /// NOLINT + ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) override; + void addStructureAndFormatToArgs( - ASTs & args, const String & structure, const String & format, ContextPtr context) override; + ASTs & args, + const String & structure, + const String & format, + ContextPtr context) override; private: void fromNamedCollection(const NamedCollection & collection) override; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 2c9831f0d29..a187a8fc54d 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -91,6 +91,7 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c void StorageObjectStorage::updateConfiguration(ContextPtr context) { + /// FIXME: we should be able to update everything apart from client if static_configuration == true. if (!configuration->isStaticConfiguration()) object_storage->applyNewSettings(context->getConfigRef(), "s3.", context); } @@ -113,7 +114,6 @@ public: const std::optional & format_settings_, bool distributed_processing_, ReadFromFormatInfo info_, - SchemaCache & schema_cache_, const bool need_only_count_, ContextPtr context_, size_t max_block_size_, @@ -121,11 +121,9 @@ public: : SourceStepWithFilter(DataStream{.header = info_.source_header}, columns_to_read, query_info_, storage_snapshot_, context_) , object_storage(object_storage_) , configuration(configuration_) - , schema_cache(schema_cache_) , info(std::move(info_)) , virtual_columns(virtual_columns_) , format_settings(format_settings_) - , query_settings(configuration->getQuerySettings(context_)) , name(name_ + "Source") , need_only_count(need_only_count_) , max_block_size(max_block_size_) @@ -154,8 +152,8 @@ public: for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( - getName(), object_storage, configuration, info, format_settings, query_settings, - context, max_block_size, iterator_wrapper, need_only_count, schema_cache); + getName(), object_storage, configuration, info, format_settings, + context, max_block_size, iterator_wrapper, need_only_count); source->setKeyCondition(filter_actions_dag, context); pipes.emplace_back(std::move(source)); @@ -175,12 +173,10 @@ private: ObjectStoragePtr object_storage; ConfigurationPtr configuration; std::shared_ptr iterator_wrapper; - SchemaCache & schema_cache; const ReadFromFormatInfo info; const NamesAndTypesList virtual_columns; const std::optional format_settings; - const StorageObjectStorage::QuerySettings query_settings; const String name; const bool need_only_count; const size_t max_block_size; @@ -233,7 +229,6 @@ void StorageObjectStorage::read( format_settings, distributed_processing, read_from_format_info, - getSchemaCache(local_context), need_only_count, local_context, max_block_size, @@ -371,11 +366,6 @@ std::pair StorageObjectStorage::resolveSchemaAn return std::pair(columns, format); } -SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context) -{ - return getSchemaCache(context, configuration->getTypeName()); -} - SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name) { if (storage_type_name == "s3") diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 46d422b26c2..3f8ff79ad54 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -92,8 +92,6 @@ public: bool parallelizeOutputAfterReading(ContextPtr context) const override; - SchemaCache & getSchemaCache(const ContextPtr & context); - static SchemaCache & getSchemaCache(const ContextPtr & context, const std::string & storage_type_name); static ColumnsDescription resolveSchemaFromData( @@ -132,7 +130,6 @@ protected: const bool distributed_processing; LoggerPtr log; - std::mutex configuration_update_mutex; }; class StorageObjectStorage::Configuration @@ -175,7 +172,7 @@ public: virtual void check(ContextPtr context) const; virtual void validateNamespace(const String & /* name */) const {} - virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly = true) = 0; /// NOLINT + virtual ObjectStoragePtr createObjectStorage(ContextPtr context, bool is_readonly) = 0; virtual ConfigurationPtr clone() = 0; virtual bool isStaticConfiguration() const { return true; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index b224afb7a58..cb3f732ce83 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -44,19 +44,16 @@ StorageObjectStorageSource::StorageObjectStorageSource( ConfigurationPtr configuration_, const ReadFromFormatInfo & info, std::optional format_settings_, - const StorageObjectStorage::QuerySettings & query_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - bool need_only_count_, - SchemaCache & schema_cache_) + bool need_only_count_) : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) , name(std::move(name_)) , object_storage(object_storage_) , configuration(configuration_) , format_settings(format_settings_) - , query_settings(query_settings_) , max_block_size(max_block_size_) , need_only_count(need_only_count_) , read_from_format_info(info) @@ -67,7 +64,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( 1/* max_threads */)) , columns_desc(info.columns_description) , file_iterator(file_iterator_) - , schema_cache(schema_cache_) + , schema_cache(StorageObjectStorage::getSchemaCache(context_, configuration->getTypeName())) , create_reader_scheduler(threadPoolCallbackRunnerUnsafe(*create_reader_pool, "Reader")) { } @@ -229,6 +226,8 @@ std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const O StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReader(size_t processor) { ObjectInfoPtr object_info; + auto query_settings = configuration->getQuerySettings(getContext()); + do { object_info = file_iterator->next(processor); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 356478422bc..a8df00bc0ac 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -32,12 +32,10 @@ public: ConfigurationPtr configuration, const ReadFromFormatInfo & info, std::optional format_settings_, - const StorageObjectStorage::QuerySettings & query_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, - bool need_only_count_, - SchemaCache & schema_cache_); + bool need_only_count_); ~StorageObjectStorageSource() override; @@ -62,7 +60,6 @@ protected: ObjectStoragePtr object_storage; const ConfigurationPtr configuration; const std::optional format_settings; - const StorageObjectStorage::QuerySettings query_settings; const UInt64 max_block_size; const bool need_only_count; const ReadFromFormatInfo read_from_format_info; diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index c23b180215e..74c8aeaad7d 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include @@ -59,7 +58,7 @@ static std::shared_ptr createStorageObjectStorage( return std::make_shared( configuration, - configuration->createObjectStorage(context), + configuration->createObjectStorage(context, /* is_readonly */false), args.getContext(), args.table_id, args.columns, diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 38934a7895a..b9c67c7d801 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -138,7 +138,7 @@ StorageS3Queue::StorageS3Queue( checkAndAdjustSettings(*s3queue_settings, context_->getSettingsRef()); - object_storage = configuration->createObjectStorage(context_); + object_storage = configuration->createObjectStorage(context_, /* is_readonly */true); FormatFactory::instance().checkFormatName(configuration->format); configuration->check(context_); @@ -361,12 +361,10 @@ std::shared_ptr StorageS3Queue::createSource( configuration, info, format_settings, - configuration->getQuerySettings(local_context), local_context, max_block_size, file_iterator, - false, - StorageObjectStorage::getSchemaCache(local_context, configuration->getTypeName())); + false); auto file_deleter = [=, this](const std::string & path) mutable { From 936f94d286f50133cf12ba449245502769a22e40 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 May 2024 14:40:45 +0200 Subject: [PATCH 046/158] Add print --- utils/keeper-bench/Runner.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 8b111f5adb9..a893dac3851 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -635,11 +635,14 @@ struct ZooKeeperRequestFromLogReader break; } case Coordination::OpNum::Check: + case Coordination::OpNum::CheckNotExists: { auto check_request = std::make_shared(); check_request->path = current_block->getPath(idx_in_block); if (auto version = current_block->getVersion(idx_in_block)) check_request->version = *version; + if (op_num == Coordination::OpNum::CheckNotExists) + check_request->not_exists = true; request_from_log.request = check_request; break; } @@ -868,10 +871,20 @@ void Runner::runBenchmarkFromLog() } ZooKeeperRequestFromLogReader request_reader(input_request_log, global_context); + + delay_watch.restart(); while (auto request_from_log = request_reader.getNextRequest()) { request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); push_request(std::move(*request_from_log)); + + if (delay > 0 && delay_watch.elapsedSeconds() > delay) + { + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + std::cerr << std::endl; + delay_watch.restart(); + } } } From 412805c99e0e789d7bc13dcb73fdf8199758ad2a Mon Sep 17 00:00:00 2001 From: Danila Puzov Date: Thu, 9 May 2024 19:38:19 +0300 Subject: [PATCH 047/158] Add serial, generateSnowflakeID, generateUUIDv7 functions --- src/Functions/generateSnowflakeID.cpp | 92 ++++++++++++++ src/Functions/generateUUIDv7.cpp | 113 +++++++++++++++++ src/Functions/serial.cpp | 171 ++++++++++++++++++++++++++ 3 files changed, 376 insertions(+) create mode 100644 src/Functions/generateSnowflakeID.cpp create mode 100644 src/Functions/generateUUIDv7.cpp create mode 100644 src/Functions/serial.cpp diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp new file mode 100644 index 00000000000..e54b720ec98 --- /dev/null +++ b/src/Functions/generateSnowflakeID.cpp @@ -0,0 +1,92 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +class FunctionSnowflakeID : public IFunction +{ +private: + mutable std::atomic machine_sequence_number{0}; + mutable std::atomic last_timestamp{0}; + +public: + static constexpr auto name = "generateSnowflakeID"; + + static FunctionPtr create(ContextPtr /*context*/) + { + return std::make_shared(); + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 0; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + bool isStateful() const override { return true; } + bool isDeterministic() const override { return false; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() > 1) { + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", + getName(), arguments.size()); + } + + return std::make_shared(); + } + + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + size_t size = input_rows_count; + vec_to.resize(size); + + auto serverUUID = ServerUUID::get(); + + // hash serverUUID into 32 bytes + Int64 h = UUIDHelpers::getHighBytes(serverUUID); + Int64 l = UUIDHelpers::getLowBytes(serverUUID); + Int64 machine_id = (h * 11) ^ (l * 17); + + for (Int64 & x : vec_to) { + const auto tm_point = std::chrono::system_clock::now(); + Int64 current_timestamp = std::chrono::duration_cast( + tm_point.time_since_epoch()).count(); + + Int64 local_machine_sequence_number = 0; + + if (current_timestamp != last_timestamp.load()) { + machine_sequence_number.store(0); + last_timestamp.store(current_timestamp); + } else { + local_machine_sequence_number = machine_sequence_number.fetch_add(1) + 1; + } + + x = (current_timestamp << 22) | (machine_id & 0x3ff000ull) | (local_machine_sequence_number & 0xfffull); + } + + return col_res; + } + +}; + +REGISTER_FUNCTION(GenerateSnowflakeID) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp new file mode 100644 index 00000000000..61d742d2fda --- /dev/null +++ b/src/Functions/generateUUIDv7.cpp @@ -0,0 +1,113 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +#define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \ +DECLARE_DEFAULT_CODE (__VA_ARGS__) \ +DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__) + +DECLARE_SEVERAL_IMPLEMENTATIONS( + +class FunctionGenerateUUIDv7 : public IFunction +{ +public: + static constexpr auto name = "generateUUIDv7"; + + String getName() const override + { + return name; + } + + size_t getNumberOfArguments() const override { return 0; } + + bool isDeterministicInScopeOfQuery() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool isVariadic() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() > 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", + getName(), arguments.size()); + + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + + size_t size = input_rows_count; + vec_to.resize(size); + + /// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces. + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UUID)); + + for (UUID & uuid : vec_to) + { + /// https://www.ietf.org/archive/id/draft-peabody-dispatch-new-uuid-format-04.html#section-5.2 + + const auto tm_point = std::chrono::system_clock::now(); + UInt64 unix_ts_ms = std::chrono::duration_cast( + tm_point.time_since_epoch()).count(); + + UUIDHelpers::getHighBytes(uuid) = (UUIDHelpers::getHighBytes(uuid) & 0x0000000000000fffull) | 0x0000000000007000ull | (unix_ts_ms << 16); + UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & 0x3fffffffffffffffull) | 0x8000000000000000ull; + } + + return col_res; + } +}; + +) // DECLARE_SEVERAL_IMPLEMENTATIONS +#undef DECLARE_SEVERAL_IMPLEMENTATIONS + +class FunctionGenerateUUIDv7 : public TargetSpecific::Default::FunctionGenerateUUIDv7 +{ +public: + explicit FunctionGenerateUUIDv7(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + + #if USE_MULTITARGET_CODE + selector.registerImplementation(); + #endif + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return selector.selectAndExecute(arguments, result_type, input_rows_count); + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(context); + } + +private: + ImplementationSelector selector; +}; + +REGISTER_FUNCTION(GenerateUUIDv7) +{ + factory.registerFunction(); +} + +} + + diff --git a/src/Functions/serial.cpp b/src/Functions/serial.cpp new file mode 100644 index 00000000000..4f336013ca8 --- /dev/null +++ b/src/Functions/serial.cpp @@ -0,0 +1,171 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "Common/Logger.h" +#include + +namespace DB { + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +class FunctionSerial : public IFunction +{ +private: + mutable zkutil::ZooKeeperPtr zk{nullptr}; + ContextPtr context; + +public: + static constexpr auto name = "serial"; + + explicit FunctionSerial(ContextPtr ctx) : context(ctx) + { + if (ctx->hasZooKeeper()) { + zk = ctx->getZooKeeper(); + } + } + + static FunctionPtr create(ContextPtr context) + { + return std::make_shared(std::move(context)); + } + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return 1; } + + bool isStateful() const override { return true; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return false; } + bool isSuitableForConstantFolding() const override { return false; } + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForNothing() const override { return false; } + bool canBeExecutedOnDefaultArguments() const override { return false; } + bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; } + bool hasInformationAboutMonotonicity() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1.", + getName(), arguments.size()); + if (!isStringOrFixedString(arguments[0])) { + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Type of argument for function {} doesn't match: passed {}, should be string", + getName(), arguments[0]->getName()); + } + + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); + size_t size = input_rows_count; + LOG_INFO(getLogger("Serial Function"), "Size = {}", size); + vec_to.resize(size); + + const auto & serial_path = "/serials/" + arguments[0].column->getDataAt(0).toString(); + + // if serial name used first time + zk->createAncestors(serial_path); + zk->createIfNotExists(serial_path, ""); + + Int64 counter; + + if (zk != nullptr) { + // Get Lock in ZooKeeper + // https://zookeeper.apache.org/doc/r3.2.2/recipes.html + + // 1. + if (zk->expired()) { + zk = context->getZooKeeper(); + } + + std::string lock_path = serial_path + "/lock-"; + std::string path_created = zk->create(lock_path, "", zkutil::CreateMode::EphemeralSequential); + Int64 created_sequence_number = std::stoll(path_created.substr(lock_path.size(), path_created.size() - lock_path.size())); + + while (true) { + // 2. + zkutil::Strings children = zk->getChildren(serial_path); + + // 3. + Int64 lowest_child_sequence_number = -1; + for (auto& child : children) { + if (child == "counter") { + continue; + } + std::string child_suffix = child.substr(5, 10); + Int64 seq_number = std::stoll(child_suffix); + + if (lowest_child_sequence_number == -1 || seq_number < lowest_child_sequence_number) { + lowest_child_sequence_number = seq_number; + } + } + + if (lowest_child_sequence_number == created_sequence_number) { + break; + // we have a lock in ZooKeeper, now can get the counter value + } + + // 4. and 5. + Int64 prev_seq_number = created_sequence_number - 1; + std::string to_wait_key = std::to_string(prev_seq_number); + while (to_wait_key.size() != 10) { + to_wait_key = "0" + to_wait_key; + } + + zk->waitForDisappear(lock_path + to_wait_key); + } + + // Now we have a lock + // Update counter in ZooKeeper + std::string counter_path = serial_path + "/counter"; + if (zk->exists(counter_path)) { + std::string counter_string = zk->get(counter_path, nullptr); + counter = std::stoll(counter_string); + + LOG_INFO(getLogger("Serial Function"), "Got counter from Zookeeper = {}", counter); + } else { + counter = 1; + } + zk->createOrUpdate(counter_path, std::to_string(counter + input_rows_count), zkutil::CreateMode::Persistent); + + // Unlock = delete node created on step 1. + zk->deleteEphemeralNodeIfContentMatches(path_created, ""); + } else { + // ZooKeeper is not available + // What to do? + + counter = 1; + } + + // Make a result + for (auto& val : vec_to) { + val = counter; + ++counter; + } + + + return col_res; + } + +}; + +REGISTER_FUNCTION(Serial) +{ + factory.registerFunction(); +} + +} From fa5898a3cd5a9b4276eb75e39c4475dfdf722e3b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 10 May 2024 13:46:56 +0200 Subject: [PATCH 048/158] Refactor data part writer --- src/Storages/MergeTree/IMergeTreeDataPart.h | 21 ++-- .../MergeTree/IMergeTreeDataPartWriter.cpp | 119 +++++++++++++++++- .../MergeTree/IMergeTreeDataPartWriter.h | 57 ++++++++- .../MergeTree/IMergedBlockOutputStream.cpp | 17 ++- .../MergeTree/IMergedBlockOutputStream.h | 15 ++- src/Storages/MergeTree/MergeTask.cpp | 3 +- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 48 ++++--- .../MergeTree/MergeTreeDataPartCompact.h | 17 +-- .../MergeTree/MergeTreeDataPartWide.cpp | 18 ++- .../MergeTree/MergeTreeDataPartWide.h | 17 +-- .../MergeTreeDataPartWriterCompact.cpp | 27 ++-- .../MergeTreeDataPartWriterCompact.h | 9 +- .../MergeTreeDataPartWriterOnDisk.cpp | 32 +++-- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 9 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 69 +++++----- .../MergeTree/MergeTreeDataPartWriterWide.h | 9 +- .../MergeTree/MergeTreeDataWriter.cpp | 4 +- src/Storages/MergeTree/MergeTreePartition.cpp | 13 +- src/Storages/MergeTree/MergeTreePartition.h | 4 +- .../MergeTree/MergedBlockOutputStream.cpp | 29 +++-- .../MergeTree/MergedBlockOutputStream.h | 2 +- .../MergedColumnOnlyOutputStream.cpp | 11 +- src/Storages/MergeTree/MutateTask.cpp | 2 +- 24 files changed, 409 insertions(+), 145 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index ba2ff2ed6fe..4ec5b3f5f8a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -74,7 +74,7 @@ public: using VirtualFields = std::unordered_map; using MergeTreeReaderPtr = std::unique_ptr; - using MergeTreeWriterPtr = std::unique_ptr; +// using MergeTreeWriterPtr = std::unique_ptr; using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; @@ -106,15 +106,16 @@ public: const ValueSizeMap & avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0; - virtual MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) = 0; +//// virtual MergeTreeWriterPtr getWriter( +//// const NamesAndTypesList & columns_list, +//// const StorageMetadataPtr & metadata_snapshot, +//// const std::vector & indices_to_recalc, +//// const Statistics & stats_to_recalc_, +//// const CompressionCodecPtr & default_codec_, +//// const MergeTreeWriterSettings & writer_settings, +//// const MergeTreeIndexGranularity & computed_index_granularity) = 0; +// TODO: remove? virtual bool isStoredOnDisk() const = 0; virtual bool isStoredOnRemoteDisk() const = 0; @@ -168,6 +169,8 @@ public: const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } + const SerializationByName & getSerializations() const { return serializations; } + SerializationPtr getSerialization(const String & column_name) const; SerializationPtr tryGetSerialization(const String & column_name) const; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 2488c63e309..c67e148d011 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,8 +1,15 @@ #include +#include "Storages/MergeTree/MergeTreeSettings.h" namespace DB { +namespace ErrorCodes +{ + extern const int NO_SUCH_COLUMN_IN_TABLE; +} + + Block getBlockAndPermute(const Block & block, const Names & names, const IColumn::Permutation * permutation) { Block result; @@ -38,13 +45,23 @@ Block permuteBlockIfNeeded(const Block & block, const IColumn::Permutation * per } IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( - const MergeTreeMutableDataPartPtr & data_part_, +// const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : data_part(data_part_) - , storage(data_part_->storage) + : data_part_name(data_part_name_) + , serializations(serializations_) + , data_part_storage(data_part_storage_) + , index_granularity_info(index_granularity_info_) + + , storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) , columns_list(columns_list_) , settings(settings_) @@ -60,6 +77,102 @@ Columns IMergeTreeDataPartWriter::releaseIndexColumns() std::make_move_iterator(index_columns.end())); } +SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const +{ + auto it = serializations.find(column_name); + if (it == serializations.end()) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column or subcolumn {} in part {}", column_name, data_part_name); + + return it->second; +} + +ASTPtr IMergeTreeDataPartWriter::getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const +{ + auto get_codec_or_default = [&](const auto & column_desc) + { + return column_desc.codec ? column_desc.codec : default_codec->getFullCodecDesc(); + }; + + const auto & columns = metadata_snapshot->getColumns(); + if (const auto * column_desc = columns.tryGet(column_name)) + return get_codec_or_default(*column_desc); + +///// TODO: is this needed? +// if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) +// return get_codec_or_default(*virtual_desc); +// + return default_codec->getFullCodecDesc(); +} + + IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default; + +MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + + + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( + MergeTreeDataPartType part_type, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity) +{ + if (part_type == MergeTreeDataPartType::Compact) + return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + else if (part_type == MergeTreeDataPartType::Wide) + return createMergeTreeDataPartWideWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); +} + } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 3f359904ddd..ec04fd5f8a8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -7,6 +7,8 @@ #include #include #include +#include "Storages/MergeTree/MergeTreeDataPartType.h" +#include "Storages/MergeTree/MergeTreeSettings.h" namespace DB @@ -22,7 +24,15 @@ class IMergeTreeDataPartWriter : private boost::noncopyable { public: IMergeTreeDataPartWriter( - const MergeTreeMutableDataPartPtr & data_part_, +// const MergeTreeMutableDataPartPtr & data_part_, + + const String & data_part_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_, @@ -39,10 +49,30 @@ public: Columns releaseIndexColumns(); const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } + SerializationPtr getSerialization(const String & column_name) const; + + ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; + + IDataPartStorage & getDataPartStorage() { return *data_part_storage; } + protected: - const MergeTreeMutableDataPartPtr data_part; - const MergeTreeData & storage; +// const MergeTreeMutableDataPartPtr data_part; // TODO: remove + + /// Serializations for every columns and subcolumns by their names. + String data_part_name; + SerializationByName serializations; + MutableDataPartStoragePtr data_part_storage; + MergeTreeIndexGranularityInfo index_granularity_info; + + +// const MergeTreeData & storage; // TODO: remove + + const MergeTreeSettingsPtr storage_settings; + const size_t low_cardinality_max_dictionary_size = 0; // TODO: pass it in ctor + const bool low_cardinality_use_single_dictionary_for_part = true; // TODO: pass it in ctor + + const StorageMetadataPtr metadata_snapshot; const NamesAndTypesList columns_list; const MergeTreeWriterSettings settings; @@ -52,4 +82,25 @@ protected: MutableColumns index_columns; }; +using MergeTreeDataPartWriterPtr = std::unique_ptr; + +MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( + MergeTreeDataPartType part_type, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity); + + } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index c8d6aa0ba65..f99adf7c4db 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -2,25 +2,30 @@ #include #include #include +#include "Storages/MergeTree/IDataPartStorage.h" +#include "Storages/StorageSet.h" namespace DB { IMergedBlockOutputStream::IMergedBlockOutputStream( - const MergeTreeMutableDataPartPtr & data_part, +// const MergeTreeMutableDataPartPtr & data_part, + const MergeTreeSettingsPtr & storage_settings_, + MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list, bool reset_columns_) - : storage(data_part->storage) + //: storage(data_part->storage) + : storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) - , data_part_storage(data_part->getDataPartStoragePtr()) + , data_part_storage(data_part_storage_)//data_part->getDataPartStoragePtr()) , reset_columns(reset_columns_) { if (reset_columns) { SerializationInfo::Settings info_settings = { - .ratio_of_defaults_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = storage_settings->ratio_of_defaults_for_sparse_serialization,//storage.getSettings()->ratio_of_defaults_for_sparse_serialization, .choose_kind = false, }; @@ -42,7 +47,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( return {}; for (const auto & column : empty_columns) - LOG_TRACE(storage.log, "Skipping expired/empty column {} for part {}", column, data_part->name); + LOG_TRACE(data_part->storage.log, "Skipping expired/empty column {} for part {}", column, data_part->name); /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; @@ -91,7 +96,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } else /// If we have no file in checksums it doesn't exist on disk { - LOG_TRACE(storage.log, "Files {} doesn't exist in checksums so it doesn't exist on disk, will not try to remove it", *itr); + LOG_TRACE(data_part->storage.log, "Files {} doesn't exist in checksums so it doesn't exist on disk, will not try to remove it", *itr); itr = remove_files.erase(itr); } } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index ca4e3899b29..b6f279e6d58 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -1,10 +1,12 @@ #pragma once #include "Storages/MergeTree/IDataPartStorage.h" +#include "Storages/MergeTree/MergeTreeSettings.h" #include #include #include #include +#include "Common/Logger.h" namespace DB { @@ -13,7 +15,9 @@ class IMergedBlockOutputStream { public: IMergedBlockOutputStream( - const MergeTreeMutableDataPartPtr & data_part, +// const MergeTreeMutableDataPartPtr & data_part, + const MergeTreeSettingsPtr & storage_settings_, + MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list, bool reset_columns_); @@ -39,11 +43,16 @@ protected: SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums); - const MergeTreeData & storage; +// const MergeTreeData & storage; // TODO: remove +//// + MergeTreeSettingsPtr storage_settings; + LoggerPtr log; +//// + StorageMetadataPtr metadata_snapshot; MutableDataPartStoragePtr data_part_storage; - IMergeTreeDataPart::MergeTreeWriterPtr writer; + MergeTreeDataPartWriterPtr writer; bool reset_columns = false; SerializationInfoByName new_serialization_infos; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 34e17e40a74..1b5ad0d81a7 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include namespace DB @@ -378,7 +379,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), MergeTreeStatisticsFactory::instance().getMany(global_ctx->metadata_snapshot->getColumns()), ctx->compression_codec, - global_ctx->txn, + global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 440c62213a3..8a96e4c9f04 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -8423,7 +8423,7 @@ std::pair MergeTreeData::createE MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), Statistics{}, - compression_codec, txn); + compression_codec, txn ? txn->tid : Tx::PrehistoricTID); bool sync_on_insert = settings->fsync_after_insert; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 418b2d8f81b..eebbe3110c0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -47,27 +47,37 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( avg_value_size_hints, profile_callback, CLOCK_MONOTONIC_COARSE); } -IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) +MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity) { - NamesAndTypesList ordered_columns_list; - std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), - [this](const auto & column) { return getColumnPosition(column.name) != std::nullopt; }); - - /// Order of writing is important in compact format - ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) - { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); - +////// TODO: fix the order of columns +//// +//// NamesAndTypesList ordered_columns_list; +//// std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), +//// [this](const auto & column) { return getColumnPosition(column.name) != std::nullopt; }); +//// +//// /// Order of writing is important in compact format +//// ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) +//// { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); +//// return std::make_unique( - shared_from_this(), ordered_columns_list, metadata_snapshot, - indices_to_recalc, stats_to_recalc_, getMarksFileExtension(), - default_codec_, writer_settings, computed_index_granularity); + data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 3a4e7b95f33..5a57d778b7d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -40,15 +40,16 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; - MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) override; +// MergeTreeWriterPtr getWriter( +// const NamesAndTypesList & columns_list, +// const StorageMetadataPtr & metadata_snapshot, +// const std::vector & indices_to_recalc, +// const Statistics & stats_to_recalc_, +// const CompressionCodecPtr & default_codec_, +// const MergeTreeWriterSettings & writer_settings, +// const MergeTreeIndexGranularity & computed_index_granularity) override; +// TODO: remove? bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index fc3108e522a..c99cff258e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -53,20 +53,26 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( profile_callback); } -IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( +MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, + const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { - return std::make_unique( - shared_from_this(), columns_list, - metadata_snapshot, indices_to_recalc, stats_to_recalc_, - getMarksFileExtension(), - default_codec_, writer_settings, computed_index_granularity); + return std::make_unique(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 84eeec4211b..45d0fbbebec 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -35,15 +35,16 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; - MergeTreeWriterPtr getWriter( - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) override; +// MergeTreeWriterPtr getWriter( +// const NamesAndTypesList & columns_list, +// const StorageMetadataPtr & metadata_snapshot, +// const std::vector & indices_to_recalc, +// const Statistics & stats_to_recalc_, +// const CompressionCodecPtr & default_codec_, +// const MergeTreeWriterSettings & writer_settings, +// const MergeTreeIndexGranularity & computed_index_granularity) override; +// TODO: remove? bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 1605e5cdb9a..6e8ea1a915b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -10,7 +10,14 @@ namespace ErrorCodes } MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( - const MergeTreeMutableDataPartPtr & data_part_, +// const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, @@ -19,23 +26,26 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, + : MergeTreeDataPartWriterOnDisk( + data_part_name_, logger_name_, serializations_, + data_part_storage_, index_granularity_info_, storage_settings_, + columns_list_, metadata_snapshot_, indices_to_recalc_, stats_to_recalc, marks_file_extension_, default_codec_, settings_, index_granularity_) - , plain_file(data_part_->getDataPartStorage().writeFile( + , plain_file(getDataPartStorage().writeFile( MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, settings_.query_write_settings)) , plain_hashing(*plain_file) { - marks_file = data_part_->getDataPartStorage().writeFile( + marks_file = getDataPartStorage().writeFile( MergeTreeDataPartCompact::DATA_FILE_NAME + marks_file_extension_, 4096, settings_.query_write_settings); marks_file_hashing = std::make_unique(*marks_file); - if (data_part_->index_granularity_info.mark_type.compressed) + if (index_granularity_info.mark_type.compressed) { marks_compressor = std::make_unique( *marks_file_hashing, @@ -45,10 +55,9 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( marks_source_hashing = std::make_unique(*marks_compressor); } - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, compression); } } @@ -81,7 +90,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, compressed_streams.emplace(stream_name, stream); }; - data_part->getSerialization(column.name)->enumerateStreams(callback, column.type); + getSerialization(column.name)->enumerateStreams(callback, column.type); } namespace @@ -230,7 +239,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeBinaryLittleEndian(static_cast(0), marks_out); writeColumnSingleGranule( - block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), + block.getByName(name_and_type->name), getSerialization(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index ddb6178dce6..3bec4c7e988 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -11,7 +11,14 @@ class MergeTreeDataPartWriterCompact : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterCompact( - const MergeTreeMutableDataPartPtr & data_part, +// const MergeTreeMutableDataPartPtr & data_part, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 491d2399b82..13892c17577 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -140,7 +140,13 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(Merg MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( - const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeIndices & indices_to_recalc_, @@ -149,7 +155,9 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(data_part_, columns_list_, metadata_snapshot_, settings_, index_granularity_) + : IMergeTreeDataPartWriter( + data_part_name_, serializations_, data_part_storage_, index_granularity_info_, + storage_settings_, columns_list_, metadata_snapshot_, settings_, index_granularity_) , skip_indices(indices_to_recalc_) , stats(stats_to_recalc_) , marks_file_extension(marks_file_extension_) @@ -157,14 +165,14 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , compute_granularity(index_granularity.empty()) , compress_primary_key(settings.compress_primary_key) , execution_stats(skip_indices.size(), stats.size()) - , log(getLogger(storage.getLogName() + " (DataPartWriter)")) + , log(getLogger(logger_name_ + " (DataPartWriter)")) { if (settings.blocks_are_granules_size && !index_granularity.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - if (!data_part->getDataPartStorage().exists()) - data_part->getDataPartStorage().createDirectories(); + if (!getDataPartStorage().exists()) + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); @@ -223,7 +231,7 @@ static size_t computeIndexGranularityImpl( size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const { - const auto storage_settings = storage.getSettings(); +// const auto storage_settings = storage.getSettings(); return computeIndexGranularityImpl( block, storage_settings->index_granularity_bytes, @@ -237,7 +245,7 @@ void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() if (metadata_snapshot->hasPrimaryKey()) { String index_name = "primary" + getIndexExtension(compress_primary_key); - index_file_stream = data_part->getDataPartStorage().writeFile(index_name, DBMS_DEFAULT_BUFFER_SIZE, settings.query_write_settings); + index_file_stream = getDataPartStorage().writeFile(index_name, DBMS_DEFAULT_BUFFER_SIZE, settings.query_write_settings); index_file_hashing_stream = std::make_unique(*index_file_stream); if (compress_primary_key) @@ -256,7 +264,7 @@ void MergeTreeDataPartWriterOnDisk::initStatistics() String stats_name = stat_ptr->getFileName(); stats_streams.emplace_back(std::make_unique>( stats_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stats_name, STAT_FILE_SUFFIX, default_codec, settings.max_compress_block_size, settings.query_write_settings)); @@ -275,7 +283,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() skip_indices_streams.emplace_back( std::make_unique>( stream_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stream_name, skip_index->getSerializedFileExtension(), stream_name, marks_file_extension, default_codec, settings.max_compress_block_size, @@ -285,7 +293,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() GinIndexStorePtr store = nullptr; if (typeid_cast(&*skip_index) != nullptr) { - store = std::make_shared(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment); + store = std::make_shared(stream_name, data_part_storage, data_part_storage, /*storage.getSettings()*/storage_settings->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; } skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings)); @@ -498,7 +506,7 @@ void MergeTreeDataPartWriterOnDisk::finishStatisticsSerialization(bool sync) } for (size_t i = 0; i < stats.size(); ++i) - LOG_DEBUG(log, "Spent {} ms calculating statistics {} for the part {}", execution_stats.statistics_build_us[i] / 1000, stats[i]->columnName(), data_part->name); + LOG_DEBUG(log, "Spent {} ms calculating statistics {} for the part {}", execution_stats.statistics_build_us[i] / 1000, stats[i]->columnName(), data_part_name); } void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums) @@ -524,7 +532,7 @@ void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization(bool sync) store.second->finalize(); for (size_t i = 0; i < skip_indices.size(); ++i) - LOG_DEBUG(log, "Spent {} ms calculating index {} for the part {}", execution_stats.skip_indices_build_us[i] / 1000, skip_indices[i]->index.name, data_part->name); + LOG_DEBUG(log, "Spent {} ms calculating index {} for the part {}", execution_stats.skip_indices_build_us[i] / 1000, skip_indices[i]->index.name, data_part_name); gin_index_stores.clear(); skip_indices_streams.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 9f2cc3970fa..39f33217b57 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -104,7 +104,14 @@ public: using StatisticStreamPtr = std::unique_ptr>; MergeTreeDataPartWriterOnDisk( - const MergeTreeMutableDataPartPtr & data_part_, +// const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 6a3b08d4d65..1f68a9d31a1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -76,7 +76,14 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, } MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( - const MergeTreeMutableDataPartPtr & data_part_, +// const MergeTreeMutableDataPartPtr & data_part_, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, @@ -85,14 +92,16 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, - indices_to_recalc_, stats_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + : MergeTreeDataPartWriterOnDisk( + data_part_name_, logger_name_, serializations_, + data_part_storage_, index_granularity_info_, storage_settings_, + columns_list_, metadata_snapshot_, + indices_to_recalc_, stats_to_recalc_, marks_file_extension_, + default_codec_, settings_, index_granularity_) { - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, compression); } } @@ -105,7 +114,7 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); - auto storage_settings = storage.getSettings(); +// auto storage_settings = storage.getSettings(); auto full_stream_name = ISerialization::getFileNameForStream(column, substream_path); String stream_name; @@ -149,7 +158,7 @@ void MergeTreeDataPartWriterWide::addStreams( column_streams[stream_name] = std::make_unique>( stream_name, - data_part->getDataPartStoragePtr(), + data_part_storage, stream_name, DATA_FILE_EXTENSION, stream_name, marks_file_extension, compression_codec, @@ -163,7 +172,7 @@ void MergeTreeDataPartWriterWide::addStreams( }; ISerialization::SubstreamPath path; - data_part->getSerialization(column.name)->enumerateStreams(callback, column.type); + getSerialization(column.name)->enumerateStreams(callback, column.type); } const String & MergeTreeDataPartWriterWide::getStreamName( @@ -264,7 +273,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm { auto & column = block_to_write.getByName(it->name); - if (data_part->getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE) + if (getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE) column.column = recursiveRemoveSparse(column.column); if (permutation) @@ -334,7 +343,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( min_compress_block_size = value->safeGet(); if (!min_compress_block_size) min_compress_block_size = settings.min_compress_block_size; - data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; auto stream_name = getStreamName(column, substream_path); @@ -368,7 +377,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = data_part->getSerialization(name_and_type.name); + const auto & serialization = getSerialization(name_and_type.name); serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. @@ -398,7 +407,7 @@ void MergeTreeDataPartWriterWide::writeColumn( const auto & [name, type] = name_and_type; auto [it, inserted] = serialization_states.emplace(name, nullptr); - auto serialization = data_part->getSerialization(name_and_type.name); + auto serialization = getSerialization(name_and_type.name); if (inserted) { @@ -407,11 +416,11 @@ void MergeTreeDataPartWriterWide::writeColumn( serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } - const auto & global_settings = storage.getContext()->getSettingsRef(); +// const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; for (const auto & granule : granules) { @@ -460,7 +469,7 @@ void MergeTreeDataPartWriterWide::writeColumn( void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePair & name_type) { const auto & [name, type] = name_type; - const auto & serialization = data_part->getSerialization(name_type.name); + const auto & serialization = getSerialization(name_type.name); if (!type->isValueRepresentedByNumber() || type->haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type->getName()); @@ -470,21 +479,21 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai String bin_path = escaped_name + DATA_FILE_EXTENSION; /// Some columns may be removed because of ttl. Skip them. - if (!data_part->getDataPartStorage().exists(mrk_path)) + if (!getDataPartStorage().exists(mrk_path)) return; - auto mrk_file_in = data_part->getDataPartStorage().readFile(mrk_path, {}, std::nullopt, std::nullopt); + auto mrk_file_in = getDataPartStorage().readFile(mrk_path, {}, std::nullopt, std::nullopt); std::unique_ptr mrk_in; - if (data_part->index_granularity_info.mark_type.compressed) + if (index_granularity_info.mark_type.compressed) mrk_in = std::make_unique(std::move(mrk_file_in)); else mrk_in = std::move(mrk_file_in); - DB::CompressedReadBufferFromFile bin_in(data_part->getDataPartStorage().readFile(bin_path, {}, std::nullopt, std::nullopt)); + DB::CompressedReadBufferFromFile bin_in(getDataPartStorage().readFile(bin_path, {}, std::nullopt, std::nullopt)); bool must_be_last = false; UInt64 offset_in_compressed_file = 0; UInt64 offset_in_decompressed_block = 0; - UInt64 index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; + UInt64 index_granularity_rows = index_granularity_info.fixed_index_granularity; size_t mark_num; @@ -500,7 +509,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai if (settings.can_use_adaptive_granularity) readBinaryLittleEndian(index_granularity_rows, *mrk_in); else - index_granularity_rows = data_part->index_granularity_info.fixed_index_granularity; + index_granularity_rows = index_granularity_info.fixed_index_granularity; if (must_be_last) { @@ -533,7 +542,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{}" " (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", - data_part->getDataPartStorage().getFullPath(), + getDataPartStorage().getFullPath(), mark_num, offset_in_compressed_file, offset_in_decompressed_block, index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity.getMarksCount()); @@ -596,10 +605,10 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) { - const auto & global_settings = storage.getContext()->getSettingsRef(); +// const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { @@ -622,7 +631,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum if (!serialization_states.empty()) { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); - data_part->getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); + getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) @@ -665,7 +674,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && data_part->getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { validateColumnOfFixedSize(column); } @@ -708,7 +717,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(column, offset_columns, 0); /// Memoize information about offsets - data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index f5ff323563d..ef9c4ab17dc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -21,7 +21,14 @@ class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterWide( - const MergeTreeMutableDataPartPtr & data_part, +// const MergeTreeMutableDataPartPtr & data_part, + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index daa163d741c..0f05c171230 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -600,7 +600,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( indices, MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, - context->getCurrentTransaction(), + context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, false, false, context->getWriteSettings()); @@ -738,7 +738,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( MergeTreeIndices{}, Statistics{}, /// TODO(hanfei): It should be helpful to write statistics for projection result. compression_codec, - NO_TRANSACTION_PTR, + Tx::PrehistoricTID, false, false, data.getContext()->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index ddeaf69136a..c2ef7f98388 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "Interpreters/Context_fwd.h" #include #include @@ -413,12 +414,14 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr MergeTreePartition::store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const +std::unique_ptr MergeTreePartition::store(/*const MergeTreeData & storage,*/ + StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, + IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { - auto metadata_snapshot = storage.getInMemoryMetadataPtr(); - const auto & context = storage.getContext(); - const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage.getContext()).sample_block; - return store(partition_key_sample, data_part_storage, checksums, context->getWriteSettings()); +// auto metadata_snapshot = storage.getInMemoryMetadataPtr(); +// const auto & context = storage.getContext(); + const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage_context).sample_block; + return store(partition_key_sample, data_part_storage, checksums, storage_context->getWriteSettings()); } std::unique_ptr MergeTreePartition::store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 78b141f26ec..04175d6f927 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -44,7 +44,9 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr store(const MergeTreeData & storage, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; + [[nodiscard]] std::unique_ptr store(//const MergeTreeData & storage, + StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, + IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; void assign(const MergeTreePartition & other) { value = other.value; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 9f641fd8eb5..2441d941952 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -21,35 +21,40 @@ MergedBlockOutputStream::MergedBlockOutputStream( const MergeTreeIndices & skip_indices, const Statistics & statistics, CompressionCodecPtr default_codec_, - const MergeTreeTransactionPtr & txn, + TransactionID tid, bool reset_columns_, bool blocks_are_granules_size, const WriteSettings & write_settings_, const MergeTreeIndexGranularity & computed_index_granularity) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, columns_list_, reset_columns_) + : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, reset_columns_) , columns_list(columns_list_) , default_codec(default_codec_) , write_settings(write_settings_) { MergeTreeWriterSettings writer_settings( - storage.getContext()->getSettings(), + data_part->storage.getContext()->getSettings(), write_settings, - storage.getSettings(), + storage_settings, data_part->index_granularity_info.mark_type.adaptive, /* rewrite_primary_key = */ true, blocks_are_granules_size); +// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart if (data_part->isStoredOnDisk()) data_part_storage->createDirectories(); - /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. - TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; +// /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. +// TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; /// NOTE do not pass context for writing to system.transactions_info_log, /// because part may have temporary name (with temporary block numbers). Will write it later. data_part->version.setCreationTID(tid, nullptr); data_part->storeVersionMetadata(); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, statistics, default_codec, writer_settings, computed_index_granularity); + writer = createMergeTreeDataPartWriter(data_part->getType(), + data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), + data_part_storage, data_part->index_granularity_info, + storage_settings, + columns_list, metadata_snapshot, skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); } /// If data is pre-sorted. @@ -208,7 +213,7 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (new_part->isProjectionPart()) { - if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) + if (new_part->storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { auto count_out = new_part->getDataPartStorage().writeFile("count.txt", 4096, write_settings); HashingWriteBuffer count_out_hashing(*count_out); @@ -234,14 +239,16 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis written_files.emplace_back(std::move(out)); } - if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) + if (new_part->storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - if (auto file = new_part->partition.store(storage, new_part->getDataPartStorage(), checksums)) + if (auto file = new_part->partition.store(//storage, + new_part->storage.getInMemoryMetadataPtr(), new_part->storage.getContext(), + new_part->getDataPartStorage(), checksums)) written_files.emplace_back(std::move(file)); if (new_part->minmax_idx->initialized) { - auto files = new_part->minmax_idx->store(storage, new_part->getDataPartStorage(), checksums); + auto files = new_part->minmax_idx->store(new_part->storage, new_part->getDataPartStorage(), checksums); for (auto & file : files) written_files.emplace_back(std::move(file)); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 540b3b3bffa..c1e3d75fefc 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -22,7 +22,7 @@ public: const MergeTreeIndices & skip_indices, const Statistics & statistics, CompressionCodecPtr default_codec_, - const MergeTreeTransactionPtr & txn, + TransactionID tid, bool reset_columns_ = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}, diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 728b2e38833..51853384012 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -20,11 +20,11 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) + : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) , header(header_) { const auto & global_settings = data_part->storage.getContext()->getSettings(); - const auto & storage_settings = data_part->storage.getSettings(); +// const auto & storage_settings = data_part->storage.getSettings(); MergeTreeWriterSettings writer_settings( global_settings, @@ -33,11 +33,16 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), /* rewrite_primary_key = */ false); - writer = data_part->getWriter( + writer = createMergeTreeDataPartWriter( + data_part->getType(), + data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), + data_part_storage, data_part->index_granularity_info, + storage_settings, header.getNamesAndTypesList(), metadata_snapshot_, indices_to_recalc, stats_to_recalc_, + data_part->getMarksFileExtension(), default_codec, writer_settings, index_granularity); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 55d845dfbb9..54077055d96 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1660,7 +1660,7 @@ private: skip_indices, stats_to_rewrite, ctx->compression_codec, - ctx->txn, + ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings(), From 32b8aba8ef1bf9a0b890065a5d719a002cee8bb5 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 10 May 2024 14:12:34 +0200 Subject: [PATCH 049/158] Style --- src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index c67e148d011..b46fbc5fc9e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -6,6 +6,7 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int NO_SUCH_COLUMN_IN_TABLE; } @@ -144,7 +145,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeIndexGranularity & computed_index_granularity); - MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( MergeTreeDataPartType part_type, const String & data_part_name_, From b20d60858f1286a5e406e2c74036e6ad244fda2b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 10 May 2024 15:48:32 +0200 Subject: [PATCH 050/158] Pass low cardinality settings --- src/Storages/MergeTree/IMergeTreeDataPartWriter.h | 2 -- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeIOSettings.h | 5 +++++ 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index ec04fd5f8a8..52e21bed2f2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -69,8 +69,6 @@ protected: // const MergeTreeData & storage; // TODO: remove const MergeTreeSettingsPtr storage_settings; - const size_t low_cardinality_max_dictionary_size = 0; // TODO: pass it in ctor - const bool low_cardinality_use_single_dictionary_for_part = true; // TODO: pass it in ctor const StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 1f68a9d31a1..713dee87fa8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -419,8 +419,8 @@ void MergeTreeDataPartWriterWide::writeColumn( // const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serialize_settings.low_cardinality_max_dictionary_size = low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; for (const auto & granule : granules) { @@ -607,8 +607,8 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum { // const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.low_cardinality_max_dictionary_size = low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 12a83703148..421c62887da 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -74,6 +74,8 @@ struct MergeTreeWriterSettings , blocks_are_granules_size(blocks_are_granules_size_) , query_write_settings(query_write_settings_) , max_threads_for_annoy_index_creation(global_settings.max_threads_for_annoy_index_creation) + , low_cardinality_max_dictionary_size(global_settings.low_cardinality_max_dictionary_size) + , low_cardinality_use_single_dictionary_for_part(global_settings.low_cardinality_use_single_dictionary_for_part) { } @@ -93,6 +95,9 @@ struct MergeTreeWriterSettings WriteSettings query_write_settings; size_t max_threads_for_annoy_index_creation; + + size_t low_cardinality_max_dictionary_size; + bool low_cardinality_use_single_dictionary_for_part; }; } From a3aff6939c0b3afeeb9e4ab9c6f2992a2c61b543 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 10 May 2024 19:21:16 +0200 Subject: [PATCH 051/158] Protected methods --- src/Storages/MergeTree/IMergeTreeDataPartWriter.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 52e21bed2f2..6854668a01e 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -49,13 +49,13 @@ public: Columns releaseIndexColumns(); const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } +protected: SerializationPtr getSerialization(const String & column_name) const; ASTPtr getCodecDescOrDefault(const String & column_name, CompressionCodecPtr default_codec) const; IDataPartStorage & getDataPartStorage() { return *data_part_storage; } -protected: // const MergeTreeMutableDataPartPtr data_part; // TODO: remove From f68b788f5900b66ab4623874c98ed1b4025b5fd0 Mon Sep 17 00:00:00 2001 From: Danila Puzov Date: Sat, 11 May 2024 15:34:13 +0300 Subject: [PATCH 052/158] Tests and docs for serial, some fixes for generateSnowflakeID --- src/Functions/generateSnowflakeID.cpp | 62 +++- src/Functions/generateUUIDv7.cpp | 284 ++++++++++++++---- src/Functions/serial.cpp | 134 ++++----- .../03129_serial_test_zookeeper.reference | 8 + .../03129_serial_test_zookeeper.sql | 20 ++ 5 files changed, 373 insertions(+), 135 deletions(-) create mode 100644 tests/queries/0_stateless/03129_serial_test_zookeeper.reference create mode 100644 tests/queries/0_stateless/03129_serial_test_zookeeper.sql diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index e54b720ec98..dd837a58325 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -11,11 +11,42 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } +namespace +{ + +/* + Snowflake ID + https://en.wikipedia.org/wiki/Snowflake_ID + + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|0| timestamp | +├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| | machine_id | machine_seq_num | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ + +- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds +- The middle 10 bits are the machine ID. +- The last 12 bits decode to number of ids processed by the machine at the given millisecond. +*/ + +constexpr auto timestamp_size = 41; +constexpr auto machine_id_size = 10; +constexpr auto machine_seq_num_size = 12; + +constexpr int64_t timestamp_mask = ((1LL << timestamp_size) - 1) << (machine_id_size + machine_seq_num_size); +constexpr int64_t machine_id_mask = ((1LL << machine_id_size) - 1) << machine_seq_num_size; +constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_size) - 1; + +} + class FunctionSnowflakeID : public IFunction { private: - mutable std::atomic machine_sequence_number{0}; - mutable std::atomic last_timestamp{0}; + mutable std::atomic state{0}; + // previous snowflake id + // state is 1 atomic value because we don't want use mutex public: static constexpr auto name = "generateSnowflakeID"; @@ -60,23 +91,28 @@ public: // hash serverUUID into 32 bytes Int64 h = UUIDHelpers::getHighBytes(serverUUID); Int64 l = UUIDHelpers::getLowBytes(serverUUID); - Int64 machine_id = (h * 11) ^ (l * 17); + Int64 machine_id = ((h * 11) ^ (l * 17)) & machine_id_mask; - for (Int64 & x : vec_to) { + for (Int64 & el : vec_to) { const auto tm_point = std::chrono::system_clock::now(); Int64 current_timestamp = std::chrono::duration_cast( - tm_point.time_since_epoch()).count(); + tm_point.time_since_epoch()).count() & ((1LL << timestamp_size) - 1); - Int64 local_machine_sequence_number = 0; + Int64 last_state, new_state; + do { + last_state = state.load(); + Int64 last_timestamp = (last_state & timestamp_mask) >> (machine_id_size + machine_seq_num_size); + Int64 machine_seq_num = last_state & machine_seq_num_mask; - if (current_timestamp != last_timestamp.load()) { - machine_sequence_number.store(0); - last_timestamp.store(current_timestamp); - } else { - local_machine_sequence_number = machine_sequence_number.fetch_add(1) + 1; - } + if (current_timestamp == last_timestamp) { + ++machine_seq_num; + } + new_state = (current_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | machine_seq_num; + } while (!state.compare_exchange_strong(last_state, new_state)); + // failed CAS => another thread updated state + // successful CAS => we have unique (timestamp, machine_seq_num) on this machine - x = (current_timestamp << 22) | (machine_id & 0x3ff000ull) | (local_machine_sequence_number & 0xfffull); + el = new_state; } return col_res; diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 61d742d2fda..411a3a076ac 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -1,13 +1,178 @@ -#include -#include #include +#include +#include +#include namespace DB { -namespace ErrorCodes +namespace { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + +/* Bit layouts of UUIDv7 + +without counter: + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | rand_a | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| rand_b | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ + +with counter: + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| unix_ts_ms | ver | counter_high_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|var| counter_low_bits | +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| rand_b | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +*/ + +/// bit counts +constexpr auto rand_a_bits_count = 12; +constexpr auto rand_b_bits_count = 62; +constexpr auto rand_b_low_bits_count = 32; +constexpr auto counter_high_bits_count = rand_a_bits_count; +constexpr auto counter_low_bits_count = 30; +constexpr auto bits_in_counter = counter_high_bits_count + counter_low_bits_count; +constexpr uint64_t counter_limit = (1ull << bits_in_counter); + +/// bit masks for UUIDv7 components +constexpr uint64_t variant_2_mask = (2ull << rand_b_bits_count); +constexpr uint64_t rand_a_bits_mask = (1ull << rand_a_bits_count) - 1; +constexpr uint64_t rand_b_bits_mask = (1ull << rand_b_bits_count) - 1; +constexpr uint64_t rand_b_with_counter_bits_mask = (1ull << rand_b_low_bits_count) - 1; +constexpr uint64_t counter_low_bits_mask = (1ull << counter_low_bits_count) - 1; +constexpr uint64_t counter_high_bits_mask = rand_a_bits_mask; + +uint64_t getTimestampMillisecond() +{ + timespec tp; + clock_gettime(CLOCK_REALTIME, &tp); + const uint64_t sec = tp.tv_sec; + return sec * 1000 + tp.tv_nsec / 1000000; +} + +void setTimestampAndVersion(UUID & uuid, uint64_t timestamp) +{ + UUIDHelpers::getHighBytes(uuid) = (UUIDHelpers::getHighBytes(uuid) & rand_a_bits_mask) | (timestamp << 16) | 0x7000; +} + +void setVariant(UUID & uuid) +{ + UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & rand_b_bits_mask) | variant_2_mask; +} + +struct FillAllRandomPolicy +{ + static constexpr auto name = "generateUUIDv7NonMonotonic"; + static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), and a random field (74 bit, including a 2-bit variant field "2") to distinguish UUIDs within a millisecond. This function is the fastest generateUUIDv7* function but it gives no monotonicity guarantees within a timestamp.)"; + struct Data + { + void generate(UUID & uuid, uint64_t ts) + { + setTimestampAndVersion(uuid, ts); + setVariant(uuid); + } + }; +}; + +struct CounterFields +{ + uint64_t last_timestamp = 0; + uint64_t counter = 0; + + void resetCounter(const UUID & uuid) + { + const uint64_t counter_low_bits = (UUIDHelpers::getLowBytes(uuid) >> rand_b_low_bits_count) & counter_low_bits_mask; + const uint64_t counter_high_bits = UUIDHelpers::getHighBytes(uuid) & counter_high_bits_mask; + counter = (counter_high_bits << 30) | counter_low_bits; + } + + void incrementCounter(UUID & uuid) + { + if (++counter == counter_limit) [[unlikely]] + { + ++last_timestamp; + resetCounter(uuid); + setTimestampAndVersion(uuid, last_timestamp); + setVariant(uuid); + } + else + { + UUIDHelpers::getHighBytes(uuid) = (last_timestamp << 16) | 0x7000 | (counter >> counter_low_bits_count); + UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & rand_b_with_counter_bits_mask) | variant_2_mask | ((counter & counter_low_bits_mask) << rand_b_low_bits_count); + } + } + + void generate(UUID & uuid, uint64_t timestamp) + { + const bool need_to_increment_counter = (last_timestamp == timestamp) || ((last_timestamp > timestamp) & (last_timestamp < timestamp + 10000)); + if (need_to_increment_counter) + { + incrementCounter(uuid); + } + else + { + last_timestamp = timestamp; + resetCounter(uuid); + setTimestampAndVersion(uuid, last_timestamp); + setVariant(uuid); + } + } +}; + + +struct GlobalCounterPolicy +{ + static constexpr auto name = "generateUUIDv7"; + static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. Function generateUUIDv7 guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + + /// Guarantee counter monotonicity within one timestamp across all threads generating UUIDv7 simultaneously. + struct Data + { + static inline CounterFields fields; + static inline SharedMutex mutex; /// works a little bit faster than std::mutex here + std::lock_guard guard; + + Data() + : guard(mutex) + {} + + void generate(UUID & uuid, uint64_t timestamp) + { + fields.generate(uuid, timestamp); + } + }; +}; + +struct ThreadLocalCounterPolicy +{ + static constexpr auto name = "generateUUIDv7ThreadMonotonic"; + static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. This function behaves like generateUUIDv7 but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate UUIDs.)"; + + /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. + struct Data + { + static inline thread_local CounterFields fields; + + void generate(UUID & uuid, uint64_t timestamp) + { + fields.generate(uuid, timestamp); + } + }; +}; + } #define DECLARE_SEVERAL_IMPLEMENTATIONS(...) \ @@ -16,77 +181,72 @@ DECLARE_AVX2_SPECIFIC_CODE(__VA_ARGS__) DECLARE_SEVERAL_IMPLEMENTATIONS( -class FunctionGenerateUUIDv7 : public IFunction +template +class FunctionGenerateUUIDv7Base : public IFunction, public FillPolicy { public: - static constexpr auto name = "generateUUIDv7"; + String getName() const final { return FillPolicy::name; } - String getName() const override + size_t getNumberOfArguments() const final { return 0; } + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const final { return false; } + bool useDefaultImplementationForNulls() const final { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const final { return false; } + bool isVariadic() const final { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - return name; - } - - size_t getNumberOfArguments() const override { return 0; } - - bool isDeterministicInScopeOfQuery() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - bool isVariadic() const override { return true; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", - getName(), arguments.size()); + FunctionArgumentDescriptors mandatory_args; + FunctionArgumentDescriptors optional_args{ + {"expr", nullptr, nullptr, "Arbitrary Expression"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); return std::make_shared(); } - bool isDeterministic() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - size_t size = input_rows_count; - vec_to.resize(size); - - /// RandImpl is target-dependent and is not the same in different TargetSpecific namespaces. - RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UUID)); - - for (UUID & uuid : vec_to) + if (input_rows_count) { - /// https://www.ietf.org/archive/id/draft-peabody-dispatch-new-uuid-format-04.html#section-5.2 + vec_to.resize(input_rows_count); - const auto tm_point = std::chrono::system_clock::now(); - UInt64 unix_ts_ms = std::chrono::duration_cast( - tm_point.time_since_epoch()).count(); + /// Not all random bytes produced here are required for the UUIDv7 but it's the simplest way to get the required number of them by using RandImpl + RandImpl::execute(reinterpret_cast(vec_to.data()), vec_to.size() * sizeof(UUID)); - UUIDHelpers::getHighBytes(uuid) = (UUIDHelpers::getHighBytes(uuid) & 0x0000000000000fffull) | 0x0000000000007000ull | (unix_ts_ms << 16); - UUIDHelpers::getLowBytes(uuid) = (UUIDHelpers::getLowBytes(uuid) & 0x3fffffffffffffffull) | 0x8000000000000000ull; + /// Note: For performance reasons, clock_gettime is called once per chunk instead of once per UUID. This reduces precision but + /// it still complies with the UUID standard. + uint64_t timestamp = getTimestampMillisecond(); + for (UUID & uuid : vec_to) + { + typename FillPolicy::Data data; + data.generate(uuid, timestamp); + } } - return col_res; } }; - ) // DECLARE_SEVERAL_IMPLEMENTATIONS #undef DECLARE_SEVERAL_IMPLEMENTATIONS -class FunctionGenerateUUIDv7 : public TargetSpecific::Default::FunctionGenerateUUIDv7 +template +class FunctionGenerateUUIDv7Base : public TargetSpecific::Default::FunctionGenerateUUIDv7Base { public: - explicit FunctionGenerateUUIDv7(ContextPtr context) : selector(context) - { - selector.registerImplementation(); + using Self = FunctionGenerateUUIDv7Base; + using Parent = TargetSpecific::Default::FunctionGenerateUUIDv7Base; - #if USE_MULTITARGET_CODE - selector.registerImplementation(); - #endif + explicit FunctionGenerateUUIDv7Base(ContextPtr context) : selector(context) + { + selector.registerImplementation(); + +#if USE_MULTITARGET_CODE + using ParentAVX2 = TargetSpecific::AVX2::FunctionGenerateUUIDv7Base; + selector.registerImplementation(); +#endif } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -96,18 +256,34 @@ public: static FunctionPtr create(ContextPtr context) { - return std::make_shared(context); + return std::make_shared(context); } private: ImplementationSelector selector; }; +template +void registerUUIDv7Generator(auto& factory) +{ + static constexpr auto doc_syntax_format = "{}([expression])"; + static constexpr auto example_format = "SELECT {}()"; + static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; + + FunctionDocumentation::Description doc_description = FillPolicy::doc_description; + FunctionDocumentation::Syntax doc_syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments doc_arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue doc_returned_value = "A value of type UUID version 7."; + FunctionDocumentation::Examples doc_examples = {{"uuid", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories doc_categories = {"UUID"}; + + factory.template registerFunction>({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); +} + REGISTER_FUNCTION(GenerateUUIDv7) { - factory.registerFunction(); + registerUUIDv7Generator(factory); + registerUUIDv7Generator(factory); + registerUUIDv7Generator(factory); } - } - - diff --git a/src/Functions/serial.cpp b/src/Functions/serial.cpp index 4f336013ca8..1745e17b5e7 100644 --- a/src/Functions/serial.cpp +++ b/src/Functions/serial.cpp @@ -7,6 +7,9 @@ #include #include #include "Common/Logger.h" +#include "Common/ZooKeeper/IKeeper.h" +#include "Common/ZooKeeper/KeeperException.h" +#include "Common/ZooKeeper/Types.h" #include namespace DB { @@ -15,6 +18,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int KEEPER_EXCEPTION; } class FunctionSerial : public IFunction @@ -69,6 +73,15 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + if (zk == nullptr) { + throw Exception(ErrorCodes::KEEPER_EXCEPTION, + "ZooKeeper is not configured for function {}", + getName()); + } + if (zk->expired()) { + zk = context->getZooKeeper(); + } + auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); size_t size = input_rows_count; @@ -77,78 +90,32 @@ public: const auto & serial_path = "/serials/" + arguments[0].column->getDataAt(0).toString(); - // if serial name used first time - zk->createAncestors(serial_path); - zk->createIfNotExists(serial_path, ""); + // CAS in ZooKeeper + // `get` value and version, `trySet` new with version check + // I didn't get how to do it with `multi` Int64 counter; + std::string counter_path = serial_path + "/counter"; - if (zk != nullptr) { - // Get Lock in ZooKeeper - // https://zookeeper.apache.org/doc/r3.2.2/recipes.html + // if serial name used first time + zk->createAncestors(counter_path); + zk->createIfNotExists(counter_path, "1"); - // 1. - if (zk->expired()) { - zk = context->getZooKeeper(); + Coordination::Stat stat; + while (true) { + std::string counter_string = zk->get(counter_path, &stat); + counter = std::stoll(counter_string); + std::string updated_counter = std::to_string(counter + input_rows_count); + Coordination::Error err = zk->trySet(counter_path, updated_counter); + if (err == Coordination::Error::ZOK) { + // CAS is done + break; } - - std::string lock_path = serial_path + "/lock-"; - std::string path_created = zk->create(lock_path, "", zkutil::CreateMode::EphemeralSequential); - Int64 created_sequence_number = std::stoll(path_created.substr(lock_path.size(), path_created.size() - lock_path.size())); - - while (true) { - // 2. - zkutil::Strings children = zk->getChildren(serial_path); - - // 3. - Int64 lowest_child_sequence_number = -1; - for (auto& child : children) { - if (child == "counter") { - continue; - } - std::string child_suffix = child.substr(5, 10); - Int64 seq_number = std::stoll(child_suffix); - - if (lowest_child_sequence_number == -1 || seq_number < lowest_child_sequence_number) { - lowest_child_sequence_number = seq_number; - } - } - - if (lowest_child_sequence_number == created_sequence_number) { - break; - // we have a lock in ZooKeeper, now can get the counter value - } - - // 4. and 5. - Int64 prev_seq_number = created_sequence_number - 1; - std::string to_wait_key = std::to_string(prev_seq_number); - while (to_wait_key.size() != 10) { - to_wait_key = "0" + to_wait_key; - } - - zk->waitForDisappear(lock_path + to_wait_key); + if (err != Coordination::Error::ZBADVERSION) { + throw Exception(ErrorCodes::KEEPER_EXCEPTION, + "ZooKeeper trySet operation failed with unexpected error = {} in function {}", + err, getName()); } - - // Now we have a lock - // Update counter in ZooKeeper - std::string counter_path = serial_path + "/counter"; - if (zk->exists(counter_path)) { - std::string counter_string = zk->get(counter_path, nullptr); - counter = std::stoll(counter_string); - - LOG_INFO(getLogger("Serial Function"), "Got counter from Zookeeper = {}", counter); - } else { - counter = 1; - } - zk->createOrUpdate(counter_path, std::to_string(counter + input_rows_count), zkutil::CreateMode::Persistent); - - // Unlock = delete node created on step 1. - zk->deleteEphemeralNodeIfContentMatches(path_created, ""); - } else { - // ZooKeeper is not available - // What to do? - - counter = 1; } // Make a result @@ -157,7 +124,6 @@ public: ++counter; } - return col_res; } @@ -165,7 +131,39 @@ public: REGISTER_FUNCTION(Serial) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation + { + .description=R"( +Generates and returns sequential numbers starting from the previous counter value. +This function takes a constant string argument - a series identifier. +The server should be configured with a ZooKeeper. +)", + .syntax = "serial(identifier)", + .arguments{ + {"series identifier", "Series identifier (String)"} + }, + .returned_value = "Sequential numbers of type Int64 starting from the previous counter value", + .examples{ + {"first call", "SELECT serial('name')", R"( +┌─serial('name')─┐ +│ 1 │ +└────────────────┘)"}, + {"second call", "SELECT serial('name')", R"( +┌─serial('name')─┐ +│ 2 │ +└────────────────┘)"}, + {"column call", "SELECT *, serial('name') FROM test_table", R"( +┌─CounterID─┬─UserID─┬─ver─┬─serial('name')─┐ +│ 1 │ 3 │ 3 │ 3 │ +│ 1 │ 1 │ 1 │ 4 │ +│ 1 │ 2 │ 2 │ 5 │ +│ 1 │ 5 │ 5 │ 6 │ +│ 1 │ 4 │ 4 │ 7 │ +└───────────┴────────┴─────┴────────────────┘ + )"}}, + .categories{"Unique identifiers"} + }); + } } diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.reference b/tests/queries/0_stateless/03129_serial_test_zookeeper.reference new file mode 100644 index 00000000000..60714f4064f --- /dev/null +++ b/tests/queries/0_stateless/03129_serial_test_zookeeper.reference @@ -0,0 +1,8 @@ +1 +2 +1 3 3 3 +1 1 1 4 +1 2 2 5 +1 5 5 6 +1 4 4 7 +1 diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql new file mode 100644 index 00000000000..3eacd1ae908 --- /dev/null +++ b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql @@ -0,0 +1,20 @@ +SELECT serial('x'); +SELECT serial('x'); + +DROP TABLE IF EXISTS default.test_table; + +CREATE TABLE test_table +( + CounterID UInt32, + UserID UInt32, + ver UInt16 +) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/1-1/test_table', 'x', ver) +PARTITION BY CounterID +ORDER BY (CounterID, intHash32(UserID)) +SAMPLE BY intHash32(UserID); + +INSERT INTO test_table VALUES (1, 1, 1), (1, 2, 2), (1, 3, 3), (1, 4, 4), (1, 5, 5); + +SELECT *, serial('x') FROM test_table; + +SELECT serial('y'); \ No newline at end of file From 9789d130a6cad5da2941037d91c69d9d63aa2733 Mon Sep 17 00:00:00 2001 From: Danila Puzov Date: Mon, 13 May 2024 01:11:23 +0300 Subject: [PATCH 053/158] Tests and docs for generateSnowflakeID and fixes --- src/Functions/generateSnowflakeID.cpp | 144 +++++++++++++----- src/Functions/serial.cpp | 36 ++--- .../03129_serial_test_zookeeper.reference | 15 +- .../03129_serial_test_zookeeper.sql | 24 +-- .../03130_generate_snowflake_id.reference | 3 + .../03130_generate_snowflake_id.sql | 11 ++ 6 files changed, 154 insertions(+), 79 deletions(-) create mode 100644 tests/queries/0_stateless/03130_generate_snowflake_id.reference create mode 100644 tests/queries/0_stateless/03130_generate_snowflake_id.sql diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index dd837a58325..1decda0ab46 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -1,7 +1,11 @@ -#include #include +#include #include +#include #include +#include +#include + namespace DB { @@ -38,15 +42,32 @@ constexpr auto machine_seq_num_size = 12; constexpr int64_t timestamp_mask = ((1LL << timestamp_size) - 1) << (machine_id_size + machine_seq_num_size); constexpr int64_t machine_id_mask = ((1LL << machine_id_size) - 1) << machine_seq_num_size; constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_size) - 1; +constexpr int64_t max_machine_seq_num = machine_seq_num_mask; + +Int64 getMachineID() +{ + auto serverUUID = ServerUUID::get(); + + // hash serverUUID into 64 bits + Int64 h = UUIDHelpers::getHighBytes(serverUUID); + Int64 l = UUIDHelpers::getLowBytes(serverUUID); + return ((h * 11) ^ (l * 17)) & machine_id_mask; +} + +Int64 getTimestamp() +{ + const auto tm_point = std::chrono::system_clock::now(); + return std::chrono::duration_cast( + tm_point.time_since_epoch()).count() & ((1LL << timestamp_size) - 1); +} } class FunctionSnowflakeID : public IFunction { private: - mutable std::atomic state{0}; - // previous snowflake id - // state is 1 atomic value because we don't want use mutex + mutable std::atomic lowest_available_snowflake_id{0}; + // 1 atomic value because we don't want to use mutex public: static constexpr auto name = "generateSnowflakeID"; @@ -58,23 +79,19 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } - + bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } bool useDefaultImplementationForNulls() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } bool isVariadic() const override { return true; } - bool isStateful() const override { return true; } - bool isDeterministic() const override { return false; } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() > 1) { + if (!arguments.empty()) { throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", + "Number of arguments for function {} doesn't match: passed {}, should be 0.", getName(), arguments.size()); } - return std::make_shared(); } @@ -83,36 +100,57 @@ public: { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - size_t size = input_rows_count; - vec_to.resize(size); + Int64 size64 = static_cast(input_rows_count); + vec_to.resize(input_rows_count); - auto serverUUID = ServerUUID::get(); + if (input_rows_count == 0) { + return col_res; + } - // hash serverUUID into 32 bytes - Int64 h = UUIDHelpers::getHighBytes(serverUUID); - Int64 l = UUIDHelpers::getLowBytes(serverUUID); - Int64 machine_id = ((h * 11) ^ (l * 17)) & machine_id_mask; + Int64 machine_id = getMachineID(); + Int64 current_timestamp = getTimestamp(); + Int64 current_machine_seq_num; - for (Int64 & el : vec_to) { - const auto tm_point = std::chrono::system_clock::now(); - Int64 current_timestamp = std::chrono::duration_cast( - tm_point.time_since_epoch()).count() & ((1LL << timestamp_size) - 1); + Int64 available_id, next_available_id; + do + { + available_id = lowest_available_snowflake_id.load(); + Int64 available_timestamp = (available_id & timestamp_mask) >> (machine_id_size + machine_seq_num_size); + Int64 available_machine_seq_num = available_id & machine_seq_num_mask; - Int64 last_state, new_state; - do { - last_state = state.load(); - Int64 last_timestamp = (last_state & timestamp_mask) >> (machine_id_size + machine_seq_num_size); - Int64 machine_seq_num = last_state & machine_seq_num_mask; + if (current_timestamp > available_timestamp) + { + current_machine_seq_num = 0; + } + else + { + current_timestamp = available_timestamp; + current_machine_seq_num = available_machine_seq_num; + } - if (current_timestamp == last_timestamp) { - ++machine_seq_num; - } - new_state = (current_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | machine_seq_num; - } while (!state.compare_exchange_strong(last_state, new_state)); - // failed CAS => another thread updated state - // successful CAS => we have unique (timestamp, machine_seq_num) on this machine + // calculate new `lowest_available_snowflake_id` + Int64 new_timestamp; + Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1); + if (size64 >= seq_nums_in_current_timestamp_left) { + new_timestamp = current_timestamp + 1 + (size64 - seq_nums_in_current_timestamp_left) / max_machine_seq_num; + } else { + new_timestamp = current_timestamp; + } + Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask; + next_available_id = (new_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | new_machine_seq_num; + } + while (!lowest_available_snowflake_id.compare_exchange_strong(available_id, next_available_id)); + // failed CAS => another thread updated `lowest_available_snowflake_id` + // successful CAS => we have our range of exclusive values - el = new_state; + for (Int64 & el : vec_to) + { + el = (current_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | current_machine_seq_num; + if (current_machine_seq_num++ == max_machine_seq_num) + { + current_machine_seq_num = 0; + ++current_timestamp; + } } return col_res; @@ -122,7 +160,41 @@ public: REGISTER_FUNCTION(GenerateSnowflakeID) { - factory.registerFunction(); + factory.registerFunction(FunctionDocumentation + { + .description=R"( +Generates Snowflake ID -- unique identificators contains: +- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds +- The middle 10 bits are the machine ID. +- The last 12 bits decode to number of ids processed by the machine at the given millisecond. + +In case the number of ids processed overflows, the timestamp field is incremented by 1 and the counter is reset to 0. +This function guarantees strict monotony on 1 machine and differences in values obtained on different machines. +)", + .syntax = "generateSnowflakeID()", + .arguments{}, + .returned_value = "Column of Int64", + .examples{ + {"single call", "SELECT generateSnowflakeID();", R"( +┌─generateSnowflakeID()─┐ +│ 7195510166884597760 │ +└───────────────────────┘)"}, + {"column call", "SELECT generateSnowflakeID() FROM numbers(10);", R"( +┌─generateSnowflakeID()─┐ +│ 7195516038159417344 │ +│ 7195516038159417345 │ +│ 7195516038159417346 │ +│ 7195516038159417347 │ +│ 7195516038159417348 │ +│ 7195516038159417349 │ +│ 7195516038159417350 │ +│ 7195516038159417351 │ +│ 7195516038159417352 │ +│ 7195516038159417353 │ +└───────────────────────┘)"}, + }, + .categories{"Unique identifiers", "Snowflake ID"} + }); } } diff --git a/src/Functions/serial.cpp b/src/Functions/serial.cpp index 1745e17b5e7..3da2f4ce218 100644 --- a/src/Functions/serial.cpp +++ b/src/Functions/serial.cpp @@ -1,18 +1,11 @@ -#include -#include -#include -#include +#include #include #include #include #include -#include "Common/Logger.h" -#include "Common/ZooKeeper/IKeeper.h" -#include "Common/ZooKeeper/KeeperException.h" -#include "Common/ZooKeeper/Types.h" -#include -namespace DB { +namespace DB +{ namespace ErrorCodes { @@ -62,30 +55,26 @@ public: throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()); - if (!isStringOrFixedString(arguments[0])) { + if (!isStringOrFixedString(arguments[0])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Type of argument for function {} doesn't match: passed {}, should be string", getName(), arguments[0]->getName()); - } return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - if (zk == nullptr) { + if (zk == nullptr) throw Exception(ErrorCodes::KEEPER_EXCEPTION, "ZooKeeper is not configured for function {}", getName()); - } - if (zk->expired()) { + if (zk->expired()) zk = context->getZooKeeper(); - } auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); size_t size = input_rows_count; - LOG_INFO(getLogger("Serial Function"), "Size = {}", size); vec_to.resize(size); const auto & serial_path = "/serials/" + arguments[0].column->getDataAt(0).toString(); @@ -102,16 +91,19 @@ public: zk->createIfNotExists(counter_path, "1"); Coordination::Stat stat; - while (true) { + while (true) + { std::string counter_string = zk->get(counter_path, &stat); counter = std::stoll(counter_string); std::string updated_counter = std::to_string(counter + input_rows_count); Coordination::Error err = zk->trySet(counter_path, updated_counter); - if (err == Coordination::Error::ZOK) { + if (err == Coordination::Error::ZOK) + { // CAS is done break; } - if (err != Coordination::Error::ZBADVERSION) { + if (err != Coordination::Error::ZBADVERSION) + { throw Exception(ErrorCodes::KEEPER_EXCEPTION, "ZooKeeper trySet operation failed with unexpected error = {} in function {}", err, getName()); @@ -119,7 +111,8 @@ public: } // Make a result - for (auto& val : vec_to) { + for (auto& val : vec_to) + { val = counter; ++counter; } @@ -163,7 +156,6 @@ The server should be configured with a ZooKeeper. )"}}, .categories{"Unique identifiers"} }); - } } diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.reference b/tests/queries/0_stateless/03129_serial_test_zookeeper.reference index 60714f4064f..479030db4be 100644 --- a/tests/queries/0_stateless/03129_serial_test_zookeeper.reference +++ b/tests/queries/0_stateless/03129_serial_test_zookeeper.reference @@ -1,8 +1,13 @@ 1 2 -1 3 3 3 -1 1 1 4 -1 2 2 5 -1 5 5 6 -1 4 4 7 1 +3 +4 +5 +6 +7 +1 1 +2 2 +3 3 +4 4 +5 5 diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql index 3eacd1ae908..c3395009477 100644 --- a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql +++ b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql @@ -1,20 +1,12 @@ +-- Tags: zookeeper + SELECT serial('x'); SELECT serial('x'); +SELECT serial('y'); +SELECT serial('x') FROM numbers(5); -DROP TABLE IF EXISTS default.test_table; +SELECT serial(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT serial('x', 'y'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT serial(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -CREATE TABLE test_table -( - CounterID UInt32, - UserID UInt32, - ver UInt16 -) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/1-1/test_table', 'x', ver) -PARTITION BY CounterID -ORDER BY (CounterID, intHash32(UserID)) -SAMPLE BY intHash32(UserID); - -INSERT INTO test_table VALUES (1, 1, 1), (1, 2, 2), (1, 3, 3), (1, 4, 4), (1, 5, 5); - -SELECT *, serial('x') FROM test_table; - -SELECT serial('y'); \ No newline at end of file +SELECT serial('z'), serial('z') FROM numbers(5); diff --git a/tests/queries/0_stateless/03130_generate_snowflake_id.reference b/tests/queries/0_stateless/03130_generate_snowflake_id.reference new file mode 100644 index 00000000000..2049ba26379 --- /dev/null +++ b/tests/queries/0_stateless/03130_generate_snowflake_id.reference @@ -0,0 +1,3 @@ +1 +1 +10 diff --git a/tests/queries/0_stateless/03130_generate_snowflake_id.sql b/tests/queries/0_stateless/03130_generate_snowflake_id.sql new file mode 100644 index 00000000000..669814c9ecb --- /dev/null +++ b/tests/queries/0_stateless/03130_generate_snowflake_id.sql @@ -0,0 +1,11 @@ +SELECT bitShiftLeft(toUInt64(generateSnowflakeID()), 52) = 0; +SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; + +SELECT generateSnowflakeID(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT count(*) +FROM +( + SELECT DISTINCT generateSnowflakeID() + FROM numbers(10) +) \ No newline at end of file From f1f668e7df24190eaf4f1d67360b9e53099289d2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 10 May 2024 14:15:01 +0200 Subject: [PATCH 054/158] Setup node generator initial --- utils/keeper-bench/Runner.cpp | 288 ++++++++++++++++++++++++++++++---- utils/keeper-bench/Runner.h | 3 + utils/keeper-bench/main.cpp | 2 + 3 files changed, 265 insertions(+), 28 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index a893dac3851..0050230b6ec 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -1,17 +1,22 @@ #include "Runner.h" #include -#include #include +#include +#include +#include #include "Common/ConcurrentBoundedQueue.h" +#include "Common/Exception.h" #include "Common/ZooKeeper/IKeeper.h" #include "Common/ZooKeeper/ZooKeeperArgs.h" #include "Common/ZooKeeper/ZooKeeperCommon.h" #include "Common/ZooKeeper/ZooKeeperConstants.h" #include #include +#include "Coordination/KeeperSnapshotManager.h" #include "Core/ColumnWithTypeAndName.h" #include "Core/ColumnsWithTypeAndName.h" +#include #include "IO/ReadBuffer.h" #include "IO/ReadBufferFromFile.h" #include "base/Decimal.h" @@ -43,12 +48,14 @@ Runner::Runner( std::optional concurrency_, const std::string & config_path, const std::string & input_request_log_, + const std::string & setup_nodes_snapshot_path_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, std::optional continue_on_error_, std::optional max_iterations_) : input_request_log(input_request_log_) + , setup_nodes_snapshot_path(setup_nodes_snapshot_path_) , info(std::make_shared()) { @@ -381,18 +388,18 @@ struct ZooKeeperRequestBlock { explicit ZooKeeperRequestBlock(DB::Block block_) : block(std::move(block_)) - , hostname_idx(block.getPositionByName("hostname")) // - , request_event_time_idx(block.getPositionByName("request_event_time")) // - , thread_id_idx(block.getPositionByName("thread_id")) // - , session_id_idx(block.getPositionByName("session_id")) // - , xid_idx(block.getPositionByName("xid")) // + , hostname_idx(block.getPositionByName("hostname")) + , request_event_time_idx(block.getPositionByName("request_event_time")) + , thread_id_idx(block.getPositionByName("thread_id")) + , session_id_idx(block.getPositionByName("session_id")) + , xid_idx(block.getPositionByName("xid")) , has_watch_idx(block.getPositionByName("has_watch")) , op_num_idx(block.getPositionByName("op_num")) , path_idx(block.getPositionByName("path")) , data_idx(block.getPositionByName("data")) , is_ephemeral_idx(block.getPositionByName("is_ephemeral")) , is_sequential_idx(block.getPositionByName("is_sequential")) - , response_event_time_idx(block.getPositionByName("response_event_time")) // + , response_event_time_idx(block.getPositionByName("response_event_time")) , error_idx(block.getPositionByName("error")) , requests_size_idx(block.getPositionByName("requests_size")) , version_idx(block.getPositionByName("version")) @@ -519,6 +526,7 @@ struct RequestFromLog { Coordination::ZooKeeperRequestPtr request; std::optional expected_result; + std::vector> subrequest_expected_results; int64_t session_id = 0; size_t executor_id = 0; bool has_watch = false; @@ -586,7 +594,6 @@ struct ZooKeeperRequestFromLogReader idx_in_block = 0; } - request_from_log.expected_result = current_block->getError(idx_in_block); request_from_log.session_id = current_block->getSessionId(idx_in_block); request_from_log.has_watch = current_block->hasWatch(idx_in_block); @@ -693,6 +700,12 @@ struct ZooKeeperRequestFromLogReader if (!subrequest_from_log) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Failed to fetch subrequest for {}, subrequest index {}", op_num, i); + if (!subrequest_from_log->expected_result && request_from_log.expected_result + && request_from_log.expected_result == Coordination::Error::ZOK) + { + subrequest_from_log->expected_result = Coordination::Error::ZOK; + } + requests.push_back(std::move(subrequest_from_log->request)); if (subrequest_from_log->session_id != request_from_log.session_id) @@ -700,6 +713,8 @@ struct ZooKeeperRequestFromLogReader if (subrequest_from_log->executor_id != request_from_log.executor_id) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Executor id mismatch for subrequest in {}, subrequest index {}", op_num, i); + + request_from_log.subrequest_expected_results.push_back(subrequest_from_log->expected_result); } request_from_log.request = std::make_shared(requests, default_acls); @@ -731,7 +746,6 @@ private: namespace { - struct RequestFromLogStats { struct Stats @@ -744,6 +758,192 @@ struct RequestFromLogStats Stats read_requests; }; +struct SetupNodeCollector +{ + explicit SetupNodeCollector(const std::string & setup_nodes_snapshot_path) + { + if (setup_nodes_snapshot_path.empty()) + return; + + keeper_context = std::make_shared(true, std::make_shared()); + keeper_context->setDigestEnabled(true); + keeper_context->setSnapshotDisk( + std::make_shared("Keeper-snapshots", setup_nodes_snapshot_path)); + + snapshot_manager.emplace(1, keeper_context); + auto snapshot_result = snapshot_manager->restoreFromLatestSnapshot(); + if (snapshot_result.storage == nullptr) + { + std::cerr << "No initial snapshot found" << std::endl; + initial_storage = std::make_unique( + /* tick_time_ms */ 500, /* superdigest */ "", keeper_context, /* initialize_system_nodes */ false); + initial_storage->initializeSystemNodes(); + } + else + { + std::cerr << "Loaded initial nodes from snapshot" << std::endl; + initial_storage = std::move(snapshot_result.storage); + } + } + + void processRequest(const RequestFromLog & request_from_log) + { + if (!request_from_log.expected_result.has_value()) + return; + + auto process_request = [&](const Coordination::ZooKeeperRequest & request, const auto expected_result) + { + const auto & path = request.getPath(); + if (processed_paths.contains(path)) + return; + + auto op_num = request.getOpNum(); + + if (op_num == Coordination::OpNum::Create) + { + if (expected_result == Coordination::Error::ZNODEEXISTS) + { + addExpectedNode(path); + processed_paths.insert(path); + } + else if (expected_result == Coordination::Error::ZOK) + { + /// we need to make sure ancestors exist + auto position = path.find_last_of('/'); + if (position != 0) + { + auto parent_path = path.substr(0, position); + if (!processed_paths.contains(parent_path)) + { + addExpectedNode(parent_path); + processed_paths.insert(parent_path); + } + } + + processed_paths.insert(path); + } + } + else if (op_num == Coordination::OpNum::Remove) + { + if (expected_result == Coordination::Error::ZOK) + { + addExpectedNode(path); + processed_paths.insert(path); + } + } + else if (op_num == Coordination::OpNum::Set) + { + if (expected_result == Coordination::Error::ZOK) + { + addExpectedNode(path); + processed_paths.insert(path); + } + } + else if (op_num == Coordination::OpNum::Check) + { + if (expected_result == Coordination::Error::ZOK) + { + addExpectedNode(path); + processed_paths.insert(path); + } + } + else if (op_num == Coordination::OpNum::CheckNotExists) + { + if (expected_result == Coordination::Error::ZNODEEXISTS) + { + addExpectedNode(path); + processed_paths.insert(path); + } + } + else if (request.isReadRequest()) + { + if (expected_result == Coordination::Error::ZOK) + { + addExpectedNode(path); + processed_paths.insert(path); + } + } + }; + + const auto & request = request_from_log.request; + if (request->getOpNum() == Coordination::OpNum::Multi || request->getOpNum() == Coordination::OpNum::MultiRead) + { + const auto & multi_request = dynamic_cast(*request); + const auto & subrequests = multi_request.requests; + + for (size_t i = 0; i < subrequests.size(); ++i) + { + const auto & zookeeper_request = dynamic_cast(*subrequests[i]); + const auto subrequest_expected_result = request_from_log.subrequest_expected_results[i]; + if (subrequest_expected_result.has_value()) + process_request(zookeeper_request, *subrequest_expected_result); + + } + } + else + process_request(*request, *request_from_log.expected_result); + } + + void addExpectedNode(const std::string & path) + { + std::lock_guard lock(nodes_mutex); + + if (initial_storage->container.contains(path)) + return; + + std::cerr << "Adding expected node " << path << std::endl; + + Coordination::Requests create_ops; + + size_t pos = 1; + while (true) + { + pos = path.find('/', pos); + if (pos == std::string::npos) + break; + + auto request = zkutil::makeCreateRequest(path.substr(0, pos), "", zkutil::CreateMode::Persistent, true); + create_ops.emplace_back(request); + ++pos; + } + + auto request = zkutil::makeCreateRequest(path, "", zkutil::CreateMode::Persistent, true); + create_ops.emplace_back(request); + + auto next_zxid = initial_storage->getNextZXID(); + + static Coordination::ACLs default_acls = [] + { + Coordination::ACL acl; + acl.permissions = Coordination::ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + return Coordination::ACLs{std::move(acl)}; + }(); + + auto multi_create_request = std::make_shared(create_ops, default_acls); + initial_storage->preprocessRequest(multi_create_request, 1, 0, next_zxid, /* check_acl = */ false); + auto responses = initial_storage->processRequest(multi_create_request, 1, next_zxid, /* check_acl = */ false); + if (responses.size() > 1 || responses[0].response->error != Coordination::Error::ZOK) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Invalid response after trying to create a node {}", responses[0].response->error); + } + + void generateSnapshot() + { + std::cerr << "Generating snapshot with starting data" << std::endl; + std::lock_guard lock(nodes_mutex); + DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(initial_storage->getZXID(), 1, std::make_shared()); + DB::KeeperStorageSnapshot snapshot(initial_storage.get(), snapshot_meta); + snapshot_manager->serializeSnapshotToDisk(snapshot); + } + + std::mutex nodes_mutex; + DB::KeeperContextPtr keeper_context; + Coordination::KeeperStoragePtr initial_storage; + std::unordered_set processed_paths; + std::optional snapshot_manager; +}; + void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_for_type) { std::cerr << fmt::format( @@ -751,7 +951,7 @@ void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_f type, stats_for_type.total, stats_for_type.unexpected_results, - static_cast(stats_for_type.unexpected_results) / stats_for_type.total * 100) + stats_for_type.total != 0 ? static_cast(stats_for_type.unexpected_results) / stats_for_type.total * 100 : 0.0) << std::endl; }; @@ -763,24 +963,40 @@ void requestFromLogExecutor(std::shared_ptr>(); last_request = request_promise->get_future(); - Coordination::ResponseCallback callback - = [&, request_promise, request = request_from_log.request, expected_result = request_from_log.expected_result]( - const Coordination::Response & response) mutable + Coordination::ResponseCallback callback = [&, + request_promise, + request = request_from_log.request, + expected_result = request_from_log.expected_result, + subrequest_expected_results = std::move(request_from_log.subrequest_expected_results)]( + const Coordination::Response & response) mutable { auto & stats = request->isReadRequest() ? request_stats.read_requests : request_stats.write_requests; stats.total.fetch_add(1, std::memory_order_relaxed); - if (*expected_result != response.error) - stats.unexpected_results.fetch_add(1, std::memory_order_relaxed); + if (expected_result) + { + if (*expected_result != response.error) + stats.unexpected_results.fetch_add(1, std::memory_order_relaxed); - //if (!expected_result) - // return; + if (*expected_result != response.error) + { + std::cerr << fmt::format( + "Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result) + << std::endl; - //if (*expected_result != response.error) - // std::cerr << fmt::format( - // "Unexpected result for {}, got {}, expected {}", request->getOpNum(), response.error, *expected_result) - // << std::endl; + if (const auto * multi_response = dynamic_cast(&response)) + { + std::string subresponses; + for (size_t i = 0; i < multi_response->responses.size(); ++i) + { + subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); + } + + std::cerr << "Subresponses\n" << subresponses << std::endl; + } + } + } request_promise->set_value(); }; @@ -827,6 +1043,9 @@ void Runner::runBenchmarkFromLog() RequestFromLogStats stats; + std::optional setup_nodes_collector; + if (!setup_nodes_snapshot_path.empty()) + setup_nodes_collector.emplace(setup_nodes_snapshot_path); std::unordered_map>> executor_id_to_queue; @@ -850,7 +1069,7 @@ void Runner::runBenchmarkFromLog() return; } - auto executor_queue = std::make_shared>(std::numeric_limits().max()); + auto executor_queue = std::make_shared>(std::numeric_limits::max()); executor_id_to_queue.emplace(request.executor_id, executor_queue); auto scheduled = pool->trySchedule([&, executor_queue]() mutable { @@ -865,6 +1084,7 @@ void Runner::runBenchmarkFromLog() throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to push to the executor's queue"); }; + if (!setup_nodes_collector) { auto setup_connection = getConnection(connection_infos[0], 0); benchmark_context.startup(*setup_connection); @@ -875,14 +1095,26 @@ void Runner::runBenchmarkFromLog() delay_watch.restart(); while (auto request_from_log = request_reader.getNextRequest()) { - request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); - push_request(std::move(*request_from_log)); + if (setup_nodes_collector) + { + setup_nodes_collector->processRequest(*request_from_log); + } + else + { + request_from_log->connection = get_zookeeper_connection(request_from_log->session_id); + push_request(std::move(*request_from_log)); + } if (delay > 0 && delay_watch.elapsedSeconds() > delay) { - dumpStats("Write", stats.write_requests); - dumpStats("Read", stats.read_requests); - std::cerr << std::endl; + if (setup_nodes_collector) + setup_nodes_collector->generateSnapshot(); + else + { + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + std::cerr << std::endl; + } delay_watch.restart(); } } @@ -906,7 +1138,7 @@ void Runner::runBenchmarkWithGenerator() for (size_t i = 0; i < concurrency; ++i) { auto thread_connections = connections; - pool->scheduleOrThrowOnError([this, connections_ = std::move(thread_connections)]() mutable { thread(connections_); }); + pool->scheduleOrThrowOnError([this, my_connections = std::move(thread_connections)]() mutable { thread(my_connections); }); } } catch (...) diff --git a/utils/keeper-bench/Runner.h b/utils/keeper-bench/Runner.h index 0c646eb2166..c19a4d82898 100644 --- a/utils/keeper-bench/Runner.h +++ b/utils/keeper-bench/Runner.h @@ -27,6 +27,7 @@ public: void startup(Coordination::ZooKeeper & zookeeper); void cleanup(Coordination::ZooKeeper & zookeeper); + private: struct Node { @@ -54,6 +55,7 @@ public: std::optional concurrency_, const std::string & config_path, const std::string & input_request_log_, + const std::string & setup_nodes_snapshot_path_, const Strings & hosts_strings_, std::optional max_time_, std::optional delay_, @@ -96,6 +98,7 @@ private: std::shared_ptr getConnection(const ConnectionInfo & connection_info, size_t connection_info_idx); std::string input_request_log; + std::string setup_nodes_snapshot_path; size_t concurrency = 1; diff --git a/utils/keeper-bench/main.cpp b/utils/keeper-bench/main.cpp index 45fc28f3bca..0b963abf406 100644 --- a/utils/keeper-bench/main.cpp +++ b/utils/keeper-bench/main.cpp @@ -38,6 +38,7 @@ int main(int argc, char *argv[]) ("help", "produce help message") ("config", value()->default_value(""), "yaml/xml file containing configuration") ("input-request-log", value()->default_value(""), "log of requests that will be replayed") + ("setup-nodes-snapshot-path", value()->default_value(""), "directory containing snapshots with starting state") ("concurrency,c", value(), "number of parallel queries") ("report-delay,d", value(), "delay between intermediate reports in seconds (set 0 to disable reports)") ("iterations,i", value(), "amount of queries to be executed") @@ -60,6 +61,7 @@ int main(int argc, char *argv[]) Runner runner(valueToOptional(options["concurrency"]), options["config"].as(), options["input-request-log"].as(), + options["setup-nodes-snapshot-path"].as(), options["hosts"].as(), valueToOptional(options["time-limit"]), valueToOptional(options["report-delay"]), From 904800afc8e77bc5567ba2096258aec4802d8cee Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 13 May 2024 17:44:14 +0200 Subject: [PATCH 055/158] Apply recent changes to storages3/hdfs/azure --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- .../ObjectStorage/StorageObjectStorage.cpp | 29 ++++++++++++------- .../ObjectStorage/StorageObjectStorage.h | 3 +- 3 files changed, 21 insertions(+), 13 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 74707b61238..c24874d0a94 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -592,7 +592,7 @@ std::unique_ptr S3ObjectStorage::cloneObjectStorage( ContextPtr context) { auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + auto new_client = getClient(config, config_prefix, context, *new_s3_settings, true); auto new_uri{uri}; new_uri.bucket = new_namespace; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index a187a8fc54d..01790760747 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -206,7 +206,7 @@ void StorageObjectStorage::read( size_t num_streams) { updateConfiguration(local_context); - if (partition_by && configuration->withWildcard()) + if (partition_by && configuration->withPartitionWildcard()) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Reading from a partitioned {} storage is not implemented yet", @@ -247,7 +247,14 @@ SinkToStoragePtr StorageObjectStorage::write( const auto sample_block = metadata_snapshot->getSampleBlock(); const auto & settings = configuration->getQuerySettings(local_context); - if (configuration->withWildcard()) + if (configuration->withGlobsIgnorePartitionWildcard()) + { + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, + "Path '{}' contains globs, so the table is in readonly mode", + configuration->getPath()); + } + + if (configuration->withPartitionWildcard()) { ASTPtr partition_by_ast = nullptr; if (auto insert_query = std::dynamic_pointer_cast(query)) @@ -265,14 +272,6 @@ SinkToStoragePtr StorageObjectStorage::write( } } - if (configuration->withGlobs()) - { - throw Exception( - ErrorCodes::DATABASE_ACCESS_DENIED, - "{} key '{}' contains globs, so the table is in readonly mode", - getName(), configuration->getPath()); - } - auto paths = configuration->getPaths(); if (auto new_key = checkAndGetNewFileOnInsertIfNeeded( *object_storage, *configuration, settings, paths.front(), paths.size())) @@ -428,13 +427,21 @@ StorageObjectStorage::Configuration::Configuration(const Configuration & other) structure = other.structure; } -bool StorageObjectStorage::Configuration::withWildcard() const +bool StorageObjectStorage::Configuration::withPartitionWildcard() const { static const String PARTITION_ID_WILDCARD = "{_partition_id}"; return getPath().find(PARTITION_ID_WILDCARD) != String::npos || getNamespace().find(PARTITION_ID_WILDCARD) != String::npos; } +bool StorageObjectStorage::Configuration::withGlobsIgnorePartitionWildcard() const +{ + if (!withPartitionWildcard()) + return withGlobs(); + else + return PartitionedSink::replaceWildcards(getPath(), "").find_first_of("*?{") != std::string::npos; +} + bool StorageObjectStorage::Configuration::isPathWithGlobs() const { return getPath().find_first_of("*?{") != std::string::npos; diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 3f8ff79ad54..a396bad9d6e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -163,8 +163,9 @@ public: virtual void addStructureAndFormatToArgs( ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; - bool withWildcard() const; + bool withPartitionWildcard() const; bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); } + bool withGlobsIgnorePartitionWildcard() const; bool isPathWithGlobs() const; bool isNamespaceWithGlobs() const; virtual std::string getPathWithoutGlobs() const; From f3b9a326fede69769811dc9309bfb5d00aefd874 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 13 May 2024 19:59:16 +0200 Subject: [PATCH 056/158] Fix build --- src/TableFunctions/TableFunctionObjectStorage.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index a997b34a75c..9f16a9a0b25 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -192,6 +192,15 @@ void registerTableFunctionObjectStorage(TableFunctionFactory & factory) #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])", "" + }} + }, .allow_readonly = false }); #endif From 0abb2be5eb55183e83c218cf352c88c7fb497939 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 May 2024 18:40:09 +0200 Subject: [PATCH 057/158] Review fixes --- docs/en/operations/settings/settings.md | 50 +++++++++++++++++++ .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 7 ++- .../ObjectStorages/HDFS/HDFSObjectStorage.h | 3 ++ src/Storages/Cache/SchemaCache.cpp | 1 - .../ObjectStorage/StorageObjectStorage.cpp | 6 +-- .../ObjectStorage/StorageObjectStorage.h | 6 ++- .../StorageObjectStorageSource.cpp | 2 +- .../StorageObjectStorageSource.h | 2 +- src/Storages/ObjectStorage/Utils.cpp | 6 +-- .../registerStorageObjectStorage.cpp | 6 +-- 10 files changed, 70 insertions(+), 19 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 91b544c6a82..72bd1ca8e2c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3665,6 +3665,16 @@ Possible values: Default value: `0`. +## s3_ignore_file_doesnt_exist {#s3_ignore_file_doesnt_exist} + +Ignore ansense of file if it does not exist when reading certain keys. + +Possible values: +- 1 — `SELECT` returns empty result. +- 0 — `SELECT` throws an exception. + +Default value: `0`. + ## hdfs_truncate_on_insert {#hdfs_truncate_on_insert} Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. @@ -3697,6 +3707,46 @@ Possible values: Default value: `0`. +## hdfs_throw_on_zero_files_match {#hdfs_throw_on_zero_files_match} + +Throw an error if matched zero files according to glob expansion rules. + +Possible values: +- 1 — `SELECT` throws an exception. +- 0 — `SELECT` returns empty result. + +Default value: `0`. + +## hdfs_ignore_file_doesnt_exist {#hdfs_ignore_file_doesnt_exist} + +Ignore ansense of file if it does not exist when reading certain keys. + +Possible values: +- 1 — `SELECT` returns empty result. +- 0 — `SELECT` throws an exception. + +Default value: `0`. + +## azure_throw_on_zero_files_match {#azure_throw_on_zero_files_match} + +Throw an error if matched zero files according to glob expansion rules. + +Possible values: +- 1 — `SELECT` throws an exception. +- 0 — `SELECT` returns empty result. + +Default value: `0`. + +## azure_ignore_file_doesnt_exist {#azure_ignore_file_doesnt_exist} + +Ignore ansense of file if it does not exist when reading certain keys. + +Possible values: +- 1 — `SELECT` returns empty result. +- 0 — `SELECT` throws an exception. + +Default value: `0`. + ## engine_url_skip_empty_files {#engine_url_skip_empty_files} Enables or disables skipping empty files in [URL](../../engines/table-engines/special/url.md) engine tables. diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 6c2f310a7d1..1f3a4bdf6c7 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -186,7 +186,6 @@ ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string & path) co void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithMetadata & children, size_t max_keys) const { initializeHDFSFS(); - auto * log = &Poco::Logger::get("HDFSObjectStorage"); LOG_TEST(log, "Trying to list files for {}", path); HDFSFileInfo ls; @@ -210,9 +209,6 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM for (int i = 0; i < ls.length; ++i) { const String file_path = fs::path(ls.file_info[i].mName).lexically_normal(); - const size_t last_slash = file_path.rfind('/'); - const String file_name = file_path.substr(last_slash); - const bool is_directory = ls.file_info[i].mKind == 'D'; if (is_directory) { @@ -227,6 +223,9 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM Poco::Timestamp::fromEpochTime(ls.file_info[i].mLastMod), {}})); } + + if (children.size() >= max_keys) + break; } } diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index e747b283400..8aae90d0721 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -39,6 +39,7 @@ public: bool lazy_initialize) : config(config_) , settings(std::move(settings_)) + , log(getLogger("HDFSObjectStorage(" + hdfs_root_path_ + ")")) { const size_t begin_of_path = hdfs_root_path_.find('/', hdfs_root_path_.find("//") + 2); url = hdfs_root_path_; @@ -134,6 +135,8 @@ private: std::string url; std::string url_without_path; std::string data_directory; + + LoggerPtr log; }; } diff --git a/src/Storages/Cache/SchemaCache.cpp b/src/Storages/Cache/SchemaCache.cpp index 5dc39f04ae0..299dd292772 100644 --- a/src/Storages/Cache/SchemaCache.cpp +++ b/src/Storages/Cache/SchemaCache.cpp @@ -1,6 +1,5 @@ #include #include -#include #include namespace ProfileEvents diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 01790760747..c5affb7989f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -61,10 +61,6 @@ StorageObjectStorage::StorageObjectStorage( metadata.setConstraints(constraints_); metadata.setComment(comment); - StoredObjects objects; - for (const auto & key : configuration->getPaths()) - objects.emplace_back(key); - setVirtuals(VirtualColumnUtils::getVirtualsForFileLikeStorage(metadata.getColumns())); setInMemoryMetadata(metadata); } @@ -93,7 +89,7 @@ void StorageObjectStorage::updateConfiguration(ContextPtr context) { /// FIXME: we should be able to update everything apart from client if static_configuration == true. if (!configuration->isStaticConfiguration()) - object_storage->applyNewSettings(context->getConfigRef(), "s3.", context); + object_storage->applyNewSettings(context->getConfigRef(), configuration->getTypeName() + ".", context); } namespace diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index a396bad9d6e..928d49f9604 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -124,7 +124,6 @@ protected: ConfigurationPtr configuration; const ObjectStoragePtr object_storage; - const std::string engine_name; const std::optional format_settings; const ASTPtr partition_by; const bool distributed_processing; @@ -148,7 +147,9 @@ public: ContextPtr local_context, bool with_table_structure); + /// Storage type: s3, hdfs, azure. virtual std::string getTypeName() const = 0; + /// Engine name: S3, HDFS, Azure. virtual std::string getEngineName() const = 0; virtual Path getPath() const = 0; @@ -158,7 +159,10 @@ public: virtual void setPaths(const Paths & paths) = 0; virtual String getDataSourceDescription() = 0; + /// Sometimes object storages have something similar to chroot or namespace, for example + /// buckets in S3. If object storage doesn't have any namepaces return empty string. virtual String getNamespace() const = 0; + virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; virtual void addStructureAndFormatToArgs( ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index cb3f732ce83..e28924617e0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -321,7 +321,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const S const bool object_too_small = object_size <= 2 * getContext()->getSettings().max_download_buffer_size; const bool use_prefetch = object_too_small && read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; read_settings.remote_fs_method = use_prefetch ? RemoteFSReadMethod::threadpool : RemoteFSReadMethod::read; - /// User's S3 object may change, don't cache it. + /// User's object may change, don't cache it. read_settings.use_page_cache_for_disks_without_file_cache = false; // Create a read buffer that will prefetch the first ~1 MB of the file. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index a8df00bc0ac..08d545f9b85 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -62,7 +62,7 @@ protected: const std::optional format_settings; const UInt64 max_block_size; const bool need_only_count; - const ReadFromFormatInfo read_from_format_info; + const ReadFromFormatInfo & read_from_format_info; const std::shared_ptr create_reader_pool; ColumnsDescription columns_desc; diff --git a/src/Storages/ObjectStorage/Utils.cpp b/src/Storages/ObjectStorage/Utils.cpp index bde3cb7e1cb..e49e14d2a0c 100644 --- a/src/Storages/ObjectStorage/Utils.cpp +++ b/src/Storages/ObjectStorage/Utils.cpp @@ -38,9 +38,9 @@ std::optional checkAndGetNewFileOnInsertIfNeeded( throw Exception( ErrorCodes::BAD_ARGUMENTS, "Object in bucket {} with key {} already exists. " - "If you want to overwrite it, enable setting s3_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", - configuration.getNamespace(), key); + "If you want to overwrite it, enable setting {}_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting {}_create_new_file_on_insert", + configuration.getNamespace(), key, configuration.getTypeName(), configuration.getTypeName()); } void resolveSchemaAndFormat( diff --git a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp index 74c8aeaad7d..bf595b2f5d4 100644 --- a/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/registerStorageObjectStorage.cpp @@ -106,17 +106,17 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) void registerStorageS3(StorageFactory & factory) { - return registerStorageS3Impl("S3", factory); + registerStorageS3Impl("S3", factory); } void registerStorageCOS(StorageFactory & factory) { - return registerStorageS3Impl("COSN", factory); + registerStorageS3Impl("COSN", factory); } void registerStorageOSS(StorageFactory & factory) { - return registerStorageS3Impl("OSS", factory); + registerStorageS3Impl("OSS", factory); } #endif From 3778cee49e1d6ac1f0f4f470ba5d63458c33df3b Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 14 May 2024 18:41:19 +0200 Subject: [PATCH 058/158] Update src/Core/Settings.h Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Core/Settings.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index aa20f68ac0d..066a551b37b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -117,9 +117,9 @@ class IColumn; M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, s3_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageS3", 0) \ - M(Bool, hdfs_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageHDFS", 0) \ - M(Bool, azure_ignore_file_doesnt_exist, false, "Ignore if files does not exits and return 0 zeros for StorageAzure", 0) \ + M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in S3 table engine", 0) \ + M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ + M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ From be693ceba7fa17e2c03c54197fb0d0f301640cc1 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 May 2024 18:46:35 +0200 Subject: [PATCH 059/158] Minor --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c5affb7989f..bc5b347d1e0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -402,7 +402,6 @@ void StorageObjectStorage::Configuration::initialize( else configuration.fromAST(engine_args, local_context, with_table_structure); - // FIXME: it should be - if (format == "auto" && get_format_from_file) if (configuration.format == "auto") configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); else From 65f404c153fb96602ec07c4f3919af14468b8d7d Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 May 2024 21:28:40 +0200 Subject: [PATCH 060/158] Review fixes --- docs/en/operations/settings/settings.md | 2 +- src/Core/Settings.h | 6 +++--- .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.h | 2 ++ .../ObjectStorage/StorageObjectStorage.h | 5 +++-- .../StorageObjectStorageSource.cpp | 19 ++++++++----------- .../StorageObjectStorageSource.h | 2 +- 7 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 72bd1ca8e2c..88e945a710c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3667,7 +3667,7 @@ Default value: `0`. ## s3_ignore_file_doesnt_exist {#s3_ignore_file_doesnt_exist} -Ignore ansense of file if it does not exist when reading certain keys. +Ignore absense of file if it does not exist when reading certain keys. Possible values: - 1 — `SELECT` returns empty result. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 066a551b37b..afadaa88f6d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -117,9 +117,9 @@ class IColumn; M(Bool, s3_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, hdfs_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ M(Bool, azure_throw_on_zero_files_match, false, "Throw an error, when ListObjects request cannot match any files", 0) \ - M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in S3 table engine", 0) \ - M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ - M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the reqested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ + M(Bool, s3_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in S3 table engine", 0) \ + M(Bool, hdfs_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in HDFS table engine", 0) \ + M(Bool, azure_ignore_file_doesnt_exist, false, "Return 0 rows when the requested files don't exist, instead of throwing an exception in AzureBlobStorage table engine", 0) \ M(Bool, s3_validate_request_settings, true, "Validate S3 request settings", 0) \ M(Bool, s3_disable_checksum, false, "Do not calculate a checksum when sending a file to S3. This speeds up writes by avoiding excessive processing passes on a file. It is mostly safe as the data of MergeTree tables is checksummed by ClickHouse anyway, and when S3 is accessed with HTTPS, the TLS layer already provides integrity while transferring through the network. While additional checksums on S3 give defense in depth.", 0) \ M(UInt64, s3_retry_attempts, 100, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries", 0) \ diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 1f3a4bdf6c7..dcb2af9d4d3 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -224,7 +224,7 @@ void HDFSObjectStorage::listObjects(const std::string & path, RelativePathsWithM {}})); } - if (children.size() >= max_keys) + if (max_keys && children.size() >= max_keys) break; } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index b28b1c226a7..0bd7f1ab108 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -15,12 +15,14 @@ public: using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; static constexpr auto type_name = "s3"; + static constexpr auto namespace_name = "bucket"; StorageS3Configuration() = default; StorageS3Configuration(const StorageS3Configuration & other); std::string getTypeName() const override { return type_name; } std::string getEngineName() const override { return url.storage_name; } + std::string getNamespaceType() const override { return namespace_name; } Path getPath() const override { return url.key; } void setPath(const Path & path) override { url.key = path; } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 928d49f9604..26b153ca0db 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -151,6 +151,9 @@ public: virtual std::string getTypeName() const = 0; /// Engine name: S3, HDFS, Azure. virtual std::string getEngineName() const = 0; + /// Sometimes object storages have something similar to chroot or namespace, for example + /// buckets in S3. If object storage doesn't have any namepaces return empty string. + virtual std::string getNamespaceType() const { return "namespace"; } virtual Path getPath() const = 0; virtual void setPath(const Path & path) = 0; @@ -159,8 +162,6 @@ public: virtual void setPaths(const Paths & paths) = 0; virtual String getDataSourceDescription() = 0; - /// Sometimes object storages have something similar to chroot or namespace, for example - /// buckets in S3. If object storage doesn't have any namepaces return empty string. virtual String getNamespace() const = 0; virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index e28924617e0..737f733615f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -43,7 +43,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( ObjectStoragePtr object_storage_, ConfigurationPtr configuration_, const ReadFromFormatInfo & info, - std::optional format_settings_, + const std::optional & format_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, @@ -95,7 +95,8 @@ std::shared_ptr StorageObjectStorageSourc local_context->getSettingsRef().max_threads); if (configuration->isNamespaceWithGlobs()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Expression can not have wildcards inside namespace name"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); auto settings = configuration->getQuerySettings(local_context); @@ -425,15 +426,13 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne { std::lock_guard lock(next_mutex); auto object_info = nextImplUnlocked(processor); - if (object_info) + if (first_iteration && !object_info && throw_on_zero_files_match) { - if (first_iteration) - first_iteration = false; - } - else if (first_iteration && throw_on_zero_files_match) - { - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Can not match any files"); + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, + "Can not match any files with path {}", + configuration->getPath()); } + first_iteration = false; return object_info; } @@ -456,8 +455,6 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne } new_batch = std::move(result.value()); - LOG_TEST(logger, "Batch size: {}", new_batch.size()); - for (auto it = new_batch.begin(); it != new_batch.end();) { if (!recursive && !re2::RE2::FullMatch((*it)->relative_path, *matcher)) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 08d545f9b85..9c67a125f5e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -31,7 +31,7 @@ public: ObjectStoragePtr object_storage_, ConfigurationPtr configuration, const ReadFromFormatInfo & info, - std::optional format_settings_, + const std::optional & format_settings_, ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, From a7b135ea8b8962ec4db318305391881ec1ff4ff8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 May 2024 12:42:38 +0200 Subject: [PATCH 061/158] Fix style check --- docs/en/operations/settings/settings.md | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 88e945a710c..131948eace9 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3667,7 +3667,7 @@ Default value: `0`. ## s3_ignore_file_doesnt_exist {#s3_ignore_file_doesnt_exist} -Ignore absense of file if it does not exist when reading certain keys. +Ignore absence of file if it does not exist when reading certain keys. Possible values: - 1 — `SELECT` returns empty result. diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 996f7da234a..3c72ef0f737 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -253,6 +253,7 @@ DockerHub DoubleDelta Doxygen Durre +doesnt ECMA Ecto EdgeAngle From 4c8bdad0e709b64ed045aed6092a429767370395 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 May 2024 12:54:59 +0200 Subject: [PATCH 062/158] Simplify glob iterator --- .../ObjectStorage/StorageObjectStorageCluster.cpp | 8 +++----- .../ObjectStorage/StorageObjectStorageSource.cpp | 15 +++------------ 2 files changed, 6 insertions(+), 17 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index 193894a1d44..a43d9da0fa3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -82,11 +82,9 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded( RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExtension( const ActionsDAG::Node * predicate, const ContextPtr & local_context) const { - const auto settings = configuration->getQuerySettings(local_context); - auto iterator = std::make_shared( - object_storage, configuration, predicate, virtual_columns, local_context, - nullptr, settings.list_object_keys_size, settings.throw_on_zero_files_match, - local_context->getFileProgressCallback()); + auto iterator = StorageObjectStorageSource::createFileIterator( + configuration, object_storage, /* distributed_processing */false, local_context, + predicate, virtual_columns, nullptr, local_context->getFileProgressCallback()); auto callback = std::make_shared>([iterator]() mutable -> String { diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 737f733615f..8d5df96ca6e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -407,18 +407,9 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } else { - const auto object_key = configuration_->getPath(); - auto object_metadata = object_storage->getObjectMetadata(object_key); - auto object_info = std::make_shared(object_key, object_metadata); - - object_infos.emplace_back(object_info); - if (read_keys) - read_keys->emplace_back(object_info); - - if (file_progress_callback) - file_progress_callback(FileProgress(0, object_metadata.size_bytes)); - - is_finished = true; + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Using glob iterator with path without globs is not allowed (used path: {})", + configuration->getPath()); } } From 53f5b958036d4ef3f69c3a22be96cf4c2e1b8c4a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 May 2024 13:25:44 +0200 Subject: [PATCH 063/158] Fix typo --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 131948eace9..1772a3aa861 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3719,7 +3719,7 @@ Default value: `0`. ## hdfs_ignore_file_doesnt_exist {#hdfs_ignore_file_doesnt_exist} -Ignore ansense of file if it does not exist when reading certain keys. +Ignore absence of file if it does not exist when reading certain keys. Possible values: - 1 — `SELECT` returns empty result. @@ -3739,7 +3739,7 @@ Default value: `0`. ## azure_ignore_file_doesnt_exist {#azure_ignore_file_doesnt_exist} -Ignore ansense of file if it does not exist when reading certain keys. +Ignore absence of file if it does not exist when reading certain keys. Possible values: - 1 — `SELECT` returns empty result. From 04fb84d4ade10df2a4fc9f6cb6f94ac4993d1ffd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 15 May 2024 21:57:15 +0200 Subject: [PATCH 064/158] Update src/Core/SettingsChangesHistory.h Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- src/Core/SettingsChangesHistory.h | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6edfcc129f8..e004e83355b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -91,13 +91,13 @@ static std::map sett {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"hdfs_throw_on_zero_files_match", false, false, "Throw an error, when ListObjects request cannot match any files"}, - {"azure_throw_on_zero_files_match", false, false, "Throw an error, when ListObjects request cannot match any files"}, - {"s3_validate_request_settings", true, true, "Validate S3 request settings"}, + {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, + {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, + {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageHDFS"}, - {"azure_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageAzureBlob"}, - {"s3_ignore_file_doesnt_exist", false, false, "Ignore if files does not exits and return 0 zeros for StorageS3"}, + {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, + {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, + {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, }}, From a63e846724f503607fe38b34fda37345ee8111c5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 May 2024 22:13:48 +0200 Subject: [PATCH 065/158] Review fixes --- docs/en/operations/settings/settings.md | 20 +++++++++++++++++++ .../StorageObjectStorageSink.cpp | 2 +- .../ObjectStorage/StorageObjectStorageSink.h | 2 +- src/Storages/S3Queue/S3QueueSource.cpp | 4 ++-- .../TableFunctionObjectStorage.cpp | 5 ++--- .../TableFunctionObjectStorage.h | 10 ++++++++-- .../TableFunctionObjectStorageCluster.h | 19 ++++++++++-------- 7 files changed, 45 insertions(+), 17 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1772a3aa861..3a79eb64c67 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3675,6 +3675,16 @@ Possible values: Default value: `0`. +## s3_validate_request_settings {#s3_validate_request_settings} + +Enables s3 request settings validation. + +Possible values: +- 1 — validate settings. +- 0 — do not validate settings. + +Default value: `1`. + ## hdfs_truncate_on_insert {#hdfs_truncate_on_insert} Enables or disables truncation before an insert in hdfs engine tables. If disabled, an exception will be thrown on an attempt to insert if a file in HDFS already exists. @@ -3747,6 +3757,16 @@ Possible values: Default value: `0`. +## azure_skip_empty_files {#azure_skip_empty_files} + +Enables or disables skipping empty files in S3 engine. + +Possible values: +- 0 — `SELECT` throws an exception if empty file is not compatible with requested format. +- 1 — `SELECT` returns empty result for empty file. + +Default value: `0`. + ## engine_url_skip_empty_files {#engine_url_skip_empty_files} Enables or disables skipping empty files in [URL](../../engines/table-engines/special/url.md) engine tables. diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp index 81bdeaa43a3..0a3cf19a590 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes StorageObjectStorageSink::StorageObjectStorageSink( ObjectStoragePtr object_storage, ConfigurationPtr configuration, - std::optional format_settings_, + const std::optional & format_settings_, const Block & sample_block_, ContextPtr context, const std::string & blob_path) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSink.h b/src/Storages/ObjectStorage/StorageObjectStorageSink.h index a3c8ef68cf0..45cf83d606f 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSink.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSink.h @@ -13,7 +13,7 @@ public: StorageObjectStorageSink( ObjectStoragePtr object_storage, ConfigurationPtr configuration, - std::optional format_settings_, + const std::optional & format_settings_, const Block & sample_block_, ContextPtr context, const std::string & blob_path = ""); diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 0cee94769c4..458f681d7b5 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -235,7 +235,7 @@ Chunk StorageS3QueueSource::generate() catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->key, getCurrentExceptionMessage(true)); + key_with_info->relative_path, getCurrentExceptionMessage(true)); } appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); @@ -262,7 +262,7 @@ Chunk StorageS3QueueSource::generate() catch (...) { LOG_ERROR(log, "Failed to set file {} as failed: {}", - key_with_info->key, getCurrentExceptionMessage(true)); + key_with_info->relative_path, getCurrentExceptionMessage(true)); } appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); diff --git a/src/TableFunctions/TableFunctionObjectStorage.cpp b/src/TableFunctions/TableFunctionObjectStorage.cpp index 9f16a9a0b25..550d9cc799b 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.cpp +++ b/src/TableFunctions/TableFunctionObjectStorage.cpp @@ -116,9 +116,8 @@ StoragePtr TableFunctionObjectStorage::executeImpl( columns, ConstraintsDescription{}, String{}, - /// No format_settings for table function Azure - std::nullopt, - /* distributed_processing */ false, + /* format_settings */std::nullopt, + /* distributed_processing */false, nullptr); storage->startup(); diff --git a/src/TableFunctions/TableFunctionObjectStorage.h b/src/TableFunctions/TableFunctionObjectStorage.h index bbc40cc6191..86b8f0d5e14 100644 --- a/src/TableFunctions/TableFunctionObjectStorage.h +++ b/src/TableFunctions/TableFunctionObjectStorage.h @@ -32,6 +32,7 @@ struct AzureDefinition " - storage_account_url, container_name, blobpath, account_name, account_key, format\n" " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression\n" " - storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure\n"; + static constexpr auto max_number_of_arguments = 8; }; struct S3Definition @@ -51,6 +52,7 @@ struct S3Definition " - url, access_key_id, secret_access_key, format, structure, compression_method\n" " - url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + static constexpr auto max_number_of_arguments = 8; }; struct GCSDefinition @@ -58,6 +60,7 @@ struct GCSDefinition static constexpr auto name = "gcs"; static constexpr auto storage_type_name = "GCS"; static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct COSNDefinition @@ -65,6 +68,7 @@ struct COSNDefinition static constexpr auto name = "cosn"; static constexpr auto storage_type_name = "COSN"; static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct OSSDefinition @@ -72,6 +76,7 @@ struct OSSDefinition static constexpr auto name = "oss"; static constexpr auto storage_type_name = "OSS"; static constexpr auto signature = S3Definition::signature; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments; }; struct HDFSDefinition @@ -82,6 +87,7 @@ struct HDFSDefinition " - uri, format\n" " - uri, format, structure\n" " - uri, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = 4; }; template @@ -91,7 +97,7 @@ public: static constexpr auto name = Definition::name; static constexpr auto signature = Definition::signature; - static size_t getMaxNumberOfArguments() { return 8; } + static size_t getMaxNumberOfArguments() { return Definition::max_number_of_arguments; } String getName() const override { return name; } @@ -105,7 +111,7 @@ public: bool supportsReadingSubsetOfColumns(const ContextPtr & context) override { - return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); + return configuration->format != "auto" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration->format, context); } std::unordered_set getVirtualsToCheckBeforeUsingStructureHint() const override diff --git a/src/TableFunctions/TableFunctionObjectStorageCluster.h b/src/TableFunctions/TableFunctionObjectStorageCluster.h index 76786fafe99..296791b8bda 100644 --- a/src/TableFunctions/TableFunctionObjectStorageCluster.h +++ b/src/TableFunctions/TableFunctionObjectStorageCluster.h @@ -17,17 +17,10 @@ class StorageAzureConfiguration; struct AzureClusterDefinition { - /** - * azureBlobStorageCluster(cluster_name, source, [access_key_id, secret_access_key,] format, compression_method, structure) - * A table function, which allows to process many files from Azure Blob Storage on a specific cluster - * On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks - * in Azure Blob Storage file path and dispatch each file dynamically. - * On worker node it asks initiator about next task to process, processes it. - * This is repeated until the tasks are finished. - */ static constexpr auto name = "azureBlobStorageCluster"; static constexpr auto storage_type_name = "AzureBlobStorageCluster"; static constexpr auto signature = " - cluster, connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure]"; + static constexpr auto max_number_of_arguments = AzureDefinition::max_number_of_arguments + 1; }; struct S3ClusterDefinition @@ -44,6 +37,7 @@ struct S3ClusterDefinition " - cluster, url, access_key_id, secret_access_key, format, structure, compression_method\n" " - cluster, url, access_key_id, secret_access_key, session_token, format, structure, compression_method\n" "All signatures supports optional headers (specified as `headers('name'='value', 'name2'='value2')`)"; + static constexpr auto max_number_of_arguments = S3Definition::max_number_of_arguments + 1; }; struct HDFSClusterDefinition @@ -54,8 +48,17 @@ struct HDFSClusterDefinition " - cluster_name, uri, format\n" " - cluster_name, uri, format, structure\n" " - cluster_name, uri, format, structure, compression_method\n"; + static constexpr auto max_number_of_arguments = HDFSDefinition::max_number_of_arguments + 1; }; +/** +* Class implementing s3/hdfs/azureBlobStorage)Cluster(...) table functions, +* which allow to process many files from S3/HDFS/Azure blob storage on a specific cluster. +* On initiator it creates a connection to _all_ nodes in cluster, discloses asterisks +* in file path and dispatch each file dynamically. +* On worker node it asks initiator about next task to process, processes it. +* This is repeated until the tasks are finished. +*/ template class TableFunctionObjectStorageCluster : public ITableFunctionCluster> { From 3ff2ec0a7d8d3006ccf90250cb95b6ac7c1e872e Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 May 2024 15:58:27 +0200 Subject: [PATCH 066/158] Fix segfault --- src/Storages/ObjectStorage/StorageObjectStorageSource.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 9c67a125f5e..abaf51edc4e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -62,7 +62,7 @@ protected: const std::optional format_settings; const UInt64 max_block_size; const bool need_only_count; - const ReadFromFormatInfo & read_from_format_info; + const ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; ColumnsDescription columns_desc; From b53e9eec7b6560ebb67a5d868689494a7f0ab008 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 May 2024 18:17:46 +0200 Subject: [PATCH 067/158] Support for archives (unfinished) --- src/IO/S3/URI.h | 1 + .../ObjectStorage/ReadBufferIterator.cpp | 34 ++-- .../ObjectStorage/S3/Configuration.cpp | 8 + src/Storages/ObjectStorage/S3/Configuration.h | 3 + .../ObjectStorage/StorageObjectStorage.cpp | 10 ++ .../ObjectStorage/StorageObjectStorage.h | 4 + .../StorageObjectStorageSource.cpp | 146 +++++++++++++++++- .../StorageObjectStorageSource.h | 70 ++++++++- 8 files changed, 255 insertions(+), 21 deletions(-) diff --git a/src/IO/S3/URI.h b/src/IO/S3/URI.h index c52e6bc1441..363f98c46f5 100644 --- a/src/IO/S3/URI.h +++ b/src/IO/S3/URI.h @@ -29,6 +29,7 @@ struct URI std::string key; std::string version_id; std::string storage_name; + /// Path (or path pattern) in archive if uri is an archive. std::optional archive_pattern; std::string uri_str; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 3705725ffe1..61575b0115a 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -244,22 +245,35 @@ ReadBufferIterator::Data ReadBufferIterator::next() } } - std::unique_ptr read_buffer = object_storage->readObject( - StoredObject(current_object_info->relative_path), - getContext()->getReadSettings(), - {}, - current_object_info->metadata->size_bytes); + std::unique_ptr read_buf; + CompressionMethod compression_method; + using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; + if (auto object_info_in_archive = dynamic_cast(current_object_info.get())) + { + compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); + 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(current_object_info->relative_path, configuration->compression_method); + read_buf = object_storage->readObject( + StoredObject(current_object_info->relative_path), + getContext()->getReadSettings(), + {}, + current_object_info->metadata->size_bytes); + } - if (!query_settings.skip_empty_files || !read_buffer->eof()) + if (!query_settings.skip_empty_files || !read_buf->eof()) { first = false; - read_buffer = wrapReadBufferWithCompressionMethod( - std::move(read_buffer), - chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method), + read_buf = wrapReadBufferWithCompressionMethod( + std::move(read_buf), + compression_method, static_cast(getContext()->getSettingsRef().zstd_window_log_max)); - return {std::move(read_buffer), std::nullopt, format}; + return {std::move(read_buf), std::nullopt, format}; } } } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 9fcbc6a6816..00d569fea9f 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -55,6 +55,14 @@ String StorageS3Configuration::getDataSourceDescription() return std::filesystem::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; } +std::string StorageS3Configuration::getPathInArchive() const +{ + if (url.archive_pattern.has_value()) + return url.archive_pattern.value(); + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not an archive", getPath()); +} + void StorageS3Configuration::check(ContextPtr context) const { validateNamespace(url.bucket); diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index 0bd7f1ab108..de6c02d5020 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -34,6 +34,9 @@ public: String getDataSourceDescription() override; StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; + bool isArchive() const override { return url.archive_pattern.has_value(); } + std::string getPathInArchive() const override; + void check(ContextPtr context) const override; void validateNamespace(const String & name) const override; ConfigurationPtr clone() override { return std::make_shared(*this); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index bc5b347d1e0..73e3d861cff 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -452,6 +452,16 @@ std::string StorageObjectStorage::Configuration::getPathWithoutGlobs() const return getPath().substr(0, getPath().find_first_of("*?{")); } +bool StorageObjectStorage::Configuration::isPathInArchiveWithGlobs() const +{ + return getPathInArchive().find_first_of("*?{") != std::string::npos; +} + +std::string StorageObjectStorage::Configuration::getPathInArchive() const +{ + throw Exception(ErrorCodes::LOGICAL_ERROR, "Path {} is not archive", getPath()); +} + void StorageObjectStorage::Configuration::assertInitialized() const { if (!initialized) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 26b153ca0db..7b118cb7e6b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -175,6 +175,10 @@ public: bool isNamespaceWithGlobs() const; virtual std::string getPathWithoutGlobs() const; + virtual bool isArchive() const { return false; } + bool isPathInArchiveWithGlobs() const; + virtual std::string getPathInArchive() const; + virtual void check(ContextPtr context) const; virtual void validateNamespace(const String & /* name */) const {} diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8d5df96ca6e..56905e6c29b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -100,10 +101,11 @@ std::shared_ptr StorageObjectStorageSourc auto settings = configuration->getQuerySettings(local_context); + std::unique_ptr iterator; if (configuration->isPathWithGlobs()) { /// Iterate through disclosed globs and make a source for each file - return std::make_shared( + iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, local_context, read_keys, settings.list_object_keys_size, settings.throw_on_zero_files_match, file_progress_callback); @@ -123,10 +125,17 @@ std::shared_ptr StorageObjectStorageSourc copy_configuration->setPaths(keys); } - return std::make_shared( + iterator = std::make_unique( object_storage, copy_configuration, virtual_columns, read_keys, settings.ignore_non_existent_file, file_progress_callback); } + + if (configuration->isArchive()) + { + return std::make_shared(object_storage, configuration, std::move(iterator), local_context, read_keys); + } + + return iterator; } void StorageObjectStorageSource::lazyInitialize(size_t processor) @@ -262,9 +271,20 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { - const auto compression_method = chooseCompressionMethod(object_info->relative_path, configuration->compression_method); + CompressionMethod compression_method; const auto max_parsing_threads = need_only_count ? std::optional(1) : std::nullopt; - read_buf = createReadBuffer(object_info->relative_path, object_info->metadata->size_bytes); + + if (auto object_info_in_archive = dynamic_cast(object_info.get())) + { + compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); + 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->relative_path, configuration->compression_method); + read_buf = createReadBuffer(*object_info); + } auto input_format = FormatFactory::instance().getInput( configuration->format, *read_buf, read_from_format_info.format_header, @@ -312,8 +332,10 @@ std::future StorageObjectStorageSource return create_reader_scheduler([=, this] { return createReader(processor); }, Priority{}); } -std::unique_ptr StorageObjectStorageSource::createReadBuffer(const String & key, size_t object_size) +std::unique_ptr StorageObjectStorageSource::createReadBuffer(const ObjectInfo & object_info) { + const auto & object_size = object_info.metadata->size_bytes; + auto read_settings = getContext()->getReadSettings().adjustBufferSize(object_size); read_settings.enable_filesystem_cache = false; /// FIXME: Changing this setting to default value breaks something around parquet reading @@ -333,7 +355,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const S LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); auto async_reader = object_storage->readObjects( - StoredObjects{StoredObject{key, /* local_path */ "", object_size}}, read_settings); + StoredObjects{StoredObject{object_info.relative_path, /* local_path */ "", object_size}}, read_settings); async_reader->setReadUntilEnd(); if (read_settings.remote_fs_prefetch) @@ -344,7 +366,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const S else { /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - return object_storage->readObject(StoredObject(key), read_settings); + return object_storage->readObject(StoredObject(object_info.relative_path, "", object_size), read_settings); } } @@ -609,4 +631,114 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::ReadTaskIterator return buffer[current_index]; } +static IArchiveReader::NameFilter createArchivePathFilter(const std::string & archive_pattern) +{ + auto matcher = std::make_shared(makeRegexpPatternFromGlobs(archive_pattern)); + if (!matcher->ok()) + { + throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP, + "Cannot compile regex from glob ({}): {}", + archive_pattern, matcher->error()); + } + return [matcher](const std::string & p) mutable { return re2::RE2::FullMatch(p, *matcher); }; +} + +StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive::ObjectInfoInArchive( + ObjectInfoPtr archive_object_, + const std::string & path_in_archive_, + std::shared_ptr archive_reader_) + : archive_object(archive_object_) + , path_in_archive(path_in_archive_) + , archive_reader(archive_reader_) +{ +} + +StorageObjectStorageSource::ArchiveIterator::ArchiveIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::unique_ptr archives_iterator_, + ContextPtr context_, + ObjectInfos * read_keys_) + : IIterator("ArchiveIterator") + , WithContext(context_) + , object_storage(object_storage_) + , is_path_in_archive_with_globs(configuration_->isPathInArchiveWithGlobs()) + , archives_iterator(std::move(archives_iterator_)) + , filter(is_path_in_archive_with_globs ? createArchivePathFilter(configuration_->getPathInArchive()) : IArchiveReader::NameFilter{}) + , path_in_archive(is_path_in_archive_with_globs ? "" : configuration_->getPathInArchive()) + , read_keys(read_keys_) +{ +} + +std::shared_ptr +StorageObjectStorageSource::ArchiveIterator::createArchiveReader(ObjectInfoPtr object_info) const +{ + const auto size = object_info->metadata->size_bytes; + return DB::createArchiveReader( + /* path_to_archive */object_info->relative_path, + /* archive_read_function */[=, this]() + { + StoredObject stored_object(object_info->relative_path, "", size); + return object_storage->readObject(stored_object, getContext()->getReadSettings()); + }, + /* archive_size */size); +} + +StorageObjectStorageSource::ObjectInfoPtr +StorageObjectStorageSource::ArchiveIterator::nextImpl(size_t processor) +{ + std::unique_lock lock{next_mutex}; + while (true) + { + if (filter) + { + if (!file_enumerator) + { + archive_object = archives_iterator->next(processor); + if (!archive_object) + return {}; + + archive_reader = createArchiveReader(archive_object); + file_enumerator = archive_reader->firstFile(); + if (!file_enumerator) + continue; + } + else if (!file_enumerator->nextFile()) + { + file_enumerator.reset(); + continue; + } + + path_in_archive = file_enumerator->getFileName(); + if (!filter(path_in_archive)) + continue; + } + else + { + archive_object = archives_iterator->next(processor); + if (!archive_object) + return {}; + + if (!archive_object->metadata) + archive_object->metadata = object_storage->getObjectMetadata(archive_object->relative_path); + + archive_reader = createArchiveReader(archive_object); + if (!archive_reader->fileExists(path_in_archive)) + continue; + } + + auto object_in_archive = std::make_shared(archive_object, path_in_archive, archive_reader); + + if (read_keys != nullptr) + read_keys->push_back(object_in_archive); + + return object_in_archive; + } +} + +size_t StorageObjectStorageSource::ArchiveIterator::estimatedKeysCount() +{ + return archives_iterator->estimatedKeysCount(); +} + } diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index abaf51edc4e..664aad56928 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -1,10 +1,11 @@ #pragma once +#include +#include +#include #include #include -#include -#include #include -#include +#include namespace DB @@ -25,6 +26,7 @@ public: class ReadTaskIterator; class GlobIterator; class KeysIterator; + class ArchiveIterator; StorageObjectStorageSource( String name_, @@ -109,7 +111,7 @@ protected: /// Recreate ReadBuffer and Pipeline for each file. ReaderHolder createReader(size_t processor = 0); std::future createReaderAsync(size_t processor = 0); - std::unique_ptr createReadBuffer(const String & key, size_t object_size); + std::unique_ptr createReadBuffer(const ObjectInfo & object_info); void addNumRowsToCache(const String & path, size_t num_rows); std::optional tryGetNumRowsFromCache(const ObjectInfoPtr & object_info); @@ -218,4 +220,64 @@ private: std::atomic index = 0; bool ignore_non_existent_files; }; + +/* + * An archives iterator. + * Allows to iterate files inside one or many archives. + * `archives_iterator` is an iterator which iterates over different archives. + * There are two ways to read files in archives: + * 1. When we want to read one concete file in each archive. + * In this case we go through all archives, check if this certain file + * exists within this archive and read it if it exists. + * 2. When we have a certain pattern of files we want to read in each archive. + * For this purpose we create a filter defined as IArchiveReader::NameFilter. + */ +class StorageObjectStorageSource::ArchiveIterator : public IIterator, private WithContext +{ +public: + explicit ArchiveIterator( + ObjectStoragePtr object_storage_, + ConfigurationPtr configuration_, + std::unique_ptr archives_iterator_, + ContextPtr context_, + ObjectInfos * read_keys_); + + size_t estimatedKeysCount() override; + + struct ObjectInfoInArchive : public ObjectInfo + { + ObjectInfoInArchive( + ObjectInfoPtr archive_object_, + const std::string & path_in_archive_, + std::shared_ptr archive_reader_); + + const ObjectInfoPtr archive_object; + const std::string path_in_archive; + const std::shared_ptr archive_reader; + }; + +private: + ObjectInfoPtr nextImpl(size_t processor) override; + std::shared_ptr createArchiveReader(ObjectInfoPtr object_info) const; + + const ObjectStoragePtr object_storage; + const bool is_path_in_archive_with_globs; + /// Iterator which iterates through different archives. + const std::unique_ptr archives_iterator; + /// Used when files inside archive are defined with a glob + const IArchiveReader::NameFilter filter = {}; + /// Current file inside the archive. + std::string path_in_archive = {}; + /// Read keys of files inside archives. + ObjectInfos * read_keys; + /// Object pointing to archive (NOT path within archive). + ObjectInfoPtr archive_object; + /// Reader of the archive. + std::shared_ptr archive_reader; + /// File enumerator inside the archive. + std::unique_ptr file_enumerator; + + std::mutex next_mutex; +}; + } From 53e992af4ff6c2df33f46c597498baa38c327ee3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 May 2024 11:42:28 +0000 Subject: [PATCH 068/158] Remove some unnecessary UNREACHABLEs --- programs/keeper-client/Commands.cpp | 3 ++- programs/main.cpp | 2 +- src/Access/AccessEntityIO.cpp | 3 +-- src/Access/AccessRights.cpp | 1 - src/Access/IAccessStorage.cpp | 9 +++------ .../AggregateFunctionGroupArray.cpp | 13 ++++++------- .../AggregateFunctionSequenceNextNode.cpp | 1 - src/AggregateFunctions/AggregateFunctionSum.h | 13 ++++++------- src/Common/DateLUTImpl.cpp | 1 - src/Common/IntervalKind.cpp | 10 ---------- src/Common/TargetSpecific.cpp | 2 -- src/Common/ThreadProfileEvents.cpp | 1 - src/Common/ZooKeeper/IKeeper.cpp | 2 -- src/Compression/CompressionCodecDeflateQpl.cpp | 1 - src/Compression/CompressionCodecDoubleDelta.cpp | 3 +-- src/Coordination/KeeperReconfiguration.cpp | 8 +++++++- src/Coordination/KeeperServer.cpp | 3 ++- src/Core/Field.cpp | 1 - src/Core/Field.h | 2 -- src/DataTypes/Serializations/ISerialization.cpp | 1 - src/Disks/IO/CachedOnDiskReadBufferFromFile.h | 1 - .../MetadataStorageTransactionState.cpp | 1 - src/Disks/VolumeJBOD.cpp | 2 -- src/Formats/EscapingRuleUtils.cpp | 1 - src/Functions/FunctionsRound.h | 4 ---- src/Functions/PolygonUtils.h | 2 -- .../UserDefinedSQLObjectsZooKeeperStorage.cpp | 1 - src/IO/CompressionMethod.cpp | 1 - src/IO/HadoopSnappyReadBuffer.h | 1 - src/Interpreters/AggregatedDataVariants.cpp | 8 -------- src/Interpreters/Cache/FileSegment.cpp | 1 - src/Interpreters/ComparisonGraph.cpp | 1 - src/Interpreters/FilesystemCacheLog.cpp | 1 - src/Interpreters/HashJoin.cpp | 3 --- .../InterpreterTransactionControlQuery.cpp | 1 - src/Interpreters/SetVariants.cpp | 4 ---- src/Parsers/ASTExplainQuery.h | 2 -- src/Parsers/Lexer.cpp | 4 ---- .../Formats/Impl/MsgPackRowInputFormat.cpp | 1 - src/Processors/IProcessor.cpp | 2 -- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 ------ src/Processors/QueryPlan/TotalsHavingStep.cpp | 2 -- src/Processors/Transforms/FillingTransform.cpp | 1 - .../Transforms/buildPushingToViewsChain.cpp | 2 -- src/Storages/MergeTree/BackgroundJobsAssignee.cpp | 1 - src/Storages/MergeTree/KeyCondition.cpp | 2 -- src/Storages/MergeTree/MergeTreeData.cpp | 2 -- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 -- src/Storages/WindowView/StorageWindowView.cpp | 1 - 49 files changed, 29 insertions(+), 112 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index ec5eaf5070c..38c3d4356f6 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int KEEPER_EXCEPTION; + extern const int UNEXPECTED_ZOOKEEPER_ERROR; } bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -441,7 +442,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient new_members = query->args[1].safeGet(); break; default: - UNREACHABLE(); + throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected operation: {}", operation); } auto response = client->zookeeper->reconfig(joining, leaving, new_members); diff --git a/programs/main.cpp b/programs/main.cpp index 4bb73399719..48985ea683f 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -155,8 +155,8 @@ auto instructionFailToString(InstructionFail fail) ret("AVX2"); case InstructionFail::AVX512: ret("AVX512"); +#undef ret } - UNREACHABLE(); } diff --git a/src/Access/AccessEntityIO.cpp b/src/Access/AccessEntityIO.cpp index b0dfd74c53b..1b073329296 100644 --- a/src/Access/AccessEntityIO.cpp +++ b/src/Access/AccessEntityIO.cpp @@ -144,8 +144,7 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String catch (Exception & e) { e.addMessage("Could not parse " + file_path); - e.rethrow(); - UNREACHABLE(); + throw; } } diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index c10931f554c..dd25d3e4ac0 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -258,7 +258,6 @@ namespace case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } - UNREACHABLE(); } } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index 8e51481e415..8d4e7d3073e 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -257,8 +257,7 @@ std::vector IAccessStorage::insert(const std::vector & mu } e.addMessage("After successfully inserting {}/{}: {}", successfully_inserted.size(), multiple_entities.size(), successfully_inserted_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } @@ -361,8 +360,7 @@ std::vector IAccessStorage::remove(const std::vector & ids, bool thr } e.addMessage("After successfully removing {}/{}: {}", removed_names.size(), ids.size(), removed_names_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } @@ -458,8 +456,7 @@ std::vector IAccessStorage::update(const std::vector & ids, const Up } e.addMessage("After successfully updating {}/{}: {}", names_of_updated.size(), ids.size(), names_of_updated_str); } - e.rethrow(); - UNREACHABLE(); + throw; } } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index d4fb7afcb78..930b2c6ce73 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -60,14 +60,13 @@ struct GroupArrayTrait template constexpr const char * getNameByTrait() { - if (Trait::last) + if constexpr (Trait::last) return "groupArrayLast"; - if (Trait::sampler == Sampler::NONE) - return "groupArray"; - else if (Trait::sampler == Sampler::RNG) - return "groupArraySample"; - - UNREACHABLE(); + switch (Trait::sampler) + { + case Sampler::NONE: return "groupArray"; + case Sampler::RNG: return "groupArraySample"; + } } template diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index bed10333af0..a9dd53a75e8 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -414,7 +414,6 @@ public: break; return (i == events_size) ? base - i : unmatched_idx; } - UNREACHABLE(); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index 58aaddf357a..2f23187d2ea 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -457,13 +457,12 @@ public: String getName() const override { - if constexpr (Type == AggregateFunctionTypeSum) - return "sum"; - else if constexpr (Type == AggregateFunctionTypeSumWithOverflow) - return "sumWithOverflow"; - else if constexpr (Type == AggregateFunctionTypeSumKahan) - return "sumKahan"; - UNREACHABLE(); + switch (Type) + { + case AggregateFunctionTypeSum: return "sum"; + case AggregateFunctionTypeSumWithOverflow: return "sumWithOverflow"; + case AggregateFunctionTypeSumKahan: return "sumKahan"; + } } explicit AggregateFunctionSum(const DataTypes & argument_types_) diff --git a/src/Common/DateLUTImpl.cpp b/src/Common/DateLUTImpl.cpp index 392ee64dcbf..c87d44a4b95 100644 --- a/src/Common/DateLUTImpl.cpp +++ b/src/Common/DateLUTImpl.cpp @@ -41,7 +41,6 @@ UInt8 getDayOfWeek(const cctz::civil_day & date) case cctz::weekday::saturday: return 6; case cctz::weekday::sunday: return 7; } - UNREACHABLE(); } inline cctz::time_point lookupTz(const cctz::time_zone & cctz_time_zone, const cctz::civil_day & date) diff --git a/src/Common/IntervalKind.cpp b/src/Common/IntervalKind.cpp index 22c7db504c3..1548d5cf9a5 100644 --- a/src/Common/IntervalKind.cpp +++ b/src/Common/IntervalKind.cpp @@ -34,8 +34,6 @@ Int64 IntervalKind::toAvgNanoseconds() const default: return toAvgSeconds() * NANOSECONDS_PER_SECOND; } - - UNREACHABLE(); } Int32 IntervalKind::toAvgSeconds() const @@ -54,7 +52,6 @@ Int32 IntervalKind::toAvgSeconds() const case IntervalKind::Kind::Quarter: return 7889238; /// Exactly 1/4 of a year. case IntervalKind::Kind::Year: return 31556952; /// The average length of a Gregorian year is equal to 365.2425 days } - UNREACHABLE(); } Float64 IntervalKind::toSeconds() const @@ -80,7 +77,6 @@ Float64 IntervalKind::toSeconds() const default: throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not possible to get precise number of seconds in non-precise interval"); } - UNREACHABLE(); } bool IntervalKind::isFixedLength() const @@ -99,7 +95,6 @@ bool IntervalKind::isFixedLength() const case IntervalKind::Kind::Quarter: case IntervalKind::Kind::Year: return false; } - UNREACHABLE(); } IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds) @@ -141,7 +136,6 @@ const char * IntervalKind::toKeyword() const case IntervalKind::Kind::Quarter: return "QUARTER"; case IntervalKind::Kind::Year: return "YEAR"; } - UNREACHABLE(); } @@ -161,7 +155,6 @@ const char * IntervalKind::toLowercasedKeyword() const case IntervalKind::Kind::Quarter: return "quarter"; case IntervalKind::Kind::Year: return "year"; } - UNREACHABLE(); } @@ -192,7 +185,6 @@ const char * IntervalKind::toDateDiffUnit() const case IntervalKind::Kind::Year: return "year"; } - UNREACHABLE(); } @@ -223,7 +215,6 @@ const char * IntervalKind::toNameOfFunctionToIntervalDataType() const case IntervalKind::Kind::Year: return "toIntervalYear"; } - UNREACHABLE(); } @@ -257,7 +248,6 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const case IntervalKind::Kind::Year: return "toYear"; } - UNREACHABLE(); } diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index 49f396c0926..8540c9a9986 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -54,8 +54,6 @@ String toString(TargetArch arch) case TargetArch::AMXTILE: return "amxtile"; case TargetArch::AMXINT8: return "amxint8"; } - - UNREACHABLE(); } } diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 6a63d484cd9..23b41f23bde 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -75,7 +75,6 @@ const char * TasksStatsCounters::metricsProviderString(MetricsProvider provider) case MetricsProvider::Netlink: return "netlink"; } - UNREACHABLE(); } bool TasksStatsCounters::checkIfAvailable() diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 7d2602bde1e..7cca262baca 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -146,8 +146,6 @@ const char * errorMessage(Error code) case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; case Error::ZNOTREADONLY: return "State-changing request is passed to read-only server"; } - - UNREACHABLE(); } bool isHardwareError(Error zk_return_code) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 7e0653c69f8..f1b5b24e866 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -466,7 +466,6 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so sw_codec->doDecompressData(source, source_size, dest, uncompressed_size); return; } - UNREACHABLE(); } void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests() diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index e6e8db4c699..78fdf5c627a 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -163,9 +163,8 @@ inline Int64 getMaxValueForByteSize(Int8 byte_size) case sizeof(UInt64): return std::numeric_limits::max(); default: - assert(false && "only 1, 2, 4 and 8 data sizes are supported"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "only 1, 2, 4 and 8 data sizes are supported"); } - UNREACHABLE(); } struct WriteSpec diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index e3642913a7a..a2a06f92283 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -5,6 +5,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int UNEXPECTED_ZOOKEEPER_ERROR; +} + ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) { ClusterUpdateActions out; @@ -79,7 +85,7 @@ String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateA new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); } else - UNREACHABLE(); + throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected update"); } for (const auto & item : cfg->get_servers()) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 8d21ce2ab01..b132c898be6 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -45,6 +45,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int LOGICAL_ERROR; extern const int INVALID_CONFIG_PARAMETER; + extern const int UNEXPECTED_ZOOKEEPER_ERROR; } using namespace std::chrono_literals; @@ -990,7 +991,7 @@ KeeperServer::ConfigUpdateState KeeperServer::applyConfigUpdate( raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return Accepted; } - UNREACHABLE(); + throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected action"); } ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 73f0703f21e..7207485c799 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -146,7 +146,6 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf) case Field::Types::CustomType: return Field(); } - UNREACHABLE(); } void readBinary(Array & x, ReadBuffer & buf) diff --git a/src/Core/Field.h b/src/Core/Field.h index 4424d669c4d..710614cd0a0 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -667,8 +667,6 @@ public: case Types::AggregateFunctionState: return f(field.template get()); case Types::CustomType: return f(field.template get()); } - - UNREACHABLE(); } String dump() const; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index a3a28f8091c..cd605c93f0d 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -36,7 +36,6 @@ String ISerialization::kindToString(Kind kind) case Kind::SPARSE: return "Sparse"; } - UNREACHABLE(); } ISerialization::Kind ISerialization::stringToKind(const String & str) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h index 3433698a162..cb34f7932c3 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.h +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.h @@ -140,7 +140,6 @@ private: case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE: return "REMOTE_FS_READ_AND_PUT_IN_CACHE"; } - UNREACHABLE(); } size_t first_offset = 0; diff --git a/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp b/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp index 245578b5d9e..a37f4ce7e65 100644 --- a/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageTransactionState.cpp @@ -17,7 +17,6 @@ std::string toString(MetadataStorageTransactionState state) case MetadataStorageTransactionState::PARTIALLY_ROLLED_BACK: return "PARTIALLY_ROLLED_BACK"; } - UNREACHABLE(); } } diff --git a/src/Disks/VolumeJBOD.cpp b/src/Disks/VolumeJBOD.cpp index a0c71583a22..e796ad6cdd7 100644 --- a/src/Disks/VolumeJBOD.cpp +++ b/src/Disks/VolumeJBOD.cpp @@ -112,7 +112,6 @@ DiskPtr VolumeJBOD::getDisk(size_t /* index */) const return disks_by_size.top().disk; } } - UNREACHABLE(); } ReservationPtr VolumeJBOD::reserve(UInt64 bytes) @@ -164,7 +163,6 @@ ReservationPtr VolumeJBOD::reserve(UInt64 bytes) return reservation; } } - UNREACHABLE(); } bool VolumeJBOD::areMergesAvoided() const diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 3edade639df..2fe29d8bebb 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -62,7 +62,6 @@ String escapingRuleToString(FormatSettings::EscapingRule escaping_rule) case FormatSettings::EscapingRule::Raw: return "Raw"; } - UNREACHABLE(); } void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 99f3a14dfec..233d4058f11 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -149,8 +149,6 @@ struct IntegerRoundingComputation return x; } } - - UNREACHABLE(); } static ALWAYS_INLINE T compute(T x, T scale) @@ -163,8 +161,6 @@ struct IntegerRoundingComputation case ScaleMode::Negative: return computeImpl(x, scale); } - - UNREACHABLE(); } static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral diff --git a/src/Functions/PolygonUtils.h b/src/Functions/PolygonUtils.h index c4851718da6..57f1243537d 100644 --- a/src/Functions/PolygonUtils.h +++ b/src/Functions/PolygonUtils.h @@ -381,8 +381,6 @@ bool PointInPolygonWithGrid::contains(CoordinateType x, Coordina case CellType::complexPolygon: return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]); } - - UNREACHABLE(); } diff --git a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp index 568e0b9b5d2..766d63eafb0 100644 --- a/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp +++ b/src/Functions/UserDefined/UserDefinedSQLObjectsZooKeeperStorage.cpp @@ -35,7 +35,6 @@ namespace case UserDefinedSQLObjectType::Function: return "function_"; } - UNREACHABLE(); } constexpr std::string_view sql_extension = ".sql"; diff --git a/src/IO/CompressionMethod.cpp b/src/IO/CompressionMethod.cpp index b8e1134d422..22913125e99 100644 --- a/src/IO/CompressionMethod.cpp +++ b/src/IO/CompressionMethod.cpp @@ -52,7 +52,6 @@ std::string toContentEncodingName(CompressionMethod method) case CompressionMethod::None: return ""; } - UNREACHABLE(); } CompressionMethod chooseHTTPCompressionMethod(const std::string & list) diff --git a/src/IO/HadoopSnappyReadBuffer.h b/src/IO/HadoopSnappyReadBuffer.h index 73e52f2c503..bbbb84dd6dd 100644 --- a/src/IO/HadoopSnappyReadBuffer.h +++ b/src/IO/HadoopSnappyReadBuffer.h @@ -88,7 +88,6 @@ public: case Status::TOO_LARGE_COMPRESSED_BLOCK: return "TOO_LARGE_COMPRESSED_BLOCK"; } - UNREACHABLE(); } explicit HadoopSnappyReadBuffer( diff --git a/src/Interpreters/AggregatedDataVariants.cpp b/src/Interpreters/AggregatedDataVariants.cpp index 87cfdda5948..8f82f15248f 100644 --- a/src/Interpreters/AggregatedDataVariants.cpp +++ b/src/Interpreters/AggregatedDataVariants.cpp @@ -117,8 +117,6 @@ size_t AggregatedDataVariants::size() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t AggregatedDataVariants::sizeWithoutOverflowRow() const @@ -136,8 +134,6 @@ size_t AggregatedDataVariants::sizeWithoutOverflowRow() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } const char * AggregatedDataVariants::getMethodName() const @@ -155,8 +151,6 @@ const char * AggregatedDataVariants::getMethodName() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } bool AggregatedDataVariants::isTwoLevel() const @@ -174,8 +168,6 @@ bool AggregatedDataVariants::isTwoLevel() const APPLY_FOR_AGGREGATED_VARIANTS(M) #undef M } - - UNREACHABLE(); } bool AggregatedDataVariants::isConvertibleToTwoLevel() const diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 9459029dc4c..61a356fa3c3 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -799,7 +799,6 @@ String FileSegment::stateToString(FileSegment::State state) case FileSegment::State::DETACHED: return "DETACHED"; } - UNREACHABLE(); } bool FileSegment::assertCorrectness() const diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 4eacbae7a30..d53ff4b0227 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -309,7 +309,6 @@ ComparisonGraphCompareResult ComparisonGraph::pathToCompareResult(Path pat case Path::GREATER: return inverse ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::GREATER; case Path::GREATER_OR_EQUAL: return inverse ? ComparisonGraphCompareResult::LESS_OR_EQUAL : ComparisonGraphCompareResult::GREATER_OR_EQUAL; } - UNREACHABLE(); } template diff --git a/src/Interpreters/FilesystemCacheLog.cpp b/src/Interpreters/FilesystemCacheLog.cpp index 80fe1c3a8ef..aa489351a98 100644 --- a/src/Interpreters/FilesystemCacheLog.cpp +++ b/src/Interpreters/FilesystemCacheLog.cpp @@ -26,7 +26,6 @@ static String typeToString(FilesystemCacheLogElement::CacheType type) case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE: return "WRITE_THROUGH_CACHE"; } - UNREACHABLE(); } ColumnsDescription FilesystemCacheLogElement::getColumnsDescription() diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 3a21c13db5e..75da8bbc3e7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -705,7 +705,6 @@ namespace APPLY_FOR_JOIN_VARIANTS(M) #undef M } - UNREACHABLE(); } } @@ -2641,8 +2640,6 @@ private: default: throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type); } - - UNREACHABLE(); } template diff --git a/src/Interpreters/InterpreterTransactionControlQuery.cpp b/src/Interpreters/InterpreterTransactionControlQuery.cpp index d31ace758c4..13872fbe3f5 100644 --- a/src/Interpreters/InterpreterTransactionControlQuery.cpp +++ b/src/Interpreters/InterpreterTransactionControlQuery.cpp @@ -33,7 +33,6 @@ BlockIO InterpreterTransactionControlQuery::execute() case ASTTransactionControl::SET_SNAPSHOT: return executeSetSnapshot(session_context, tcl.snapshot); } - UNREACHABLE(); } BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context) diff --git a/src/Interpreters/SetVariants.cpp b/src/Interpreters/SetVariants.cpp index 64796a013f1..c600d096160 100644 --- a/src/Interpreters/SetVariants.cpp +++ b/src/Interpreters/SetVariants.cpp @@ -41,8 +41,6 @@ size_t SetVariantsTemplate::getTotalRowCount() const APPLY_FOR_SET_VARIANTS(M) #undef M } - - UNREACHABLE(); } template @@ -57,8 +55,6 @@ size_t SetVariantsTemplate::getTotalByteCount() const APPLY_FOR_SET_VARIANTS(M) #undef M } - - UNREACHABLE(); } template diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 701bde8cebd..eb095b5dbbc 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -40,8 +40,6 @@ public: case TableOverride: return "EXPLAIN TABLE OVERRIDE"; case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION"; } - - UNREACHABLE(); } static ExplainKind fromString(const String & str) diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 9ac6e623803..30717550713 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -41,8 +41,6 @@ Token quotedString(const char *& pos, const char * const token_begin, const char ++pos; continue; } - - UNREACHABLE(); } } @@ -538,8 +536,6 @@ const char * getTokenName(TokenType type) APPLY_FOR_TOKENS(M) #undef M } - - UNREACHABLE(); } diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 98cbdeaaa4b..6b7f1f5206c 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -657,7 +657,6 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {:x} is not supported", object_ext.type()); } } - UNREACHABLE(); } std::optional MsgPackSchemaReader::readRowAndGetDataTypes() diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index 8b160153733..5ab5e5277aa 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -36,8 +36,6 @@ std::string IProcessor::statusToName(Status status) case Status::ExpandPipeline: return "ExpandPipeline"; } - - UNREACHABLE(); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e523a2c243c..2f7927681aa 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1136,8 +1136,6 @@ static void addMergingFinal( return std::make_shared(header, num_outputs, sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now); } - - UNREACHABLE(); }; pipe.addTransform(get_merging_processor()); @@ -2143,8 +2141,6 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type) case ReadFromMergeTree::IndexType::Skip: return "Skip"; } - - UNREACHABLE(); } static const char * readTypeToString(ReadFromMergeTree::ReadType type) @@ -2160,8 +2156,6 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type) case ReadFromMergeTree::ReadType::ParallelReplicas: return "Parallel"; } - - UNREACHABLE(); } void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index d1bd70fd0b2..ac5e144bf4a 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -86,8 +86,6 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr case TotalsMode::AFTER_HAVING_AUTO: return "after_having_auto threshold " + std::to_string(auto_include_threshold); } - - UNREACHABLE(); } void TotalsHavingStep::describeActions(FormatSettings & settings) const diff --git a/src/Processors/Transforms/FillingTransform.cpp b/src/Processors/Transforms/FillingTransform.cpp index 05fd2a7254f..bb38c3e1dc5 100644 --- a/src/Processors/Transforms/FillingTransform.cpp +++ b/src/Processors/Transforms/FillingTransform.cpp @@ -67,7 +67,6 @@ static FillColumnDescription::StepFunction getStepFunction( FOR_EACH_INTERVAL_KIND(DECLARE_CASE) #undef DECLARE_CASE } - UNREACHABLE(); } static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 5e8ecdca95e..20977b801d3 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -897,8 +897,6 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St { return std::current_exception(); } - - UNREACHABLE(); } void FinalizingViewsTransform::work() diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 56a4378cf9a..0a69bf1109f 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -93,7 +93,6 @@ String BackgroundJobsAssignee::toString(Type type) case Type::Moving: return "Moving"; } - UNREACHABLE(); } void BackgroundJobsAssignee::start() diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 849240502e4..dbc98404569 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -2957,8 +2957,6 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri case ALWAYS_TRUE: return "true"; } - - UNREACHABLE(); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f448a9a820d..6b6adf56cd2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1175,8 +1175,6 @@ String MergeTreeData::MergingParams::getModeName() const case Graphite: return "Graphite"; case VersionedCollapsing: return "VersionedCollapsing"; } - - UNREACHABLE(); } Int64 MergeTreeData::getMaxBlockNumber() const diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index daa163d741c..395d27558f3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -360,8 +360,6 @@ Block MergeTreeDataWriter::mergeBlock( return std::make_shared( block, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, merging_params.graphite_params, time(nullptr)); } - - UNREACHABLE(); }; auto merging_algorithm = get_merging_algorithm(); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a9ec1f6c694..4e11787cecf 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -297,7 +297,6 @@ namespace CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } class AddingAggregatedChunkInfoTransform : public ISimpleTransform From e560bd8a1a9c57640af1303a95f0a81d864c75e3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 May 2024 14:37:47 +0000 Subject: [PATCH 069/158] Incorporate review feedback --- src/Access/AccessRights.cpp | 1 + src/AggregateFunctions/AggregateFunctionSum.h | 12 ++++++------ src/Compression/CompressionCodecDoubleDelta.cpp | 4 ++-- src/Coordination/KeeperReconfiguration.cpp | 4 ++-- src/Coordination/KeeperServer.cpp | 2 +- src/Core/Field.cpp | 1 + src/Functions/FunctionsTimeWindow.cpp | 2 -- src/Parsers/Lexer.cpp | 2 ++ 8 files changed, 15 insertions(+), 13 deletions(-) diff --git a/src/Access/AccessRights.cpp b/src/Access/AccessRights.cpp index dd25d3e4ac0..2127f4ada70 100644 --- a/src/Access/AccessRights.cpp +++ b/src/Access/AccessRights.cpp @@ -258,6 +258,7 @@ namespace case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel(); case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel(); } + chassert(false); } } diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index 2f23187d2ea..2ce03c530c2 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -457,12 +457,12 @@ public: String getName() const override { - switch (Type) - { - case AggregateFunctionTypeSum: return "sum"; - case AggregateFunctionTypeSumWithOverflow: return "sumWithOverflow"; - case AggregateFunctionTypeSumKahan: return "sumKahan"; - } + if constexpr (Type == AggregateFunctionTypeSum) + return "sum"; + else if constexpr (Type == AggregateFunctionTypeSumWithOverflow) + return "sumWithOverflow"; + else if constexpr (Type == AggregateFunctionTypeSumKahan) + return "sumKahan"; } explicit AggregateFunctionSum(const DataTypes & argument_types_) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 78fdf5c627a..443b9d33532 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -142,9 +142,9 @@ namespace ErrorCodes { extern const int CANNOT_COMPRESS; extern const int CANNOT_DECOMPRESS; - extern const int BAD_ARGUMENTS; extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE; extern const int ILLEGAL_CODEC_PARAMETER; + extern const int LOGICAL_ERROR; } namespace @@ -163,7 +163,7 @@ inline Int64 getMaxValueForByteSize(Int8 byte_size) case sizeof(UInt64): return std::numeric_limits::max(); default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "only 1, 2, 4 and 8 data sizes are supported"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "only 1, 2, 4 and 8 data sizes are supported"); } } diff --git a/src/Coordination/KeeperReconfiguration.cpp b/src/Coordination/KeeperReconfiguration.cpp index a2a06f92283..05211af6704 100644 --- a/src/Coordination/KeeperReconfiguration.cpp +++ b/src/Coordination/KeeperReconfiguration.cpp @@ -8,7 +8,7 @@ namespace DB namespace ErrorCodes { - extern const int UNEXPECTED_ZOOKEEPER_ERROR; + extern const int LOGICAL_ERROR; } ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining) @@ -85,7 +85,7 @@ String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateA new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)}); } else - throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected update"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected update"); } for (const auto & item : cfg->get_servers()) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b132c898be6..953072c5b0e 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -991,7 +991,7 @@ KeeperServer::ConfigUpdateState KeeperServer::applyConfigUpdate( raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return Accepted; } - throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected action"); + chassert(false); } ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) diff --git a/src/Core/Field.cpp b/src/Core/Field.cpp index 7207485c799..73f0703f21e 100644 --- a/src/Core/Field.cpp +++ b/src/Core/Field.cpp @@ -146,6 +146,7 @@ inline Field getBinaryValue(UInt8 type, ReadBuffer & buf) case Field::Types::CustomType: return Field(); } + UNREACHABLE(); } void readBinary(Array & x, ReadBuffer & buf) diff --git a/src/Functions/FunctionsTimeWindow.cpp b/src/Functions/FunctionsTimeWindow.cpp index 1c9f28c9724..f93a885ee65 100644 --- a/src/Functions/FunctionsTimeWindow.cpp +++ b/src/Functions/FunctionsTimeWindow.cpp @@ -232,7 +232,6 @@ struct TimeWindowImpl default: throw Exception(ErrorCodes::SYNTAX_ERROR, "Fraction seconds are unsupported by windows yet"); } - UNREACHABLE(); } template @@ -422,7 +421,6 @@ struct TimeWindowImpl default: throw Exception(ErrorCodes::SYNTAX_ERROR, "Fraction seconds are unsupported by windows yet"); } - UNREACHABLE(); } template diff --git a/src/Parsers/Lexer.cpp b/src/Parsers/Lexer.cpp index 30717550713..d669c8a4690 100644 --- a/src/Parsers/Lexer.cpp +++ b/src/Parsers/Lexer.cpp @@ -41,6 +41,8 @@ Token quotedString(const char *& pos, const char * const token_begin, const char ++pos; continue; } + + chassert(false); } } From f266bdb88e1891e484add0431e9e5ca56c963635 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 17 May 2024 14:44:17 +0000 Subject: [PATCH 070/158] Fix more places --- src/Functions/FunctionsRound.h | 4 ---- src/Interpreters/HashJoin.h | 6 ------ .../MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 2 -- src/Storages/WindowView/StorageWindowView.cpp | 2 -- 4 files changed, 14 deletions(-) diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 233d4058f11..dde57e8320d 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -243,8 +243,6 @@ inline float roundWithMode(float x, RoundingMode mode) case RoundingMode::Ceil: return ceilf(x); case RoundingMode::Trunc: return truncf(x); } - - UNREACHABLE(); } inline double roundWithMode(double x, RoundingMode mode) @@ -256,8 +254,6 @@ inline double roundWithMode(double x, RoundingMode mode) case RoundingMode::Ceil: return ceil(x); case RoundingMode::Trunc: return trunc(x); } - - UNREACHABLE(); } template diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 86db8943926..a0996556f9a 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -322,8 +322,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t getTotalByteCountImpl(Type which) const @@ -338,8 +336,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } size_t getBufferSizeInCells(Type which) const @@ -354,8 +350,6 @@ public: APPLY_FOR_JOIN_VARIANTS(M) #undef M } - - UNREACHABLE(); } /// NOLINTEND(bugprone-macro-parentheses) }; diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 78fcfabb704..4228d7b70b6 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -616,8 +616,6 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st } } } - - UNREACHABLE(); } void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 4e11787cecf..8bca1c97aad 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -919,7 +919,6 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) @@ -947,7 +946,6 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } - UNREACHABLE(); } void StorageWindowView::addFireSignal(std::set & signals) From d964b4b78667a1437dd74836432828c5dda1be7e Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 May 2024 16:50:38 +0200 Subject: [PATCH 071/158] Finish archives related changes --- src/Disks/ObjectStorages/IObjectStorage.h | 6 +++ .../ObjectStorages/S3/S3ObjectStorage.cpp | 11 ++++- .../ObjectStorage/ReadBufferIterator.cpp | 40 ++++++++++++------- .../ObjectStorage/StorageObjectStorage.cpp | 7 +++- .../StorageObjectStorageCluster.cpp | 2 +- .../StorageObjectStorageSource.cpp | 37 +++++++++-------- .../StorageObjectStorageSource.h | 19 ++++++++- src/Storages/S3Queue/S3QueueSource.h | 2 +- 8 files changed, 88 insertions(+), 36 deletions(-) diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 43c7cf19adf..5724ae8929c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -37,6 +37,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } class ReadBufferFromFileBase; @@ -64,6 +65,11 @@ struct RelativePathWithMetadata {} virtual ~RelativePathWithMetadata() = default; + + virtual std::string getFileName() const { return std::filesystem::path(relative_path).filename(); } + virtual std::string getPath() const { return relative_path; } + virtual bool isArchive() const { return false; } + virtual std::string getPathToArchive() const { throw Exception(ErrorCodes::LOGICAL_ERROR, "Not an archive"); } }; struct ObjectKeyWithMetadata diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index c24874d0a94..983bb1834b8 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -457,7 +457,16 @@ std::optional S3ObjectStorage::tryGetObjectMetadata(const std::s ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const { auto settings_ptr = s3_settings.get(); - auto object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); + S3::ObjectInfo object_info; + try + { + object_info = S3::getObjectInfo(*client.get(), uri.bucket, path, {}, settings_ptr->request_settings, /* with_metadata= */ true); + } + catch (DB::Exception & e) + { + e.addMessage("while reading " + path); + throw; + } ObjectMetadata result; result.size_bytes = object_info.size; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 61575b0115a..e065de16e55 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -50,7 +50,7 @@ SchemaCache::Keys ReadBufferIterator::getKeysForSchemaCache() const std::back_inserter(sources), [&](const auto & elem) { - return std::filesystem::path(configuration->getDataSourceDescription()) / elem->relative_path; + return std::filesystem::path(configuration->getDataSourceDescription()) / elem->getPath(); }); return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); } @@ -67,8 +67,9 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( const auto & object_info = (*it); auto get_last_mod_time = [&] -> std::optional { + const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); if (!object_info->metadata) - object_info->metadata = object_storage->tryGetObjectMetadata(object_info->relative_path); + object_info->metadata = object_storage->tryGetObjectMetadata(path); return object_info->metadata ? std::optional(object_info->metadata->last_modified.epochTime()) @@ -77,7 +78,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( if (format) { - auto cache_key = getKeyForSchemaCache(object_info->relative_path, *format); + auto cache_key = getKeyForSchemaCache(object_info->getPath(), *format); if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) return columns; } @@ -88,7 +89,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( /// If we have such entry for some format, we can use this format to read the file. for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) { - auto cache_key = getKeyForSchemaCache(object_info->relative_path, format_name); + auto cache_key = getKeyForSchemaCache(object_info->getPath(), format_name); if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) { /// Now format is known. It should be the same for all files. @@ -105,7 +106,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) { if (query_settings.schema_inference_use_cache) - schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->relative_path, *format), num_rows); + schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->getPath(), *format), num_rows); } void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) @@ -113,7 +114,7 @@ void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) if (query_settings.schema_inference_use_cache && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) { - schema_cache.addColumns(getKeyForSchemaCache(current_object_info->relative_path, *format), columns); + schema_cache.addColumns(getKeyForSchemaCache(current_object_info->getPath(), *format), columns); } } @@ -134,7 +135,7 @@ void ReadBufferIterator::setFormatName(const String & format_name) String ReadBufferIterator::getLastFileName() const { if (current_object_info) - return current_object_info->relative_path; + return current_object_info->getFileName(); else return ""; } @@ -142,9 +143,13 @@ String ReadBufferIterator::getLastFileName() const std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() { auto context = getContext(); - auto impl = object_storage->readObject(StoredObject(current_object_info->relative_path), context->getReadSettings()); - const auto compression_method = chooseCompressionMethod(current_object_info->relative_path, configuration->compression_method); + + const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); + auto impl = object_storage->readObject(StoredObject(), context->getReadSettings()); + + const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); const auto zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); + return wrapReadBufferWithCompressionMethod(std::move(impl), compression_method, zstd_window_log_max); } @@ -158,7 +163,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() { for (const auto & object_info : read_keys) { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->relative_path)) + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(object_info->getFileName())) { format = format_from_file_name; break; @@ -170,7 +175,9 @@ ReadBufferIterator::Data ReadBufferIterator::next() if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT) { if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end())) + { return {nullptr, cached_columns, format}; + } } } @@ -178,7 +185,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() { current_object_info = file_iterator->next(0); - if (!current_object_info || current_object_info->relative_path.empty()) + if (!current_object_info) { if (first) { @@ -203,6 +210,9 @@ ReadBufferIterator::Data ReadBufferIterator::next() return {nullptr, std::nullopt, format}; } + const auto filename = current_object_info->getFileName(); + chassert(!filename.empty()); + /// file iterator could get new keys after new iteration if (read_keys.size() > prev_read_keys_size) { @@ -211,7 +221,7 @@ ReadBufferIterator::Data ReadBufferIterator::next() { for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it) { - if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->relative_path)) + if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->getFileName())) { format = format_from_file_name; break; @@ -250,15 +260,15 @@ ReadBufferIterator::Data ReadBufferIterator::next() using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; if (auto object_info_in_archive = dynamic_cast(current_object_info.get())) { - compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); + compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); 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(current_object_info->relative_path, configuration->compression_method); + compression_method = chooseCompressionMethod(filename, configuration->compression_method); read_buf = object_storage->readObject( - StoredObject(current_object_info->relative_path), + StoredObject(current_object_info->getPath()), getContext()->getReadSettings(), {}, current_object_info->metadata->size_bytes); diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 73e3d861cff..c45752c10f5 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -403,7 +403,12 @@ void StorageObjectStorage::Configuration::initialize( configuration.fromAST(engine_args, local_context, with_table_structure); if (configuration.format == "auto") - configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.getPath()).value_or("auto"); + { + configuration.format = FormatFactory::instance().tryGetFormatFromFileName( + configuration.isArchive() + ? configuration.getPathInArchive() + : configuration.getPath()).value_or("auto"); + } else FormatFactory::instance().checkFormatName(configuration.format); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp index a43d9da0fa3..78f568d8ae2 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.cpp @@ -90,7 +90,7 @@ RemoteQueryExecutor::Extension StorageObjectStorageCluster::getTaskIteratorExten { auto object_info = iterator->next(0); if (object_info) - return object_info->relative_path; + return object_info->getPath(); else return ""; }); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 56905e6c29b..d3b67876224 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -100,6 +100,7 @@ std::shared_ptr StorageObjectStorageSourc "Expression can not have wildcards inside {} name", configuration->getNamespaceType()); auto settings = configuration->getQuerySettings(local_context); + const bool is_archive = configuration->isArchive(); std::unique_ptr iterator; if (configuration->isPathWithGlobs()) @@ -107,7 +108,7 @@ std::shared_ptr StorageObjectStorageSourc /// Iterate through disclosed globs and make a source for each file iterator = std::make_unique( object_storage, configuration, predicate, virtual_columns, - local_context, read_keys, settings.list_object_keys_size, + local_context, is_archive ? nullptr : read_keys, settings.list_object_keys_size, settings.throw_on_zero_files_match, file_progress_callback); } else @@ -126,11 +127,11 @@ std::shared_ptr StorageObjectStorageSourc } iterator = std::make_unique( - object_storage, copy_configuration, virtual_columns, read_keys, + object_storage, copy_configuration, virtual_columns, is_archive ? nullptr : read_keys, settings.ignore_non_existent_file, file_progress_callback); } - if (configuration->isArchive()) + if (is_archive) { return std::make_shared(object_storage, configuration, std::move(iterator), local_context, read_keys); } @@ -175,12 +176,13 @@ Chunk StorageObjectStorageSource::generate() progress(num_rows, chunk_size ? chunk_size : chunk.bytes()); const auto & object_info = reader.getObjectInfo(); + const auto & filename = object_info.getFileName(); chassert(object_info.metadata); VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, fs::path(configuration->getNamespace()) / reader.getRelativePath(), - object_info.metadata->size_bytes); + object_info.metadata->size_bytes, &filename); return chunk; } @@ -219,7 +221,7 @@ void StorageObjectStorageSource::addNumRowsToCache(const String & path, size_t n std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfoPtr & object_info) { const auto cache_key = getKeyForSchemaCache( - fs::path(configuration->getDataSourceDescription()) / object_info->relative_path, + fs::path(configuration->getDataSourceDescription()) / object_info->getPath(), configuration->format, format_settings, getContext()); @@ -242,11 +244,14 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade { object_info = file_iterator->next(processor); - if (!object_info || object_info->relative_path.empty()) + if (!object_info || object_info->getFileName().empty()) return {}; if (!object_info->metadata) - object_info->metadata = object_storage->getObjectMetadata(object_info->relative_path); + { + const auto & path = object_info->isArchive() ? object_info->getPathToArchive() : object_info->getPath(); + object_info->metadata = object_storage->getObjectMetadata(path); + } } while (query_settings.skip_empty_files && object_info->metadata->size_bytes == 0); @@ -282,7 +287,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } else { - compression_method = chooseCompressionMethod(object_info->relative_path, configuration->compression_method); + compression_method = chooseCompressionMethod(object_info->getFileName(), configuration->compression_method); read_buf = createReadBuffer(*object_info); } @@ -355,7 +360,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const O LOG_TRACE(log, "Downloading object of size {} with initial prefetch", object_size); auto async_reader = object_storage->readObjects( - StoredObjects{StoredObject{object_info.relative_path, /* local_path */ "", object_size}}, read_settings); + StoredObjects{StoredObject{object_info.getPath(), /* local_path */ "", object_size}}, read_settings); async_reader->setReadUntilEnd(); if (read_settings.remote_fs_prefetch) @@ -366,7 +371,7 @@ std::unique_ptr StorageObjectStorageSource::createReadBuffer(const O else { /// FIXME: this is inconsistent that readObject always reads synchronously ignoring read_method setting. - return object_storage->readObject(StoredObject(object_info.relative_path, "", object_size), read_settings); + return object_storage->readObject(StoredObject(object_info.getPath(), "", object_size), read_settings); } } @@ -381,7 +386,7 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::IIterator::next( if (object_info) { - LOG_TEST(logger, "Next key: {}", object_info->relative_path); + LOG_TEST(logger, "Next key: {}", object_info->getFileName()); } return object_info; @@ -470,7 +475,7 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne new_batch = std::move(result.value()); for (auto it = new_batch.begin(); it != new_batch.end();) { - if (!recursive && !re2::RE2::FullMatch((*it)->relative_path, *matcher)) + if (!recursive && !re2::RE2::FullMatch((*it)->getPath(), *matcher)) it = new_batch.erase(it); else ++it; @@ -487,7 +492,7 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne for (const auto & object_info : new_batch) { chassert(object_info); - paths.push_back(fs::path(configuration->getNamespace()) / object_info->relative_path); + paths.push_back(fs::path(configuration->getNamespace()) / object_info->getPath()); } VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); @@ -675,10 +680,10 @@ StorageObjectStorageSource::ArchiveIterator::createArchiveReader(ObjectInfoPtr o { const auto size = object_info->metadata->size_bytes; return DB::createArchiveReader( - /* path_to_archive */object_info->relative_path, + /* path_to_archive */object_info->getPath(), /* archive_read_function */[=, this]() { - StoredObject stored_object(object_info->relative_path, "", size); + StoredObject stored_object(object_info->getPath(), "", size); return object_storage->readObject(stored_object, getContext()->getReadSettings()); }, /* archive_size */size); @@ -720,7 +725,7 @@ StorageObjectStorageSource::ArchiveIterator::nextImpl(size_t processor) return {}; if (!archive_object->metadata) - archive_object->metadata = object_storage->getObjectMetadata(archive_object->relative_path); + archive_object->metadata = object_storage->getObjectMetadata(archive_object->getPath()); archive_reader = createArchiveReader(archive_object); if (!archive_reader->fileExists(path_in_archive)) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 664aad56928..fb0ad3e32f1 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -92,7 +92,7 @@ protected: PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - const String & getRelativePath() const { return object_info->relative_path; } + std::string getRelativePath() const { return object_info->getPath(); } const ObjectInfo & getObjectInfo() const { return *object_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } @@ -251,6 +251,23 @@ public: const std::string & path_in_archive_, std::shared_ptr archive_reader_); + std::string getFileName() const override + { + return path_in_archive; + } + + std::string getPath() const override + { + return archive_object->getPath() + "::" + path_in_archive; + } + + std::string getPathToArchive() const override + { + return archive_object->getPath(); + } + + bool isArchive() const override { return true; } + const ObjectInfoPtr archive_object; const std::string path_in_archive; const std::shared_ptr archive_reader; diff --git a/src/Storages/S3Queue/S3QueueSource.h b/src/Storages/S3Queue/S3QueueSource.h index fdeed8d46d2..663577e055b 100644 --- a/src/Storages/S3Queue/S3QueueSource.h +++ b/src/Storages/S3Queue/S3QueueSource.h @@ -29,7 +29,7 @@ public: using FileStatusPtr = S3QueueFilesMetadata::FileStatusPtr; using ReaderHolder = StorageObjectStorageSource::ReaderHolder; using Metadata = S3QueueFilesMetadata; - using ObjectInfo = RelativePathWithMetadata; + using ObjectInfo = StorageObjectStorageSource::ObjectInfo; using ObjectInfoPtr = std::shared_ptr; using ObjectInfos = std::vector; From 4909c3ea2393c66226c23cd03847f1c5e5b05ff7 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Fri, 17 May 2024 18:24:21 +0200 Subject: [PATCH 072/158] Cleanups --- src/Storages/MergeTree/IMergeTreeDataPart.h | 11 ------ .../MergeTree/IMergeTreeDataPartWriter.cpp | 7 ---- .../MergeTree/IMergeTreeDataPartWriter.h | 39 ++++++------------- .../MergeTree/IMergedBlockOutputStream.cpp | 8 +--- .../MergeTree/IMergedBlockOutputStream.h | 10 ++--- src/Storages/MergeTree/MergeTask.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 29 +++++++------- .../MergeTree/MergeTreeDataPartCompact.h | 9 ----- .../MergeTree/MergeTreeDataPartWide.cpp | 15 ++++--- .../MergeTree/MergeTreeDataPartWide.h | 9 ----- .../MergeTreeDataPartWriterCompact.cpp | 18 ++++----- .../MergeTreeDataPartWriterCompact.h | 6 +-- .../MergeTreeDataPartWriterOnDisk.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 13 ++----- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 29 ++++++-------- .../MergeTree/MergeTreeDataPartWriterWide.h | 6 +-- src/Storages/MergeTree/MergeTreeIOSettings.h | 2 +- src/Storages/MergeTree/MergeTreePartition.cpp | 5 +-- src/Storages/MergeTree/MergeTreePartition.h | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 1 + .../MergedColumnOnlyOutputStream.cpp | 9 ++--- src/Storages/MergeTree/MutateTask.cpp | 1 + 22 files changed, 76 insertions(+), 159 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 4ec5b3f5f8a..091a7ceb5bd 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -43,7 +43,6 @@ class IReservation; using ReservationPtr = std::unique_ptr; class IMergeTreeReader; -class IMergeTreeDataPartWriter; class MarkCache; class UncompressedCache; class MergeTreeTransaction; @@ -74,7 +73,6 @@ public: using VirtualFields = std::unordered_map; using MergeTreeReaderPtr = std::unique_ptr; -// using MergeTreeWriterPtr = std::unique_ptr; using ColumnSizeByName = std::unordered_map; using NameToNumber = std::unordered_map; @@ -106,15 +104,6 @@ public: const ValueSizeMap & avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0; -//// virtual MergeTreeWriterPtr getWriter( -//// const NamesAndTypesList & columns_list, -//// const StorageMetadataPtr & metadata_snapshot, -//// const std::vector & indices_to_recalc, -//// const Statistics & stats_to_recalc_, -//// const CompressionCodecPtr & default_codec_, -//// const MergeTreeWriterSettings & writer_settings, -//// const MergeTreeIndexGranularity & computed_index_granularity) = 0; - // TODO: remove? virtual bool isStoredOnDisk() const = 0; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index b46fbc5fc9e..e01572715d6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -1,5 +1,4 @@ #include -#include "Storages/MergeTree/MergeTreeSettings.h" namespace DB { @@ -46,12 +45,10 @@ Block permuteBlockIfNeeded(const Block & block, const IColumn::Permutation * per } IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( -// const MergeTreeMutableDataPartPtr & data_part_, const String & data_part_name_, const SerializationByName & serializations_, MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, @@ -61,7 +58,6 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( , serializations(serializations_) , data_part_storage(data_part_storage_) , index_granularity_info(index_granularity_info_) - , storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) , columns_list(columns_list_) @@ -117,7 +113,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, @@ -134,7 +129,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, @@ -153,7 +147,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 6854668a01e..3245a23339b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -1,14 +1,12 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include "Storages/MergeTree/MergeTreeDataPartType.h" -#include "Storages/MergeTree/MergeTreeSettings.h" +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -24,15 +22,11 @@ class IMergeTreeDataPartWriter : private boost::noncopyable { public: IMergeTreeDataPartWriter( -// const MergeTreeMutableDataPartPtr & data_part_, - const String & data_part_name_, const SerializationByName & serializations_, MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_, @@ -42,7 +36,7 @@ public: virtual void write(const Block & block, const IColumn::Permutation * permutation) = 0; - virtual void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) = 0; + virtual void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) = 0; virtual void finish(bool sync) = 0; @@ -56,21 +50,12 @@ protected: IDataPartStorage & getDataPartStorage() { return *data_part_storage; } - -// const MergeTreeMutableDataPartPtr data_part; // TODO: remove - /// Serializations for every columns and subcolumns by their names. - String data_part_name; - SerializationByName serializations; + const String data_part_name; + const SerializationByName serializations; MutableDataPartStoragePtr data_part_storage; - MergeTreeIndexGranularityInfo index_granularity_info; - - -// const MergeTreeData & storage; // TODO: remove - + const MergeTreeIndexGranularityInfo index_granularity_info; const MergeTreeSettingsPtr storage_settings; - - const StorageMetadataPtr metadata_snapshot; const NamesAndTypesList columns_list; const MergeTreeWriterSettings settings; @@ -90,7 +75,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, @@ -100,5 +84,4 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity); - } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index f99adf7c4db..89c813ab233 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -2,30 +2,26 @@ #include #include #include -#include "Storages/MergeTree/IDataPartStorage.h" -#include "Storages/StorageSet.h" namespace DB { IMergedBlockOutputStream::IMergedBlockOutputStream( -// const MergeTreeMutableDataPartPtr & data_part, const MergeTreeSettingsPtr & storage_settings_, MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, const NamesAndTypesList & columns_list, bool reset_columns_) - //: storage(data_part->storage) : storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) - , data_part_storage(data_part_storage_)//data_part->getDataPartStoragePtr()) + , data_part_storage(data_part_storage_) , reset_columns(reset_columns_) { if (reset_columns) { SerializationInfo::Settings info_settings = { - .ratio_of_defaults_for_sparse = storage_settings->ratio_of_defaults_for_sparse_serialization,//storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = storage_settings->ratio_of_defaults_for_sparse_serialization, .choose_kind = false, }; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index b6f279e6d58..a9b058418ea 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -1,12 +1,12 @@ #pragma once -#include "Storages/MergeTree/IDataPartStorage.h" -#include "Storages/MergeTree/MergeTreeSettings.h" +#include +#include #include #include #include #include -#include "Common/Logger.h" +#include namespace DB { @@ -15,7 +15,6 @@ class IMergedBlockOutputStream { public: IMergedBlockOutputStream( -// const MergeTreeMutableDataPartPtr & data_part, const MergeTreeSettingsPtr & storage_settings_, MutableDataPartStoragePtr data_part_storage_, const StorageMetadataPtr & metadata_snapshot_, @@ -43,11 +42,8 @@ protected: SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums); -// const MergeTreeData & storage; // TODO: remove -//// MergeTreeSettingsPtr storage_settings; LoggerPtr log; -//// StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 1b5ad0d81a7..2ce74bde1d5 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -9,7 +9,7 @@ #include #include #include - +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index eebbe3110c0..373ad6c23ea 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -48,21 +48,20 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( } MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( - const String & data_part_name_, - const String & logger_name_, - const SerializationByName & serializations_, - MutableDataPartStoragePtr data_part_storage_, - const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, - - const NamesAndTypesList & columns_list, - const StorageMetadataPtr & metadata_snapshot, - const std::vector & indices_to_recalc, - const Statistics & stats_to_recalc_, - const String & marks_file_extension_, - const CompressionCodecPtr & default_codec_, - const MergeTreeWriterSettings & writer_settings, - const MergeTreeIndexGranularity & computed_index_granularity) + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, + const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, + const std::vector & indices_to_recalc, + const Statistics & stats_to_recalc_, + const String & marks_file_extension_, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity) { ////// TODO: fix the order of columns //// diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 5a57d778b7d..ca88edba7b3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -40,15 +40,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; -// MergeTreeWriterPtr getWriter( -// const NamesAndTypesList & columns_list, -// const StorageMetadataPtr & metadata_snapshot, -// const std::vector & indices_to_recalc, -// const Statistics & stats_to_recalc_, -// const CompressionCodecPtr & default_codec_, -// const MergeTreeWriterSettings & writer_settings, -// const MergeTreeIndexGranularity & computed_index_granularity) override; - // TODO: remove? bool isStoredOnDisk() const override { return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index c99cff258e0..34a3f30c4ba 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -54,18 +54,17 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( } MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( - const String & data_part_name_, - const String & logger_name_, - const SerializationByName & serializations_, - MutableDataPartStoragePtr data_part_storage_, - const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, - + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, - const String & marks_file_extension_, + const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 45d0fbbebec..e3cb3f04335 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -35,15 +35,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; -// MergeTreeWriterPtr getWriter( -// const NamesAndTypesList & columns_list, -// const StorageMetadataPtr & metadata_snapshot, -// const std::vector & indices_to_recalc, -// const Statistics & stats_to_recalc_, -// const CompressionCodecPtr & default_codec_, -// const MergeTreeWriterSettings & writer_settings, -// const MergeTreeIndexGranularity & computed_index_granularity) override; - // TODO: remove? bool isStoredOnDisk() const override { return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 6e8ea1a915b..3f08d8eea21 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -10,14 +10,12 @@ namespace ErrorCodes } MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( -// const MergeTreeMutableDataPartPtr & data_part_, - const String & data_part_name_, - const String & logger_name_, - const SerializationByName & serializations_, - MutableDataPartStoragePtr data_part_storage_, - const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, - + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, @@ -250,7 +248,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G } } -void MergeTreeDataPartWriterCompact::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums) +void MergeTreeDataPartWriterCompact::fillDataChecksums(MergeTreeDataPartChecksums & checksums) { if (columns_buffer.size() != 0) { @@ -420,7 +418,7 @@ size_t MergeTreeDataPartWriterCompact::ColumnsBuffer::size() const return accumulated_columns.at(0)->size(); } -void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & /*checksums_to_remove*/) +void MergeTreeDataPartWriterCompact::fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & /*checksums_to_remove*/) { // If we don't have anything to write, skip finalization. if (!columns_list.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 3bec4c7e988..03804ff4966 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -11,14 +11,12 @@ class MergeTreeDataPartWriterCompact : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterCompact( -// const MergeTreeMutableDataPartPtr & data_part, const String & data_part_name_, const String & logger_name_, const SerializationByName & serializations_, MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, @@ -30,12 +28,12 @@ public: void write(const Block & block, const IColumn::Permutation * permutation) override; - void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) override; + void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) override; void finish(bool sync) override; private: /// Finish serialization of the data. Flush rows in buffer to disk, compute checksums. - void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums); + void fillDataChecksums(MergeTreeDataPartChecksums & checksums); void finishDataSerialization(bool sync); void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 13892c17577..25eb83a82c0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -146,7 +146,6 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const MergeTreeIndices & indices_to_recalc_, @@ -231,7 +230,6 @@ static size_t computeIndexGranularityImpl( size_t MergeTreeDataPartWriterOnDisk::computeIndexGranularity(const Block & block) const { -// const auto storage_settings = storage.getSettings(); return computeIndexGranularityImpl( block, storage_settings->index_granularity_bytes, @@ -293,7 +291,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() GinIndexStorePtr store = nullptr; if (typeid_cast(&*skip_index) != nullptr) { - store = std::make_shared(stream_name, data_part_storage, data_part_storage, /*storage.getSettings()*/storage_settings->max_digestion_size_per_segment); + store = std::make_shared(stream_name, data_part_storage, data_part_storage, storage_settings->max_digestion_size_per_segment); gin_index_stores[stream_name] = store; } skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings)); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 39f33217b57..e17724fa1d0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -5,9 +5,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -97,21 +94,19 @@ public: void sync() const; - void addToChecksums(IMergeTreeDataPart::Checksums & checksums); + void addToChecksums(MergeTreeDataPartChecksums & checksums); }; using StreamPtr = std::unique_ptr>; using StatisticStreamPtr = std::unique_ptr>; MergeTreeDataPartWriterOnDisk( -// const MergeTreeMutableDataPartPtr & data_part_, const String & data_part_name_, const String & logger_name_, const SerializationByName & serializations_, MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, @@ -140,13 +135,13 @@ protected: void calculateAndSerializeStatistics(const Block & stats_block); /// Finishes primary index serialization: write final primary index row (if required) and compute checksums - void fillPrimaryIndexChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillPrimaryIndexChecksums(MergeTreeDataPartChecksums & checksums); void finishPrimaryIndexSerialization(bool sync); /// Finishes skip indices serialization: write all accumulated data to disk and compute checksums - void fillSkipIndicesChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillSkipIndicesChecksums(MergeTreeDataPartChecksums & checksums); void finishSkipIndicesSerialization(bool sync); - void fillStatisticsChecksums(MergeTreeData::DataPart::Checksums & checksums); + void fillStatisticsChecksums(MergeTreeDataPartChecksums & checksums); void finishStatisticsSerialization(bool sync); /// Get global number of the current which we are writing (or going to start to write) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 713dee87fa8..a57bf7d2037 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -76,14 +76,12 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, } MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( -// const MergeTreeMutableDataPartPtr & data_part_, - const String & data_part_name_, - const String & logger_name_, - const SerializationByName & serializations_, - MutableDataPartStoragePtr data_part_storage_, - const MergeTreeIndexGranularityInfo & index_granularity_info_, - const MergeTreeSettingsPtr & storage_settings_, - + const String & data_part_name_, + const String & logger_name_, + const SerializationByName & serializations_, + MutableDataPartStoragePtr data_part_storage_, + const MergeTreeIndexGranularityInfo & index_granularity_info_, + const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, @@ -114,7 +112,6 @@ void MergeTreeDataPartWriterWide::addStreams( { assert(!substream_path.empty()); -// auto storage_settings = storage.getSettings(); auto full_stream_name = ISerialization::getFileNameForStream(column, substream_path); String stream_name; @@ -416,11 +413,10 @@ void MergeTreeDataPartWriterWide::writeColumn( serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } -// const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part; for (const auto & granule : granules) { @@ -603,12 +599,11 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai } -void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) +void MergeTreeDataPartWriterWide::fillDataChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) { -// const auto & global_settings = storage.getContext()->getSettingsRef(); ISerialization::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size;//global_settings.low_cardinality_max_dictionary_size; - serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part;//global_settings.low_cardinality_use_single_dictionary_for_part != 0; + serialize_settings.low_cardinality_max_dictionary_size = settings.low_cardinality_max_dictionary_size; + serialize_settings.low_cardinality_use_single_dictionary_for_part = settings.low_cardinality_use_single_dictionary_for_part; WrittenOffsetColumns offset_columns; if (rows_written_in_last_mark > 0) { @@ -683,7 +678,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) } -void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) +void MergeTreeDataPartWriterWide::fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) { // If we don't have anything to write, skip finalization. if (!columns_list.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index ef9c4ab17dc..5789213c910 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -21,14 +21,12 @@ class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterWide( -// const MergeTreeMutableDataPartPtr & data_part, const String & data_part_name_, const String & logger_name_, const SerializationByName & serializations_, MutableDataPartStoragePtr data_part_storage_, const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, - const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, @@ -40,14 +38,14 @@ public: void write(const Block & block, const IColumn::Permutation * permutation) override; - void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) final; + void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) final; void finish(bool sync) final; private: /// Finish serialization of data: write final mark if required and compute checksums /// Also validate written data in debug mode - void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove); + void fillDataChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove); void finishDataSerialization(bool sync); /// Write data of one column. diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 421c62887da..2b7d5c366f2 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -75,7 +75,7 @@ struct MergeTreeWriterSettings , query_write_settings(query_write_settings_) , max_threads_for_annoy_index_creation(global_settings.max_threads_for_annoy_index_creation) , low_cardinality_max_dictionary_size(global_settings.low_cardinality_max_dictionary_size) - , low_cardinality_use_single_dictionary_for_part(global_settings.low_cardinality_use_single_dictionary_for_part) + , low_cardinality_use_single_dictionary_for_part(global_settings.low_cardinality_use_single_dictionary_for_part != 0) { } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index c2ef7f98388..c7b7557fe52 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -12,7 +12,6 @@ #include #include #include -#include "Interpreters/Context_fwd.h" #include #include @@ -414,12 +413,10 @@ void MergeTreePartition::load(const MergeTreeData & storage, const PartMetadataM partition_key_sample.getByPosition(i).type->getDefaultSerialization()->deserializeBinary(value[i], *file, {}); } -std::unique_ptr MergeTreePartition::store(/*const MergeTreeData & storage,*/ +std::unique_ptr MergeTreePartition::store( StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const { -// auto metadata_snapshot = storage.getInMemoryMetadataPtr(); -// const auto & context = storage.getContext(); const auto & partition_key_sample = adjustPartitionKey(metadata_snapshot, storage_context).sample_block; return store(partition_key_sample, data_part_storage, checksums, storage_context->getWriteSettings()); } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 04175d6f927..44def70bdd9 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -44,7 +44,7 @@ public: /// Store functions return write buffer with written but not finalized data. /// User must call finish() for returned object. - [[nodiscard]] std::unique_ptr store(//const MergeTreeData & storage, + [[nodiscard]] std::unique_ptr store( StorageMetadataPtr metadata_snapshot, ContextPtr storage_context, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums) const; [[nodiscard]] std::unique_ptr store(const Block & partition_key_sample, IDataPartStorage & data_part_storage, MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 2441d941952..e0fb4f703a0 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 51853384012..1c75d81eca5 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -24,7 +24,6 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( , header(header_) { const auto & global_settings = data_part->storage.getContext()->getSettings(); -// const auto & storage_settings = data_part->storage.getSettings(); MergeTreeWriterSettings writer_settings( global_settings, @@ -34,10 +33,10 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( /* rewrite_primary_key = */ false); writer = createMergeTreeDataPartWriter( - data_part->getType(), - data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), - data_part_storage, data_part->index_granularity_info, - storage_settings, + data_part->getType(), + data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), + data_part_storage, data_part->index_granularity_info, + storage_settings, header.getNamesAndTypesList(), metadata_snapshot_, indices_to_recalc, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 54077055d96..7d6b68c7359 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include From 79b3f52dc5189d6def125cf5ed9b1fb2e37267e4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 18 May 2024 23:18:41 +0000 Subject: [PATCH 073/158] only interpolate expression should be used for DAG --- src/Planner/PlannerExpressionAnalysis.cpp | 7 +++---- .../03155_analyzer_interpolate.reference | 13 +++++++++++++ .../0_stateless/03155_analyzer_interpolate.sql | 7 +++++++ 3 files changed, 23 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03155_analyzer_interpolate.reference create mode 100644 tests/queries/0_stateless/03155_analyzer_interpolate.sql diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 6e194b2c03e..6ff56f36933 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -439,20 +439,19 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, auto & interpolate_list_node = query_node.getInterpolate()->as(); PlannerActionsVisitor interpolate_actions_visitor(planner_context); - auto interpolate_actions_dag = std::make_shared(); + auto interpolate_expression_dag = std::make_shared(); for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); - interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); - interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); + interpolate_actions_visitor.visit(interpolate_expression_dag, interpolate_node_typed.getInterpolateExpression()); } std::unordered_map before_sort_actions_inputs_name_to_node; for (const auto & node : before_sort_actions->getInputs()) before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); - for (const auto & node : interpolate_actions_dag->getNodes()) + for (const auto & node : interpolate_expression_dag->getNodes()) { if (before_sort_actions_dag_output_node_names.contains(node.result_name) || node.type != ActionsDAG::ActionType::INPUT) diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.reference b/tests/queries/0_stateless/03155_analyzer_interpolate.reference new file mode 100644 index 00000000000..791aaa5b2a2 --- /dev/null +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.reference @@ -0,0 +1,13 @@ +0 [5] +0.5 [5] +1 [1] +1.5 [5] +2 [5] +2.5 [5] +3 [5] +3.5 [5] +4 [4] +4.5 [5] +5 [5] +5.5 [5] +7 [7] diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql new file mode 100644 index 00000000000..9b56106f2b4 --- /dev/null +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -0,0 +1,7 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/62464 +SET allow_experimental_analyzer = 1; + +SELECT n, [number] as inter FROM ( + SELECT toFloat32(number % 10) AS n, number + FROM numbers(10) WHERE number % 3 = 1 +) group by n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS [5]); From a67418bcc8abb685a1c0271f8f34d5434bb0a113 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 19 May 2024 07:14:37 +0000 Subject: [PATCH 074/158] add NOT_AN_AGGREGATE exception for interpolate expression columns --- src/Planner/PlannerExpressionAnalysis.cpp | 16 ++++++++++++++-- .../0_stateless/03155_analyzer_interpolate.sql | 9 +++++++-- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 6ff56f36933..e7d553af944 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NOT_AN_AGGREGATE; } namespace @@ -397,7 +398,8 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, SortAnalysisResult analyzeSort(const QueryNode & query_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, - ActionsChain & actions_chain) + ActionsChain & actions_chain, + std::optional aggregation_analysis_result_optional) { ActionsDAGPtr before_sort_actions = std::make_shared(input_columns); auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); @@ -451,6 +453,10 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (const auto & node : before_sort_actions->getInputs()) before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); + std::unordered_set aggregation_keys; + if (aggregation_analysis_result_optional) + aggregation_keys.insert(aggregation_analysis_result_optional->aggregation_keys.begin(), aggregation_analysis_result_optional->aggregation_keys.end()); + for (const auto & node : interpolate_expression_dag->getNodes()) { if (before_sort_actions_dag_output_node_names.contains(node.result_name) || @@ -466,6 +472,12 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, input_node_it = it; } + if (aggregation_analysis_result_optional) + if (!aggregation_keys.contains(node.result_name)) + throw Exception(ErrorCodes::NOT_AN_AGGREGATE, + "Column {} is not under aggregate function and not in GROUP BY keys. In query {}", + node.result_name, query_node.formatASTForErrorMessage()); + before_sort_actions_outputs.push_back(input_node_it->second); before_sort_actions_dag_output_node_names.insert(node.result_name); } @@ -567,7 +579,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) { - sort_analysis_result_optional = analyzeSort(query_node, current_output_columns, planner_context, actions_chain); + sort_analysis_result_optional = analyzeSort(query_node, current_output_columns, planner_context, actions_chain, aggregation_analysis_result_optional); current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } diff --git a/tests/queries/0_stateless/03155_analyzer_interpolate.sql b/tests/queries/0_stateless/03155_analyzer_interpolate.sql index 9b56106f2b4..b3c1d233f47 100644 --- a/tests/queries/0_stateless/03155_analyzer_interpolate.sql +++ b/tests/queries/0_stateless/03155_analyzer_interpolate.sql @@ -1,7 +1,12 @@ -- https://github.com/ClickHouse/ClickHouse/issues/62464 SET allow_experimental_analyzer = 1; -SELECT n, [number] as inter FROM ( +SELECT n, [number] AS inter FROM ( SELECT toFloat32(number % 10) AS n, number FROM numbers(10) WHERE number % 3 = 1 -) group by n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS [5]); +) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS [5]); + +SELECT n, number+5 AS inter FROM ( -- { serverError NOT_AN_AGGREGATE } + SELECT toFloat32(number % 10) AS n, number, number*2 AS mn + FROM numbers(10) WHERE number % 3 = 1 +) GROUP BY n, inter ORDER BY n WITH FILL FROM 0 TO 5.51 STEP 0.5 INTERPOLATE (inter AS mn * 2); From f065128ef2d67dfa4709f5d783d3c5a33b6f1e42 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 07:16:07 +0000 Subject: [PATCH 075/158] Fix style --- src/Compression/CompressionCodecDoubleDelta.cpp | 5 +++++ src/Coordination/KeeperServer.cpp | 1 - 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDoubleDelta.cpp b/src/Compression/CompressionCodecDoubleDelta.cpp index 443b9d33532..cbd8cd57a62 100644 --- a/src/Compression/CompressionCodecDoubleDelta.cpp +++ b/src/Compression/CompressionCodecDoubleDelta.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + /** NOTE DoubleDelta is surprisingly bad name. The only excuse is that it comes from an academic paper. * Most people will think that "double delta" is just applying delta transform twice. * But in fact it is something more than applying delta transform twice. diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 953072c5b0e..b07c90b8660 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -45,7 +45,6 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int LOGICAL_ERROR; extern const int INVALID_CONFIG_PARAMETER; - extern const int UNEXPECTED_ZOOKEEPER_ERROR; } using namespace std::chrono_literals; From f143ae6969c77b5ebe44ec4865251caaa18db7fa Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 14:31:21 +0000 Subject: [PATCH 076/158] Fix build --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index b07c90b8660..736a01443ce 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -990,7 +990,7 @@ KeeperServer::ConfigUpdateState KeeperServer::applyConfigUpdate( raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true); return Accepted; } - chassert(false); + std::unreachable(); } ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) From 513900cb524d7b3e96cfbe8b8b56d9b0b0eb6070 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 19 May 2024 15:44:19 +0000 Subject: [PATCH 077/158] assume columns from projection are aggregates --- src/Planner/PlannerExpressionAnalysis.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index e7d553af944..399bbfc67cf 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -454,6 +454,13 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); std::unordered_set aggregation_keys; + + auto projection_expression_dag = std::make_shared(); + for (const auto & node : query_node.getProjection()) + actions_visitor.visit(projection_expression_dag, node); + for (const auto & node : projection_expression_dag->getNodes()) + aggregation_keys.insert(node.result_name); + if (aggregation_analysis_result_optional) aggregation_keys.insert(aggregation_analysis_result_optional->aggregation_keys.begin(), aggregation_analysis_result_optional->aggregation_keys.end()); From 1293a0f79572213f2cd90f5a6f09fbe39d8dbf9e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 18:47:58 +0000 Subject: [PATCH 078/158] Cosmetics, pt. I --- src/Functions/generateSnowflakeID.cpp | 95 +++++++++++++-------------- 1 file changed, 45 insertions(+), 50 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 1decda0ab46..28fc2eb6b05 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -18,8 +18,7 @@ namespace ErrorCodes namespace { -/* - Snowflake ID +/* Snowflake ID https://en.wikipedia.org/wiki/Snowflake_ID 0 1 2 3 @@ -30,35 +29,34 @@ namespace | | machine_id | machine_seq_num | ├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ -- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds -- The middle 10 bits are the machine ID. -- The last 12 bits decode to number of ids processed by the machine at the given millisecond. +- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) +- The middle 10 bits are the machine ID +- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes */ -constexpr auto timestamp_size = 41; -constexpr auto machine_id_size = 10; -constexpr auto machine_seq_num_size = 12; +constexpr auto timestamp_bits_count = 41; +constexpr auto machine_id_bits_count = 10; +constexpr auto machine_seq_num_bits_count = 12; -constexpr int64_t timestamp_mask = ((1LL << timestamp_size) - 1) << (machine_id_size + machine_seq_num_size); -constexpr int64_t machine_id_mask = ((1LL << machine_id_size) - 1) << machine_seq_num_size; -constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_size) - 1; +constexpr int64_t timestamp_mask = ((1LL << timestamp_bits_count) - 1) << (machine_id_bits_count + machine_seq_num_bits_count); +constexpr int64_t machine_id_mask = ((1LL << machine_id_bits_count) - 1) << machine_seq_num_bits_count; +constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_bits_count) - 1; constexpr int64_t max_machine_seq_num = machine_seq_num_mask; Int64 getMachineID() { - auto serverUUID = ServerUUID::get(); - - // hash serverUUID into 64 bits - Int64 h = UUIDHelpers::getHighBytes(serverUUID); - Int64 l = UUIDHelpers::getLowBytes(serverUUID); - return ((h * 11) ^ (l * 17)) & machine_id_mask; + UUID server_uuid = ServerUUID::get(); + /// hash into 64 bits + UInt64 hi = UUIDHelpers::getHighBytes(server_uuid); + UInt64 lo = UUIDHelpers::getLowBytes(server_uuid); + return ((hi * 11) ^ (lo * 17)) & machine_id_mask; } Int64 getTimestamp() { - const auto tm_point = std::chrono::system_clock::now(); - return std::chrono::duration_cast( - tm_point.time_since_epoch()).count() & ((1LL << timestamp_size) - 1); + auto now = std::chrono::system_clock::now(); + auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); + return ticks_since_epoch & ((1LL << timestamp_bits_count) - 1); } } @@ -66,16 +64,11 @@ Int64 getTimestamp() class FunctionSnowflakeID : public IFunction { private: - mutable std::atomic lowest_available_snowflake_id{0}; - // 1 atomic value because we don't want to use mutex + mutable std::atomic lowest_available_snowflake_id = 0; /// atomic to avoid a mutex public: static constexpr auto name = "generateSnowflakeID"; - - static FunctionPtr create(ContextPtr /*context*/) - { - return std::make_shared(); - } + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -95,31 +88,34 @@ public: return std::make_shared(); } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override { auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - Int64 size64 = static_cast(input_rows_count); + vec_to.resize(input_rows_count); if (input_rows_count == 0) { return col_res; } - Int64 machine_id = getMachineID(); + const Int64 machine_id = getMachineID(); Int64 current_timestamp = getTimestamp(); Int64 current_machine_seq_num; - Int64 available_id, next_available_id; + Int64 available_snowflake_id, next_available_snowflake_id; + + const Int64 size64 = static_cast(input_rows_count); + do { - available_id = lowest_available_snowflake_id.load(); - Int64 available_timestamp = (available_id & timestamp_mask) >> (machine_id_size + machine_seq_num_size); - Int64 available_machine_seq_num = available_id & machine_seq_num_mask; + available_snowflake_id = lowest_available_snowflake_id.load(); + const Int64 available_timestamp = (available_snowflake_id & timestamp_mask) >> (machine_id_bits_count + machine_seq_num_bits_count); + const Int64 available_machine_seq_num = available_snowflake_id & machine_seq_num_mask; if (current_timestamp > available_timestamp) { + /// handle overflow current_machine_seq_num = 0; } else @@ -128,24 +124,23 @@ public: current_machine_seq_num = available_machine_seq_num; } - // calculate new `lowest_available_snowflake_id` + /// calculate new lowest_available_snowflake_id + const Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1); Int64 new_timestamp; - Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1); - if (size64 >= seq_nums_in_current_timestamp_left) { + if (size64 >= seq_nums_in_current_timestamp_left) new_timestamp = current_timestamp + 1 + (size64 - seq_nums_in_current_timestamp_left) / max_machine_seq_num; - } else { + else new_timestamp = current_timestamp; - } - Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask; - next_available_id = (new_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | new_machine_seq_num; + const Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask; + next_available_snowflake_id = (new_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | new_machine_seq_num; } - while (!lowest_available_snowflake_id.compare_exchange_strong(available_id, next_available_id)); - // failed CAS => another thread updated `lowest_available_snowflake_id` - // successful CAS => we have our range of exclusive values + while (!lowest_available_snowflake_id.compare_exchange_strong(available_snowflake_id, next_available_snowflake_id)); + /// failed CAS => another thread updated `lowest_available_snowflake_id` + /// successful CAS => we have our range of exclusive values - for (Int64 & el : vec_to) + for (Int64 & to_row : vec_to) { - el = (current_timestamp << (machine_id_size + machine_seq_num_size)) | machine_id | current_machine_seq_num; + to_row = (current_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | current_machine_seq_num; if (current_machine_seq_num++ == max_machine_seq_num) { current_machine_seq_num = 0; @@ -163,10 +158,10 @@ REGISTER_FUNCTION(GenerateSnowflakeID) factory.registerFunction(FunctionDocumentation { .description=R"( -Generates Snowflake ID -- unique identificators contains: -- The first 41 (+ 1 top zero bit) bits is timestamp in Unix time milliseconds -- The middle 10 bits are the machine ID. -- The last 12 bits decode to number of ids processed by the machine at the given millisecond. +Generates a SnowflakeID -- unique identificators contains: +- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) +- The middle 10 bits are the machine ID +- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes In case the number of ids processed overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function guarantees strict monotony on 1 machine and differences in values obtained on different machines. From 08a3c16a5aca95c73cc0ea1aaf2d57edb6acaef2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 18:53:51 +0000 Subject: [PATCH 079/158] Cosmetics, pt. II --- src/Functions/generateSnowflakeID.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 28fc2eb6b05..d70b8349cd8 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -105,7 +105,7 @@ public: Int64 available_snowflake_id, next_available_snowflake_id; - const Int64 size64 = static_cast(input_rows_count); + const Int64 input_rows_count_signed = static_cast(input_rows_count); do { @@ -127,11 +127,11 @@ public: /// calculate new lowest_available_snowflake_id const Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1); Int64 new_timestamp; - if (size64 >= seq_nums_in_current_timestamp_left) - new_timestamp = current_timestamp + 1 + (size64 - seq_nums_in_current_timestamp_left) / max_machine_seq_num; + if (input_rows_count_signed >= seq_nums_in_current_timestamp_left) + new_timestamp = current_timestamp + 1 + (input_rows_count_signed - seq_nums_in_current_timestamp_left) / max_machine_seq_num; else new_timestamp = current_timestamp; - const Int64 new_machine_seq_num = (current_machine_seq_num + size64) & machine_seq_num_mask; + const Int64 new_machine_seq_num = (current_machine_seq_num + input_rows_count_signed) & machine_seq_num_mask; next_available_snowflake_id = (new_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | new_machine_seq_num; } while (!lowest_available_snowflake_id.compare_exchange_strong(available_snowflake_id, next_available_snowflake_id)); From e8d66bf4d79d4ee1f3b18a4ccb1865f3f7ce7294 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 19 May 2024 19:16:24 +0000 Subject: [PATCH 080/158] Cosmetics, pt. III --- src/Functions/serial.cpp | 39 +++++++++++++++++++-------------------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/src/Functions/serial.cpp b/src/Functions/serial.cpp index 3da2f4ce218..de3036ad242 100644 --- a/src/Functions/serial.cpp +++ b/src/Functions/serial.cpp @@ -17,16 +17,16 @@ namespace ErrorCodes class FunctionSerial : public IFunction { private: - mutable zkutil::ZooKeeperPtr zk{nullptr}; + mutable zkutil::ZooKeeperPtr zk; ContextPtr context; public: static constexpr auto name = "serial"; - explicit FunctionSerial(ContextPtr ctx) : context(ctx) + explicit FunctionSerial(ContextPtr context_) : context(context_) { - if (ctx->hasZooKeeper()) { - zk = ctx->getZooKeeper(); + if (context->hasZooKeeper()) { + zk = context->getZooKeeper(); } } @@ -37,7 +37,6 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } - bool isStateful() const override { return true; } bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } @@ -74,14 +73,14 @@ public: auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - size_t size = input_rows_count; - vec_to.resize(size); + + vec_to.resize(input_rows_count); const auto & serial_path = "/serials/" + arguments[0].column->getDataAt(0).toString(); - // CAS in ZooKeeper - // `get` value and version, `trySet` new with version check - // I didn't get how to do it with `multi` + /// CAS in ZooKeeper + /// `get` value and version, `trySet` new with version check + /// I didn't get how to do it with `multi` Int64 counter; std::string counter_path = serial_path + "/counter"; @@ -93,10 +92,10 @@ public: Coordination::Stat stat; while (true) { - std::string counter_string = zk->get(counter_path, &stat); + const String counter_string = zk->get(counter_path, &stat); counter = std::stoll(counter_string); - std::string updated_counter = std::to_string(counter + input_rows_count); - Coordination::Error err = zk->trySet(counter_path, updated_counter); + String updated_counter = std::to_string(counter + input_rows_count); + const Coordination::Error err = zk->trySet(counter_path, updated_counter); if (err == Coordination::Error::ZOK) { // CAS is done @@ -111,7 +110,7 @@ public: } // Make a result - for (auto& val : vec_to) + for (auto & val : vec_to) { val = counter; ++counter; @@ -137,16 +136,16 @@ The server should be configured with a ZooKeeper. }, .returned_value = "Sequential numbers of type Int64 starting from the previous counter value", .examples{ - {"first call", "SELECT serial('name')", R"( -┌─serial('name')─┐ + {"first call", "SELECT serial('id1')", R"( +┌─serial('id1')──┐ │ 1 │ └────────────────┘)"}, - {"second call", "SELECT serial('name')", R"( -┌─serial('name')─┐ + {"second call", "SELECT serial('id1')", R"( +┌─serial('id1')──┐ │ 2 │ └────────────────┘)"}, - {"column call", "SELECT *, serial('name') FROM test_table", R"( -┌─CounterID─┬─UserID─┬─ver─┬─serial('name')─┐ + {"column call", "SELECT *, serial('id1') FROM test_table", R"( +┌─CounterID─┬─UserID─┬─ver─┬─serial('id1')──┐ │ 1 │ 3 │ 3 │ 3 │ │ 1 │ 1 │ 1 │ 4 │ │ 1 │ 2 │ 2 │ 5 │ From e1fef7ecd77da0b1eaed4b0dbc7a73b36cd228ac Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 21 May 2024 12:54:46 +0200 Subject: [PATCH 081/158] Group const fields --- src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPartWriter.h | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index e01572715d6..b3e33e94073 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -56,14 +56,14 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeIndexGranularity & index_granularity_) : data_part_name(data_part_name_) , serializations(serializations_) - , data_part_storage(data_part_storage_) , index_granularity_info(index_granularity_info_) , storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) , columns_list(columns_list_) , settings(settings_) - , index_granularity(index_granularity_) , with_final_mark(settings.can_use_adaptive_granularity) + , data_part_storage(data_part_storage_) + , index_granularity(index_granularity_) { } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 3245a23339b..d2bf03483c9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -50,19 +50,19 @@ protected: IDataPartStorage & getDataPartStorage() { return *data_part_storage; } - /// Serializations for every columns and subcolumns by their names. const String data_part_name; + /// Serializations for every columns and subcolumns by their names. const SerializationByName serializations; - MutableDataPartStoragePtr data_part_storage; const MergeTreeIndexGranularityInfo index_granularity_info; const MergeTreeSettingsPtr storage_settings; const StorageMetadataPtr metadata_snapshot; const NamesAndTypesList columns_list; const MergeTreeWriterSettings settings; - MergeTreeIndexGranularity index_granularity; const bool with_final_mark; + MutableDataPartStoragePtr data_part_storage; MutableColumns index_columns; + MergeTreeIndexGranularity index_granularity; }; using MergeTreeDataPartWriterPtr = std::unique_ptr; From ffa38ecd8bbabff099e1bfb916b4699c9fde1054 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 21 May 2024 13:28:20 +0200 Subject: [PATCH 082/158] Cleanups --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/MergeTreeDataPartCompact.cpp | 7 ++++--- src/Storages/MergeTree/MergeTreeDataPartCompact.h | 1 - src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 9 ++++++--- src/Storages/MergeTree/MergeTreeDataPartWide.h | 1 - src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- 6 files changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 091a7ceb5bd..f4889d64179 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -104,7 +104,6 @@ public: const ValueSizeMap & avg_value_size_hints_, const ReadBufferFromFileBase::ProfileCallback & profile_callback_) const = 0; -// TODO: remove? virtual bool isStoredOnDisk() const = 0; virtual bool isStoredOnRemoteDisk() const = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 373ad6c23ea..fb1c2fe35ed 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -74,9 +74,10 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( //// { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); //// return std::make_unique( - data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, - marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, + indices_to_recalc, stats_to_recalc_, marks_file_extension_, + default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index ca88edba7b3..1fb84424774 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -40,7 +40,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; -// TODO: remove? bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 34a3f30c4ba..74cab30064a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -69,9 +69,12 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { - return std::make_unique(data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, - marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + return std::make_unique( + data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, + metadata_snapshot, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, + default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index e3cb3f04335..7465e08b7c4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -35,7 +35,6 @@ public: const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; -// TODO: remove? bool isStoredOnDisk() const override { return true; } bool isStoredOnRemoteDisk() const override; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index e0fb4f703a0..0fe3ee30a0d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -40,7 +40,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( /* rewrite_primary_key = */ true, blocks_are_granules_size); -// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart + /// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart if (data_part->isStoredOnDisk()) data_part_storage->createDirectories(); From 98b89323c8239ce71153f88f6232806993b1a411 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 21 May 2024 16:14:48 +0200 Subject: [PATCH 083/158] Pass virtual columns descriptions to writer --- .../MergeTree/IMergeTreeDataPartWriter.cpp | 16 ++++++++++------ .../MergeTree/IMergeTreeDataPartWriter.h | 4 ++++ .../MergeTree/MergeTreeDataPartCompact.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterCompact.h | 1 + .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 + .../MergeTree/MergedBlockOutputStream.cpp | 3 ++- .../MergeTree/MergedColumnOnlyOutputStream.cpp | 1 + 12 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index b3e33e94073..27da53de9b0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -52,6 +52,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : data_part_name(data_part_name_) @@ -59,6 +60,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( , index_granularity_info(index_granularity_info_) , storage_settings(storage_settings_) , metadata_snapshot(metadata_snapshot_) + , virtual_columns(virtual_columns_) , columns_list(columns_list_) , settings(settings_) , with_final_mark(settings.can_use_adaptive_granularity) @@ -95,10 +97,9 @@ ASTPtr IMergeTreeDataPartWriter::getCodecDescOrDefault(const String & column_nam if (const auto * column_desc = columns.tryGet(column_name)) return get_codec_or_default(*column_desc); -///// TODO: is this needed? -// if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) -// return get_codec_or_default(*virtual_desc); -// + if (const auto * virtual_desc = virtual_columns->tryGetDescription(column_name)) + return get_codec_or_default(*virtual_desc); + return default_codec->getFullCodecDesc(); } @@ -115,6 +116,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -131,6 +133,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -149,6 +152,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -158,11 +162,11 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( { if (part_type == MergeTreeDataPartType::Compact) return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); else if (part_type == MergeTreeDataPartType::Wide) return createMergeTreeDataPartWideWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, indices_to_recalc, stats_to_recalc_, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); else throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index d2bf03483c9..5dcc7ddc599 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -29,6 +30,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_ = {}); @@ -56,6 +58,7 @@ protected: const MergeTreeIndexGranularityInfo index_granularity_info; const MergeTreeSettingsPtr storage_settings; const StorageMetadataPtr metadata_snapshot; + const VirtualsDescriptionPtr virtual_columns; const NamesAndTypesList columns_list; const MergeTreeWriterSettings settings; const bool with_final_mark; @@ -77,6 +80,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fb1c2fe35ed..332b7d04f7f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -56,6 +56,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -75,7 +76,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( //// return std::make_unique( data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, + index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 74cab30064a..d4630d3dd3f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -62,6 +62,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -72,7 +73,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( return std::make_unique( data_part_name_, logger_name_, serializations_, data_part_storage_, index_granularity_info_, storage_settings_, columns_list, - metadata_snapshot, indices_to_recalc, stats_to_recalc_, + metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 3f08d8eea21..328e3118ba9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -18,6 +18,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc, const String & marks_file_extension_, @@ -27,7 +28,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( : MergeTreeDataPartWriterOnDisk( data_part_name_, logger_name_, serializations_, data_part_storage_, index_granularity_info_, storage_settings_, - columns_list_, metadata_snapshot_, + columns_list_, metadata_snapshot_, virtual_columns_, indices_to_recalc_, stats_to_recalc, marks_file_extension_, default_codec_, settings_, index_granularity_) , plain_file(getDataPartStorage().writeFile( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 03804ff4966..f62f060fde2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -19,6 +19,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 25eb83a82c0..30f01c1acd6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -148,6 +148,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const MergeTreeIndices & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -156,7 +157,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter( data_part_name_, serializations_, data_part_storage_, index_granularity_info_, - storage_settings_, columns_list_, metadata_snapshot_, settings_, index_granularity_) + storage_settings_, columns_list_, metadata_snapshot_, virtual_columns_, settings_, index_granularity_) , skip_indices(indices_to_recalc_) , stats(stats_to_recalc_) , marks_file_extension(marks_file_extension_) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index e17724fa1d0..a60fcd43a58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -109,6 +109,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a57bf7d2037..001f09b81b3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -84,6 +84,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -93,7 +94,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( : MergeTreeDataPartWriterOnDisk( data_part_name_, logger_name_, serializations_, data_part_storage_, index_granularity_info_, storage_settings_, - columns_list_, metadata_snapshot_, + columns_list_, metadata_snapshot_, virtual_columns_, indices_to_recalc_, stats_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 5789213c910..8dc488788c6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -29,6 +29,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, + const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 0fe3ee30a0d..5ef967d930a 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -55,7 +55,8 @@ MergedBlockOutputStream::MergedBlockOutputStream( data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), data_part_storage, data_part->index_granularity_info, storage_settings, - columns_list, metadata_snapshot, skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); + columns_list, metadata_snapshot, data_part->storage.getVirtualsPtr(), + skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); } /// If data is pre-sorted. diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1c75d81eca5..1d1783b1b43 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -39,6 +39,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( storage_settings, header.getNamesAndTypesList(), metadata_snapshot_, + data_part->storage.getVirtualsPtr(), indices_to_recalc, stats_to_recalc_, data_part->getMarksFileExtension(), From d4430b583c4e4531ad1372fd3e40ff6bad5a414d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 21 May 2024 16:19:14 +0200 Subject: [PATCH 084/158] Create snapshot --- utils/keeper-bench/Runner.cpp | 100 +++++++++++++++++----------------- 1 file changed, 50 insertions(+), 50 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index 0050230b6ec..a625a7f157d 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -628,7 +628,11 @@ struct ZooKeeperRequestFromLogReader set_request->path = current_block->getPath(idx_in_block); set_request->data = current_block->getData(idx_in_block); if (auto version = current_block->getVersion(idx_in_block)) - set_request->version = *version; + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + set_request->version = std::numeric_limits::max(); + } request_from_log.request = set_request; break; } @@ -637,7 +641,11 @@ struct ZooKeeperRequestFromLogReader auto remove_request = std::make_shared(); remove_request->path = current_block->getPath(idx_in_block); if (auto version = current_block->getVersion(idx_in_block)) - remove_request->version = *version; + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + remove_request->version = std::numeric_limits::max(); + } request_from_log.request = remove_request; break; } @@ -647,7 +655,11 @@ struct ZooKeeperRequestFromLogReader auto check_request = std::make_shared(); check_request->path = current_block->getPath(idx_in_block); if (auto version = current_block->getVersion(idx_in_block)) - check_request->version = *version; + { + /// we just need to make sure that the request with version that need to fail, fail when replaying + if (request_from_log.expected_result == Coordination::Error::ZBADVERSION) + check_request->version = std::numeric_limits::max(); + } if (op_num == Coordination::OpNum::CheckNotExists) check_request->not_exists = true; request_from_log.request = check_request; @@ -791,10 +803,12 @@ struct SetupNodeCollector if (!request_from_log.expected_result.has_value()) return; + auto process_request = [&](const Coordination::ZooKeeperRequest & request, const auto expected_result) { const auto & path = request.getPath(); - if (processed_paths.contains(path)) + + if (nodes_created_during_replay.contains(path)) return; auto op_num = request.getOpNum(); @@ -804,64 +818,43 @@ struct SetupNodeCollector if (expected_result == Coordination::Error::ZNODEEXISTS) { addExpectedNode(path); - processed_paths.insert(path); } else if (expected_result == Coordination::Error::ZOK) { + nodes_created_during_replay.insert(path); /// we need to make sure ancestors exist auto position = path.find_last_of('/'); if (position != 0) { auto parent_path = path.substr(0, position); - if (!processed_paths.contains(parent_path)) - { - addExpectedNode(parent_path); - processed_paths.insert(parent_path); - } + addExpectedNode(parent_path); } - - processed_paths.insert(path); } } else if (op_num == Coordination::OpNum::Remove) { - if (expected_result == Coordination::Error::ZOK) - { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) addExpectedNode(path); - processed_paths.insert(path); - } } else if (op_num == Coordination::OpNum::Set) { - if (expected_result == Coordination::Error::ZOK) - { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) addExpectedNode(path); - processed_paths.insert(path); - } } else if (op_num == Coordination::OpNum::Check) { - if (expected_result == Coordination::Error::ZOK) - { + if (expected_result == Coordination::Error::ZOK || expected_result == Coordination::Error::ZBADVERSION) addExpectedNode(path); - processed_paths.insert(path); - } } else if (op_num == Coordination::OpNum::CheckNotExists) { - if (expected_result == Coordination::Error::ZNODEEXISTS) - { + if (expected_result == Coordination::Error::ZNODEEXISTS || expected_result == Coordination::Error::ZBADVERSION) addExpectedNode(path); - processed_paths.insert(path); - } } else if (request.isReadRequest()) { if (expected_result == Coordination::Error::ZOK) - { addExpectedNode(path); - processed_paths.insert(path); - } } }; @@ -940,7 +933,7 @@ struct SetupNodeCollector std::mutex nodes_mutex; DB::KeeperContextPtr keeper_context; Coordination::KeeperStoragePtr initial_storage; - std::unordered_set processed_paths; + std::unordered_set nodes_created_during_replay; std::optional snapshot_manager; }; @@ -979,23 +972,23 @@ void requestFromLogExecutor(std::shared_ptrtoString(), response.error, *expected_result) - << std::endl; + //if (*expected_result != response.error) + //{ + // std::cerr << fmt::format( + // "Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result) + // << std::endl; - if (const auto * multi_response = dynamic_cast(&response)) - { - std::string subresponses; - for (size_t i = 0; i < multi_response->responses.size(); ++i) - { - subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); - } + // if (const auto * multi_response = dynamic_cast(&response)) + // { + // std::string subresponses; + // for (size_t i = 0; i < multi_response->responses.size(); ++i) + // { + // subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); + // } - std::cerr << "Subresponses\n" << subresponses << std::endl; - } - } + // std::cerr << "Subresponses\n" << subresponses << std::endl; + // } + //} } request_promise->set_value(); @@ -1049,7 +1042,7 @@ void Runner::runBenchmarkFromLog() std::unordered_map>> executor_id_to_queue; - SCOPE_EXIT({ + SCOPE_EXIT_SAFE({ for (const auto & [executor_id, executor_queue] : executor_id_to_queue) executor_queue->finish(); @@ -1262,8 +1255,15 @@ Runner::~Runner() if (pool) pool->wait(); - auto connection = getConnection(connection_infos[0], 0); - benchmark_context.cleanup(*connection); + try + { + auto connection = getConnection(connection_infos[0], 0); + benchmark_context.cleanup(*connection); + } + catch (...) + { + DB::tryLogCurrentException("While trying to clean nodes"); + } } namespace From 828885c66c8a06d24c34b0d92c6cddda3525b30f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 May 2024 17:20:52 +0200 Subject: [PATCH 085/158] Fix applyNewSettings --- .../AzureBlobStorage/AzureObjectStorage.cpp | 4 +++- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 3 ++- src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp | 5 +++-- src/Disks/ObjectStorages/Cached/CachedObjectStorage.h | 3 ++- src/Disks/ObjectStorages/DiskObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 9 +++++++-- src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp | 5 ----- src/Disks/ObjectStorages/Local/LocalObjectStorage.h | 5 ----- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 5 +++-- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 3 ++- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 5 ----- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 5 ----- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 5 ++--- 13 files changed, 25 insertions(+), 34 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c09cb5e24e1..e7ecf7cd515 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -398,7 +398,9 @@ void AzureObjectStorage::copyObject( /// NOLINT dest_blob_client.CopyFromUri(source_blob_client.GetUrl(), copy_options); } -void AzureObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) +void AzureObjectStorage::applyNewSettings( + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr context, const ApplyNewSettingsOptions &) { auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context); settings.set(std::move(new_settings)); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index c38b5906f4e..e09f5e6753d 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -143,7 +143,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; String getObjectsNamespace() const override { return object_namespace ; } diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index c834ef56644..f2f33684fde 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -192,9 +192,10 @@ void CachedObjectStorage::shutdown() } void CachedObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) + const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, + ContextPtr context, const ApplyNewSettingsOptions & options) { - object_storage->applyNewSettings(config, config_prefix, context); + object_storage->applyNewSettings(config, config_prefix, context, options); } String CachedObjectStorage::getObjectsNamespace() const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index ed78eb90ef4..a4d263e92eb 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -91,7 +91,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; String getObjectsNamespace() const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index f6980d1e8f1..27e0cc78a38 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -536,7 +536,7 @@ void DiskObjectStorage::applyNewSettings( { /// FIXME we cannot use config_prefix that was passed through arguments because the disk may be wrapped with cache and we need another name const auto config_prefix = "storage_configuration.disks." + name; - object_storage->applyNewSettings(config, config_prefix, context_); + object_storage->applyNewSettings(config, config_prefix, context_, IObjectStorage::ApplyNewSettingsOptions{ .allow_client_change = true }); { std::unique_lock lock(resource_mutex); diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 5724ae8929c..d4ac6ea0239 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -199,10 +199,15 @@ public: virtual void startup() = 0; /// Apply new settings, in most cases reiniatilize client and some other staff + struct ApplyNewSettingsOptions + { + bool allow_client_change = true; + }; virtual void applyNewSettings( - const Poco::Util::AbstractConfiguration &, + const Poco::Util::AbstractConfiguration & /* config */, const std::string & /*config_prefix*/, - ContextPtr) {} + ContextPtr /* context */, + const ApplyNewSettingsOptions & /* options */) {} /// Sometimes object storages have something similar to chroot or namespace, for example /// buckets in S3. If object storage doesn't have any namepaces return empty string. diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp index fa27e08f404..a247d86ddce 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.cpp @@ -222,11 +222,6 @@ std::unique_ptr LocalObjectStorage::cloneObjectStorage( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage() is not implemented for LocalObjectStorage"); } -void LocalObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, ContextPtr /* context */) -{ -} - ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const { constexpr size_t key_name_total_size = 32; diff --git a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h index 4c667818c88..371cd37f8b2 100644 --- a/src/Disks/ObjectStorages/Local/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/Local/LocalObjectStorage.h @@ -73,11 +73,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 7891be64b06..d18468411ea 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -572,7 +572,8 @@ void S3ObjectStorage::startup() void S3ObjectStorage::applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) + ContextPtr context, + const ApplyNewSettingsOptions & options) { auto new_s3_settings = getSettings(config, config_prefix, context); if (!static_headers.empty()) @@ -586,7 +587,7 @@ void S3ObjectStorage::applyNewSettings( new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); auto current_s3_settings = s3_settings.get(); - if (current_s3_settings->auth_settings.hasUpdates(new_s3_settings->auth_settings) || for_disk_s3) + if (options.allow_client_change && (current_s3_settings->auth_settings.hasUpdates(new_s3_settings->auth_settings) || for_disk_s3)) { auto new_client = getClient(config, config_prefix, context, *new_s3_settings, for_disk_s3, &uri); client.set(std::move(new_client)); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 74bc5bef3c7..1fff6d67e23 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -149,7 +149,8 @@ public: void applyNewSettings( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - ContextPtr context) override; + ContextPtr context, + const ApplyNewSettingsOptions & options) override; std::string getObjectsNamespace() const override { return uri.bucket; } diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 69f6137cd2d..e837e056acc 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -344,11 +344,6 @@ void WebObjectStorage::startup() { } -void WebObjectStorage::applyNewSettings( - const Poco::Util::AbstractConfiguration & /* config */, const std::string & /* config_prefix */, ContextPtr /* context */) -{ -} - ObjectMetadata WebObjectStorage::getObjectMetadata(const std::string & /* path */) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Metadata is not supported for {}", getName()); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index b8ab510a6fb..9d3b9a3a8f0 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -72,11 +72,6 @@ public: void startup() override; - void applyNewSettings( - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - ContextPtr context) override; - String getObjectsNamespace() const override { return ""; } std::unique_ptr cloneObjectStorage( diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index c45752c10f5..ba91f3038b6 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -87,9 +87,8 @@ bool StorageObjectStorage::supportsSubsetOfColumns(const ContextPtr & context) c void StorageObjectStorage::updateConfiguration(ContextPtr context) { - /// FIXME: we should be able to update everything apart from client if static_configuration == true. - if (!configuration->isStaticConfiguration()) - object_storage->applyNewSettings(context->getConfigRef(), configuration->getTypeName() + ".", context); + IObjectStorage::ApplyNewSettingsOptions options{ .allow_client_change = !configuration->isStaticConfiguration() }; + object_storage->applyNewSettings(context->getConfigRef(), configuration->getTypeName() + ".", context, options); } namespace From a38bb095d800686c27cdf45275af7dc7a5dde149 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 May 2024 18:12:22 +0200 Subject: [PATCH 086/158] Disallow write and truncate if archive --- .../ObjectStorage/StorageObjectStorage.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index ba91f3038b6..b38636e9144 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -242,6 +242,13 @@ SinkToStoragePtr StorageObjectStorage::write( const auto sample_block = metadata_snapshot->getSampleBlock(); const auto & settings = configuration->getQuerySettings(local_context); + if (configuration->isArchive()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Path '{}' contains archive. Write into archive is not supported", + configuration->getPath()); + } + if (configuration->withGlobsIgnorePartitionWildcard()) { throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, @@ -289,6 +296,13 @@ void StorageObjectStorage::truncate( ContextPtr /* context */, TableExclusiveLockHolder & /* table_holder */) { + if (configuration->isArchive()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Path '{}' contains archive. Table cannot be truncated", + configuration->getPath()); + } + if (configuration->withGlobs()) { throw Exception( From 532fe901293968b8dc4fa49299ff09079a9b3cd2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 May 2024 18:32:19 +0200 Subject: [PATCH 087/158] Remove redundant includes --- src/Storages/ObjectStorage/StorageObjectStorageCluster.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index b38eb722df5..1c244b1ca36 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -1,10 +1,7 @@ #pragma once - -// #include #include #include #include -// #include namespace DB { From 96715f611bd54127f43f29123b9a06757d3d7daa Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 May 2024 18:43:53 +0200 Subject: [PATCH 088/158] Apply change from PR #63642 (https://github.com/ClickHouse/ClickHouse/pull/63642) --- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index b38636e9144..dba4aedf7b7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -131,7 +131,7 @@ public: void applyFilters(ActionDAGNodes added_filter_nodes) override { - filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes); + SourceStepWithFilter::applyFilters(std::move(added_filter_nodes)); const ActionsDAG::Node * predicate = nullptr; if (filter_actions_dag) predicate = filter_actions_dag->getOutputs().at(0); From c1920130bb308e2d329117113ddf6ada3da2b908 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 May 2024 19:28:49 +0200 Subject: [PATCH 089/158] Apply changes from PR #62120 --- .../ObjectStorageIteratorAsync.cpp | 1 - .../ObjectStorage/StorageObjectStorage.cpp | 18 +++++++++-- .../StorageObjectStorageSource.cpp | 31 ++++++++++++++++--- .../StorageObjectStorageSource.h | 7 ++++- src/Storages/S3Queue/StorageS3Queue.cpp | 1 + 5 files changed, 49 insertions(+), 9 deletions(-) diff --git a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp index 3fb615b2a5c..0420de0f8dd 100644 --- a/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp +++ b/src/Disks/ObjectStorages/ObjectStorageIteratorAsync.cpp @@ -93,7 +93,6 @@ std::future IObjectStorageIterator }, Priority{}); } - bool IObjectStorageIteratorAsync::isValid() { if (!is_initialized) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index dba4aedf7b7..5de7f41b4f7 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -141,14 +141,28 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override { createIterator(nullptr); + Pipes pipes; auto context = getContext(); + const size_t max_threads = context->getSettingsRef().max_threads; + size_t estimated_keys_count = iterator_wrapper->estimatedKeysCount(); + + if (estimated_keys_count > 1) + num_streams = std::min(num_streams, estimated_keys_count); + else + { + /// The amount of keys (zero) was probably underestimated. + /// We will keep one stream for this particular case. + num_streams = 1; + } + + const size_t max_parsing_threads = num_streams >= max_threads ? 1 : (max_threads / std::max(num_streams, 1ul)); for (size_t i = 0; i < num_streams; ++i) { auto source = std::make_shared( getName(), object_storage, configuration, info, format_settings, - context, max_block_size, iterator_wrapper, need_only_count); + context, max_block_size, iterator_wrapper, max_parsing_threads, need_only_count); source->setKeyCondition(filter_actions_dag, context); pipes.emplace_back(std::move(source)); @@ -175,7 +189,7 @@ private: const String name; const bool need_only_count; const size_t max_block_size; - const size_t num_streams; + size_t num_streams; const bool distributed_processing; void createIterator(const ActionsDAG::Node * predicate) diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index d3b67876224..8d946f515a3 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -48,6 +48,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, + size_t max_parsing_threads_, bool need_only_count_) : SourceWithKeyCondition(info.source_header, false) , WithContext(context_) @@ -57,6 +58,7 @@ StorageObjectStorageSource::StorageObjectStorageSource( , format_settings(format_settings_) , max_block_size(max_block_size_) , need_only_count(need_only_count_) + , max_parsing_threads(max_parsing_threads_) , read_from_format_info(info) , create_reader_pool(std::make_shared( CurrentMetrics::StorageObjectStorageThreads, @@ -277,8 +279,6 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { CompressionMethod compression_method; - const auto max_parsing_threads = need_only_count ? std::optional(1) : std::nullopt; - if (auto object_info_in_archive = dynamic_cast(object_info.get())) { compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); @@ -292,9 +292,17 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade } auto input_format = FormatFactory::instance().getInput( - configuration->format, *read_buf, read_from_format_info.format_header, - getContext(), max_block_size, format_settings, max_parsing_threads, - std::nullopt, /* is_remote_fs */ true, compression_method); + configuration->format, + *read_buf, + read_from_format_info.format_header, + getContext(), + max_block_size, + format_settings, + need_only_count ? 1 : max_parsing_threads, + std::nullopt, + true/* is_remote_fs */, + compression_method, + need_only_count); if (key_condition) input_format->setKeyCondition(key_condition); @@ -440,6 +448,19 @@ StorageObjectStorageSource::GlobIterator::GlobIterator( } } +size_t StorageObjectStorageSource::GlobIterator::estimatedKeysCount() +{ + if (object_infos.empty() && !is_finished && object_storage_iterator->isValid()) + { + /// 1000 files were listed, and we cannot make any estimation of _how many more_ there are (because we list bucket lazily); + /// If there are more objects in the bucket, limiting the number of streams is the last thing we may want to do + /// as it would lead to serious slow down of the execution, since objects are going + /// to be fetched sequentially rather than in-parallel with up to times. + return std::numeric_limits::max(); + } + return object_infos.size(); +} + StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::nextImpl(size_t processor) { std::lock_guard lock(next_mutex); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index fb0ad3e32f1..8dbb31fdfba 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -37,6 +37,7 @@ public: ContextPtr context_, UInt64 max_block_size_, std::shared_ptr file_iterator_, + size_t max_parsing_threads_, bool need_only_count_); ~StorageObjectStorageSource() override; @@ -64,6 +65,7 @@ protected: const std::optional format_settings; const UInt64 max_block_size; const bool need_only_count; + const size_t max_parsing_threads; const ReadFromFormatInfo read_from_format_info; const std::shared_ptr create_reader_pool; @@ -165,12 +167,13 @@ public: ~GlobIterator() override = default; - size_t estimatedKeysCount() override { return object_infos.size(); } + size_t estimatedKeysCount() override; private: ObjectInfoPtr nextImpl(size_t processor) override; ObjectInfoPtr nextImplUnlocked(size_t processor); void createFilterAST(const String & any_key); + void fillBufferForKey(const std::string & uri_key); const ObjectStoragePtr object_storage; const ConfigurationPtr configuration; @@ -184,6 +187,8 @@ private: ActionsDAGPtr filter_dag; ObjectStorageIteratorPtr object_storage_iterator; bool recursive{false}; + std::vector expanded_keys; + std::vector::iterator expanded_keys_iter; std::unique_ptr matcher; diff --git a/src/Storages/S3Queue/StorageS3Queue.cpp b/src/Storages/S3Queue/StorageS3Queue.cpp index 867f22ef5fe..f8eb288921c 100644 --- a/src/Storages/S3Queue/StorageS3Queue.cpp +++ b/src/Storages/S3Queue/StorageS3Queue.cpp @@ -359,6 +359,7 @@ std::shared_ptr StorageS3Queue::createSource( local_context, max_block_size, file_iterator, + local_context->getSettingsRef().max_download_threads, false); auto file_deleter = [=, this](const std::string & path) mutable From c9d29213d8e6af3569fef6be235f0074888a0261 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 21 May 2024 21:04:28 +0200 Subject: [PATCH 090/158] Update InterpreterCreateQuery.cpp --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 4fdd804452d..541717f1c04 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1493,7 +1493,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, validateVirtualColumns(*res); - if (!res->supportsDynamicSubcolumns() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns())) + if (!res->supportsDynamicSubcolumns() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns()) && mode <= LoadingStrictnessLevel::CREATE) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column of type Object, " From 42efc4e2f641b1abec484a36aa32b2cc97e6b49d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 21 May 2024 21:31:52 +0200 Subject: [PATCH 091/158] Pass column position to compact part writer --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 4 +++- .../MergeTree/IMergeTreeDataPartWriter.h | 2 ++ .../MergeTree/MergeTreeDataPartCompact.cpp | 21 +++++++++---------- .../MergeTree/MergedBlockOutputStream.cpp | 8 +++---- .../MergedColumnOnlyOutputStream.cpp | 1 + 6 files changed, 20 insertions(+), 17 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f4889d64179..15c8760141a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -186,6 +186,7 @@ public: /// take place, you must take original name of column for this part from /// storage and pass it to this method. std::optional getColumnPosition(const String & column_name) const; + const NameToNumber & getColumnPositions() const { return column_name_to_position; } /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 27da53de9b0..e8792be6293 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -115,6 +115,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, @@ -151,6 +152,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, @@ -162,7 +164,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( { if (part_type == MergeTreeDataPartType::Compact) return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, + index_granularity_info_, storage_settings_, columns_list, column_positions, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); else if (part_type == MergeTreeDataPartType::Wide) return createMergeTreeDataPartWideWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 5dcc7ddc599..8eb546c4f2c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -69,6 +69,7 @@ protected: }; using MergeTreeDataPartWriterPtr = std::unique_ptr; +using ColumnPositions = std::unordered_map; MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( MergeTreeDataPartType part_type, @@ -79,6 +80,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, const VirtualsDescriptionPtr virtual_columns_, const std::vector & indices_to_recalc, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 332b7d04f7f..98eda5573ce 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -55,6 +55,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const MergeTreeIndexGranularityInfo & index_granularity_info_, const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, + const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, const VirtualsDescriptionPtr virtual_columns, const std::vector & indices_to_recalc, @@ -64,19 +65,17 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) { -////// TODO: fix the order of columns -//// -//// NamesAndTypesList ordered_columns_list; -//// std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), -//// [this](const auto & column) { return getColumnPosition(column.name) != std::nullopt; }); -//// -//// /// Order of writing is important in compact format -//// ordered_columns_list.sort([this](const auto & lhs, const auto & rhs) -//// { return *getColumnPosition(lhs.name) < *getColumnPosition(rhs.name); }); -//// + NamesAndTypesList ordered_columns_list; + std::copy_if(columns_list.begin(), columns_list.end(), std::back_inserter(ordered_columns_list), + [&column_positions](const auto & column) { return column_positions.contains(column.name); }); + + /// Order of writing is important in compact format + ordered_columns_list.sort([&column_positions](const auto & lhs, const auto & rhs) + { return column_positions.at(lhs.name) < column_positions.at(rhs.name); }); + return std::make_unique( data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, metadata_snapshot, virtual_columns, + index_granularity_info_, storage_settings_, ordered_columns_list, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 5ef967d930a..ee5c197336d 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -44,8 +44,6 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (data_part->isStoredOnDisk()) data_part_storage->createDirectories(); -// /// We should write version metadata on part creation to distinguish it from parts that were created without transaction. -// TransactionID tid = txn ? txn->tid : Tx::PrehistoricTID; /// NOTE do not pass context for writing to system.transactions_info_log, /// because part may have temporary name (with temporary block numbers). Will write it later. data_part->version.setCreationTID(tid, nullptr); @@ -55,7 +53,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( data_part->name, data_part->storage.getLogName(), data_part->getSerializations(), data_part_storage, data_part->index_granularity_info, storage_settings, - columns_list, metadata_snapshot, data_part->storage.getVirtualsPtr(), + columns_list, data_part->getColumnPositions(), metadata_snapshot, data_part->storage.getVirtualsPtr(), skip_indices, statistics, data_part->getMarksFileExtension(), default_codec, writer_settings, computed_index_granularity); } @@ -243,9 +241,9 @@ MergedBlockOutputStream::WrittenFiles MergedBlockOutputStream::finalizePartOnDis if (new_part->storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { - if (auto file = new_part->partition.store(//storage, + if (auto file = new_part->partition.store( new_part->storage.getInMemoryMetadataPtr(), new_part->storage.getContext(), - new_part->getDataPartStorage(), checksums)) + new_part->getDataPartStorage(), checksums)) written_files.emplace_back(std::move(file)); if (new_part->minmax_idx->initialized) diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 1d1783b1b43..674a9bd498f 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -38,6 +38,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( data_part_storage, data_part->index_granularity_info, storage_settings, header.getNamesAndTypesList(), + data_part->getColumnPositions(), metadata_snapshot_, data_part->storage.getVirtualsPtr(), indices_to_recalc, From 3f46e4e4305693c9542001fb9e718f2fb098a137 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Wed, 22 May 2024 04:35:06 +0000 Subject: [PATCH 092/158] better exception message in delete table with projection --- src/Interpreters/InterpreterDeleteQuery.cpp | 15 ++++++++++++++- src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/IMergeTreeDataPart.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 15 +++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ .../03161_lightweight_delete_projection.reference | 0 .../03161_lightweight_delete_projection.sql | 15 +++++++++++++++ 7 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03161_lightweight_delete_projection.reference create mode 100644 tests/queries/0_stateless/03161_lightweight_delete_projection.sql diff --git a/src/Interpreters/InterpreterDeleteQuery.cpp b/src/Interpreters/InterpreterDeleteQuery.cpp index ee774994145..9cfb8e486cb 100644 --- a/src/Interpreters/InterpreterDeleteQuery.cpp +++ b/src/Interpreters/InterpreterDeleteQuery.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int TABLE_IS_READ_ONLY; extern const int SUPPORT_IS_DISABLED; extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } @@ -107,7 +108,19 @@ BlockIO InterpreterDeleteQuery::execute() } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "DELETE query is not supported for table {}", table->getStorageID().getFullTableName()); + /// Currently just better exception for the case of a table with projection, + /// can act differently according to the setting. + if (table->hasProjection()) + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "DELETE query is not supported for table {} as it has projections. " + "User should drop all the projections manually before running the query", + table->getStorageID().getFullTableName()); + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "DELETE query is not supported for table {}", + table->getStorageID().getFullTableName()); } } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 87a04c3fcc6..37613704c6a 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -259,6 +259,9 @@ public: /// Return true if storage can execute lightweight delete mutations. virtual bool supportsLightweightDelete() const { return false; } + /// Return true if storage has any projection. + virtual bool hasProjection() const { return false; } + /// Return true if storage can execute 'DELETE FROM' mutations. This is different from lightweight delete /// because those are internally translated into 'ALTER UDPATE' mutations. virtual bool supportsDelete() const { return false; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c380f99060e..f38a80455c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -442,6 +442,8 @@ public: bool hasProjection(const String & projection_name) const { return projection_parts.contains(projection_name); } + bool hasProjection() const { return !projection_parts.empty(); } + bool hasBrokenProjection(const String & projection_name) const; /// Return true, if all projections were loaded successfully and none was marked as broken. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 167160db317..1f7e0a19b3a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6133,6 +6133,21 @@ bool MergeTreeData::supportsLightweightDelete() const return true; } +bool MergeTreeData::hasProjection() const +{ + auto lock = lockParts(); + for (const auto & part : data_parts_by_info) + { + if (part->getState() == MergeTreeDataPartState::Outdated + || part->getState() == MergeTreeDataPartState::Deleting) + continue; + + if (part->hasProjection()) + return true; + } + return false; +} + MergeTreeData::ProjectionPartsVector MergeTreeData::getAllProjectionPartsVector(MergeTreeData::DataPartStateVector * out_states) const { ProjectionPartsVector res; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2f9283659e3..ff93c7c5ae4 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -438,6 +438,8 @@ public: bool supportsLightweightDelete() const override; + bool hasProjection() const override; + bool areAsynchronousInsertsEnabled() const override { return getSettings()->async_insert; } bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override; diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.reference b/tests/queries/0_stateless/03161_lightweight_delete_projection.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03161_lightweight_delete_projection.sql b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql new file mode 100644 index 00000000000..cd29fae8fd7 --- /dev/null +++ b/tests/queries/0_stateless/03161_lightweight_delete_projection.sql @@ -0,0 +1,15 @@ + +DROP TABLE IF EXISTS users; + +CREATE TABLE users ( + uid Int16, + name String, + age Int16, + projection p1 (select count(), age group by age) +) ENGINE = MergeTree order by uid; + +INSERT INTO users VALUES (1231, 'John', 33); +INSERT INTO users VALUES (6666, 'Ksenia', 48); +INSERT INTO users VALUES (8888, 'Alice', 50); + +DELETE FROM users WHERE 1; -- { serverError NOT_IMPLEMENTED } From 12ce276b8af09da46cb89ed9e2e15bb9ceff758a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 22 May 2024 08:51:41 +0200 Subject: [PATCH 093/158] clang-tidy fix --- src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp | 8 ++++---- src/Storages/MergeTree/IMergeTreeDataPartWriter.h | 4 ++-- src/Storages/MergeTree/MergeTreeDataPartCompact.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 2 +- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index e8792be6293..891ba1b9660 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -52,7 +52,7 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : data_part_name(data_part_name_) @@ -117,7 +117,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const NamesAndTypesList & columns_list, const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -134,7 +134,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, @@ -154,7 +154,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const NamesAndTypesList & columns_list, const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 8eb546c4f2c..f04beb37ebb 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -30,7 +30,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_ = {}); @@ -82,7 +82,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const NamesAndTypesList & columns_list, const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 98eda5573ce..4a160e5e229 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -57,7 +57,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartCompactWriter( const NamesAndTypesList & columns_list, const ColumnPositions & column_positions, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index d4630d3dd3f..149f86cef00 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -62,7 +62,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWideWriter( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns, + const VirtualsDescriptionPtr & virtual_columns, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension_, From 58e655e07b128c4dfd26ffe60ad9d9ee285b3fa9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 22 May 2024 07:24:42 +0000 Subject: [PATCH 094/158] Incorporate review feedback --- programs/keeper-client/Commands.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 3c649cad0d3..860840a2d06 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -10,8 +10,8 @@ namespace DB namespace ErrorCodes { + extern const int LOGICAL_ERROR; extern const int KEEPER_EXCEPTION; - extern const int UNEXPECTED_ZOOKEEPER_ERROR; } bool LSCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -442,7 +442,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient new_members = query->args[1].safeGet(); break; default: - throw Exception(ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, "Unexpected operation: {}", operation); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected operation: {}", operation); } auto response = client->zookeeper->reconfig(joining, leaving, new_members); From 7f46eae7b4961b3d58e2d592bc42ba5a32297f7c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Wed, 22 May 2024 11:31:01 +0200 Subject: [PATCH 095/158] clang-tidy fix --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterWide.h | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 328e3118ba9..2d86e0f0770 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -18,7 +18,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc, const String & marks_file_extension_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index f62f060fde2..ebf96c1ebb2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -19,7 +19,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 30f01c1acd6..0a8920790e0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -148,7 +148,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const MergeTreeIndices & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index a60fcd43a58..0c31cabc8c4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -109,7 +109,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 001f09b81b3..9df6cc5e2f7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -84,7 +84,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list_, const StorageMetadataPtr & metadata_snapshot_, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc_, const Statistics & stats_to_recalc_, const String & marks_file_extension_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 8dc488788c6..63205775c58 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -29,7 +29,7 @@ public: const MergeTreeSettingsPtr & storage_settings_, const NamesAndTypesList & columns_list, const StorageMetadataPtr & metadata_snapshot, - const VirtualsDescriptionPtr virtual_columns_, + const VirtualsDescriptionPtr & virtual_columns_, const std::vector & indices_to_recalc, const Statistics & stats_to_recalc_, const String & marks_file_extension, From 03fc077be7d8576c4e3e550842f2fd7c6d06a78f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 May 2024 14:12:37 +0200 Subject: [PATCH 096/158] Fxi --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 2 +- src/Storages/ObjectStorage/ReadBufferIterator.cpp | 6 +++--- src/Storages/ObjectStorage/StorageObjectStorage.cpp | 1 - src/Storages/ObjectStorage/StorageObjectStorageSource.cpp | 4 ++-- 4 files changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index d18468411ea..c07313b52db 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -575,7 +575,7 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto new_s3_settings = getSettings(config, config_prefix, context); + auto new_s3_settings = getSettings(config, config_prefix, context, context->getSettingsRef().s3_validate_request_settings); if (!static_headers.empty()) { new_s3_settings->auth_settings.headers.insert( diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index e065de16e55..5a8a4735fe1 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -145,7 +145,7 @@ std::unique_ptr ReadBufferIterator::recreateLastReadBuffer() auto context = getContext(); const auto & path = current_object_info->isArchive() ? current_object_info->getPathToArchive() : current_object_info->getPath(); - auto impl = object_storage->readObject(StoredObject(), context->getReadSettings()); + auto impl = object_storage->readObject(StoredObject(path), context->getReadSettings()); const auto compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); const auto zstd_window_log_max = static_cast(context->getSettingsRef().zstd_window_log_max); @@ -258,10 +258,10 @@ ReadBufferIterator::Data ReadBufferIterator::next() std::unique_ptr read_buf; CompressionMethod compression_method; using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; - if (auto object_info_in_archive = dynamic_cast(current_object_info.get())) + if (const auto * object_info_in_archive = dynamic_cast(current_object_info.get())) { compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); - auto & archive_reader = object_info_in_archive->archive_reader; + 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 diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.cpp b/src/Storages/ObjectStorage/StorageObjectStorage.cpp index 5de7f41b4f7..2c8e60b49d0 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorage.cpp @@ -17,7 +17,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 8d946f515a3..a2b3ca5b69e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -279,10 +279,10 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade else { CompressionMethod compression_method; - if (auto object_info_in_archive = dynamic_cast(object_info.get())) + if (const auto * object_info_in_archive = dynamic_cast(object_info.get())) { compression_method = chooseCompressionMethod(configuration->getPathInArchive(), configuration->compression_method); - auto & archive_reader = object_info_in_archive->archive_reader; + 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 From b899bd07cfdee3a2919583482c0da2354bbb348a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 22 May 2024 16:12:33 +0200 Subject: [PATCH 097/158] Better --- utils/keeper-bench/Runner.cpp | 90 +++++++++++++++++++++-------------- 1 file changed, 54 insertions(+), 36 deletions(-) diff --git a/utils/keeper-bench/Runner.cpp b/utils/keeper-bench/Runner.cpp index a625a7f157d..ed7e09685f0 100644 --- a/utils/keeper-bench/Runner.cpp +++ b/utils/keeper-bench/Runner.cpp @@ -4,30 +4,28 @@ #include #include +#include #include -#include "Common/ConcurrentBoundedQueue.h" -#include "Common/Exception.h" -#include "Common/ZooKeeper/IKeeper.h" -#include "Common/ZooKeeper/ZooKeeperArgs.h" -#include "Common/ZooKeeper/ZooKeeperCommon.h" -#include "Common/ZooKeeper/ZooKeeperConstants.h" -#include -#include -#include "Coordination/KeeperSnapshotManager.h" -#include "Core/ColumnWithTypeAndName.h" -#include "Core/ColumnsWithTypeAndName.h" +#include +#include #include -#include "IO/ReadBuffer.h" -#include "IO/ReadBufferFromFile.h" -#include "base/Decimal.h" -#include "base/types.h" -#include +#include +#include +#include +#include #include #include #include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace CurrentMetrics @@ -884,6 +882,7 @@ struct SetupNodeCollector if (initial_storage->container.contains(path)) return; + new_nodes = true; std::cerr << "Adding expected node " << path << std::endl; Coordination::Requests create_ops; @@ -923,11 +922,19 @@ struct SetupNodeCollector void generateSnapshot() { - std::cerr << "Generating snapshot with starting data" << std::endl; std::lock_guard lock(nodes_mutex); + if (!new_nodes) + { + std::cerr << "No new nodes added" << std::endl; + return; + } + + std::cerr << "Generating snapshot with starting data" << std::endl; DB::SnapshotMetadataPtr snapshot_meta = std::make_shared(initial_storage->getZXID(), 1, std::make_shared()); DB::KeeperStorageSnapshot snapshot(initial_storage.get(), snapshot_meta); snapshot_manager->serializeSnapshotToDisk(snapshot); + + new_nodes = false; } std::mutex nodes_mutex; @@ -935,6 +942,7 @@ struct SetupNodeCollector Coordination::KeeperStoragePtr initial_storage; std::unordered_set nodes_created_during_replay; std::optional snapshot_manager; + bool new_nodes = false; }; void dumpStats(std::string_view type, const RequestFromLogStats::Stats & stats_for_type) @@ -972,23 +980,25 @@ void requestFromLogExecutor(std::shared_ptrtoString(), response.error, *expected_result) - // << std::endl; +#if 0 + if (*expected_result != response.error) + { + std::cerr << fmt::format( + "Unexpected result for {}\ngot {}, expected {}\n", request->toString(), response.error, *expected_result) + << std::endl; - // if (const auto * multi_response = dynamic_cast(&response)) - // { - // std::string subresponses; - // for (size_t i = 0; i < multi_response->responses.size(); ++i) - // { - // subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); - // } + if (const auto * multi_response = dynamic_cast(&response)) + { + std::string subresponses; + for (size_t i = 0; i < multi_response->responses.size(); ++i) + { + subresponses += fmt::format("{} = {}\n", i, multi_response->responses[i]->error); + } - // std::cerr << "Subresponses\n" << subresponses << std::endl; - // } - //} + std::cerr << "Subresponses\n" << subresponses << std::endl; + } + } +#endif } request_promise->set_value(); @@ -1048,8 +1058,16 @@ void Runner::runBenchmarkFromLog() pool->wait(); - dumpStats("Write", stats.write_requests); - dumpStats("Read", stats.read_requests); + + if (setup_nodes_collector) + { + setup_nodes_collector->generateSnapshot(); + } + else + { + dumpStats("Write", stats.write_requests); + dumpStats("Read", stats.read_requests); + } }); auto push_request = [&](RequestFromLog request) From 332f449a0cec30616180266d4a43a4e658794b1f Mon Sep 17 00:00:00 2001 From: Danila Puzov Date: Wed, 22 May 2024 18:59:39 +0300 Subject: [PATCH 098/158] Issues --- src/Functions/generateSnowflakeID.cpp | 272 +++++++++++------- src/Functions/serial.cpp | 67 +++-- .../03129_serial_test_zookeeper.sql | 16 +- .../03130_generateSnowflakeId.reference | 11 + .../0_stateless/03130_generateSnowflakeId.sql | 29 ++ .../03130_generate_snowflake_id.reference | 3 - .../03130_generate_snowflake_id.sql | 11 - 7 files changed, 252 insertions(+), 157 deletions(-) create mode 100644 tests/queries/0_stateless/03130_generateSnowflakeId.reference create mode 100644 tests/queries/0_stateless/03130_generateSnowflakeId.sql delete mode 100644 tests/queries/0_stateless/03130_generate_snowflake_id.reference delete mode 100644 tests/queries/0_stateless/03130_generate_snowflake_id.sql diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index d70b8349cd8..6ae5dc13af0 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -5,6 +5,7 @@ #include #include #include +#include "base/types.h" namespace DB @@ -34,43 +35,153 @@ namespace - The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes */ +/// bit counts constexpr auto timestamp_bits_count = 41; constexpr auto machine_id_bits_count = 10; constexpr auto machine_seq_num_bits_count = 12; -constexpr int64_t timestamp_mask = ((1LL << timestamp_bits_count) - 1) << (machine_id_bits_count + machine_seq_num_bits_count); -constexpr int64_t machine_id_mask = ((1LL << machine_id_bits_count) - 1) << machine_seq_num_bits_count; -constexpr int64_t machine_seq_num_mask = (1LL << machine_seq_num_bits_count) - 1; -constexpr int64_t max_machine_seq_num = machine_seq_num_mask; +/// bits masks for Snowflake ID components +// constexpr uint64_t timestamp_mask = ((1ULL << timestamp_bits_count) - 1) << (machine_id_bits_count + machine_seq_num_bits_count); // unused +constexpr uint64_t machine_id_mask = ((1ULL << machine_id_bits_count) - 1) << machine_seq_num_bits_count; +constexpr uint64_t machine_seq_num_mask = (1ULL << machine_seq_num_bits_count) - 1; -Int64 getMachineID() +/// max values +constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; + +uint64_t getMachineID() { UUID server_uuid = ServerUUID::get(); /// hash into 64 bits - UInt64 hi = UUIDHelpers::getHighBytes(server_uuid); - UInt64 lo = UUIDHelpers::getLowBytes(server_uuid); - return ((hi * 11) ^ (lo * 17)) & machine_id_mask; + uint64_t hi = UUIDHelpers::getHighBytes(server_uuid); + uint64_t lo = UUIDHelpers::getLowBytes(server_uuid); + /// return only 10 bits + return (((hi * 11) ^ (lo * 17)) & machine_id_mask) >> machine_seq_num_bits_count; } -Int64 getTimestamp() +uint64_t getTimestamp() { auto now = std::chrono::system_clock::now(); auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); - return ticks_since_epoch & ((1LL << timestamp_bits_count) - 1); + return static_cast(ticks_since_epoch) & ((1ULL << timestamp_bits_count) - 1); } +struct SnowflakeComponents { + uint64_t timestamp; + uint64_t machind_id; + uint64_t machine_seq_num; +}; + +SnowflakeComponents toComponents(uint64_t snowflake) { + return { + .timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)), + .machind_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), + .machine_seq_num = (snowflake & machine_seq_num_mask) + }; } -class FunctionSnowflakeID : public IFunction +uint64_t toSnowflakeID(SnowflakeComponents components) { + return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) | + components.machind_id << (machine_seq_num_bits_count) | + components.machine_seq_num); +} + +struct RangeOfSnowflakeIDs { + /// [begin, end) + SnowflakeComponents begin, end; +}; + +/* Get range of `input_rows_count` Snowflake IDs from `max(available, now)` + +1. Calculate Snowflake ID by current timestamp (`now`) +2. `begin = max(available, now)` +3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow +*/ +RangeOfSnowflakeIDs getRangeOfAvailableIDs(const SnowflakeComponents& available, size_t input_rows_count) { -private: - mutable std::atomic lowest_available_snowflake_id = 0; /// atomic to avoid a mutex + /// 1. `now` + SnowflakeComponents begin = { + .timestamp = getTimestamp(), + .machind_id = getMachineID(), + .machine_seq_num = 0 + }; -public: + /// 2. `begin` + if (begin.timestamp <= available.timestamp) + { + begin.timestamp = available.timestamp; + begin.machine_seq_num = available.machine_seq_num; + } + + /// 3. `end = begin + input_rows_count` + SnowflakeComponents end; + const uint64_t seq_nums_in_current_timestamp_left = (max_machine_seq_num - begin.machine_seq_num + 1); + if (input_rows_count >= seq_nums_in_current_timestamp_left) + /// if sequence numbers in current timestamp is not enough for rows => update timestamp + end.timestamp = begin.timestamp + 1 + (input_rows_count - seq_nums_in_current_timestamp_left) / (max_machine_seq_num + 1); + else + end.timestamp = begin.timestamp; + + end.machind_id = begin.machind_id; + end.machine_seq_num = (begin.machine_seq_num + input_rows_count) & machine_seq_num_mask; + + return {begin, end}; +} + +struct GlobalCounterPolicy +{ static constexpr auto name = "generateSnowflakeID"; - static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } + static constexpr auto doc_description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; - String getName() const override { return name; } + /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. + struct Data + { + static inline std::atomic lowest_available_snowflake_id = 0; + + SnowflakeComponents reserveRange(size_t input_rows_count) + { + uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); + RangeOfSnowflakeIDs range; + do + { + range = getRangeOfAvailableIDs(toComponents(available_snowflake_id), input_rows_count); + } + while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, toSnowflakeID(range.end))); + /// if `compare_exhange` failed => another thread updated `lowest_available_snowflake_id` and we should try again + /// completed => range of IDs [begin, end) is reserved, can return the beginning of the range + + return range.begin; + } + }; +}; + +struct ThreadLocalCounterPolicy +{ + static constexpr auto name = "generateSnowflakeIDThreadMonotonic"; + static constexpr auto doc_description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)"; + + /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. + struct Data + { + static inline thread_local uint64_t lowest_available_snowflake_id = 0; + + SnowflakeComponents reserveRange(size_t input_rows_count) + { + RangeOfSnowflakeIDs range = getRangeOfAvailableIDs(toComponents(lowest_available_snowflake_id), input_rows_count); + lowest_available_snowflake_id = toSnowflakeID(range.end); + return range.begin; + } + }; +}; + +} + +template +class FunctionGenerateSnowflakeID : public IFunction, public FillPolicy +{ +public: + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } + + String getName() const override { return FillPolicy::name; } size_t getNumberOfArguments() const override { return 0; } bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } @@ -80,71 +191,36 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!arguments.empty()) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 0.", - getName(), arguments.size()); - } - return std::make_shared(); + FunctionArgumentDescriptors mandatory_args; + FunctionArgumentDescriptors optional_args{ + {"expr", nullptr, nullptr, "Arbitrary Expression"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); + + return std::make_shared(); } ColumnPtr executeImpl(const ColumnsWithTypeAndName & /*arguments*/, const DataTypePtr &, size_t input_rows_count) const override { - auto col_res = ColumnVector::create(); - typename ColumnVector::Container & vec_to = col_res->getData(); + auto col_res = ColumnVector::create(); + typename ColumnVector::Container & vec_to = col_res->getData(); vec_to.resize(input_rows_count); - if (input_rows_count == 0) { - return col_res; - } - - const Int64 machine_id = getMachineID(); - Int64 current_timestamp = getTimestamp(); - Int64 current_machine_seq_num; - - Int64 available_snowflake_id, next_available_snowflake_id; - - const Int64 input_rows_count_signed = static_cast(input_rows_count); - - do + if (input_rows_count != 0) { - available_snowflake_id = lowest_available_snowflake_id.load(); - const Int64 available_timestamp = (available_snowflake_id & timestamp_mask) >> (machine_id_bits_count + machine_seq_num_bits_count); - const Int64 available_machine_seq_num = available_snowflake_id & machine_seq_num_mask; + typename FillPolicy::Data data; + /// get the begin of available snowflake ids range + SnowflakeComponents snowflake_id = data.reserveRange(input_rows_count); - if (current_timestamp > available_timestamp) + for (UInt64 & to_row : vec_to) { - /// handle overflow - current_machine_seq_num = 0; - } - else - { - current_timestamp = available_timestamp; - current_machine_seq_num = available_machine_seq_num; - } - - /// calculate new lowest_available_snowflake_id - const Int64 seq_nums_in_current_timestamp_left = (max_machine_seq_num - current_machine_seq_num + 1); - Int64 new_timestamp; - if (input_rows_count_signed >= seq_nums_in_current_timestamp_left) - new_timestamp = current_timestamp + 1 + (input_rows_count_signed - seq_nums_in_current_timestamp_left) / max_machine_seq_num; - else - new_timestamp = current_timestamp; - const Int64 new_machine_seq_num = (current_machine_seq_num + input_rows_count_signed) & machine_seq_num_mask; - next_available_snowflake_id = (new_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | new_machine_seq_num; - } - while (!lowest_available_snowflake_id.compare_exchange_strong(available_snowflake_id, next_available_snowflake_id)); - /// failed CAS => another thread updated `lowest_available_snowflake_id` - /// successful CAS => we have our range of exclusive values - - for (Int64 & to_row : vec_to) - { - to_row = (current_timestamp << (machine_id_bits_count + machine_seq_num_bits_count)) | machine_id | current_machine_seq_num; - if (current_machine_seq_num++ == max_machine_seq_num) - { - current_machine_seq_num = 0; - ++current_timestamp; + to_row = toSnowflakeID(snowflake_id); + if (snowflake_id.machine_seq_num++ == max_machine_seq_num) + { + snowflake_id.machine_seq_num = 0; + ++snowflake_id.timestamp; + } } } @@ -153,43 +229,27 @@ public: }; +template +void registerSnowflakeIDGenerator(auto& factory) +{ + static constexpr auto doc_syntax_format = "{}([expression])"; + static constexpr auto example_format = "SELECT {}()"; + static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; + + FunctionDocumentation::Description doc_description = FillPolicy::doc_description; + FunctionDocumentation::Syntax doc_syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments doc_arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue doc_returned_value = "A value of type UInt64"; + FunctionDocumentation::Examples doc_examples = {{"uuid", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories doc_categories = {"Snowflake ID"}; + + factory.template registerFunction>({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); +} + REGISTER_FUNCTION(GenerateSnowflakeID) { - factory.registerFunction(FunctionDocumentation - { - .description=R"( -Generates a SnowflakeID -- unique identificators contains: -- The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) -- The middle 10 bits are the machine ID -- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes - -In case the number of ids processed overflows, the timestamp field is incremented by 1 and the counter is reset to 0. -This function guarantees strict monotony on 1 machine and differences in values obtained on different machines. -)", - .syntax = "generateSnowflakeID()", - .arguments{}, - .returned_value = "Column of Int64", - .examples{ - {"single call", "SELECT generateSnowflakeID();", R"( -┌─generateSnowflakeID()─┐ -│ 7195510166884597760 │ -└───────────────────────┘)"}, - {"column call", "SELECT generateSnowflakeID() FROM numbers(10);", R"( -┌─generateSnowflakeID()─┐ -│ 7195516038159417344 │ -│ 7195516038159417345 │ -│ 7195516038159417346 │ -│ 7195516038159417347 │ -│ 7195516038159417348 │ -│ 7195516038159417349 │ -│ 7195516038159417350 │ -│ 7195516038159417351 │ -│ 7195516038159417352 │ -│ 7195516038159417353 │ -└───────────────────────┘)"}, - }, - .categories{"Unique identifiers", "Snowflake ID"} - }); + registerSnowflakeIDGenerator(factory); + registerSnowflakeIDGenerator(factory); } } diff --git a/src/Functions/serial.cpp b/src/Functions/serial.cpp index de3036ad242..d65df83c9f9 100644 --- a/src/Functions/serial.cpp +++ b/src/Functions/serial.cpp @@ -1,9 +1,12 @@ +#include "Common/Exception.h" #include #include #include #include +#include #include + namespace DB { @@ -14,6 +17,9 @@ namespace ErrorCodes extern const int KEEPER_EXCEPTION; } +constexpr auto function_node_name = "/serial_ids/"; +constexpr size_t MAX_SERIES_NUMBER = 1000; // ? + class FunctionSerial : public IFunction { private: @@ -21,7 +27,7 @@ private: ContextPtr context; public: - static constexpr auto name = "serial"; + static constexpr auto name = "generateSerialID"; explicit FunctionSerial(ContextPtr context_) : context(context_) { @@ -48,16 +54,12 @@ public: bool hasInformationAboutMonotonicity() const override { return true; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 1.", - getName(), arguments.size()); - if (!isStringOrFixedString(arguments[0])) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Type of argument for function {} doesn't match: passed {}, should be string", - getName(), arguments[0]->getName()); + FunctionArgumentDescriptors mandatory_args{ + {"series identifier", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"} + }; + validateFunctionArgumentTypes(*this, arguments, mandatory_args); return std::make_shared(); } @@ -71,12 +73,19 @@ public: if (zk->expired()) zk = context->getZooKeeper(); + // slow? + if (zk->exists(function_node_name) && zk->getChildren(function_node_name).size() == MAX_SERIES_NUMBER) { + throw Exception(ErrorCodes::KEEPER_EXCEPTION, + "At most {} serial nodes can be created", + MAX_SERIES_NUMBER); + } + auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); vec_to.resize(input_rows_count); - const auto & serial_path = "/serials/" + arguments[0].column->getDataAt(0).toString(); + const auto & serial_path = function_node_name + arguments[0].column->getDataAt(0).toString(); /// CAS in ZooKeeper /// `get` value and version, `trySet` new with version check @@ -130,28 +139,28 @@ Generates and returns sequential numbers starting from the previous counter valu This function takes a constant string argument - a series identifier. The server should be configured with a ZooKeeper. )", - .syntax = "serial(identifier)", + .syntax = "generateSerialID(identifier)", .arguments{ - {"series identifier", "Series identifier (String)"} + {"series identifier", "Series identifier (String or FixedString)"} }, .returned_value = "Sequential numbers of type Int64 starting from the previous counter value", .examples{ - {"first call", "SELECT serial('id1')", R"( -┌─serial('id1')──┐ -│ 1 │ -└────────────────┘)"}, - {"second call", "SELECT serial('id1')", R"( -┌─serial('id1')──┐ -│ 2 │ -└────────────────┘)"}, - {"column call", "SELECT *, serial('id1') FROM test_table", R"( -┌─CounterID─┬─UserID─┬─ver─┬─serial('id1')──┐ -│ 1 │ 3 │ 3 │ 3 │ -│ 1 │ 1 │ 1 │ 4 │ -│ 1 │ 2 │ 2 │ 5 │ -│ 1 │ 5 │ 5 │ 6 │ -│ 1 │ 4 │ 4 │ 7 │ -└───────────┴────────┴─────┴────────────────┘ + {"first call", "SELECT generateSerialID('id1')", R"( +┌─generateSerialID('id1')──┐ +│ 1 │ +└──────────────────────────┘)"}, + {"second call", "SELECT generateSerialID('id1')", R"( +┌─generateSerialID('id1')──┐ +│ 2 │ +└──────────────────────────┘)"}, + {"column call", "SELECT *, generateSerialID('id1') FROM test_table", R"( +┌─CounterID─┬─UserID─┬─ver─┬─generateSerialID('id1')──┐ +│ 1 │ 3 │ 3 │ 3 │ +│ 1 │ 1 │ 1 │ 4 │ +│ 1 │ 2 │ 2 │ 5 │ +│ 1 │ 5 │ 5 │ 6 │ +│ 1 │ 4 │ 4 │ 7 │ +└───────────┴────────┴─────┴──────────────────────────┘ )"}}, .categories{"Unique identifiers"} }); diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql index c3395009477..2bd60656259 100644 --- a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql +++ b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql @@ -1,12 +1,12 @@ -- Tags: zookeeper -SELECT serial('x'); -SELECT serial('x'); -SELECT serial('y'); -SELECT serial('x') FROM numbers(5); +SELECT generateSerialID('x'); +SELECT generateSerialID('x'); +SELECT generateSerialID('y'); +SELECT generateSerialID('x') FROM numbers(5); -SELECT serial(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT serial('x', 'y'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT serial(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT generateSerialID(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT generateSerialID('x', 'y'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT generateSerialID(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT serial('z'), serial('z') FROM numbers(5); +SELECT generateSerialID('z'), generateSerialID('z') FROM numbers(5); diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference new file mode 100644 index 00000000000..8cdced96770 --- /dev/null +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -0,0 +1,11 @@ +-- generateSnowflakeID -- +1 +1 +0 +0 +1 +100 +-- generateSnowflakeIDThreadMonotonic -- +1 +1 +100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql new file mode 100644 index 00000000000..3e994149d2b --- /dev/null +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -0,0 +1,29 @@ +SELECT '-- generateSnowflakeID --'; +SELECT bitShiftLeft(toUInt64(generateSnowflakeID()), 52) = 0; -- check machine sequence number is zero +SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero + +SELECT generateSnowflakeID(1) = generateSnowflakeID(2); +SELECT generateSnowflakeID() = generateSnowflakeID(1); +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); + +SELECT generateSnowflakeID(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT count(*) +FROM +( + SELECT DISTINCT generateSnowflakeID() + FROM numbers(100) +); + +SELECT '-- generateSnowflakeIDThreadMonotonic --'; +SELECT bitShiftLeft(toUInt64(generateSnowflakeIDThreadMonotonic()), 52) = 0; -- check machine sequence number is zero +SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero + +SELECT generateSnowflakeIDThreadMonotonic(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } + +SELECT count(*) +FROM +( + SELECT DISTINCT generateSnowflakeIDThreadMonotonic() + FROM numbers(100) +); \ No newline at end of file diff --git a/tests/queries/0_stateless/03130_generate_snowflake_id.reference b/tests/queries/0_stateless/03130_generate_snowflake_id.reference deleted file mode 100644 index 2049ba26379..00000000000 --- a/tests/queries/0_stateless/03130_generate_snowflake_id.reference +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -10 diff --git a/tests/queries/0_stateless/03130_generate_snowflake_id.sql b/tests/queries/0_stateless/03130_generate_snowflake_id.sql deleted file mode 100644 index 669814c9ecb..00000000000 --- a/tests/queries/0_stateless/03130_generate_snowflake_id.sql +++ /dev/null @@ -1,11 +0,0 @@ -SELECT bitShiftLeft(toUInt64(generateSnowflakeID()), 52) = 0; -SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; - -SELECT generateSnowflakeID(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } - -SELECT count(*) -FROM -( - SELECT DISTINCT generateSnowflakeID() - FROM numbers(10) -) \ No newline at end of file From b6aa841e575a6594d159be2cc2a5fbc1391190ce Mon Sep 17 00:00:00 2001 From: Danila Puzov Date: Wed, 22 May 2024 19:26:48 +0300 Subject: [PATCH 099/158] Docs for generateSnowflakeID --- .../sql-reference/functions/uuid-functions.md | 126 ++++++++++++++++++ src/Functions/generateSnowflakeID.cpp | 2 +- 2 files changed, 127 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index d1b833c2439..80d7215b9ef 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -690,6 +690,132 @@ serverUUID() Type: [UUID](../data-types/uuid.md). +## generateSnowflakeID + +Generates a [Snowflake ID](https://github.com/twitter-archive/snowflake/tree/b3f6a3c6ca8e1b6847baa6ff42bf72201e2c2231). + +Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. +In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. + +Function `generateSnowflakeID` guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries. + +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|0| timestamp | +├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| | machine_id | machine_seq_num | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateSnowflakeID([expr]) +``` + +**Arguments** + +- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional. + +**Returned value** + +A value of type UInt64. + +**Example** + +First, create a table with a column of type UInt64, then insert a generated Snowflake ID into the table. + +``` sql +CREATE TABLE tab (id UInt64) ENGINE = Memory; + +INSERT INTO tab SELECT generateSnowflakeID(); + +SELECT * FROM tab; +``` + +Result: + +```response +┌──────────────────id─┐ +│ 7199081390080409600 │ +└─────────────────────┘ +``` + +**Example with multiple Snowflake IDs generated per row** + +```sql +SELECT generateSnowflakeID(1), generateSnowflakeID(2); + +┌─generateSnowflakeID(1)─┬─generateSnowflakeID(2)─┐ +│ 7199081609652224000 │ 7199081609652224001 │ +└────────────────────────┴────────────────────────┘ +``` + +## generateSnowflakeIDThreadMonotonic + +Generates a [Snowflake ID](https://github.com/twitter-archive/snowflake/tree/b3f6a3c6ca8e1b6847baa6ff42bf72201e2c2231). + +Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. + +This function behaves like `generateSnowflakeID` but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs. + +``` + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +|0| timestamp | +├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +| | machine_id | machine_seq_num | +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ +``` + +**Syntax** + +``` sql +generateSnowflakeIDThreadMonotonic([expr]) +``` + +**Arguments** + +- `expr` — An arbitrary [expression](../../sql-reference/syntax.md#syntax-expressions) used to bypass [common subexpression elimination](../../sql-reference/functions/index.md#common-subexpression-elimination) if the function is called multiple times in a query. The value of the expression has no effect on the returned Snowflake ID. Optional. + +**Returned value** + +A value of type UInt64. + +**Example** + +First, create a table with a column of type UInt64, then insert a generated Snowflake ID into the table. + +``` sql +CREATE TABLE tab (id UInt64) ENGINE = Memory; + +INSERT INTO tab SELECT generateSnowflakeIDThreadMonotonic(); + +SELECT * FROM tab; +``` + +Result: + +```response +┌──────────────────id─┐ +│ 7199082832006627328 │ +└─────────────────────┘ +``` + +**Example with multiple Snowflake IDs generated per row** + +```sql +SELECT generateSnowflakeIDThreadMonotonic(1), generateSnowflakeIDThreadMonotonic(2); + +┌─generateSnowflakeIDThreadMonotonic(1)─┬─generateSnowflakeIDThreadMonotonic(2)─┐ +│ 7199082940311945216 │ 7199082940316139520 │ +└───────────────────────────────────────┴───────────────────────────────────────┘ +``` + ## See also - [dictGetUUID](../../sql-reference/functions/ext-dict-functions.md#ext_dict_functions-other) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 6ae5dc13af0..1b26bf44adb 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -28,7 +28,7 @@ namespace |0| timestamp | ├─┼ ┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ | | machine_id | machine_seq_num | -├─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┼─┤ +└─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┴─┘ - The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) - The middle 10 bits are the machine ID From a73d60bae5b49bf6b09e4acc05f59cecd528a007 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 22 May 2024 18:35:28 +0200 Subject: [PATCH 100/158] tests for qps_limit_exceeded --- contrib/aws | 2 +- .../integration/helpers/s3_mocks/broken_s3.py | 40 +++- .../test_checking_s3_blobs_paranoid/test.py | 206 +++++++++--------- 3 files changed, 143 insertions(+), 105 deletions(-) diff --git a/contrib/aws b/contrib/aws index 2e12d7c6daf..b7ae6e5bf48 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 2e12d7c6dafa81311ee3d73ac6a178550ffa75be +Subproject commit b7ae6e5bf48fb4981f24476bdd187cd35df1e2c6 diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 206f960293f..238b8aac112 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -165,11 +165,35 @@ class _ServerRuntime: '' "" "ExpectedError" - "mock s3 injected error" + "mock s3 injected unretryable error" "txfbd566d03042474888193-00608d7537" "" ) - request_handler.write_error(data) + request_handler.write_error(500, data) + + class SlowDownAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "SlowDown" + "Slow Down." + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(429, data) + + class QpsLimitExceededAction: + def inject_error(self, request_handler): + data = ( + '' + "" + "QpsLimitExceeded" + "Please reduce your request rate." + "txfbd566d03042474888193-00608d7537" + "" + ) + request_handler.write_error(429, data) class RedirectAction: def __init__(self, host="localhost", port=1): @@ -239,6 +263,10 @@ class _ServerRuntime: self.error_handler = _ServerRuntime.BrokenPipeAction() elif self.action == "redirect_to": self.error_handler = _ServerRuntime.RedirectAction(*self.action_args) + elif self.action == "slow_down": + self.error_handler = _ServerRuntime.SlowDownAction(*self.action_args) + elif self.action == "qps_limit_exceeded": + self.error_handler = _ServerRuntime.QpsLimitExceededAction(*self.action_args) else: self.error_handler = _ServerRuntime.Expected500ErrorAction() @@ -344,12 +372,12 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.end_headers() self.wfile.write(b"Redirected") - def write_error(self, data, content_length=None): + def write_error(self, http_code, data, content_length=None): if content_length is None: content_length = len(data) self.log_message("write_error %s", data) self.read_all_input() - self.send_response(500) + self.send_response(http_code) self.send_header("Content-Type", "text/xml") self.send_header("Content-Length", str(content_length)) self.end_headers() @@ -418,7 +446,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): path = [x for x in parts.path.split("/") if x] assert path[0] == "mock_settings", path if len(path) < 2: - return self.write_error("_mock_settings: wrong command") + return self.write_error(400, "_mock_settings: wrong command") if path[1] == "at_part_upload": params = urllib.parse.parse_qs(parts.query, keep_blank_values=False) @@ -477,7 +505,7 @@ class RequestHandler(http.server.BaseHTTPRequestHandler): self.log_message("reset") return self._ok() - return self.write_error("_mock_settings: wrong command") + return self.write_error(400, "_mock_settings: wrong command") def do_GET(self): if self.path == "/": diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 22d6d263d23..97fc5de65e7 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -91,7 +91,7 @@ def get_multipart_counters(node, query_id, log_type="ExceptionWhileProcessing"): SELECT ProfileEvents['S3CreateMultipartUpload'], ProfileEvents['S3UploadPart'], - ProfileEvents['S3WriteRequestsErrors'], + ProfileEvents['S3WriteRequestsErrors'] + ProfileEvents['S3WriteRequestsThrottling'], FROM system.query_log WHERE query_id='{query_id}' AND type='{log_type}' @@ -148,7 +148,7 @@ def test_upload_s3_fail_create_multi_part_upload(cluster, broken_s3, compression ) assert "Code: 499" in error, error - assert "mock s3 injected error" in error, error + assert "mock s3 injected unretryable error" in error, error create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id @@ -190,7 +190,7 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( ) assert "Code: 499" in error, error - assert "mock s3 injected error" in error, error + assert "mock s3 injected unretryable error" in error, error create_multipart, upload_parts, s3_errors = get_multipart_counters( node, insert_query_id @@ -200,18 +200,28 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( assert s3_errors >= 2 -def test_when_s3_connection_refused_is_retried(cluster, broken_s3): +@pytest.mark.parametrize( + "action_and_message", [ + ("slow_down", "DB::Exception: Slow Down."), + ("qps_limit_exceeded", "DB::Exception: Please reduce your request rate."), + ("connection_refused", "Poco::Exception. Code: 1000, e.code() = 111, Connection refused"), + ], + ids=lambda x: x[0] +) +def test_when_error_is_retried(cluster, broken_s3, action_and_message): node = cluster.instances["node"] - broken_s3.setup_fake_multpartuploads() - broken_s3.setup_at_part_upload(count=3, after=2, action="connection_refused") + action, message = action_and_message - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED" + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload(count=3, after=2, action=action) + + insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED" node.query( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'http://resolver:8083/root/data/test_when_{action}_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -234,13 +244,13 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): assert upload_parts == 39 assert s3_errors == 3 - broken_s3.setup_at_part_upload(count=1000, after=2, action="connection_refused") - insert_query_id = f"INSERT_INTO_TABLE_FUNCTION_CONNECTION_REFUSED_RETRIED_1" + broken_s3.setup_at_part_upload(count=1000, after=2, action=action) + insert_query_id = f"INSERT_INTO_TABLE_{action}_RETRIED_1" error = node.query_and_get_error( f""" INSERT INTO TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_refused_at_write_retried', + 'http://resolver:8083/root/data/test_when_{action}_retried', 'minio', 'minio123', 'CSV', auto, 'none' ) @@ -258,7 +268,79 @@ def test_when_s3_connection_refused_is_retried(cluster, broken_s3): assert "Code: 499" in error, error assert ( - "Poco::Exception. Code: 1000, e.code() = 111, Connection refused" in error + message in error + ), error + + +def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): + node = cluster.instances["node"] + + broken_s3.setup_fake_multpartuploads() + broken_s3.setup_at_part_upload( + count=3, + after=2, + action="broken_pipe", + ) + + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" + node.query( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + create_multipart, upload_parts, s3_errors = get_multipart_counters( + node, insert_query_id, log_type="QueryFinish" + ) + + assert create_multipart == 1 + assert upload_parts == 7 + assert s3_errors == 3 + + broken_s3.setup_at_part_upload( + count=1000, + after=2, + action="broken_pipe", + ) + insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" + error = node.query_and_get_error( + f""" + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=1000000, + s3_check_objects_after_upload=0 + """, + query_id=insert_query_id, + ) + + assert "Code: 1000" in error, error + assert ( + "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" + in error ), error @@ -401,20 +483,20 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ) error = node.query_and_get_error( f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=100, - s3_check_objects_after_upload=0 + INSERT INTO + TABLE FUNCTION s3( + 'http://resolver:8083/root/data/test_when_s3_connection_reset_by_peer_at_create_mpu_retried', + 'minio', 'minio123', + 'CSV', auto, 'none' + ) + SELECT + * + FROM system.numbers + LIMIT 1000 + SETTINGS + s3_max_single_part_upload_size=100, + s3_min_upload_part_size=100, + s3_check_objects_after_upload=0 """, query_id=insert_query_id, ) @@ -427,78 +509,6 @@ def test_when_s3_connection_reset_by_peer_at_create_mpu_retried( ), error -def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): - node = cluster.instances["node"] - - broken_s3.setup_fake_multpartuploads() - broken_s3.setup_at_part_upload( - count=3, - after=2, - action="broken_pipe", - ) - - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD" - node.query( - f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, - s3_check_objects_after_upload=0 - """, - query_id=insert_query_id, - ) - - create_multipart, upload_parts, s3_errors = get_multipart_counters( - node, insert_query_id, log_type="QueryFinish" - ) - - assert create_multipart == 1 - assert upload_parts == 7 - assert s3_errors == 3 - - broken_s3.setup_at_part_upload( - count=1000, - after=2, - action="broken_pipe", - ) - insert_query_id = f"TEST_WHEN_S3_BROKEN_PIPE_AT_UPLOAD_1" - error = node.query_and_get_error( - f""" - INSERT INTO - TABLE FUNCTION s3( - 'http://resolver:8083/root/data/test_when_s3_broken_pipe_at_upload_is_retried', - 'minio', 'minio123', - 'CSV', auto, 'none' - ) - SELECT - * - FROM system.numbers - LIMIT 1000000 - SETTINGS - s3_max_single_part_upload_size=100, - s3_min_upload_part_size=1000000, - s3_check_objects_after_upload=0 - """, - query_id=insert_query_id, - ) - - assert "Code: 1000" in error, error - assert ( - "DB::Exception: Poco::Exception. Code: 1000, e.code() = 32, I/O error: Broken pipe" - in error - ), error - - def test_query_is_canceled_with_inf_retries(cluster, broken_s3): node = cluster.instances["node_with_inf_s3_retries"] From 52fe1fab97a5f39c99c33deb1054bf319fbbf230 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 22 May 2024 16:46:02 +0000 Subject: [PATCH 101/158] Automatic style fix --- tests/integration/helpers/s3_mocks/broken_s3.py | 4 +++- .../test_checking_s3_blobs_paranoid/test.py | 14 ++++++++------ 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/integration/helpers/s3_mocks/broken_s3.py b/tests/integration/helpers/s3_mocks/broken_s3.py index 238b8aac112..7d0127bc1c4 100644 --- a/tests/integration/helpers/s3_mocks/broken_s3.py +++ b/tests/integration/helpers/s3_mocks/broken_s3.py @@ -266,7 +266,9 @@ class _ServerRuntime: elif self.action == "slow_down": self.error_handler = _ServerRuntime.SlowDownAction(*self.action_args) elif self.action == "qps_limit_exceeded": - self.error_handler = _ServerRuntime.QpsLimitExceededAction(*self.action_args) + self.error_handler = _ServerRuntime.QpsLimitExceededAction( + *self.action_args + ) else: self.error_handler = _ServerRuntime.Expected500ErrorAction() diff --git a/tests/integration/test_checking_s3_blobs_paranoid/test.py b/tests/integration/test_checking_s3_blobs_paranoid/test.py index 97fc5de65e7..a7fe02b16de 100644 --- a/tests/integration/test_checking_s3_blobs_paranoid/test.py +++ b/tests/integration/test_checking_s3_blobs_paranoid/test.py @@ -201,12 +201,16 @@ def test_upload_s3_fail_upload_part_when_multi_part_upload( @pytest.mark.parametrize( - "action_and_message", [ + "action_and_message", + [ ("slow_down", "DB::Exception: Slow Down."), ("qps_limit_exceeded", "DB::Exception: Please reduce your request rate."), - ("connection_refused", "Poco::Exception. Code: 1000, e.code() = 111, Connection refused"), + ( + "connection_refused", + "Poco::Exception. Code: 1000, e.code() = 111, Connection refused", + ), ], - ids=lambda x: x[0] + ids=lambda x: x[0], ) def test_when_error_is_retried(cluster, broken_s3, action_and_message): node = cluster.instances["node"] @@ -267,9 +271,7 @@ def test_when_error_is_retried(cluster, broken_s3, action_and_message): ) assert "Code: 499" in error, error - assert ( - message in error - ), error + assert message in error, error def test_when_s3_broken_pipe_at_upload_is_retried(cluster, broken_s3): From 1e5069b5dc6f07d7b29b3a94eaad1c15c9842635 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 22 May 2024 19:21:27 +0200 Subject: [PATCH 102/158] Fix duplicate include --- src/TableFunctions/ITableFunctionDataLake.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/ITableFunctionDataLake.h b/src/TableFunctions/ITableFunctionDataLake.h index 6ad8689a9b4..fe6e5b3e593 100644 --- a/src/TableFunctions/ITableFunctionDataLake.h +++ b/src/TableFunctions/ITableFunctionDataLake.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From 6be79a35b6a55e88103056058ce9833ac62be77e Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 22 May 2024 20:30:19 +0200 Subject: [PATCH 103/158] update contrib/aws to the last head --- contrib/aws | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/aws b/contrib/aws index b7ae6e5bf48..eb96e740453 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit b7ae6e5bf48fb4981f24476bdd187cd35df1e2c6 +Subproject commit eb96e740453ae27afa1f367ba19f99bdcb38484d From f1c191a3cb2d2037de4346683fbc90a58a98a8a6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 May 2024 13:48:23 +0200 Subject: [PATCH 104/158] Better --- .../ObjectStorage/Azure/Configuration.cpp | 4 ++++ .../ObjectStorage/ReadBufferIterator.cpp | 23 +++++++++++------- .../ObjectStorage/ReadBufferIterator.h | 3 ++- .../StorageObjectStorageSource.cpp | 20 +++++++--------- .../StorageObjectStorageSource.h | 5 ++-- src/Storages/S3Queue/S3QueueSource.cpp | 24 ++++++++++--------- 6 files changed, 44 insertions(+), 35 deletions(-) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.cpp b/src/Storages/ObjectStorage/Azure/Configuration.cpp index cca94488a30..ada3e2e9323 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.cpp +++ b/src/Storages/ObjectStorage/Azure/Configuration.cpp @@ -100,6 +100,10 @@ AzureObjectStorage::SettingsPtr StorageAzureConfiguration::createSettings(Contex settings_ptr->max_single_part_upload_size = context_settings.azure_max_single_part_upload_size; settings_ptr->max_single_read_retries = context_settings.azure_max_single_read_retries; settings_ptr->list_object_keys_size = static_cast(context_settings.azure_list_object_keys_size); + settings_ptr->strict_upload_part_size = context_settings.azure_strict_upload_part_size; + settings_ptr->max_upload_part_size = context_settings.azure_max_upload_part_size; + settings_ptr->max_blocks_in_multipart_upload = context_settings.azure_max_blocks_in_multipart_upload; + settings_ptr->min_upload_part_size = context_settings.azure_min_upload_part_size; return settings_ptr; } diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 5a8a4735fe1..50d69129883 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -35,9 +35,10 @@ ReadBufferIterator::ReadBufferIterator( format = configuration->format; } -SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const String & path, const String & format_name) const +SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const { - auto source = std::filesystem::path(configuration->getDataSourceDescription()) / path; + chassert(!object_info.getPath().starts_with("/")); + auto source = std::filesystem::path(configuration->getDataSourceDescription()) / object_info.getPath(); return DB::getKeyForSchemaCache(source, format_name, format_settings, getContext()); } @@ -50,6 +51,7 @@ SchemaCache::Keys ReadBufferIterator::getKeysForSchemaCache() const std::back_inserter(sources), [&](const auto & elem) { + chassert(!elem->getPath().starts_with("/")); return std::filesystem::path(configuration->getDataSourceDescription()) / elem->getPath(); }); return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); @@ -78,7 +80,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( if (format) { - auto cache_key = getKeyForSchemaCache(object_info->getPath(), *format); + const auto cache_key = getKeyForSchemaCache(*object_info, *format); if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) return columns; } @@ -89,7 +91,7 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( /// If we have such entry for some format, we can use this format to read the file. for (const auto & format_name : FormatFactory::instance().getAllInputFormats()) { - auto cache_key = getKeyForSchemaCache(object_info->getPath(), format_name); + const auto cache_key = getKeyForSchemaCache(*object_info, format_name); if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time)) { /// Now format is known. It should be the same for all files. @@ -99,14 +101,13 @@ std::optional ReadBufferIterator::tryGetColumnsFromCache( } } } - return std::nullopt; } void ReadBufferIterator::setNumRowsToLastFile(size_t num_rows) { if (query_settings.schema_inference_use_cache) - schema_cache.addNumRows(getKeyForSchemaCache(current_object_info->getPath(), *format), num_rows); + schema_cache.addNumRows(getKeyForSchemaCache(*current_object_info, *format), num_rows); } void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) @@ -114,7 +115,7 @@ void ReadBufferIterator::setSchemaToLastFile(const ColumnsDescription & columns) if (query_settings.schema_inference_use_cache && query_settings.schema_inference_mode == SchemaInferenceMode::UNION) { - schema_cache.addColumns(getKeyForSchemaCache(current_object_info->getPath(), *format), columns); + schema_cache.addColumns(getKeyForSchemaCache(*current_object_info, *format), columns); } } @@ -135,7 +136,7 @@ void ReadBufferIterator::setFormatName(const String & format_name) String ReadBufferIterator::getLastFileName() const { if (current_object_info) - return current_object_info->getFileName(); + return current_object_info->getPath(); else return ""; } @@ -255,17 +256,21 @@ ReadBufferIterator::Data ReadBufferIterator::next() } } + LOG_TEST(getLogger("KSSENII"), "Will read columns from {}", current_object_info->getPath()); + std::unique_ptr read_buf; CompressionMethod compression_method; using ObjectInfoInArchive = StorageObjectStorageSource::ArchiveIterator::ObjectInfoInArchive; if (const auto * object_info_in_archive = dynamic_cast(current_object_info.get())) { - compression_method = chooseCompressionMethod(current_object_info->getFileName(), configuration->compression_method); + LOG_TEST(getLogger("KSSENII"), "Will read columns from {} from archive", current_object_info->getPath()); + compression_method = chooseCompressionMethod(filename, configuration->compression_method); 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 { + LOG_TEST(getLogger("KSSENII"), "Will read columns from {} from s3", current_object_info->getPath()); compression_method = chooseCompressionMethod(filename, configuration->compression_method); read_buf = object_storage->readObject( StoredObject(current_object_info->getPath()), diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.h b/src/Storages/ObjectStorage/ReadBufferIterator.h index 287e316e243..6eeb52ec2ed 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.h +++ b/src/Storages/ObjectStorage/ReadBufferIterator.h @@ -13,6 +13,7 @@ public: using FileIterator = std::shared_ptr; using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; using ObjectInfoPtr = StorageObjectStorage::ObjectInfoPtr; + using ObjectInfo = StorageObjectStorage::ObjectInfo; using ObjectInfos = StorageObjectStorage::ObjectInfos; ReadBufferIterator( @@ -41,7 +42,7 @@ public: std::unique_ptr recreateLastReadBuffer() override; private: - SchemaCache::Key getKeyForSchemaCache(const String & path, const String & format_name) const; + SchemaCache::Key getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const; SchemaCache::Keys getKeysForSchemaCache() const; std::optional tryGetColumnsFromCache( const ObjectInfos::iterator & begin, const ObjectInfos::iterator & end); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index a2b3ca5b69e..7332574b246 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -183,14 +183,14 @@ Chunk StorageObjectStorageSource::generate() VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, - fs::path(configuration->getNamespace()) / reader.getRelativePath(), + fs::path(configuration->getNamespace()) / reader.getObjectInfo().getPath(), object_info.metadata->size_bytes, &filename); return chunk; } if (reader.getInputFormat() && getContext()->getSettingsRef().use_cache_for_count_from_files) - addNumRowsToCache(reader.getRelativePath(), total_rows_in_file); + addNumRowsToCache(reader.getObjectInfo(), total_rows_in_file); total_rows_in_file = 0; @@ -209,29 +209,28 @@ Chunk StorageObjectStorageSource::generate() return {}; } -void StorageObjectStorageSource::addNumRowsToCache(const String & path, size_t num_rows) +void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows) { const auto cache_key = getKeyForSchemaCache( - fs::path(configuration->getDataSourceDescription()) / path, + fs::path(configuration->getDataSourceDescription()) / object_info.getPath(), configuration->format, format_settings, getContext()); - schema_cache.addNumRows(cache_key, num_rows); } -std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfoPtr & object_info) +std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfo & object_info) { const auto cache_key = getKeyForSchemaCache( - fs::path(configuration->getDataSourceDescription()) / object_info->getPath(), + fs::path(configuration->getDataSourceDescription()) / object_info.getPath(), configuration->format, format_settings, getContext()); auto get_last_mod_time = [&]() -> std::optional { - return object_info->metadata - ? std::optional(object_info->metadata->last_modified.epochTime()) + return object_info.metadata + ? std::optional(object_info.metadata->last_modified.epochTime()) : std::nullopt; }; return schema_cache.tryGetNumRows(cache_key, get_last_mod_time); @@ -263,7 +262,7 @@ StorageObjectStorageSource::ReaderHolder StorageObjectStorageSource::createReade std::optional num_rows_from_cache = need_only_count && getContext()->getSettingsRef().use_cache_for_count_from_files - ? tryGetNumRowsFromCache(object_info) + ? tryGetNumRowsFromCache(*object_info) : std::nullopt; if (num_rows_from_cache) @@ -505,7 +504,6 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne index = 0; - LOG_TEST(logger, "Filter: {}", filter_dag != nullptr); if (filter_dag) { std::vector paths; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index 8dbb31fdfba..e9635ff4dce 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -94,7 +94,6 @@ protected: PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } - std::string getRelativePath() const { return object_info->getPath(); } const ObjectInfo & getObjectInfo() const { return *object_info; } const IInputFormat * getInputFormat() const { return dynamic_cast(source.get()); } @@ -115,8 +114,8 @@ protected: std::future createReaderAsync(size_t processor = 0); std::unique_ptr createReadBuffer(const ObjectInfo & object_info); - void addNumRowsToCache(const String & path, size_t num_rows); - std::optional tryGetNumRowsFromCache(const ObjectInfoPtr & object_info); + void addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows); + std::optional tryGetNumRowsFromCache(const ObjectInfo & object_info); void lazyInitialize(size_t processor); }; diff --git a/src/Storages/S3Queue/S3QueueSource.cpp b/src/Storages/S3Queue/S3QueueSource.cpp index 458f681d7b5..c8aaece0711 100644 --- a/src/Storages/S3Queue/S3QueueSource.cpp +++ b/src/Storages/S3Queue/S3QueueSource.cpp @@ -238,12 +238,14 @@ Chunk StorageS3QueueSource::generate() key_with_info->relative_path, getCurrentExceptionMessage(true)); } - appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); + appendLogElement(reader.getObjectInfo().getPath(), *file_status, processed_rows_from_file, false); } break; } + const auto & path = reader.getObjectInfo().getPath(); + if (shutdown_called) { if (processed_rows_from_file == 0) @@ -253,7 +255,7 @@ Chunk StorageS3QueueSource::generate() { LOG_DEBUG( log, "Table is being dropped, {} rows are already processed from {}, but file is not fully processed", - processed_rows_from_file, reader.getRelativePath()); + processed_rows_from_file, path); try { @@ -265,7 +267,7 @@ Chunk StorageS3QueueSource::generate() key_with_info->relative_path, getCurrentExceptionMessage(true)); } - appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); + appendLogElement(path, *file_status, processed_rows_from_file, false); /// Leave the file half processed. Table is being dropped, so we do not care. break; @@ -273,7 +275,7 @@ Chunk StorageS3QueueSource::generate() LOG_DEBUG(log, "Shutdown called, but file {} is partially processed ({} rows). " "Will process the file fully and then shutdown", - reader.getRelativePath(), processed_rows_from_file); + path, processed_rows_from_file); } auto * prev_scope = CurrentThread::get().attachProfileCountersScope(&file_status->profile_counters); @@ -287,31 +289,31 @@ Chunk StorageS3QueueSource::generate() Chunk chunk; if (reader->pull(chunk)) { - LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), reader.getRelativePath()); + LOG_TEST(log, "Read {} rows from file: {}", chunk.getNumRows(), path); file_status->processed_rows += chunk.getNumRows(); processed_rows_from_file += chunk.getNumRows(); VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( - chunk, requested_virtual_columns, reader.getRelativePath(), reader.getObjectInfo().metadata->size_bytes); + chunk, requested_virtual_columns, path, reader.getObjectInfo().metadata->size_bytes); return chunk; } } catch (...) { const auto message = getCurrentExceptionMessage(true); - LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", reader.getRelativePath(), message); + LOG_ERROR(log, "Got an error while pulling chunk. Will set file {} as failed. Error: {} ", path, message); files_metadata->setFileFailed(key_with_info->processing_holder, message); - appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, false); + appendLogElement(path, *file_status, processed_rows_from_file, false); throw; } files_metadata->setFileProcessed(key_with_info->processing_holder); - applyActionAfterProcessing(reader.getRelativePath()); + applyActionAfterProcessing(path); - appendLogElement(reader.getRelativePath(), *file_status, processed_rows_from_file, true); + appendLogElement(path, *file_status, processed_rows_from_file, true); file_status.reset(); processed_rows_from_file = 0; @@ -327,7 +329,7 @@ Chunk StorageS3QueueSource::generate() if (!reader) break; - file_status = files_metadata->getFileStatus(reader.getRelativePath()); + file_status = files_metadata->getFileStatus(reader.getObjectInfo().getPath()); /// Even if task is finished the thread may be not freed in pool. /// So wait until it will be freed before scheduling a new task. From c150c20512afef6ae816606f197b1ab0a2160712 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Thu, 23 May 2024 13:53:36 +0200 Subject: [PATCH 105/158] adjust tests in test_merge_tree_s3 --- tests/integration/test_merge_tree_s3/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 9216b08f942..0bf81e81383 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -857,9 +857,9 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli error = node.query_and_get_error( "OPTIMIZE TABLE test_merge_canceled_by_s3_errors FINAL", ) - assert "ExpectedError Message: mock s3 injected error" in error, error + assert "ExpectedError Message: mock s3 injected unretryable error" in error, error - node.wait_for_log_line("ExpectedError Message: mock s3 injected error") + node.wait_for_log_line("ExpectedError Message: mock s3 injected unretryable error") table_uuid = node.query( "SELECT uuid FROM system.tables WHERE database = 'default' AND name = 'test_merge_canceled_by_s3_errors' LIMIT 1" @@ -867,7 +867,7 @@ def test_merge_canceled_by_s3_errors(cluster, broken_s3, node_name, storage_poli node.query("SYSTEM FLUSH LOGS") error_count_in_blob_log = node.query( - f"SELECT count() FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' AND error like '%mock s3 injected error%'" + f"SELECT count() FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' AND error like '%mock s3 injected unretryable error%'" ).strip() assert int(error_count_in_blob_log) > 0, node.query( f"SELECT * FROM system.blob_storage_log WHERE query_id like '{table_uuid}::%' FORMAT PrettyCompactMonoBlock" @@ -911,7 +911,7 @@ def test_merge_canceled_by_s3_errors_when_move(cluster, broken_s3, node_name): node.query("OPTIMIZE TABLE merge_canceled_by_s3_errors_when_move FINAL") - node.wait_for_log_line("ExpectedError Message: mock s3 injected error") + node.wait_for_log_line("ExpectedError Message: mock s3 injected unretryable error") count = node.query("SELECT count() FROM merge_canceled_by_s3_errors_when_move") assert int(count) == 2000, count From dd7f3d1ba23bf2e18545ece2675f9836d84d7f69 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 May 2024 14:11:30 +0200 Subject: [PATCH 106/158] Fix test --- tests/integration/test_storage_s3/test.py | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index dc929b7db46..09b27fff1e8 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -1816,27 +1816,13 @@ def test_schema_inference_cache(started_cluster): check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses( - instance, - files, - storage_name, - started_cluster, - bucket, - 4 if storage_name == "url" else 1, - ) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) instance.query("system drop schema cache") check_cache(instance, []) run_describe_query(instance, files, storage_name, started_cluster, bucket) - check_cache_misses( - instance, - files, - storage_name, - started_cluster, - bucket, - 4 if storage_name == "url" else 1, - ) + check_cache_misses(instance, files, storage_name, started_cluster, bucket, 4) instance.query("system drop schema cache") From 8df4da5efaa014f7866288e1aac799f40f52a8c2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 May 2024 14:21:38 +0000 Subject: [PATCH 107/158] Print query in explain plan with parallel replicas --- src/Interpreters/ClusterProxy/executeQuery.cpp | 4 ++++ src/Processors/QueryPlan/ReadFromRemote.cpp | 2 ++ 2 files changed, 6 insertions(+) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 4bbda982f5b..13e6fa87051 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -403,6 +403,10 @@ void executeQueryWithParallelReplicas( ContextPtr context, std::shared_ptr storage_limits) { + auto logger = getLogger("executeQueryWithParallelReplicas"); + LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas", + storage_id.getNameForLogs(), header.dumpStructure(), query_ast->formatForLogging(), processed_stage); + const auto & settings = context->getSettingsRef(); /// check cluster for parallel replicas diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index b4e35af85d6..84c2515e8ca 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -386,6 +386,8 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( chassert(cluster->getShardCount() == 1); std::vector description; + description.push_back(fmt::format("query: {}", formattedAST(query_ast))); + for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) description.push_back(fmt::format("Replica: {}", pool->getHost())); From 71ce01404ddb4bf26f88d910452e70bb4a27a842 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 23 May 2024 16:34:52 +0200 Subject: [PATCH 108/158] Fix validation --- src/Analyzer/ValidationUtils.cpp | 3 +++ src/Planner/PlannerExpressionAnalysis.cpp | 24 ++++------------------- 2 files changed, 7 insertions(+), 20 deletions(-) diff --git a/src/Analyzer/ValidationUtils.cpp b/src/Analyzer/ValidationUtils.cpp index 9e977964755..59157838edf 100644 --- a/src/Analyzer/ValidationUtils.cpp +++ b/src/Analyzer/ValidationUtils.cpp @@ -276,6 +276,9 @@ void validateAggregates(const QueryTreeNodePtr & query_node, AggregatesValidatio if (query_node_typed.hasOrderBy()) validate_group_by_columns_visitor.visit(query_node_typed.getOrderByNode()); + if (query_node_typed.hasInterpolate()) + validate_group_by_columns_visitor.visit(query_node_typed.getInterpolate()); + validate_group_by_columns_visitor.visit(query_node_typed.getProjectionNode()); } diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 399bbfc67cf..1cdff0a26aa 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -441,30 +441,20 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, auto & interpolate_list_node = query_node.getInterpolate()->as(); PlannerActionsVisitor interpolate_actions_visitor(planner_context); - auto interpolate_expression_dag = std::make_shared(); + auto interpolate_actions_dag = std::make_shared(); for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); - interpolate_actions_visitor.visit(interpolate_expression_dag, interpolate_node_typed.getInterpolateExpression()); + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); + interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); } std::unordered_map before_sort_actions_inputs_name_to_node; for (const auto & node : before_sort_actions->getInputs()) before_sort_actions_inputs_name_to_node.emplace(node->result_name, node); - std::unordered_set aggregation_keys; - - auto projection_expression_dag = std::make_shared(); - for (const auto & node : query_node.getProjection()) - actions_visitor.visit(projection_expression_dag, node); - for (const auto & node : projection_expression_dag->getNodes()) - aggregation_keys.insert(node.result_name); - - if (aggregation_analysis_result_optional) - aggregation_keys.insert(aggregation_analysis_result_optional->aggregation_keys.begin(), aggregation_analysis_result_optional->aggregation_keys.end()); - - for (const auto & node : interpolate_expression_dag->getNodes()) + for (const auto & node : interpolate_actions_dag->getNodes()) { if (before_sort_actions_dag_output_node_names.contains(node.result_name) || node.type != ActionsDAG::ActionType::INPUT) @@ -479,12 +469,6 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, input_node_it = it; } - if (aggregation_analysis_result_optional) - if (!aggregation_keys.contains(node.result_name)) - throw Exception(ErrorCodes::NOT_AN_AGGREGATE, - "Column {} is not under aggregate function and not in GROUP BY keys. In query {}", - node.result_name, query_node.formatASTForErrorMessage()); - before_sort_actions_outputs.push_back(input_node_it->second); before_sort_actions_dag_output_node_names.insert(node.result_name); } From 21f831da0d823b9f00b02100bedb847d7af6720e Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 23 May 2024 16:36:11 +0200 Subject: [PATCH 109/158] Remove unneeded changes --- src/Planner/PlannerExpressionAnalysis.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 1cdff0a26aa..6e194b2c03e 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -28,7 +28,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int NOT_AN_AGGREGATE; } namespace @@ -398,8 +397,7 @@ ProjectionAnalysisResult analyzeProjection(const QueryNode & query_node, SortAnalysisResult analyzeSort(const QueryNode & query_node, const ColumnsWithTypeAndName & input_columns, const PlannerContextPtr & planner_context, - ActionsChain & actions_chain, - std::optional aggregation_analysis_result_optional) + ActionsChain & actions_chain) { ActionsDAGPtr before_sort_actions = std::make_shared(input_columns); auto & before_sort_actions_outputs = before_sort_actions->getOutputs(); @@ -570,7 +568,7 @@ PlannerExpressionsAnalysisResult buildExpressionAnalysisResult(const QueryTreeNo std::optional sort_analysis_result_optional; if (query_node.hasOrderBy()) { - sort_analysis_result_optional = analyzeSort(query_node, current_output_columns, planner_context, actions_chain, aggregation_analysis_result_optional); + sort_analysis_result_optional = analyzeSort(query_node, current_output_columns, planner_context, actions_chain); current_output_columns = actions_chain.getLastStepAvailableOutputColumns(); } From 47578772e4558ec044b676e13f5be6ae89d6c49f Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 May 2024 16:39:16 +0200 Subject: [PATCH 110/158] Fix hdfs assertion --- .../ObjectStorage/Azure/Configuration.h | 2 +- .../ObjectStorage/HDFS/Configuration.h | 2 +- .../ObjectStorage/ReadBufferIterator.cpp | 6 ++--- .../ObjectStorage/S3/Configuration.cpp | 2 +- src/Storages/ObjectStorage/S3/Configuration.h | 2 +- .../ObjectStorage/StorageObjectStorage.h | 2 +- .../StorageObjectStorageSource.cpp | 23 +++++++++++++++---- .../StorageObjectStorageSource.h | 6 +++++ 8 files changed, 32 insertions(+), 13 deletions(-) diff --git a/src/Storages/ObjectStorage/Azure/Configuration.h b/src/Storages/ObjectStorage/Azure/Configuration.h index 19b9cf56f93..35b19079ca9 100644 --- a/src/Storages/ObjectStorage/Azure/Configuration.h +++ b/src/Storages/ObjectStorage/Azure/Configuration.h @@ -36,7 +36,7 @@ public: void setPaths(const Paths & paths) override { blobs_paths = paths; } String getNamespace() const override { return container; } - String getDataSourceDescription() override { return std::filesystem::path(connection_url) / container; } + String getDataSourceDescription() const override { return std::filesystem::path(connection_url) / container; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; diff --git a/src/Storages/ObjectStorage/HDFS/Configuration.h b/src/Storages/ObjectStorage/HDFS/Configuration.h index dc06e754c44..01a8b9c5e3b 100644 --- a/src/Storages/ObjectStorage/HDFS/Configuration.h +++ b/src/Storages/ObjectStorage/HDFS/Configuration.h @@ -31,7 +31,7 @@ public: std::string getPathWithoutGlobs() const override; String getNamespace() const override { return ""; } - String getDataSourceDescription() override { return url; } + String getDataSourceDescription() const override { return url; } StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; void check(ContextPtr context) const override; diff --git a/src/Storages/ObjectStorage/ReadBufferIterator.cpp b/src/Storages/ObjectStorage/ReadBufferIterator.cpp index 50d69129883..5e89a0a1b9d 100644 --- a/src/Storages/ObjectStorage/ReadBufferIterator.cpp +++ b/src/Storages/ObjectStorage/ReadBufferIterator.cpp @@ -37,8 +37,7 @@ ReadBufferIterator::ReadBufferIterator( SchemaCache::Key ReadBufferIterator::getKeyForSchemaCache(const ObjectInfo & object_info, const String & format_name) const { - chassert(!object_info.getPath().starts_with("/")); - auto source = std::filesystem::path(configuration->getDataSourceDescription()) / object_info.getPath(); + auto source = StorageObjectStorageSource::getUniqueStoragePathIdentifier(*configuration, object_info); return DB::getKeyForSchemaCache(source, format_name, format_settings, getContext()); } @@ -51,8 +50,7 @@ SchemaCache::Keys ReadBufferIterator::getKeysForSchemaCache() const std::back_inserter(sources), [&](const auto & elem) { - chassert(!elem->getPath().starts_with("/")); - return std::filesystem::path(configuration->getDataSourceDescription()) / elem->getPath(); + return StorageObjectStorageSource::getUniqueStoragePathIdentifier(*configuration, *elem); }); return DB::getKeysForSchemaCache(sources, *format, format_settings, getContext()); } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 00d569fea9f..6b6cde0c431 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -50,7 +50,7 @@ static const std::unordered_set optional_configuration_keys = "no_sign_request" }; -String StorageS3Configuration::getDataSourceDescription() +String StorageS3Configuration::getDataSourceDescription() const { return std::filesystem::path(url.uri.getHost() + std::to_string(url.uri.getPort())) / url.bucket; } diff --git a/src/Storages/ObjectStorage/S3/Configuration.h b/src/Storages/ObjectStorage/S3/Configuration.h index de6c02d5020..906d10a1a9a 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.h +++ b/src/Storages/ObjectStorage/S3/Configuration.h @@ -31,7 +31,7 @@ public: void setPaths(const Paths & paths) override { keys = paths; } String getNamespace() const override { return url.bucket; } - String getDataSourceDescription() override; + String getDataSourceDescription() const override; StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const override; bool isArchive() const override { return url.archive_pattern.has_value(); } diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index 7b118cb7e6b..de75af5035b 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -161,7 +161,7 @@ public: virtual const Paths & getPaths() const = 0; virtual void setPaths(const Paths & paths) = 0; - virtual String getDataSourceDescription() = 0; + virtual String getDataSourceDescription() const = 0; virtual String getNamespace() const = 0; virtual StorageObjectStorage::QuerySettings getQuerySettings(const ContextPtr &) const = 0; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp index 7332574b246..b31d0f8a92e 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.cpp @@ -82,6 +82,21 @@ void StorageObjectStorageSource::setKeyCondition(const ActionsDAGPtr & filter_ac setKeyConditionImpl(filter_actions_dag, context_, read_from_format_info.format_header); } +std::string StorageObjectStorageSource::getUniqueStoragePathIdentifier( + const Configuration & configuration, + const ObjectInfo & object_info, + bool include_connection_info) +{ + auto path = object_info.getPath(); + if (path.starts_with("/")) + path = path.substr(1); + + if (include_connection_info) + return fs::path(configuration.getDataSourceDescription()) / path; + else + return fs::path(configuration.getNamespace()) / path; +} + std::shared_ptr StorageObjectStorageSource::createFileIterator( ConfigurationPtr configuration, ObjectStoragePtr object_storage, @@ -183,7 +198,7 @@ Chunk StorageObjectStorageSource::generate() VirtualColumnUtils::addRequestedPathFileAndSizeVirtualsToChunk( chunk, read_from_format_info.requested_virtual_columns, - fs::path(configuration->getNamespace()) / reader.getObjectInfo().getPath(), + getUniqueStoragePathIdentifier(*configuration, reader.getObjectInfo(), false), object_info.metadata->size_bytes, &filename); return chunk; @@ -212,7 +227,7 @@ Chunk StorageObjectStorageSource::generate() void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_info, size_t num_rows) { const auto cache_key = getKeyForSchemaCache( - fs::path(configuration->getDataSourceDescription()) / object_info.getPath(), + getUniqueStoragePathIdentifier(*configuration, object_info), configuration->format, format_settings, getContext()); @@ -222,7 +237,7 @@ void StorageObjectStorageSource::addNumRowsToCache(const ObjectInfo & object_inf std::optional StorageObjectStorageSource::tryGetNumRowsFromCache(const ObjectInfo & object_info) { const auto cache_key = getKeyForSchemaCache( - fs::path(configuration->getDataSourceDescription()) / object_info.getPath(), + getUniqueStoragePathIdentifier(*configuration, object_info), configuration->format, format_settings, getContext()); @@ -511,7 +526,7 @@ StorageObjectStorage::ObjectInfoPtr StorageObjectStorageSource::GlobIterator::ne for (const auto & object_info : new_batch) { chassert(object_info); - paths.push_back(fs::path(configuration->getNamespace()) / object_info->getPath()); + paths.push_back(getUniqueStoragePathIdentifier(*configuration, *object_info, false)); } VirtualColumnUtils::filterByPathOrFile(new_batch, paths, filter_dag, virtual_columns, getContext()); diff --git a/src/Storages/ObjectStorage/StorageObjectStorageSource.h b/src/Storages/ObjectStorage/StorageObjectStorageSource.h index e9635ff4dce..fd7c7aa7102 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageSource.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageSource.h @@ -17,6 +17,7 @@ class StorageObjectStorageSource : public SourceWithKeyCondition, WithContext { friend class StorageS3QueueSource; public: + using Configuration = StorageObjectStorage::Configuration; using ConfigurationPtr = StorageObjectStorage::ConfigurationPtr; using ObjectInfo = StorageObjectStorage::ObjectInfo; using ObjectInfos = StorageObjectStorage::ObjectInfos; @@ -58,6 +59,11 @@ public: ObjectInfos * read_keys, std::function file_progress_callback = {}); + static std::string getUniqueStoragePathIdentifier( + const Configuration & configuration, + const ObjectInfo & object_info, + bool include_connection_info = true); + protected: const String name; ObjectStoragePtr object_storage; From 60e94af1ecd1e2b3e5b3f3194901d001653b7991 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 23 May 2024 16:55:02 +0200 Subject: [PATCH 111/158] Return one line change --- src/Planner/PlannerExpressionAnalysis.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index 6e194b2c03e..7984d97a1ea 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -444,7 +444,6 @@ SortAnalysisResult analyzeSort(const QueryNode & query_node, for (auto & interpolate_node : interpolate_list_node.getNodes()) { auto & interpolate_node_typed = interpolate_node->as(); - interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getExpression()); interpolate_actions_visitor.visit(interpolate_actions_dag, interpolate_node_typed.getInterpolateExpression()); } From a4903e6b5583b172496be8fa0dbf6cead2b51d86 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 May 2024 16:55:48 +0200 Subject: [PATCH 112/158] Add supportsDynamicSubcolumns() --- src/Storages/ObjectStorage/StorageObjectStorage.h | 2 ++ src/Storages/ObjectStorage/StorageObjectStorageCluster.h | 2 ++ 2 files changed, 4 insertions(+) diff --git a/src/Storages/ObjectStorage/StorageObjectStorage.h b/src/Storages/ObjectStorage/StorageObjectStorage.h index de75af5035b..f45d8c1f01a 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorage.h +++ b/src/Storages/ObjectStorage/StorageObjectStorage.h @@ -84,6 +84,8 @@ public: bool supportsSubcolumns() const override { return true; } + bool supportsDynamicSubcolumns() const override { return true; } + bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } bool supportsSubsetOfColumns(const ContextPtr & context) const; diff --git a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h index 1c244b1ca36..69fec2b3c77 100644 --- a/src/Storages/ObjectStorage/StorageObjectStorageCluster.h +++ b/src/Storages/ObjectStorage/StorageObjectStorageCluster.h @@ -26,6 +26,8 @@ public: bool supportsSubcolumns() const override { return true; } + bool supportsDynamicSubcolumns() const override { return true; } + bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; } RemoteQueryExecutor::Extension getTaskIteratorExtension( From c1950236ced0b110e679c4042d1fab2c7df26f2f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 15:24:18 +0000 Subject: [PATCH 113/158] Cosmetics, pt. IV --- src/Functions/{serial.cpp => generateSerialID.cpp} | 2 -- 1 file changed, 2 deletions(-) rename src/Functions/{serial.cpp => generateSerialID.cpp} (98%) diff --git a/src/Functions/serial.cpp b/src/Functions/generateSerialID.cpp similarity index 98% rename from src/Functions/serial.cpp rename to src/Functions/generateSerialID.cpp index d65df83c9f9..db26d0d684b 100644 --- a/src/Functions/serial.cpp +++ b/src/Functions/generateSerialID.cpp @@ -12,8 +12,6 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int KEEPER_EXCEPTION; } From e6f135089f300a6e5cc0d1276e748750f2b59454 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 15:25:38 +0000 Subject: [PATCH 114/158] Cosmetics, pt. V --- src/Functions/generateSnowflakeID.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 1b26bf44adb..bbae41e4f49 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -11,11 +11,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - namespace { @@ -81,7 +76,7 @@ SnowflakeComponents toComponents(uint64_t snowflake) { uint64_t toSnowflakeID(SnowflakeComponents components) { return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) | - components.machind_id << (machine_seq_num_bits_count) | + components.machind_id << (machine_seq_num_bits_count) | components.machine_seq_num); } @@ -120,7 +115,7 @@ RangeOfSnowflakeIDs getRangeOfAvailableIDs(const SnowflakeComponents& available, end.timestamp = begin.timestamp + 1 + (input_rows_count - seq_nums_in_current_timestamp_left) / (max_machine_seq_num + 1); else end.timestamp = begin.timestamp; - + end.machind_id = begin.machind_id; end.machine_seq_num = (begin.machine_seq_num + input_rows_count) & machine_seq_num_mask; From 4611a44c1f76873482fff498f7e7f8414f24e375 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 15:53:14 +0000 Subject: [PATCH 115/158] Cosmetics, pt. VI --- src/Functions/generateSnowflakeID.cpp | 100 +++++++++++++------------- src/Functions/generateUUIDv7.cpp | 25 ++++--- 2 files changed, 60 insertions(+), 65 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index bbae41e4f49..4e61bd9fb1c 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -27,7 +27,7 @@ namespace - The first 41 (+ 1 top zero bit) bits is the timestamp (millisecond since Unix epoch 1 Jan 1970) - The middle 10 bits are the machine ID -- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by differen processes +- The last 12 bits are a counter to disambiguate multiple snowflakeIDs generated within the same millisecond by different processes */ /// bit counts @@ -36,14 +36,13 @@ constexpr auto machine_id_bits_count = 10; constexpr auto machine_seq_num_bits_count = 12; /// bits masks for Snowflake ID components -// constexpr uint64_t timestamp_mask = ((1ULL << timestamp_bits_count) - 1) << (machine_id_bits_count + machine_seq_num_bits_count); // unused -constexpr uint64_t machine_id_mask = ((1ULL << machine_id_bits_count) - 1) << machine_seq_num_bits_count; -constexpr uint64_t machine_seq_num_mask = (1ULL << machine_seq_num_bits_count) - 1; +constexpr uint64_t machine_id_mask = ((1ull << machine_id_bits_count) - 1) << machine_seq_num_bits_count; +constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) - 1; /// max values constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; -uint64_t getMachineID() +uint64_t getMachineId() { UUID server_uuid = ServerUUID::get(); /// hash into 64 bits @@ -57,48 +56,44 @@ uint64_t getTimestamp() { auto now = std::chrono::system_clock::now(); auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); - return static_cast(ticks_since_epoch) & ((1ULL << timestamp_bits_count) - 1); + return static_cast(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1); } -struct SnowflakeComponents { +struct SnowflakeId +{ uint64_t timestamp; uint64_t machind_id; uint64_t machine_seq_num; }; -SnowflakeComponents toComponents(uint64_t snowflake) { - return { - .timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)), - .machind_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), - .machine_seq_num = (snowflake & machine_seq_num_mask) - }; +SnowflakeId toSnowflakeId(uint64_t snowflake) +{ + return {.timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)), + .machind_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), + .machine_seq_num = (snowflake & machine_seq_num_mask)}; } -uint64_t toSnowflakeID(SnowflakeComponents components) { +uint64_t fromSnowflakeId(SnowflakeId components) +{ return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) | components.machind_id << (machine_seq_num_bits_count) | components.machine_seq_num); } -struct RangeOfSnowflakeIDs { - /// [begin, end) - SnowflakeComponents begin, end; +struct SnowflakeIdRange +{ + SnowflakeId begin; /// inclusive + SnowflakeId end; /// exclusive }; -/* Get range of `input_rows_count` Snowflake IDs from `max(available, now)` - -1. Calculate Snowflake ID by current timestamp (`now`) -2. `begin = max(available, now)` -3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow -*/ -RangeOfSnowflakeIDs getRangeOfAvailableIDs(const SnowflakeComponents& available, size_t input_rows_count) +/// To get the range of `input_rows_count` Snowflake IDs from `max(available, now)`: +/// 1. calculate Snowflake ID by current timestamp (`now`) +/// 2. `begin = max(available, now)` +/// 3. Calculate `end = begin + input_rows_count` handling `machine_seq_num` overflow +SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count) { /// 1. `now` - SnowflakeComponents begin = { - .timestamp = getTimestamp(), - .machind_id = getMachineID(), - .machine_seq_num = 0 - }; + SnowflakeId begin = {.timestamp = getTimestamp(), .machind_id = getMachineId(), .machine_seq_num = 0}; /// 2. `begin` if (begin.timestamp <= available.timestamp) @@ -108,7 +103,7 @@ RangeOfSnowflakeIDs getRangeOfAvailableIDs(const SnowflakeComponents& available, } /// 3. `end = begin + input_rows_count` - SnowflakeComponents end; + SnowflakeId end; const uint64_t seq_nums_in_current_timestamp_left = (max_machine_seq_num - begin.machine_seq_num + 1); if (input_rows_count >= seq_nums_in_current_timestamp_left) /// if sequence numbers in current timestamp is not enough for rows => update timestamp @@ -125,22 +120,22 @@ RangeOfSnowflakeIDs getRangeOfAvailableIDs(const SnowflakeComponents& available, struct GlobalCounterPolicy { static constexpr auto name = "generateSnowflakeID"; - static constexpr auto doc_description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. Function generateSnowflakeID guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; /// Guarantee counter monotonicity within one timestamp across all threads generating Snowflake IDs simultaneously. struct Data { static inline std::atomic lowest_available_snowflake_id = 0; - SnowflakeComponents reserveRange(size_t input_rows_count) + SnowflakeId reserveRange(size_t input_rows_count) { uint64_t available_snowflake_id = lowest_available_snowflake_id.load(); - RangeOfSnowflakeIDs range; + SnowflakeIdRange range; do { - range = getRangeOfAvailableIDs(toComponents(available_snowflake_id), input_rows_count); + range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count); } - while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, toSnowflakeID(range.end))); + while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); /// if `compare_exhange` failed => another thread updated `lowest_available_snowflake_id` and we should try again /// completed => range of IDs [begin, end) is reserved, can return the beginning of the range @@ -152,17 +147,17 @@ struct GlobalCounterPolicy struct ThreadLocalCounterPolicy { static constexpr auto name = "generateSnowflakeIDThreadMonotonic"; - static constexpr auto doc_description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)"; + static constexpr auto description = R"(Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. This function behaves like generateSnowflakeID but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs.)"; /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. struct Data { static inline thread_local uint64_t lowest_available_snowflake_id = 0; - SnowflakeComponents reserveRange(size_t input_rows_count) + SnowflakeId reserveRange(size_t input_rows_count) { - RangeOfSnowflakeIDs range = getRangeOfAvailableIDs(toComponents(lowest_available_snowflake_id), input_rows_count); - lowest_available_snowflake_id = toSnowflakeID(range.end); + SnowflakeIdRange range = getRangeOfAvailableIds(toSnowflakeId(lowest_available_snowflake_id), input_rows_count); + lowest_available_snowflake_id = fromSnowflakeId(range.end); return range.begin; } }; @@ -188,7 +183,7 @@ public: { FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ - {"expr", nullptr, nullptr, "Arbitrary Expression"} + {"expr", nullptr, nullptr, "Arbitrary expression"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -200,17 +195,18 @@ public: auto col_res = ColumnVector::create(); typename ColumnVector::Container & vec_to = col_res->getData(); - vec_to.resize(input_rows_count); - if (input_rows_count != 0) { + vec_to.resize(input_rows_count); + typename FillPolicy::Data data; + /// get the begin of available snowflake ids range - SnowflakeComponents snowflake_id = data.reserveRange(input_rows_count); + SnowflakeId snowflake_id = data.reserveRange(input_rows_count); for (UInt64 & to_row : vec_to) { - to_row = toSnowflakeID(snowflake_id); + to_row = fromSnowflakeId(snowflake_id); if (snowflake_id.machine_seq_num++ == max_machine_seq_num) { snowflake_id.machine_seq_num = 0; @@ -225,20 +221,20 @@ public: }; template -void registerSnowflakeIDGenerator(auto& factory) +void registerSnowflakeIDGenerator(auto & factory) { static constexpr auto doc_syntax_format = "{}([expression])"; static constexpr auto example_format = "SELECT {}()"; static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; - FunctionDocumentation::Description doc_description = FillPolicy::doc_description; - FunctionDocumentation::Syntax doc_syntax = fmt::format(doc_syntax_format, FillPolicy::name); - FunctionDocumentation::Arguments doc_arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; - FunctionDocumentation::ReturnedValue doc_returned_value = "A value of type UInt64"; - FunctionDocumentation::Examples doc_examples = {{"uuid", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; - FunctionDocumentation::Categories doc_categories = {"Snowflake ID"}; + FunctionDocumentation::Description description = FillPolicy::description; + FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue returned_value = "A value of type UInt64"; + FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories categories = {"Snowflake ID"}; - factory.template registerFunction>({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); + factory.template registerFunction>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive); } REGISTER_FUNCTION(GenerateSnowflakeID) diff --git a/src/Functions/generateUUIDv7.cpp b/src/Functions/generateUUIDv7.cpp index 411a3a076ac..f2a82431c0a 100644 --- a/src/Functions/generateUUIDv7.cpp +++ b/src/Functions/generateUUIDv7.cpp @@ -76,7 +76,7 @@ void setVariant(UUID & uuid) struct FillAllRandomPolicy { static constexpr auto name = "generateUUIDv7NonMonotonic"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), and a random field (74 bit, including a 2-bit variant field "2") to distinguish UUIDs within a millisecond. This function is the fastest generateUUIDv7* function but it gives no monotonicity guarantees within a timestamp.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), and a random field (74 bit, including a 2-bit variant field "2") to distinguish UUIDs within a millisecond. This function is the fastest generateUUIDv7* function but it gives no monotonicity guarantees within a timestamp.)"; struct Data { void generate(UUID & uuid, uint64_t ts) @@ -136,7 +136,7 @@ struct CounterFields struct GlobalCounterPolicy { static constexpr auto name = "generateUUIDv7"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. Function generateUUIDv7 guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. Function generateUUIDv7 guarantees that the counter field within a timestamp increments monotonically across all function invocations in concurrently running threads and queries.)"; /// Guarantee counter monotonicity within one timestamp across all threads generating UUIDv7 simultaneously. struct Data @@ -159,7 +159,7 @@ struct GlobalCounterPolicy struct ThreadLocalCounterPolicy { static constexpr auto name = "generateUUIDv7ThreadMonotonic"; - static constexpr auto doc_description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. This function behaves like generateUUIDv7 but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate UUIDs.)"; + static constexpr auto description = R"(Generates a UUID of version 7. The generated UUID contains the current Unix timestamp in milliseconds (48 bits), followed by version "7" (4 bits), a counter (42 bit, including a variant field "2", 2 bit) to distinguish UUIDs within a millisecond, and a random field (32 bits). For any given timestamp (unix_ts_ms), the counter starts at a random value and is incremented by 1 for each new UUID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to a random new start value. This function behaves like generateUUIDv7 but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate UUIDs.)"; /// Guarantee counter monotonicity within one timestamp within the same thread. Faster than GlobalCounterPolicy if a query uses multiple threads. struct Data @@ -186,7 +186,6 @@ class FunctionGenerateUUIDv7Base : public IFunction, public FillPolicy { public: String getName() const final { return FillPolicy::name; } - size_t getNumberOfArguments() const final { return 0; } bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const final { return false; } @@ -198,7 +197,7 @@ public: { FunctionArgumentDescriptors mandatory_args; FunctionArgumentDescriptors optional_args{ - {"expr", nullptr, nullptr, "Arbitrary Expression"} + {"expr", nullptr, nullptr, "Arbitrary expression"} }; validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args); @@ -264,20 +263,20 @@ private: }; template -void registerUUIDv7Generator(auto& factory) +void registerUUIDv7Generator(auto & factory) { static constexpr auto doc_syntax_format = "{}([expression])"; static constexpr auto example_format = "SELECT {}()"; static constexpr auto multiple_example_format = "SELECT {f}(1), {f}(2)"; - FunctionDocumentation::Description doc_description = FillPolicy::doc_description; - FunctionDocumentation::Syntax doc_syntax = fmt::format(doc_syntax_format, FillPolicy::name); - FunctionDocumentation::Arguments doc_arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; - FunctionDocumentation::ReturnedValue doc_returned_value = "A value of type UUID version 7."; - FunctionDocumentation::Examples doc_examples = {{"uuid", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; - FunctionDocumentation::Categories doc_categories = {"UUID"}; + FunctionDocumentation::Description description = FillPolicy::description; + FunctionDocumentation::Syntax syntax = fmt::format(doc_syntax_format, FillPolicy::name); + FunctionDocumentation::Arguments arguments = {{"expression", "The expression is used to bypass common subexpression elimination if the function is called multiple times in a query but otherwise ignored. Optional."}}; + FunctionDocumentation::ReturnedValue returned_value = "A value of type UUID version 7."; + FunctionDocumentation::Examples examples = {{"single", fmt::format(example_format, FillPolicy::name), ""}, {"multiple", fmt::format(multiple_example_format, fmt::arg("f", FillPolicy::name)), ""}}; + FunctionDocumentation::Categories categories = {"UUID"}; - factory.template registerFunction>({doc_description, doc_syntax, doc_arguments, doc_returned_value, doc_examples, doc_categories}, FunctionFactory::CaseInsensitive); + factory.template registerFunction>({description, syntax, arguments, returned_value, examples, categories}, FunctionFactory::CaseInsensitive); } REGISTER_FUNCTION(GenerateUUIDv7) From 2315991504b1e95d7bb2594e54e3c6f749897d79 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 23 May 2024 18:41:14 +0200 Subject: [PATCH 116/158] Build fix --- src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index df8fb6f6656..fb0f0ba9154 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -67,12 +67,11 @@ void MergeTreeDataPartWriterCompact::initDynamicStreamsIfNeeded(const Block & bl return; is_dynamic_streams_initialized = true; - auto storage_snapshot = std::make_shared(data_part->storage, metadata_snapshot); for (const auto & column : columns_list) { if (column.type->hasDynamicSubcolumns()) { - auto compression = storage_snapshot->getCodecDescOrDefault(column.name, default_codec); + auto compression = getCodecDescOrDefault(column.name, default_codec); addStreams(column, block.getByName(column.name).column, compression); } } From dab090e629afd3730457599d84e147bb512a1e81 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 17:14:06 +0000 Subject: [PATCH 117/158] Cosmetics, pt. VII (includes a move of all snowflake-related functions in one document) --- .../functions/type-conversion-functions.md | 140 ---------------- .../sql-reference/functions/uuid-functions.md | 155 +++++++++++++++++- 2 files changed, 149 insertions(+), 146 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index ea08ffa50e7..bab92ff1e67 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -1979,143 +1979,3 @@ Result: │ 2,"good" │ └───────────────────────────────────────────┘ ``` - -## snowflakeToDateTime - -Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](/docs/en/sql-reference/data-types/datetime.md) format. - -**Syntax** - -``` sql -snowflakeToDateTime(value[, time_zone]) -``` - -**Arguments** - -- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md). -- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md). - -**Returned value** - -- The timestamp component of `value` as a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value. - -**Example** - -Query: - -``` sql -SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC'); -``` - -Result: - -```response - -┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐ -│ 2021-08-15 10:57:56 │ -└──────────────────────────────────────────────────────────────────┘ -``` - -## snowflakeToDateTime64 - -Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) format. - -**Syntax** - -``` sql -snowflakeToDateTime64(value[, time_zone]) -``` - -**Arguments** - -- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md). -- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md). - -**Returned value** - -- The timestamp component of `value` as a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) with scale = 3, i.e. millisecond precision. - -**Example** - -Query: - -``` sql -SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC'); -``` - -Result: - -```response - -┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐ -│ 2021-08-15 10:58:19.841 │ -└────────────────────────────────────────────────────────────────────┘ -``` - -## dateTimeToSnowflake - -Converts a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. - -**Syntax** - -``` sql -dateTimeToSnowflake(value) -``` - -**Arguments** - -- `value` — Date with time. [DateTime](/docs/en/sql-reference/data-types/datetime.md). - -**Returned value** - -- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time. - -**Example** - -Query: - -``` sql -WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dateTimeToSnowflake(dt); -``` - -Result: - -```response -┌─dateTimeToSnowflake(dt)─┐ -│ 1426860702823350272 │ -└─────────────────────────┘ -``` - -## dateTime64ToSnowflake - -Convert a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. - -**Syntax** - -``` sql -dateTime64ToSnowflake(value) -``` - -**Arguments** - -- `value` — Date with time. [DateTime64](/docs/en/sql-reference/data-types/datetime64.md). - -**Returned value** - -- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time. - -**Example** - -Query: - -``` sql -WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dateTime64ToSnowflake(dt64); -``` - -Result: - -```response -┌─dateTime64ToSnowflake(dt64)─┐ -│ 1426860704886947840 │ -└─────────────────────────────┘ -``` diff --git a/docs/en/sql-reference/functions/uuid-functions.md b/docs/en/sql-reference/functions/uuid-functions.md index 80d7215b9ef..7c264450ef0 100644 --- a/docs/en/sql-reference/functions/uuid-functions.md +++ b/docs/en/sql-reference/functions/uuid-functions.md @@ -674,7 +674,7 @@ Result: └──────────────────────────────────────────────────────────────────────────────────────┘ ``` -## serverUUID() +## serverUUID Returns the random UUID generated during the first start of the ClickHouse server. The UUID is stored in file `uuid` in the ClickHouse server directory (e.g. `/var/lib/clickhouse/`) and retained between server restarts. @@ -692,9 +692,9 @@ Type: [UUID](../data-types/uuid.md). ## generateSnowflakeID -Generates a [Snowflake ID](https://github.com/twitter-archive/snowflake/tree/b3f6a3c6ca8e1b6847baa6ff42bf72201e2c2231). +Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID). -Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. @@ -756,11 +756,14 @@ SELECT generateSnowflakeID(1), generateSnowflakeID(2); ## generateSnowflakeIDThreadMonotonic -Generates a [Snowflake ID](https://github.com/twitter-archive/snowflake/tree/b3f6a3c6ca8e1b6847baa6ff42bf72201e2c2231). +Generates a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID). -Generates a Snowflake ID. The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +The generated Snowflake ID contains the current Unix timestamp in milliseconds 41 (+ 1 top zero bit) bits, followed by machine id (10 bits), a counter (12 bits) to distinguish IDs within a millisecond. +For any given timestamp (unix_ts_ms), the counter starts at 0 and is incremented by 1 for each new Snowflake ID until the timestamp changes. +In case the counter overflows, the timestamp field is incremented by 1 and the counter is reset to 0. -This function behaves like `generateSnowflakeID` but gives no guarantee on counter monotony across different simultaneous requests. Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs. +This function behaves like `generateSnowflakeID` but gives no guarantee on counter monotony across different simultaneous requests. +Monotonicity within one timestamp is guaranteed only within the same thread calling this function to generate Snowflake IDs. ``` 0 1 2 3 @@ -816,6 +819,146 @@ SELECT generateSnowflakeIDThreadMonotonic(1), generateSnowflakeIDThreadMonotonic └───────────────────────────────────────┴───────────────────────────────────────┘ ``` +## snowflakeToDateTime + +Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime](/docs/en/sql-reference/data-types/datetime.md) format. + +**Syntax** + +``` sql +snowflakeToDateTime(value[, time_zone]) +``` + +**Arguments** + +- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC'); +``` + +Result: + +```response + +┌─snowflakeToDateTime(CAST('1426860702823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:57:56 │ +└──────────────────────────────────────────────────────────────────┘ +``` + +## snowflakeToDateTime64 + +Extracts the timestamp component of a [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) in [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) format. + +**Syntax** + +``` sql +snowflakeToDateTime64(value[, time_zone]) +``` + +**Arguments** + +- `value` — Snowflake ID. [Int64](/docs/en/sql-reference/data-types/int-uint.md). +- `time_zone` — [Timezone](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](/docs/en/sql-reference/data-types/string.md). + +**Returned value** + +- The timestamp component of `value` as a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) with scale = 3, i.e. millisecond precision. + +**Example** + +Query: + +``` sql +SELECT snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC'); +``` + +Result: + +```response + +┌─snowflakeToDateTime64(CAST('1426860802823350272', 'Int64'), 'UTC')─┐ +│ 2021-08-15 10:58:19.841 │ +└────────────────────────────────────────────────────────────────────┘ +``` + +## dateTimeToSnowflake + +Converts a [DateTime](/docs/en/sql-reference/data-types/datetime.md) value to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTimeToSnowflake(value) +``` + +**Arguments** + +- `value` — Date with time. [DateTime](/docs/en/sql-reference/data-types/datetime.md). + +**Returned value** + +- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time. + +**Example** + +Query: + +``` sql +WITH toDateTime('2021-08-15 18:57:56', 'Asia/Shanghai') AS dt SELECT dateTimeToSnowflake(dt); +``` + +Result: + +```response +┌─dateTimeToSnowflake(dt)─┐ +│ 1426860702823350272 │ +└─────────────────────────┘ +``` + +## dateTime64ToSnowflake + +Convert a [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) to the first [Snowflake ID](https://en.wikipedia.org/wiki/Snowflake_ID) at the giving time. + +**Syntax** + +``` sql +dateTime64ToSnowflake(value) +``` + +**Arguments** + +- `value` — Date with time. [DateTime64](/docs/en/sql-reference/data-types/datetime64.md). + +**Returned value** + +- Input value converted to the [Int64](/docs/en/sql-reference/data-types/int-uint.md) data type as the first Snowflake ID at that time. + +**Example** + +Query: + +``` sql +WITH toDateTime64('2021-08-15 18:57:56.492', 3, 'Asia/Shanghai') AS dt64 SELECT dateTime64ToSnowflake(dt64); +``` + +Result: + +```response +┌─dateTime64ToSnowflake(dt64)─┐ +│ 1426860704886947840 │ +└─────────────────────────────┘ +``` + ## See also - [dictGetUUID](../../sql-reference/functions/ext-dict-functions.md#ext_dict_functions-other) From 5d82a94615ef8a9fb7c39787d0e2b191641cbcb8 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 17:22:59 +0000 Subject: [PATCH 118/158] Revert generateSerialID --- src/Functions/generateSerialID.cpp | 167 ------------------ .../03129_serial_test_zookeeper.reference | 13 -- .../03129_serial_test_zookeeper.sql | 12 -- 3 files changed, 192 deletions(-) delete mode 100644 src/Functions/generateSerialID.cpp delete mode 100644 tests/queries/0_stateless/03129_serial_test_zookeeper.reference delete mode 100644 tests/queries/0_stateless/03129_serial_test_zookeeper.sql diff --git a/src/Functions/generateSerialID.cpp b/src/Functions/generateSerialID.cpp deleted file mode 100644 index db26d0d684b..00000000000 --- a/src/Functions/generateSerialID.cpp +++ /dev/null @@ -1,167 +0,0 @@ -#include "Common/Exception.h" -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int KEEPER_EXCEPTION; -} - -constexpr auto function_node_name = "/serial_ids/"; -constexpr size_t MAX_SERIES_NUMBER = 1000; // ? - -class FunctionSerial : public IFunction -{ -private: - mutable zkutil::ZooKeeperPtr zk; - ContextPtr context; - -public: - static constexpr auto name = "generateSerialID"; - - explicit FunctionSerial(ContextPtr context_) : context(context_) - { - if (context->hasZooKeeper()) { - zk = context->getZooKeeper(); - } - } - - static FunctionPtr create(ContextPtr context) - { - return std::make_shared(std::move(context)); - } - - String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } - bool isStateful() const override { return true; } - bool isDeterministic() const override { return false; } - bool isDeterministicInScopeOfQuery() const override { return false; } - bool isSuitableForConstantFolding() const override { return false; } - bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForNothing() const override { return false; } - bool canBeExecutedOnDefaultArguments() const override { return false; } - bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; } - bool hasInformationAboutMonotonicity() const override { return true; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - FunctionArgumentDescriptors mandatory_args{ - {"series identifier", static_cast(&isStringOrFixedString), nullptr, "String or FixedString"} - }; - validateFunctionArgumentTypes(*this, arguments, mandatory_args); - - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - if (zk == nullptr) - throw Exception(ErrorCodes::KEEPER_EXCEPTION, - "ZooKeeper is not configured for function {}", - getName()); - if (zk->expired()) - zk = context->getZooKeeper(); - - // slow? - if (zk->exists(function_node_name) && zk->getChildren(function_node_name).size() == MAX_SERIES_NUMBER) { - throw Exception(ErrorCodes::KEEPER_EXCEPTION, - "At most {} serial nodes can be created", - MAX_SERIES_NUMBER); - } - - auto col_res = ColumnVector::create(); - typename ColumnVector::Container & vec_to = col_res->getData(); - - vec_to.resize(input_rows_count); - - const auto & serial_path = function_node_name + arguments[0].column->getDataAt(0).toString(); - - /// CAS in ZooKeeper - /// `get` value and version, `trySet` new with version check - /// I didn't get how to do it with `multi` - - Int64 counter; - std::string counter_path = serial_path + "/counter"; - - // if serial name used first time - zk->createAncestors(counter_path); - zk->createIfNotExists(counter_path, "1"); - - Coordination::Stat stat; - while (true) - { - const String counter_string = zk->get(counter_path, &stat); - counter = std::stoll(counter_string); - String updated_counter = std::to_string(counter + input_rows_count); - const Coordination::Error err = zk->trySet(counter_path, updated_counter); - if (err == Coordination::Error::ZOK) - { - // CAS is done - break; - } - if (err != Coordination::Error::ZBADVERSION) - { - throw Exception(ErrorCodes::KEEPER_EXCEPTION, - "ZooKeeper trySet operation failed with unexpected error = {} in function {}", - err, getName()); - } - } - - // Make a result - for (auto & val : vec_to) - { - val = counter; - ++counter; - } - - return col_res; - } - -}; - -REGISTER_FUNCTION(Serial) -{ - factory.registerFunction(FunctionDocumentation - { - .description=R"( -Generates and returns sequential numbers starting from the previous counter value. -This function takes a constant string argument - a series identifier. -The server should be configured with a ZooKeeper. -)", - .syntax = "generateSerialID(identifier)", - .arguments{ - {"series identifier", "Series identifier (String or FixedString)"} - }, - .returned_value = "Sequential numbers of type Int64 starting from the previous counter value", - .examples{ - {"first call", "SELECT generateSerialID('id1')", R"( -┌─generateSerialID('id1')──┐ -│ 1 │ -└──────────────────────────┘)"}, - {"second call", "SELECT generateSerialID('id1')", R"( -┌─generateSerialID('id1')──┐ -│ 2 │ -└──────────────────────────┘)"}, - {"column call", "SELECT *, generateSerialID('id1') FROM test_table", R"( -┌─CounterID─┬─UserID─┬─ver─┬─generateSerialID('id1')──┐ -│ 1 │ 3 │ 3 │ 3 │ -│ 1 │ 1 │ 1 │ 4 │ -│ 1 │ 2 │ 2 │ 5 │ -│ 1 │ 5 │ 5 │ 6 │ -│ 1 │ 4 │ 4 │ 7 │ -└───────────┴────────┴─────┴──────────────────────────┘ - )"}}, - .categories{"Unique identifiers"} - }); -} - -} diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.reference b/tests/queries/0_stateless/03129_serial_test_zookeeper.reference deleted file mode 100644 index 479030db4be..00000000000 --- a/tests/queries/0_stateless/03129_serial_test_zookeeper.reference +++ /dev/null @@ -1,13 +0,0 @@ -1 -2 -1 -3 -4 -5 -6 -7 -1 1 -2 2 -3 3 -4 4 -5 5 diff --git a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql b/tests/queries/0_stateless/03129_serial_test_zookeeper.sql deleted file mode 100644 index 2bd60656259..00000000000 --- a/tests/queries/0_stateless/03129_serial_test_zookeeper.sql +++ /dev/null @@ -1,12 +0,0 @@ --- Tags: zookeeper - -SELECT generateSerialID('x'); -SELECT generateSerialID('x'); -SELECT generateSerialID('y'); -SELECT generateSerialID('x') FROM numbers(5); - -SELECT generateSerialID(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT generateSerialID('x', 'y'); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT generateSerialID(1); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } - -SELECT generateSerialID('z'), generateSerialID('z') FROM numbers(5); From 12f60a4969acda49422aef5d5d6fc431a71109f7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 18:00:53 +0000 Subject: [PATCH 119/158] Cosmetics, pt. VIII --- src/Functions/generateSnowflakeID.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 4e61bd9fb1c..617693f017c 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -42,6 +42,13 @@ constexpr uint64_t machine_seq_num_mask = (1ull << machine_seq_num_bits_count) - /// max values constexpr uint64_t max_machine_seq_num = machine_seq_num_mask; +uint64_t getTimestamp() +{ + auto now = std::chrono::system_clock::now(); + auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); + return static_cast(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1); +} + uint64_t getMachineId() { UUID server_uuid = ServerUUID::get(); @@ -52,31 +59,24 @@ uint64_t getMachineId() return (((hi * 11) ^ (lo * 17)) & machine_id_mask) >> machine_seq_num_bits_count; } -uint64_t getTimestamp() -{ - auto now = std::chrono::system_clock::now(); - auto ticks_since_epoch = std::chrono::duration_cast(now.time_since_epoch()).count(); - return static_cast(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1); -} - struct SnowflakeId { uint64_t timestamp; - uint64_t machind_id; + uint64_t machine_id; uint64_t machine_seq_num; }; SnowflakeId toSnowflakeId(uint64_t snowflake) { return {.timestamp = (snowflake >> (machine_id_bits_count + machine_seq_num_bits_count)), - .machind_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), + .machine_id = ((snowflake & machine_id_mask) >> machine_seq_num_bits_count), .machine_seq_num = (snowflake & machine_seq_num_mask)}; } uint64_t fromSnowflakeId(SnowflakeId components) { return (components.timestamp << (machine_id_bits_count + machine_seq_num_bits_count) | - components.machind_id << (machine_seq_num_bits_count) | + components.machine_id << (machine_seq_num_bits_count) | components.machine_seq_num); } @@ -93,7 +93,7 @@ struct SnowflakeIdRange SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t input_rows_count) { /// 1. `now` - SnowflakeId begin = {.timestamp = getTimestamp(), .machind_id = getMachineId(), .machine_seq_num = 0}; + SnowflakeId begin = {.timestamp = getTimestamp(), .machine_id = getMachineId(), .machine_seq_num = 0}; /// 2. `begin` if (begin.timestamp <= available.timestamp) @@ -111,7 +111,7 @@ SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t in else end.timestamp = begin.timestamp; - end.machind_id = begin.machind_id; + end.machine_id = begin.machine_id; end.machine_seq_num = (begin.machine_seq_num + input_rows_count) & machine_seq_num_mask; return {begin, end}; From ae8ceaa35e0cb6804774881e05bccf07ab23aa19 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 18:38:30 +0000 Subject: [PATCH 120/158] Cosmetics, pt. IX and cached machineId computation --- src/Functions/generateSnowflakeID.cpp | 25 +++++++++++++------ .../03130_generateSnowflakeId.reference | 4 +-- .../0_stateless/03130_generateSnowflakeId.sql | 14 ++++++----- 3 files changed, 27 insertions(+), 16 deletions(-) diff --git a/src/Functions/generateSnowflakeID.cpp b/src/Functions/generateSnowflakeID.cpp index 617693f017c..c3f7701a05a 100644 --- a/src/Functions/generateSnowflakeID.cpp +++ b/src/Functions/generateSnowflakeID.cpp @@ -49,7 +49,7 @@ uint64_t getTimestamp() return static_cast(ticks_since_epoch) & ((1ull << timestamp_bits_count) - 1); } -uint64_t getMachineId() +uint64_t getMachineIdImpl() { UUID server_uuid = ServerUUID::get(); /// hash into 64 bits @@ -59,6 +59,12 @@ uint64_t getMachineId() return (((hi * 11) ^ (lo * 17)) & machine_id_mask) >> machine_seq_num_bits_count; } +uint64_t getMachineId() +{ + static uint64_t machine_id = getMachineIdImpl(); + return machine_id; +} + struct SnowflakeId { uint64_t timestamp; @@ -106,7 +112,7 @@ SnowflakeIdRange getRangeOfAvailableIds(const SnowflakeId & available, size_t in SnowflakeId end; const uint64_t seq_nums_in_current_timestamp_left = (max_machine_seq_num - begin.machine_seq_num + 1); if (input_rows_count >= seq_nums_in_current_timestamp_left) - /// if sequence numbers in current timestamp is not enough for rows => update timestamp + /// if sequence numbers in current timestamp is not enough for rows --> depending on how many elements input_rows_count overflows, forward timestamp by at least 1 tick end.timestamp = begin.timestamp + 1 + (input_rows_count - seq_nums_in_current_timestamp_left) / (max_machine_seq_num + 1); else end.timestamp = begin.timestamp; @@ -136,8 +142,8 @@ struct GlobalCounterPolicy range = getRangeOfAvailableIds(toSnowflakeId(available_snowflake_id), input_rows_count); } while (!lowest_available_snowflake_id.compare_exchange_weak(available_snowflake_id, fromSnowflakeId(range.end))); - /// if `compare_exhange` failed => another thread updated `lowest_available_snowflake_id` and we should try again - /// completed => range of IDs [begin, end) is reserved, can return the beginning of the range + /// if CAS failed --> another thread updated `lowest_available_snowflake_id` and we re-try + /// else --> our thread reserved ID range [begin, end) and return the beginning of the range return range.begin; } @@ -200,18 +206,21 @@ public: vec_to.resize(input_rows_count); typename FillPolicy::Data data; - - /// get the begin of available snowflake ids range - SnowflakeId snowflake_id = data.reserveRange(input_rows_count); + SnowflakeId snowflake_id = data.reserveRange(input_rows_count); /// returns begin of available snowflake ids range for (UInt64 & to_row : vec_to) { to_row = fromSnowflakeId(snowflake_id); - if (snowflake_id.machine_seq_num++ == max_machine_seq_num) + if (snowflake_id.machine_seq_num == max_machine_seq_num) { + /// handle overflow snowflake_id.machine_seq_num = 0; ++snowflake_id.timestamp; } + else + { + ++snowflake_id.machine_seq_num; + } } } diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 8cdced96770..6ec0cafab16 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,11 +1,11 @@ --- generateSnowflakeID -- +-- generateSnowflakeID 1 1 0 0 1 100 --- generateSnowflakeIDThreadMonotonic -- +-- generateSnowflakeIDThreadMonotonic 1 1 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index 3e994149d2b..903be5b786c 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -1,10 +1,11 @@ -SELECT '-- generateSnowflakeID --'; +SELECT '-- generateSnowflakeID'; + SELECT bitShiftLeft(toUInt64(generateSnowflakeID()), 52) = 0; -- check machine sequence number is zero SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero -SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -SELECT generateSnowflakeID() = generateSnowflakeID(1); -SELECT generateSnowflakeID(1) = generateSnowflakeID(1); +SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs +SELECT generateSnowflakeID() = generateSnowflakeID(1); -- same as ^^ +SELECT generateSnowflakeID(1) = generateSnowflakeID(1); -- enabled common subexpression elimination SELECT generateSnowflakeID(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } @@ -15,7 +16,8 @@ FROM FROM numbers(100) ); -SELECT '-- generateSnowflakeIDThreadMonotonic --'; +SELECT '-- generateSnowflakeIDThreadMonotonic'; + SELECT bitShiftLeft(toUInt64(generateSnowflakeIDThreadMonotonic()), 52) = 0; -- check machine sequence number is zero SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero @@ -26,4 +28,4 @@ FROM ( SELECT DISTINCT generateSnowflakeIDThreadMonotonic() FROM numbers(100) -); \ No newline at end of file +); From 40753ddefb0324d50bb8d455615da74828c7be76 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 23 May 2024 21:10:40 +0200 Subject: [PATCH 121/158] Update hdfs test --- tests/integration/test_storage_hdfs/test.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 6ee12a87ebf..eeffa8ed00b 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -326,7 +326,7 @@ def test_virtual_columns(started_cluster): hdfs_api.write_data("/file1", "1\n") hdfs_api.write_data("/file2", "2\n") hdfs_api.write_data("/file3", "3\n") - expected = "1\tfile1\t/file1\n2\tfile2\t/file2\n3\tfile3\t/file3\n" + expected = "1\tfile1\tfile1\n2\tfile2\tfile2\n3\tfile3\tfile3\n" assert ( node1.query( "select id, _file as file_name, _path as file_path from virtual_cols order by id" @@ -493,13 +493,13 @@ def test_hdfsCluster(started_cluster): actual = node1.query( "select id, _file as file_name, _path as file_path from hdfs('hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\t/test_hdfsCluster/file1\n2\tfile2\t/test_hdfsCluster/file2\n3\tfile3\t/test_hdfsCluster/file3\n" + expected = "1\tfile1\ttest_hdfsCluster/file1\n2\tfile2\ttest_hdfsCluster/file2\n3\tfile3\ttest_hdfsCluster/file3\n" assert actual == expected actual = node1.query( "select id, _file as file_name, _path as file_path from hdfsCluster('test_cluster_two_shards', 'hdfs://hdfs1:9000/test_hdfsCluster/file*', 'TSV', 'id UInt32') order by id" ) - expected = "1\tfile1\t/test_hdfsCluster/file1\n2\tfile2\t/test_hdfsCluster/file2\n3\tfile3\t/test_hdfsCluster/file3\n" + expected = "1\tfile1\ttest_hdfsCluster/file1\n2\tfile2\ttest_hdfsCluster/file2\n3\tfile3\ttest_hdfsCluster/file3\n" assert actual == expected fs.delete(dir, recursive=True) @@ -665,7 +665,7 @@ def test_virtual_columns_2(started_cluster): node1.query(f"insert into table function {table_function} SELECT 1, 'kek'") result = node1.query(f"SELECT _path FROM {table_function}") - assert result.strip() == "/parquet_2" + assert result.strip() == "parquet_2" table_function = ( f"hdfs('hdfs://hdfs1:9000/parquet_3', 'Parquet', 'a Int32, _path String')" @@ -978,25 +978,25 @@ def test_read_subcolumns(started_cluster): f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.tsv', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "2\t/test_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" + assert res == "2\ttest_subcolumns.tsv\t(1,2)\ttest_subcolumns.tsv\t3\n" res = node.query( f"select a.b.d, _path, a.b, _file, a.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'a Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "2\t/test_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" + assert res == "2\ttest_subcolumns.jsonl\t(1,2)\ttest_subcolumns.jsonl\t3\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32)')" ) - assert res == "0\t/test_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" + assert res == "0\ttest_subcolumns.jsonl\t(0,0)\ttest_subcolumns.jsonl\t0\n" res = node.query( f"select x.b.d, _path, x.b, _file, x.e from hdfs('hdfs://hdfs1:9000/test_subcolumns.jsonl', auto, 'x Tuple(b Tuple(c UInt32, d UInt32), e UInt32) default ((42, 42), 42)')" ) - assert res == "42\t/test_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" + assert res == "42\ttest_subcolumns.jsonl\t(42,42)\ttest_subcolumns.jsonl\t42\n" def test_union_schema_inference_mode(started_cluster): From dac31fb92a80982ec0a98472485fa02c4b917c07 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 17:29:00 +0000 Subject: [PATCH 122/158] Include settings into query cache key --- src/Interpreters/Cache/QueryCache.cpp | 37 ++++++++-- src/Interpreters/Cache/QueryCache.h | 5 +- src/Interpreters/executeQuery.cpp | 4 +- .../02494_query_cache_key.reference | 6 ++ .../0_stateless/02494_query_cache_key.sql | 70 +++++++++++++++++++ .../02494_query_cache_use_database.reference | 2 - .../02494_query_cache_use_database.sql | 30 -------- 7 files changed, 113 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_key.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_key.sql delete mode 100644 tests/queries/0_stateless/02494_query_cache_use_database.reference delete mode 100644 tests/queries/0_stateless/02494_query_cache_use_database.sql diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 4b10bfd3dcd..a3fe8c2e779 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -126,6 +126,11 @@ bool astContainsSystemTables(ASTPtr ast, ContextPtr context) namespace { +bool isQueryCacheRelatedSetting(const String & setting_name) +{ + return setting_name.starts_with("query_cache_") || setting_name.ends_with("_query_cache"); +} + class RemoveQueryCacheSettingsMatcher { public: @@ -141,7 +146,7 @@ public: auto is_query_cache_related_setting = [](const auto & change) { - return change.name.starts_with("query_cache_") || change.name.ends_with("_query_cache"); + return isQueryCacheRelatedSetting(change.name); }; std::erase_if(set_clause->changes, is_query_cache_related_setting); @@ -177,11 +182,11 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } -IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) +IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database, const Settings & settings) { ast = removeQueryCacheSettings(ast); - /// Hash the AST, it must consider aliases (issue #56258) + /// Hash the AST, we must consider aliases (issue #56258) SipHash hash; ast->updateTreeHash(hash, /*ignore_aliases=*/ false); @@ -189,6 +194,25 @@ IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) /// tables (issue #64136) hash.update(current_database); + /// Finally, hash the (changed) settings as they might affect the query result (e.g. think of settings `additional_table_filters` and `limit`). + /// Note: allChanged() returns the settings in random order. Also, update()-s of the composite hash must be done in deterministic order. + /// Therefore, collect and sort the settings first, then hash them. + Settings::Range changed_settings = settings.allChanged(); + std::vector> changed_settings_sorted; /// (name, value) + for (const auto & setting : changed_settings) + { + const String & name = setting.getName(); + const String & value = setting.getValueString(); + if (!isQueryCacheRelatedSetting(name)) /// see removeQueryCacheSettings() why this is a good idea + changed_settings_sorted.push_back({name, value}); + } + std::sort(changed_settings_sorted.begin(), changed_settings_sorted.end(), [](auto & lhs, auto & rhs) { return lhs.first < rhs.first; }); + for (const auto & setting : changed_settings_sorted) + { + hash.update(setting.first); + hash.update(setting.second); + } + return getSipHash128AsPair(hash); } @@ -204,12 +228,13 @@ String queryStringFromAST(ASTPtr ast) QueryCache::Key::Key( ASTPtr ast_, const String & current_database, + const Settings & settings, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, bool is_compressed_) - : ast_hash(calculateAstHash(ast_, current_database)) + : ast_hash(calculateAstHash(ast_, current_database, settings)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -220,8 +245,8 @@ QueryCache::Key::Key( { } -QueryCache::Key::Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_) - : QueryCache::Key(ast_, current_database, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles +QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_) + : QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index b5b6f477137..461197cac32 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -14,6 +14,8 @@ namespace DB { +struct Settings; + /// Does AST contain non-deterministic functions like rand() and now()? bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context); @@ -89,6 +91,7 @@ public: /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, const String & current_database, + const Settings & settings, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, @@ -96,7 +99,7 @@ public: bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 56f08dbb902..0b5f68f27f6 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1101,7 +1101,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), settings, context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1224,7 +1224,7 @@ static std::tuple executeQueryImpl( && (!ast_contains_system_tables || system_table_handling == QueryCacheSystemTableHandling::Save)) { QueryCache::Key key( - ast, context->getCurrentDatabase(), res.pipeline.getHeader(), + ast, context->getCurrentDatabase(), settings, res.pipeline.getHeader(), context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), diff --git a/tests/queries/0_stateless/02494_query_cache_key.reference b/tests/queries/0_stateless/02494_query_cache_key.reference new file mode 100644 index 00000000000..8f5b61192d5 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_key.reference @@ -0,0 +1,6 @@ +Test (1) +1 +2 +Test (2) +4 +4 diff --git a/tests/queries/0_stateless/02494_query_cache_key.sql b/tests/queries/0_stateless/02494_query_cache_key.sql new file mode 100644 index 00000000000..d8c68e0d267 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_key.sql @@ -0,0 +1,70 @@ +-- Tags: no-parallel +-- Tag no-parallel: Messes with internal cache + +-- Tests that the key of the query cache is not only formed by the query AST but also by +-- (1) the current database (`USE db`, issue #64136), +-- (2) the query settings + + +SELECT 'Test (1)'; + +SYSTEM DROP QUERY CACHE; + +DROP DATABASE IF EXISTS db1; +DROP DATABASE IF EXISTS db2; + +CREATE DATABASE db1; +CREATE DATABASE db2; + +CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a); +CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a); + +INSERT INTO db1.tab values(1); +INSERT INTO db2.tab values(2); + +USE db1; +SELECT * FROM tab SETTINGS use_query_cache=1; + +USE db2; +SELECT * FROM tab SETTINGS use_query_cache=1; + +DROP DATABASE db1; +DROP DATABASE db2; + +SYSTEM DROP QUERY CACHE; + + +SELECT 'Test (2)'; + +-- test with query-level settings +SELECT 1 SETTINGS use_query_cache = 1, limit = 1, use_skip_indexes = 0 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 0 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 1 Format Null; +SELECT 1 SETTINGS use_query_cache = 1, max_block_size = 1 Format Null; + +-- 4x the same query but with different settings each. There should yield four entries in the query cache. +SELECT count(query) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; + +-- test with mixed session-level/query-level settings +SET use_query_cache = 1; +SET limit = 1; +SELECT 1 SETTINGS use_skip_indexes = 0 Format Null; +SET limit = default; +SET use_skip_indexes = 0; +SELECT 1 Format Null; +SET use_skip_indexes = 1; +SELECT 1 SETTINGS use_skip_indexes = 1 Format Null; +SET use_skip_indexes = default; +SET max_block_size = 1; +SELECT 1 Format Null; +SET max_block_size = default; + +SET use_query_cache = default; + +-- 4x the same query but with different settings each. There should yield four entries in the query cache. +SELECT count(query) FROM system.query_cache; + +SYSTEM DROP QUERY CACHE; + diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.reference b/tests/queries/0_stateless/02494_query_cache_use_database.reference deleted file mode 100644 index 1191247b6d9..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_use_database.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -2 diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.sql b/tests/queries/0_stateless/02494_query_cache_use_database.sql deleted file mode 100644 index df560f82ebb..00000000000 --- a/tests/queries/0_stateless/02494_query_cache_use_database.sql +++ /dev/null @@ -1,30 +0,0 @@ --- Tags: no-parallel, no-fasttest --- Tag no-fasttest: Depends on OpenSSL --- Tag no-parallel: Messes with internal cache - --- Test for issue #64136 - -SYSTEM DROP QUERY CACHE; - -DROP DATABASE IF EXISTS db1; -DROP DATABASE IF EXISTS db2; - -CREATE DATABASE db1; -CREATE DATABASE db2; - -CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a); -CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a); - -INSERT INTO db1.tab values(1); -INSERT INTO db2.tab values(2); - -USE db1; -SELECT * FROM tab SETTINGS use_query_cache=1; - -USE db2; -SELECT * FROM tab SETTINGS use_query_cache=1; - -DROP DATABASE db1; -DROP DATABASE db2; - -SYSTEM DROP QUERY CACHE; From 6e6e2944b56245cd5eefd14deb7dba7b8459b935 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 23 May 2024 21:26:33 +0000 Subject: [PATCH 123/158] Fix glitch in #62696 --- src/Functions/FunctionHelpers.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index 3b057779ffe..d85bb0e7060 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -21,8 +21,6 @@ namespace ErrorCodes const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) { - if (!column) - return {}; if (!isColumnConst(*column)) return {}; From 251010f109a538c770f830bc254e031924486c46 Mon Sep 17 00:00:00 2001 From: TTPO100AJIEX Date: Fri, 24 May 2024 02:14:26 +0300 Subject: [PATCH 124/158] Move protocol-server and inter-server management into separate classes Co-authored-by: Alex Koledaev --- programs/server/Server.cpp | 987 +----------------- programs/server/Server.h | 95 +- src/CMakeLists.txt | 1 + src/Server/ServersManager/IServersManager.cpp | 268 +++++ src/Server/ServersManager/IServersManager.h | 74 ++ .../ServersManager/InterServersManager.cpp | 327 ++++++ .../ServersManager/InterServersManager.h | 45 + .../ServersManager/ProtocolServersManager.cpp | 523 ++++++++++ .../ServersManager/ProtocolServersManager.h | 37 + 9 files changed, 1325 insertions(+), 1032 deletions(-) create mode 100644 src/Server/ServersManager/IServersManager.cpp create mode 100644 src/Server/ServersManager/IServersManager.h create mode 100644 src/Server/ServersManager/InterServersManager.cpp create mode 100644 src/Server/ServersManager/InterServersManager.h create mode 100644 src/Server/ServersManager/ProtocolServersManager.cpp create mode 100644 src/Server/ServersManager/ProtocolServersManager.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 223bc1f77e7..b62ae40924c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -6,8 +6,6 @@ #include #include #include -#include -#include #include #include #include @@ -44,11 +42,9 @@ #include #include #include -#include #include #include #include -#include #include #include #include @@ -83,29 +79,19 @@ #include #include #include -#include #include "MetricsTransmitter.h" #include -#include -#include #include #include #include #include #include -#include -#include -#include -#include -#include -#include -#include -#include #include +#include +#include #include #include #include -#include #include "config.h" #include @@ -119,19 +105,9 @@ #endif #if USE_SSL -# include # include #endif -#if USE_GRPC -# include -#endif - -#if USE_NURAFT -# include -# include -#endif - #if USE_JEMALLOC # include #endif @@ -159,18 +135,6 @@ namespace ProfileEvents { extern const Event MainConfigLoads; extern const Event ServerStartupMilliseconds; - extern const Event InterfaceNativeSendBytes; - extern const Event InterfaceNativeReceiveBytes; - extern const Event InterfaceHTTPSendBytes; - extern const Event InterfaceHTTPReceiveBytes; - extern const Event InterfacePrometheusSendBytes; - extern const Event InterfacePrometheusReceiveBytes; - extern const Event InterfaceInterserverSendBytes; - extern const Event InterfaceInterserverReceiveBytes; - extern const Event InterfaceMySQLSendBytes; - extern const Event InterfaceMySQLReceiveBytes; - extern const Event InterfacePostgreSQLSendBytes; - extern const Event InterfacePostgreSQLReceiveBytes; } namespace fs = std::filesystem; @@ -238,11 +202,9 @@ namespace DB namespace ErrorCodes { extern const int NO_ELEMENTS_IN_CONFIG; - extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; - extern const int NETWORK_ERROR; extern const int CORRUPTED_DATA; } @@ -257,115 +219,6 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } -Poco::Net::SocketAddress Server::socketBindListen( - const Poco::Util::AbstractConfiguration & config, - Poco::Net::ServerSocket & socket, - const std::string & host, - UInt16 port, - [[maybe_unused]] bool secure) const -{ - auto address = makeSocketAddress(host, port, &logger()); - socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config.getBool("listen_reuse_port", false)); - /// If caller requests any available port from the OS, discover it after binding. - if (port == 0) - { - address = socket.address(); - LOG_DEBUG(&logger(), "Requested any available port (port == 0), actual port is {:d}", address.port()); - } - - socket.listen(/* backlog = */ config.getUInt("listen_backlog", 4096)); - - return address; -} - -Strings getListenHosts(const Poco::Util::AbstractConfiguration & config) -{ - auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host"); - if (listen_hosts.empty()) - { - listen_hosts.emplace_back("::1"); - listen_hosts.emplace_back("127.0.0.1"); - } - return listen_hosts; -} - -Strings getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) -{ - auto interserver_listen_hosts = DB::getMultipleValuesFromConfig(config, "", "interserver_listen_host"); - if (!interserver_listen_hosts.empty()) - return interserver_listen_hosts; - - /// Use more general restriction in case of emptiness - return getListenHosts(config); -} - -bool getListenTry(const Poco::Util::AbstractConfiguration & config) -{ - bool listen_try = config.getBool("listen_try", false); - if (!listen_try) - { - Poco::Util::AbstractConfiguration::Keys protocols; - config.keys("protocols", protocols); - listen_try = - DB::getMultipleValuesFromConfig(config, "", "listen_host").empty() && - std::none_of(protocols.begin(), protocols.end(), [&](const auto & protocol) - { - return config.has("protocols." + protocol + ".host") && config.has("protocols." + protocol + ".port"); - }); - } - return listen_try; -} - - -void Server::createServer( - Poco::Util::AbstractConfiguration & config, - const std::string & listen_host, - const char * port_name, - bool listen_try, - bool start_server, - std::vector & servers, - CreateServerFunc && func) const -{ - /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. - if (config.getString(port_name, "").empty()) - return; - - /// If we already have an active server for this listen_host/port_name, don't create it again - for (const auto & server : servers) - { - if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name) - return; - } - - auto port = config.getInt(port_name); - try - { - servers.push_back(func(port)); - if (start_server) - { - servers.back().start(); - LOG_INFO(&logger(), "Listening for {}", servers.back().getDescription()); - } - global_context->registerServerPort(port_name, port); - } - catch (const Poco::Exception &) - { - if (listen_try) - { - LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " - "then consider to " - "specify not disabled IPv4 or IPv6 address to listen in element of configuration " - "file. Example for disabled IPv6: 0.0.0.0 ." - " Example for disabled IPv4: ::", - listen_host, port, getCurrentExceptionMessage(false)); - } - else - { - throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); - } - } -} - #if defined(OS_LINUX) namespace @@ -665,6 +518,7 @@ try ServerSettings server_settings; server_settings.loadSettingsFromConfig(config()); + Poco::ThreadPool server_pool(3, server_settings.max_connections); ASTAlterCommand::setFormatAlterCommandsWithParentheses(server_settings.format_alter_operations_with_parentheses); @@ -721,11 +575,6 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); - std::mutex servers_lock; - std::vector servers; - std::vector servers_to_start_before_tables; - /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -775,6 +624,10 @@ try bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); + std::mutex servers_lock; + ProtocolServersManager servers(context(), &logger()); + InterServersManager servers_to_start_before_tables(context(), &logger()); + // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. @@ -806,32 +659,7 @@ try LOG_DEBUG(log, "Shut down storages."); - if (!servers_to_start_before_tables.empty()) - { - LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); - size_t current_connections = 0; - { - std::lock_guard lock(servers_lock); - for (auto & server : servers_to_start_before_tables) - { - server.stop(); - current_connections += server.currentConnections(); - } - } - - if (current_connections) - LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); - else - LOG_INFO(log, "Closed all listening sockets."); - - if (current_connections > 0) - current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings.shutdown_wait_unfinished); - - if (current_connections) - LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); - else - LOG_INFO(log, "Closed connections to servers for tables."); - } + servers_to_start_before_tables.stopServers(server_settings, servers_lock); global_context->shutdownKeeperDispatcher(); @@ -928,19 +756,13 @@ try server_settings.asynchronous_heavy_metrics_update_period_s, [&]() -> std::vector { - std::vector metrics; - std::lock_guard lock(servers_lock); - metrics.reserve(servers_to_start_before_tables.size() + servers.size()); - - for (const auto & server : servers_to_start_before_tables) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - - for (const auto & server : servers) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - return metrics; - } - ); + std::vector metrics1 = servers_to_start_before_tables.getMetrics(); + std::vector metrics2 = servers.getMetrics(); + metrics1.reserve(metrics1.size() + metrics2.size()); + metrics1.insert(metrics1.end(), std::make_move_iterator(metrics2.begin()), std::make_move_iterator(metrics2.end())); + return metrics1; + }); zkutil::validateZooKeeperConfig(config()); bool has_zookeeper = zkutil::hasZooKeeperConfig(config()); @@ -1588,7 +1410,8 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); + servers.updateServers(*config, *this, servers_lock, server_pool, async_metrics, latest_config); + servers_to_start_before_tables.updateServers(*config, *this, servers_lock, server_pool, async_metrics, latest_config); } } @@ -1635,141 +1458,17 @@ try /// Must be the last. latest_config = config; }, - /* already_loaded = */ false); /// Reload it right now (initial loading) + /* already_loaded = */ false); /// Reload it right now (initial loading) - const auto listen_hosts = getListenHosts(config()); - const auto interserver_listen_hosts = getInterserverListenHosts(config()); - const auto listen_try = getListenTry(config()); - - if (config().has("keeper_server.server_id")) - { -#if USE_NURAFT - //// If we don't have configured connection probably someone trying to use clickhouse-server instead - //// of clickhouse-keeper, so start synchronously. - bool can_initialize_keeper_async = false; - - if (has_zookeeper) /// We have configured connection to some zookeeper cluster - { - /// If we cannot connect to some other node from our cluster then we have to wait our Keeper start - /// synchronously. - can_initialize_keeper_async = global_context->tryCheckClientConnectionToMyKeeperCluster(); - } - /// Initialize keeper RAFT. - global_context->initializeKeeperDispatcher(can_initialize_keeper_async); - FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - - auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & - { - return global_context->getConfigRef(); - }; - - for (const auto & listen_host : listen_hosts) - { - /// TCP Keeper - const char * port_name = "keeper_server.tcp_port"; - createServer( - config(), listen_host, port_name, listen_try, /* start_server: */ false, - servers_to_start_before_tables, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config(), socket, listen_host, port); - socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); - socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); - return ProtocolServerAdapter( - listen_host, - port_name, - "Keeper (tcp): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), - global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), - false), server_pool, socket)); - }); - - const char * secure_port_name = "keeper_server.tcp_port_secure"; - createServer( - config(), listen_host, secure_port_name, listen_try, /* start_server: */ false, - servers_to_start_before_tables, - [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config(), socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); - socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); - return ProtocolServerAdapter( - listen_host, - secure_port_name, - "Keeper with secure protocol (tcp_secure): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, global_context->getKeeperDispatcher(), - global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); -#else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - - /// HTTP control endpoints - port_name = "keeper_server.http_control.port"; - createServer(config(), listen_host, port_name, listen_try, /* start_server: */ false, - servers_to_start_before_tables, - [&](UInt16 port) -> ProtocolServerAdapter - { - auto http_context = httpContext(); - Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(http_context->getReceiveTimeout()); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config(), socket, listen_host, port); - socket.setReceiveTimeout(http_context->getReceiveTimeout()); - socket.setSendTimeout(http_context->getSendTimeout()); - return ProtocolServerAdapter( - listen_host, - port_name, - "HTTP Control: http://" + address.toString(), - std::make_unique( - std::move(http_context), - createKeeperHTTPControlMainHandlerFactory( - config_getter(), - global_context->getKeeperDispatcher(), - "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); - }); - } -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); -#endif - - } - - { - std::lock_guard lock(servers_lock); - /// We should start interserver communications before (and more important shutdown after) tables. - /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. - /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can - /// communicate with zookeeper, execute merges, etc. - createInterserverServers( - config(), - interserver_listen_hosts, - listen_try, - server_pool, - async_metrics, - servers_to_start_before_tables, - /* start_servers= */ false); - - - for (auto & server : servers_to_start_before_tables) - { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); - } - } + servers_to_start_before_tables.createServers( + config(), + *this, + servers_lock, + server_pool, + async_metrics, + /* start_servers= */ false, + ServerType(ServerType::Type::QUERIES_ALL) + ); /// Initialize access storages. auto & access_control = global_context->getAccessControl(); @@ -1799,19 +1498,18 @@ try global_context->setStopServersCallback([&](const ServerType & server_type) { std::lock_guard lock(servers_lock); - stopServers(servers, server_type); + servers.stopServers(server_type); }); global_context->setStartServersCallback([&](const ServerType & server_type) { std::lock_guard lock(servers_lock); - createServers( + servers.createServers( config(), - listen_hosts, - listen_try, + *this, + servers_lock, server_pool, async_metrics, - servers, /* start_servers= */ true, server_type); }); @@ -2024,18 +1722,21 @@ try { std::lock_guard lock(servers_lock); - createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); + servers.createServers( + config(), + *this, + servers_lock, + server_pool, + async_metrics, + false, + ServerType(ServerType::Type::QUERIES_ALL)); if (servers.empty()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " - "to configuration file.)"); + throw Exception( + ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "No servers started (add valid listen_host and 'tcp_port' " + "or 'http_port' to configuration file.)"); } - if (servers.empty()) - throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " - "to configuration file.)"); - #if USE_SSL CertificateReloader::instance().tryLoad(config()); #endif @@ -2107,12 +1808,7 @@ try { std::lock_guard lock(servers_lock); - for (auto & server : servers) - { - server.start(); - LOG_INFO(log, "Listening for {}", server.getDescription()); - } - + servers.startServers(); global_context->setServerCompletelyStarted(); LOG_INFO(log, "Ready for connections."); } @@ -2148,46 +1844,10 @@ try access_control.stopPeriodicReloading(); is_cancelled = true; - - LOG_DEBUG(log, "Waiting for current connections to close."); - - size_t current_connections = 0; - { - std::lock_guard lock(servers_lock); - for (auto & server : servers) - { - server.stop(); - current_connections += server.currentConnections(); - } - } - - if (current_connections) - LOG_WARNING(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); - else - LOG_INFO(log, "Closed all listening sockets."); - - /// Wait for unfinished backups and restores. - /// This must be done after closing listening sockets (no more backups/restores) but before ProcessList::killAllQueries - /// (because killAllQueries() will cancel all running backups/restores). - if (server_settings.shutdown_wait_backups_and_restores) - global_context->waitAllBackupsAndRestores(); - - /// Killing remaining queries. - if (!server_settings.shutdown_wait_unfinished_queries) - global_context->getProcessList().killAllQueries(); - - if (current_connections) - current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); - - if (current_connections) - LOG_WARNING(log, "Closed connections. But {} remain." - " Tip: To increase wait time add to config: 60", current_connections); - else - LOG_INFO(log, "Closed connections."); - + const auto remaining_connections = servers.stopServers(server_settings, servers_lock); dns_cache_updater.reset(); - if (current_connections) + if (remaining_connections) { /// There is no better way to force connections to close in Poco. /// Otherwise connection handlers will continue to live @@ -2221,561 +1881,4 @@ catch (...) return code ? code : -1; } -std::unique_ptr Server::buildProtocolStackFromConfig( - const Poco::Util::AbstractConfiguration & config, - const std::string & protocol, - Poco::Net::HTTPServerParams::Ptr http_params, - AsynchronousMetrics & async_metrics, - bool & is_secure) -{ - auto create_factory = [&](const std::string & type, const std::string & conf_name) -> TCPServerConnectionFactory::Ptr - { - if (type == "tcp") - return TCPServerConnectionFactory::Ptr(new TCPHandlerFactory(*this, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes)); - - if (type == "tls") -#if USE_SSL - return TCPServerConnectionFactory::Ptr(new TLSHandlerFactory(*this, conf_name)); -#else - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - - if (type == "proxy1") - return TCPServerConnectionFactory::Ptr(new ProxyV1HandlerFactory(*this, conf_name)); - if (type == "mysql") - return TCPServerConnectionFactory::Ptr(new MySQLHandlerFactory(*this, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes)); - if (type == "postgres") - return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes)); - if (type == "http") - return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes) - ); - if (type == "prometheus") - return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), ProfileEvents::InterfacePrometheusReceiveBytes, ProfileEvents::InterfacePrometheusSendBytes) - ); - if (type == "interserver") - return TCPServerConnectionFactory::Ptr( - new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), ProfileEvents::InterfaceInterserverReceiveBytes, ProfileEvents::InterfaceInterserverSendBytes) - ); - - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); - }; - - std::string conf_name = "protocols." + protocol; - std::string prefix = conf_name + "."; - std::unordered_set pset {conf_name}; - - auto stack = std::make_unique(*this, conf_name); - - while (true) - { - // if there is no "type" - it's a reference to another protocol and this is just an endpoint - if (config.has(prefix + "type")) - { - std::string type = config.getString(prefix + "type"); - if (type == "tls") - { - if (is_secure) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' contains more than one TLS layer", protocol); - is_secure = true; - } - - stack->append(create_factory(type, conf_name)); - } - - if (!config.has(prefix + "impl")) - break; - - conf_name = "protocols." + config.getString(prefix + "impl"); - prefix = conf_name + "."; - - if (!pset.insert(conf_name).second) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); - } - - return stack; -} - -HTTPContextPtr Server::httpContext() const -{ - return std::make_shared(context()); -} - -void Server::createServers( - Poco::Util::AbstractConfiguration & config, - const Strings & listen_hosts, - bool listen_try, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - bool start_servers, - const ServerType & server_type) -{ - const Settings & settings = global_context->getSettingsRef(); - - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); - - Poco::Util::AbstractConfiguration::Keys protocols; - config.keys("protocols", protocols); - - for (const auto & protocol : protocols) - { - if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) - continue; - - std::string prefix = "protocols." + protocol + "."; - std::string port_name = prefix + "port"; - std::string description {" protocol"}; - if (config.has(prefix + "description")) - description = config.getString(prefix + "description"); - - if (!config.has(prefix + "port")) - continue; - - std::vector hosts; - if (config.has(prefix + "host")) - hosts.push_back(config.getString(prefix + "host")); - else - hosts = listen_hosts; - - for (const auto & host : hosts) - { - bool is_secure = false; - auto stack = buildProtocolStackFromConfig(config, protocol, http_params, async_metrics, is_secure); - - if (stack->empty()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' stack empty", protocol); - - createServer(config, host, port_name.c_str(), listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, host, port, is_secure); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - - return ProtocolServerAdapter( - host, - port_name.c_str(), - description + ": " + address.toString(), - std::make_unique( - stack.release(), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - } - - for (const auto & listen_host : listen_hosts) - { - const char * port_name; - - if (server_type.shouldStart(ServerType::Type::HTTP)) - { - /// HTTP - port_name = "http_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - - return ProtocolServerAdapter( - listen_host, - port_name, - "http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes)); - }); - } - - if (server_type.shouldStart(ServerType::Type::HTTPS)) - { - /// HTTPS - port_name = "https_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "https://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes)); -#else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP)) - { - /// TCP - port_name = "tcp_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) - { - /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt - port_name = "tcp_with_proxy_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp) with PROXY: " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) - { - /// TCP with SSL - port_name = "tcp_port_secure"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - #if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "secure native protocol (tcp_secure): " + address.toString(), - std::make_unique( - new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - #else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); - #endif - }); - } - - if (server_type.shouldStart(ServerType::Type::MYSQL)) - { - port_name = "mysql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "MySQL compatibility protocol: " + address.toString(), - std::make_unique(new MySQLHandlerFactory(*this, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes), server_pool, socket, new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) - { - port_name = "postgresql_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "PostgreSQL compatibility protocol: " + address.toString(), - std::make_unique(new PostgreSQLHandlerFactory(*this, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes), server_pool, socket, new Poco::Net::TCPServerParams)); - }); - } - -#if USE_GRPC - if (server_type.shouldStart(ServerType::Type::GRPC)) - { - port_name = "grpc_port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::SocketAddress server_address(listen_host, port); - return ProtocolServerAdapter( - listen_host, - port_name, - "gRPC protocol: " + server_address.toString(), - std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); - }); - } -#endif - if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) - { - /// Prometheus (if defined and not setup yet with http_port) - port_name = "prometheus.port"; - createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfacePrometheusReceiveBytes, ProfileEvents::InterfacePrometheusSendBytes)); - }); - } - } -} - -void Server::createInterserverServers( - Poco::Util::AbstractConfiguration & config, - const Strings & interserver_listen_hosts, - bool listen_try, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - bool start_servers, - const ServerType & server_type) -{ - const Settings & settings = global_context->getSettingsRef(); - - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); - - /// Now iterate over interserver_listen_hosts - for (const auto & interserver_listen_host : interserver_listen_hosts) - { - const char * port_name; - - if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) - { - /// Interserver IO HTTP - port_name = "interserver_http_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "replica communication (interserver): http://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceInterserverReceiveBytes, - ProfileEvents::InterfaceInterserverSendBytes)); - }); - } - - if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) - { - port_name = "interserver_https_port"; - createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "secure replica communication (interserver): https://" + address.toString(), - std::make_unique( - httpContext(), - createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceInterserverReceiveBytes, - ProfileEvents::InterfaceInterserverSendBytes)); -#else - UNUSED(port); - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - } - } -} - -void Server::stopServers( - std::vector & servers, - const ServerType & server_type -) const -{ - LoggerRawPtr log = &logger(); - - /// Remove servers once all their connections are closed - auto check_server = [&log](const char prefix[], auto & server) - { - if (!server.isStopping()) - return false; - size_t current_connections = server.currentConnections(); - LOG_DEBUG(log, "Server {}{}: {} ({} connections)", - server.getDescription(), - prefix, - !current_connections ? "finished" : "waiting", - current_connections); - return !current_connections; - }; - - std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); - - for (auto & server : servers) - { - if (!server.isStopping()) - { - const std::string server_port_name = server.getPortName(); - - if (server_type.shouldStop(server_port_name)) - server.stop(); - } - } - - std::erase_if(servers, std::bind_front(check_server, "")); -} - -void Server::updateServers( - Poco::Util::AbstractConfiguration & config, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - std::vector & servers_to_start_before_tables) -{ - LoggerRawPtr log = &logger(); - - const auto listen_hosts = getListenHosts(config); - const auto interserver_listen_hosts = getInterserverListenHosts(config); - const auto listen_try = getListenTry(config); - - /// Remove servers once all their connections are closed - auto check_server = [&log](const char prefix[], auto & server) - { - if (!server.isStopping()) - return false; - size_t current_connections = server.currentConnections(); - LOG_DEBUG(log, "Server {}{}: {} ({} connections)", - server.getDescription(), - prefix, - !current_connections ? "finished" : "waiting", - current_connections); - return !current_connections; - }; - - std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); - - Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); - - std::vector all_servers; - all_servers.reserve(servers.size() + servers_to_start_before_tables.size()); - for (auto & server : servers) - all_servers.push_back(&server); - - for (auto & server : servers_to_start_before_tables) - all_servers.push_back(&server); - - for (auto * server : all_servers) - { - if (!server->isStopping()) - { - std::string port_name = server->getPortName(); - bool has_host = false; - bool is_http = false; - if (port_name.starts_with("protocols.")) - { - std::string protocol = port_name.substr(0, port_name.find_last_of('.')); - has_host = config.has(protocol + ".host"); - - std::string conf_name = protocol; - std::string prefix = protocol + "."; - std::unordered_set pset {conf_name}; - while (true) - { - if (config.has(prefix + "type")) - { - std::string type = config.getString(prefix + "type"); - if (type == "http") - { - is_http = true; - break; - } - } - - if (!config.has(prefix + "impl")) - break; - - conf_name = "protocols." + config.getString(prefix + "impl"); - prefix = conf_name + "."; - - if (!pset.insert(conf_name).second) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); - } - } - else - { - /// NOTE: better to compare using getPortName() over using - /// dynamic_cast<> since HTTPServer is also used for prometheus and - /// internal replication communications. - is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port"; - } - - if (!has_host) - has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end(); - bool has_port = !config.getString(port_name, "").empty(); - bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); - if (force_restart) - LOG_TRACE(log, " had been changed, will reload {}", server->getDescription()); - - if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart) - { - server->stop(); - LOG_INFO(log, "Stopped listening for {}", server->getDescription()); - } - } - } - - createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); - createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true); - - std::erase_if(servers, std::bind_front(check_server, "")); - std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, "")); -} - } diff --git a/programs/server/Server.h b/programs/server/Server.h index 3f03dd137ef..b4931ce53d1 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -1,15 +1,10 @@ #pragma once #include - #include -#include -#include -#include -#include /** Server provides three interfaces: - * 1. HTTP - simple interface for any applications. + * 1. HTTP, GRPC - simple interfaces for any applications. * 2. TCP - interface for native clickhouse-client and for server to server internal communications. * More rich and efficient, but less compatible * - data is transferred by columns; @@ -18,43 +13,21 @@ * 3. Interserver HTTP - for replication. */ -namespace Poco -{ - namespace Net - { - class ServerSocket; - } -} - namespace DB { -class AsynchronousMetrics; -class ProtocolServerAdapter; class Server : public BaseDaemon, public IServer { public: using ServerApplication::run; - Poco::Util::LayeredConfiguration & config() const override - { - return BaseDaemon::config(); - } + Poco::Util::LayeredConfiguration & config() const override { return BaseDaemon::config(); } - Poco::Logger & logger() const override - { - return BaseDaemon::logger(); - } + Poco::Logger & logger() const override { return BaseDaemon::logger(); } - ContextMutablePtr context() const override - { - return global_context; - } + ContextMutablePtr context() const override { return global_context; } - bool isCancelled() const override - { - return BaseDaemon::isCancelled(); - } + bool isCancelled() const override { return BaseDaemon::isCancelled(); } void defineOptions(Poco::Util::OptionSet & _options) override; @@ -73,64 +46,6 @@ private: ContextMutablePtr global_context; /// Updated/recent config, to compare http_handlers ConfigurationPtr latest_config; - - HTTPContextPtr httpContext() const; - - Poco::Net::SocketAddress socketBindListen( - const Poco::Util::AbstractConfiguration & config, - Poco::Net::ServerSocket & socket, - const std::string & host, - UInt16 port, - [[maybe_unused]] bool secure = false) const; - - std::unique_ptr buildProtocolStackFromConfig( - const Poco::Util::AbstractConfiguration & config, - const std::string & protocol, - Poco::Net::HTTPServerParams::Ptr http_params, - AsynchronousMetrics & async_metrics, - bool & is_secure); - - using CreateServerFunc = std::function; - void createServer( - Poco::Util::AbstractConfiguration & config, - const std::string & listen_host, - const char * port_name, - bool listen_try, - bool start_server, - std::vector & servers, - CreateServerFunc && func) const; - - void createServers( - Poco::Util::AbstractConfiguration & config, - const Strings & listen_hosts, - bool listen_try, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - bool start_servers = false, - const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); - - void createInterserverServers( - Poco::Util::AbstractConfiguration & config, - const Strings & interserver_listen_hosts, - bool listen_try, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - bool start_servers = false, - const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); - - void updateServers( - Poco::Util::AbstractConfiguration & config, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - std::vector & servers, - std::vector & servers_to_start_before_tables); - - void stopServers( - std::vector & servers, - const ServerType & server_type - ) const; }; } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e8946facda..826204111a0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -234,6 +234,7 @@ add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) add_object_library(clickhouse_server_http Server/HTTP) +add_object_library(clickhouse_server_manager Server/ServersManager) add_object_library(clickhouse_formats Formats) add_object_library(clickhouse_processors Processors) add_object_library(clickhouse_processors_executors Processors/Executors) diff --git a/src/Server/ServersManager/IServersManager.cpp b/src/Server/ServersManager/IServersManager.cpp new file mode 100644 index 00000000000..c903d90f766 --- /dev/null +++ b/src/Server/ServersManager/IServersManager.cpp @@ -0,0 +1,268 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NETWORK_ERROR; +extern const int INVALID_CONFIG_PARAMETER; +} + +IServersManager::IServersManager(ContextMutablePtr l_global_context, Poco::Logger * l_logger) + : global_context(l_global_context), logger(l_logger) +{ +} + + +bool IServersManager::empty() const +{ + return servers.empty(); +} + +std::vector IServersManager::getMetrics() const +{ + std::vector metrics; + metrics.reserve(servers.size()); + for (const auto & server : servers) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + return metrics; +} + +void IServersManager::startServers() +{ + for (auto & server : servers) + { + server.start(); + LOG_INFO(logger, "Listening for {}", server.getDescription()); + } +} + +void IServersManager::stopServers(const ServerType & server_type) +{ + /// Remove servers once all their connections are closed + auto check_server = [&](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG( + logger, + "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); + + for (auto & server : servers) + { + if (!server.isStopping() && server_type.shouldStop(server.getPortName())) + server.stop(); + } + + std::erase_if(servers, std::bind_front(check_server, "")); +} + +void IServersManager::updateServers( + const Poco::Util::AbstractConfiguration & config, + IServer & iserver, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + ConfigurationPtr latest_config) +{ + stopServersForUpdate(config, latest_config); + createServers(config, iserver, servers_lock, server_pool, async_metrics, true, ServerType(ServerType::Type::QUERIES_ALL)); +} + +Poco::Net::SocketAddress IServersManager::socketBindListen( + const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port) const +{ + auto address = makeSocketAddress(host, port, logger); + socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config.getBool("listen_reuse_port", false)); + /// If caller requests any available port from the OS, discover it after binding. + if (port == 0) + { + address = socket.address(); + LOG_DEBUG(logger, "Requested any available port (port == 0), actual port is {:d}", address.port()); + } + + socket.listen(/* backlog = */ config.getUInt("listen_backlog", 4096)); + return address; +} + +void IServersManager::createServer( + const Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + CreateServerFunc && func, + bool start_server) +{ + /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. + if (config.getString(port_name, "").empty()) + return; + + /// If we already have an active server for this listen_host/port_name, don't create it again + for (const auto & server : servers) + { + if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name) + return; + } + + auto port = config.getInt(port_name); + try + { + servers.push_back(func(port)); + if (start_server) + { + servers.back().start(); + LOG_INFO(logger, "Listening for {}", servers.back().getDescription()); + } + global_context->registerServerPort(port_name, port); + } + catch (const Poco::Exception &) + { + if (!getListenTry(config)) + { + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); + } + LOG_WARNING( + logger, + "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " + "specify not disabled IPv4 or IPv6 address to listen in element of configuration " + "file. Example for disabled IPv6: 0.0.0.0 ." + " Example for disabled IPv4: ::", + listen_host, + port, + getCurrentExceptionMessage(false)); + } +} + +void IServersManager::stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config) +{ + /// Remove servers once all their connections are closed + auto check_server = [&](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG( + logger, + "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); + + const auto listen_hosts = getListenHosts(config); + const Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : config; + + for (auto & server : servers) + { + if (server.isStopping()) + return; + std::string port_name = server.getPortName(); + bool has_host = false; + bool is_http = false; + if (port_name.starts_with("protocols.")) + { + std::string protocol = port_name.substr(0, port_name.find_last_of('.')); + has_host = config.has(protocol + ".host"); + + std::string conf_name = protocol; + std::string prefix = protocol + "."; + std::unordered_set pset{conf_name}; + while (true) + { + if (config.has(prefix + "type")) + { + std::string type = config.getString(prefix + "type"); + if (type == "http") + { + is_http = true; + break; + } + } + + if (!config.has(prefix + "impl")) + break; + + conf_name = "protocols." + config.getString(prefix + "impl"); + prefix = conf_name + "."; + + if (!pset.insert(conf_name).second) + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); + } + } + else + { + /// NOTE: better to compare using getPortName() over using + /// dynamic_cast<> since HTTPServer is also used for prometheus and + /// internal replication communications. + is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port"; + } + + if (!has_host) + has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); + bool has_port = !config.getString(port_name, "").empty(); + bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); + if (force_restart) + LOG_TRACE(logger, " had been changed, will reload {}", server.getDescription()); + + if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart) + { + server.stop(); + LOG_INFO(logger, "Stopped listening for {}", server.getDescription()); + } + } + + std::erase_if(servers, std::bind_front(check_server, "")); +} + +Strings IServersManager::getListenHosts(const Poco::Util::AbstractConfiguration & config) const +{ + auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host"); + if (listen_hosts.empty()) + { + listen_hosts.emplace_back("::1"); + listen_hosts.emplace_back("127.0.0.1"); + } + return listen_hosts; +} + +bool IServersManager::getListenTry(const Poco::Util::AbstractConfiguration & config) const +{ + bool listen_try = config.getBool("listen_try", false); + if (!listen_try) + { + Poco::Util::AbstractConfiguration::Keys protocols; + config.keys("protocols", protocols); + listen_try = DB::getMultipleValuesFromConfig(config, "", "listen_host").empty() + && std::none_of( + protocols.begin(), + protocols.end(), + [&](const auto & protocol) + { return config.has("protocols." + protocol + ".host") && config.has("protocols." + protocol + ".port"); }); + } + return listen_try; +} + +} diff --git a/src/Server/ServersManager/IServersManager.h b/src/Server/ServersManager/IServersManager.h new file mode 100644 index 00000000000..5218ab63554 --- /dev/null +++ b/src/Server/ServersManager/IServersManager.h @@ -0,0 +1,74 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class IServersManager +{ +public: + IServersManager(ContextMutablePtr global_context, Poco::Logger * logger); + virtual ~IServersManager() = default; + + bool empty() const; + std::vector getMetrics() const; + + virtual void createServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) + = 0; + + virtual void startServers(); + + virtual void stopServers(const ServerType & server_type); + virtual size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) = 0; + + virtual void updateServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + ConfigurationPtr latest_config); + +protected: + ContextMutablePtr global_context; + Poco::Logger * logger; + + std::vector servers; + + Poco::Net::SocketAddress socketBindListen( + const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port) const; + + using CreateServerFunc = std::function; + virtual void createServer( + const Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + CreateServerFunc && func, + bool start_server); + + virtual void stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config); + + Strings getListenHosts(const Poco::Util::AbstractConfiguration & config) const; + bool getListenTry(const Poco::Util::AbstractConfiguration & config) const; +}; + +} diff --git a/src/Server/ServersManager/InterServersManager.cpp b/src/Server/ServersManager/InterServersManager.cpp new file mode 100644 index 00000000000..28491a4f4f4 --- /dev/null +++ b/src/Server/ServersManager/InterServersManager.cpp @@ -0,0 +1,327 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if USE_SSL +# include +#endif + +#if USE_NURAFT +# include +# include +#endif + +namespace ProfileEvents +{ +extern const Event InterfaceInterserverSendBytes; +extern const Event InterfaceInterserverReceiveBytes; +} + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int SUPPORT_IS_DISABLED; +} + +void InterServersManager::createServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) +{ + if (config.has("keeper_server.server_id")) + { +#if USE_NURAFT + //// If we don't have configured connection probably someone trying to use clickhouse-server instead + //// of clickhouse-keeper, so start synchronously. + bool can_initialize_keeper_async = false; + + if (zkutil::hasZooKeeperConfig(config)) /// We have configured connection to some zookeeper cluster + { + /// If we cannot connect to some other node from our cluster then we have to wait our Keeper start + /// synchronously. + can_initialize_keeper_async = global_context->tryCheckClientConnectionToMyKeeperCluster(); + } + /// Initialize keeper RAFT. + global_context->initializeKeeperDispatcher(can_initialize_keeper_async); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); + + auto config_getter = [this]() -> const Poco::Util::AbstractConfiguration & { return global_context->getConfigRef(); }; + + for (const auto & listen_host : getListenHosts(config)) + { + /// TCP Keeper + constexpr auto port_name = "keeper_server.tcp_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout( + Poco::Timespan(config.getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); + socket.setSendTimeout( + Poco::Timespan(config.getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); + return ProtocolServerAdapter( + listen_host, + port_name, + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, + global_context->getKeeperDispatcher(), + global_context->getSettingsRef().receive_timeout.totalSeconds(), + global_context->getSettingsRef().send_timeout.totalSeconds(), + false), + server_pool, + socket)); + }, + /* start_server = */ false); + + constexpr auto secure_port_name = "keeper_server.tcp_port_secure"; + createServer( + config, + listen_host, + secure_port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { +# if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout( + Poco::Timespan(config.getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); + socket.setSendTimeout( + Poco::Timespan(config.getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); + return ProtocolServerAdapter( + listen_host, + secure_port_name, + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, + global_context->getKeeperDispatcher(), + global_context->getSettingsRef().receive_timeout.totalSeconds(), + global_context->getSettingsRef().send_timeout.totalSeconds(), + true), + server_pool, + socket)); +# else + UNUSED(port); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +# endif + }, + /* start_server: */ false); + + /// HTTP control endpoints + createServer( + config, + listen_host, + /* port_name = */ "keeper_server.http_control.port", + [&](UInt16 port) -> ProtocolServerAdapter + { + auto http_context = std::make_shared(global_context); + Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(http_context->getReceiveTimeout()); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(http_context->getReceiveTimeout()); + socket.setSendTimeout(http_context->getSendTimeout()); + return ProtocolServerAdapter( + listen_host, + port_name, + "HTTP Control: http://" + address.toString(), + std::make_unique( + std::move(http_context), + createKeeperHTTPControlMainHandlerFactory( + config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), + server_pool, + socket, + http_params)); + }, + /* start_server: */ false); + } +#else + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); +#endif + } + + { + std::lock_guard lock(servers_lock); + /// We should start interserver communications before (and more important shutdown after) tables. + /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. + /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can + /// communicate with zookeeper, execute merges, etc. + createInterserverServers(config, server, server_pool, async_metrics, start_servers, server_type); + startServers(); + } +} + +size_t InterServersManager::stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) +{ + if (servers.empty()) + { + return 0; + } + + LOG_DEBUG(logger, "Waiting for current connections to servers for tables to finish."); + + size_t current_connections = 0; + { + std::lock_guard lock(servers_lock); + for (auto & server : servers) + { + server.stop(); + current_connections += server.currentConnections(); + } + } + + if (current_connections) + LOG_INFO(logger, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); + else + LOG_INFO(logger, "Closed all listening sockets."); + + if (current_connections > 0) + current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); + + if (current_connections) + LOG_INFO( + logger, + "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections " + "after context shutdown.", + current_connections); + else + LOG_INFO(logger, "Closed connections to servers for tables."); + return current_connections; +} + +void InterServersManager::updateServers( + const Poco::Util::AbstractConfiguration & config, + IServer & iserver, + std::mutex & /*servers_lock*/, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + ConfigurationPtr latest_config) +{ + stopServersForUpdate(config, latest_config); + createInterserverServers(config, iserver, server_pool, async_metrics, true, ServerType(ServerType::Type::QUERIES_ALL)); +} + +Strings InterServersManager::getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) const +{ + auto interserver_listen_hosts = DB::getMultipleValuesFromConfig(config, "", "interserver_listen_host"); + if (!interserver_listen_hosts.empty()) + return interserver_listen_hosts; + + /// Use more general restriction in case of emptiness + return getListenHosts(config); +} + +void InterServersManager::createInterserverServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + + /// Now iterate over interserver_listen_hosts + for (const auto & interserver_listen_host : getInterserverListenHosts(config)) + { + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) + { + /// Interserver IO HTTP + constexpr auto port_name = "interserver_http_port"; + createServer( + config, + interserver_listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "replica communication (interserver): http://" + address.toString(), + std::make_unique( + std::make_shared(global_context), + createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceInterserverReceiveBytes, + ProfileEvents::InterfaceInterserverSendBytes)); + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) + { + constexpr auto port_name = "interserver_https_port"; + createServer( + config, + interserver_listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "secure replica communication (interserver): https://" + address.toString(), + std::make_unique( + std::make_shared(global_context), + createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPSHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceInterserverReceiveBytes, + ProfileEvents::InterfaceInterserverSendBytes)); +#else + UNUSED(port); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + }, + start_servers); + } + } +} + +} diff --git a/src/Server/ServersManager/InterServersManager.h b/src/Server/ServersManager/InterServersManager.h new file mode 100644 index 00000000000..2a389e28c22 --- /dev/null +++ b/src/Server/ServersManager/InterServersManager.h @@ -0,0 +1,45 @@ +#pragma once + +#include + +namespace DB +{ + +class InterServersManager : public IServersManager +{ +public: + using IServersManager::IServersManager; + + void createServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) override; + + using IServersManager::stopServers; + size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) override; + + void updateServers( + const Poco::Util::AbstractConfiguration & config, + IServer & iserver, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + ConfigurationPtr latest_config) override; + +private: + Strings getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) const; + + void createInterserverServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type); +}; + +} diff --git a/src/Server/ServersManager/ProtocolServersManager.cpp b/src/Server/ServersManager/ProtocolServersManager.cpp new file mode 100644 index 00000000000..17b028eddbb --- /dev/null +++ b/src/Server/ServersManager/ProtocolServersManager.cpp @@ -0,0 +1,523 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#if USE_SSL +# include +#endif + +#if USE_GRPC +# include +#endif + +namespace ProfileEvents +{ +extern const Event InterfaceNativeSendBytes; +extern const Event InterfaceNativeReceiveBytes; +extern const Event InterfaceHTTPSendBytes; +extern const Event InterfaceHTTPReceiveBytes; +extern const Event InterfacePrometheusSendBytes; +extern const Event InterfacePrometheusReceiveBytes; +extern const Event InterfaceMySQLSendBytes; +extern const Event InterfaceMySQLReceiveBytes; +extern const Event InterfacePostgreSQLSendBytes; +extern const Event InterfacePostgreSQLReceiveBytes; +extern const Event InterfaceInterserverSendBytes; +extern const Event InterfaceInterserverReceiveBytes; +} + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int SUPPORT_IS_DISABLED; +extern const int INVALID_CONFIG_PARAMETER; +} + +void ProtocolServersManager::createServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & /*servers_lock*/, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) +{ + auto listen_hosts = getListenHosts(config); + const Settings & settings = global_context->getSettingsRef(); + + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + + Poco::Util::AbstractConfiguration::Keys protocols; + config.keys("protocols", protocols); + + for (const auto & protocol : protocols) + { + if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + continue; + + std::string prefix = "protocols." + protocol + "."; + std::string port_name = prefix + "port"; + std::string description{" protocol"}; + if (config.has(prefix + "description")) + description = config.getString(prefix + "description"); + + if (!config.has(prefix + "port")) + continue; + + std::vector hosts; + if (config.has(prefix + "host")) + hosts.push_back(config.getString(prefix + "host")); + else + hosts = listen_hosts; + + for (const auto & host : hosts) + { + bool is_secure = false; + auto stack = buildProtocolStackFromConfig(config, server, protocol, http_params, async_metrics, is_secure); + + if (stack->empty()) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' stack empty", protocol); + + createServer( + config, + host, + port_name.c_str(), + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + host, + port_name.c_str(), + description + ": " + address.toString(), + std::make_unique(stack.release(), server_pool, socket, new Poco::Net::TCPServerParams)); + }, + start_servers); + } + } + + for (const auto & listen_host : listen_hosts) + { + if (server_type.shouldStart(ServerType::Type::HTTP)) + { + /// HTTP + constexpr auto port_name = "http_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "http://" + address.toString(), + std::make_unique( + std::make_shared(global_context), + createHandlerFactory(server, config, async_metrics, "HTTPHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceHTTPReceiveBytes, + ProfileEvents::InterfaceHTTPSendBytes)); + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::HTTPS)) + { + /// HTTPS + constexpr auto port_name = "https_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "https://" + address.toString(), + std::make_unique( + std::make_shared(global_context), + createHandlerFactory(server, config, async_metrics, "HTTPSHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceHTTPReceiveBytes, + ProfileEvents::InterfaceHTTPSendBytes)); +#else + UNUSED(port); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "HTTPS protocol is disabled because Poco library was built without NetSSL support."); +#endif + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::TCP)) + { + /// TCP + constexpr auto port_name = "tcp_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp): " + address.toString(), + std::make_unique( + new TCPHandlerFactory( + server, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) + { + /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt + constexpr auto port_name = "tcp_with_proxy_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp) with PROXY: " + address.toString(), + std::make_unique( + new TCPHandlerFactory( + server, false, true, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) + { + /// TCP with SSL + constexpr auto port_name = "tcp_port_secure"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure native protocol (tcp_secure): " + address.toString(), + std::make_unique( + new TCPHandlerFactory( + server, true, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); +#else + UNUSED(port); + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::MYSQL)) + { + constexpr auto port_name = "mysql_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "MySQL compatibility protocol: " + address.toString(), + std::make_unique( + new MySQLHandlerFactory( + server, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }, + start_servers); + } + + if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) + { + constexpr auto port_name = "postgresql_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "PostgreSQL compatibility protocol: " + address.toString(), + std::make_unique( + new PostgreSQLHandlerFactory( + server, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }, + start_servers); + } + +#if USE_GRPC + if (server_type.shouldStart(ServerType::Type::GRPC)) + { + constexpr auto port_name = "grpc_port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::SocketAddress server_address(listen_host, port); + return ProtocolServerAdapter( + listen_host, + port_name, + "gRPC protocol: " + server_address.toString(), + std::make_unique(server, makeSocketAddress(listen_host, port, logger))); + }, + start_servers); + } +#endif + if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) + { + /// Prometheus (if defined and not setup yet with http_port) + constexpr auto port_name = "prometheus.port"; + createServer( + config, + listen_host, + port_name, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + std::make_shared(global_context), + createHandlerFactory(server, config, async_metrics, "PrometheusHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfacePrometheusReceiveBytes, + ProfileEvents::InterfacePrometheusSendBytes)); + }, + start_servers); + } + } +} + +size_t ProtocolServersManager::stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) +{ + if (servers.empty()) + { + return 0; + } + + LOG_DEBUG(logger, "Waiting for current connections to close."); + + size_t current_connections = 0; + { + std::lock_guard lock(servers_lock); + for (auto & server : servers) + { + server.stop(); + current_connections += server.currentConnections(); + } + } + + if (current_connections) + LOG_WARNING(logger, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); + else + LOG_INFO(logger, "Closed all listening sockets."); + + /// Wait for unfinished backups and restores. + /// This must be done after closing listening sockets (no more backups/restores) but before ProcessList::killAllQueries + /// (because killAllQueries() will cancel all running backups/restores). + if (server_settings.shutdown_wait_backups_and_restores) + global_context->waitAllBackupsAndRestores(); + /// Killing remaining queries. + if (!server_settings.shutdown_wait_unfinished_queries) + global_context->getProcessList().killAllQueries(); + + if (current_connections) + current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); + + if (current_connections) + LOG_WARNING( + logger, + "Closed connections. But {} remain." + " Tip: To increase wait time add to config: 60", + current_connections); + else + LOG_INFO(logger, "Closed connections."); + return current_connections; +} + +std::unique_ptr ProtocolServersManager::buildProtocolStackFromConfig( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + const std::string & protocol, + Poco::Net::HTTPServerParams::Ptr http_params, + AsynchronousMetrics & async_metrics, + bool & is_secure) const +{ + auto create_factory = [&](const std::string & type, const std::string & conf_name) -> TCPServerConnectionFactory::Ptr + { + if (type == "tcp") + return TCPServerConnectionFactory::Ptr(new TCPHandlerFactory( + server, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes)); + + if (type == "tls") +#if USE_SSL + return TCPServerConnectionFactory::Ptr(new TLSHandlerFactory(server, conf_name)); +#else + throw Exception( + ErrorCodes::SUPPORT_IS_DISABLED, + "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + + if (type == "proxy1") + return TCPServerConnectionFactory::Ptr(new ProxyV1HandlerFactory(server, conf_name)); + if (type == "mysql") + return TCPServerConnectionFactory::Ptr( + new MySQLHandlerFactory(server, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes)); + if (type == "postgres") + return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory( + server, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes)); + if (type == "http") + return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( + std::make_shared(global_context), + http_params, + createHandlerFactory(server, config, async_metrics, "HTTPHandler-factory"), + ProfileEvents::InterfaceHTTPReceiveBytes, + ProfileEvents::InterfaceHTTPSendBytes)); + if (type == "prometheus") + return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( + std::make_shared(global_context), + http_params, + createHandlerFactory(server, config, async_metrics, "PrometheusHandler-factory"), + ProfileEvents::InterfacePrometheusReceiveBytes, + ProfileEvents::InterfacePrometheusSendBytes)); + if (type == "interserver") + return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( + std::make_shared(global_context), + http_params, + createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPHandler-factory"), + ProfileEvents::InterfaceInterserverReceiveBytes, + ProfileEvents::InterfaceInterserverSendBytes)); + + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); + }; + + std::string conf_name = "protocols." + protocol; + std::string prefix = conf_name + "."; + std::unordered_set pset{conf_name}; + + auto stack = std::make_unique(server, conf_name); + + while (true) + { + // if there is no "type" - it's a reference to another protocol and this is just an endpoint + if (config.has(prefix + "type")) + { + std::string type = config.getString(prefix + "type"); + if (type == "tls") + { + if (is_secure) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' contains more than one TLS layer", protocol); + is_secure = true; + } + + stack->append(create_factory(type, conf_name)); + } + + if (!config.has(prefix + "impl")) + break; + + conf_name = "protocols." + config.getString(prefix + "impl"); + prefix = conf_name + "."; + + if (!pset.insert(conf_name).second) + throw Exception( + ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); + } + + return stack; +} + +} diff --git a/src/Server/ServersManager/ProtocolServersManager.h b/src/Server/ServersManager/ProtocolServersManager.h new file mode 100644 index 00000000000..e9eaaeb2184 --- /dev/null +++ b/src/Server/ServersManager/ProtocolServersManager.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class ProtocolServersManager : public IServersManager +{ +public: + using IServersManager::IServersManager; + + void createServers( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + std::mutex & servers_lock, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + bool start_servers, + const ServerType & server_type) override; + + using IServersManager::stopServers; + size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) override; + +private: + std::unique_ptr buildProtocolStackFromConfig( + const Poco::Util::AbstractConfiguration & config, + IServer & server, + const std::string & protocol, + Poco::Net::HTTPServerParams::Ptr http_params, + AsynchronousMetrics & async_metrics, + bool & is_secure) const; +}; + +} From 27627f603fcfcd6df06bfb5210463c1fff8763c6 Mon Sep 17 00:00:00 2001 From: jsc0218 Date: Fri, 24 May 2024 03:04:36 +0000 Subject: [PATCH 125/158] fix --- .../0_stateless/02319_lightweight_delete_on_merge_tree.sql | 2 +- tests/queries/0_stateless/02792_drop_projection_lwd.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql index 050b8e37722..f82f79dbe44 100644 --- a/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql +++ b/tests/queries/0_stateless/02319_lightweight_delete_on_merge_tree.sql @@ -102,7 +102,7 @@ ALTER TABLE t_proj ADD PROJECTION p_1 (SELECT avg(a), avg(b), count()) SETTINGS INSERT INTO t_proj SELECT number + 1, number + 1 FROM numbers(1000); -DELETE FROM t_proj WHERE a < 100; -- { serverError BAD_ARGUMENTS } +DELETE FROM t_proj WHERE a < 100; -- { serverError NOT_IMPLEMENTED } SELECT avg(a), avg(b), count() FROM t_proj; diff --git a/tests/queries/0_stateless/02792_drop_projection_lwd.sql b/tests/queries/0_stateless/02792_drop_projection_lwd.sql index a1d8a9c90f3..dcde7dcc600 100644 --- a/tests/queries/0_stateless/02792_drop_projection_lwd.sql +++ b/tests/queries/0_stateless/02792_drop_projection_lwd.sql @@ -7,7 +7,7 @@ CREATE TABLE t_projections_lwd (a UInt32, b UInt32, PROJECTION p (SELECT * ORDER INSERT INTO t_projections_lwd SELECT number, number FROM numbers(100); -- LWD does not work, as expected -DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError BAD_ARGUMENTS } +DELETE FROM t_projections_lwd WHERE a = 1; -- { serverError NOT_IMPLEMENTED } KILL MUTATION WHERE database = currentDatabase() AND table = 't_projections_lwd' SYNC FORMAT Null; -- drop projection From d4fb2d50e95762838b46356a79e7ba8ecd3e4c5e Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 11:36:28 +0200 Subject: [PATCH 126/158] CI: Sync, Merge check, CI gh's statuses fixes --- .github/workflows/master.yml | 21 +++++++++++---------- .github/workflows/pull_request.yml | 7 +++++-- tests/ci/ci.py | 29 +++++++++++++++++++++++++++-- tests/ci/commit_status_helper.py | 13 +++++++------ tests/ci/finish_check.py | 2 +- tests/ci/merge_pr.py | 1 - 6 files changed, 51 insertions(+), 22 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 11ec484d208..7c55098bdfd 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -27,15 +27,16 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 sync_pr.py --merge || : - - name: Python unit tests - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - echo "Testing the main ci directory" - python3 -m unittest discover -s . -p 'test_*.py' - for dir in *_lambda/; do - echo "Testing $dir" - python3 -m unittest discover -s "$dir" -p 'test_*.py' - done +# Runs in MQ: +# - name: Python unit tests +# run: | +# cd "$GITHUB_WORKSPACE/tests/ci" +# echo "Testing the main ci directory" +# python3 -m unittest discover -s . -p 'test_*.py' +# for dir in *_lambda/; do +# echo "Testing $dir" +# python3 -m unittest discover -s "$dir" -p 'test_*.py' +# done - name: PrepareRunConfig id: runconfig run: | @@ -162,7 +163,7 @@ jobs: python3 mark_release_ready.py FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index aa570c3ce2f..7d22554473e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -33,9 +33,12 @@ jobs: clear-repository: true # to ensure correct digests fetch-depth: 0 # to get a version filter: tree:0 - - name: Cancel Sync PR workflow + - name: Cancel previous Sync PR workflow run: | python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --cancel-previous-run + - name: Set pending Sync status + run: | + python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --set-pending-status - name: Labels check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -177,7 +180,7 @@ jobs: ################################# Stage Final ################################# # FinishCheck: - if: ${{ !failure() && !cancelled() }} + if: ${{ !cancelled() }} needs: [RunConfig, BuildDockers, StyleCheck, FastTest, Builds_1, Builds_2, Builds_1_Report, Builds_2_Report, Tests_1, Tests_2, Tests_3] runs-on: [self-hosted, style-checker-aarch64] steps: diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 4afd3f46f9d..fc25bee354d 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -17,7 +17,7 @@ from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union import docker_images_helper import upload_result_helper from build_check import get_release_or_pr -from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames +from ci_config import CI_CONFIG, Build, CILabels, CIStages, JobNames, StatusNames from ci_utils import GHActions, is_hex, normalize_string from clickhouse_helper import ( CiLogsCredentials, @@ -52,7 +52,7 @@ from git_helper import GIT_PREFIX, Git from git_helper import Runner as GitRunner from github_helper import GitHub from pr_info import PRInfo -from report import ERROR, SUCCESS, BuildResult, JobReport +from report import ERROR, SUCCESS, BuildResult, JobReport, PENDING from s3_helper import S3Helper from ci_metadata import CiMetadata from version_helper import get_version_from_repo @@ -996,6 +996,11 @@ def parse_args(parser: argparse.ArgumentParser) -> argparse.Namespace: action="store_true", help="Action that cancels previous running PR workflow if PR added into the Merge Queue", ) + parser.add_argument( + "--set-pending-status", + action="store_true", + help="Action to set needed pending statuses in the beginning of CI workflow, e.g. for Sync wf", + ) parser.add_argument( "--configure", action="store_true", @@ -1930,6 +1935,19 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No ) +def _set_pending_statuses(pr_info: PRInfo) -> None: + commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) + try: + commit.create_status( + state=PENDING, + target_url="", + description="", + context=StatusNames.SYNC, + ) + except Exception as ex: + print(f"ERROR: failed to set GH commit status, ex: {ex}") + + def main() -> int: logging.basicConfig(level=logging.INFO) exit_code = 0 @@ -2265,6 +2283,13 @@ def main() -> int: else: assert False, "BUG! Not supported scenario" + ### SET PENDING STATUS + elif args.cancel_previous_run: + if pr_info.is_pr: + _set_pending_statuses(pr_info) + else: + assert False, "BUG! Not supported scenario" + ### print results _print_results(result, args.outfile, args.pretty) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index e1c47353743..22cc0085781 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -433,11 +433,8 @@ def set_mergeable_check( commit: Commit, description: str = "", state: StatusType = SUCCESS, - hide_url: bool = False, ) -> CommitStatus: - report_url = GITHUB_RUN_URL - if hide_url: - report_url = "" + report_url = "" return post_commit_status( commit, state, @@ -469,7 +466,6 @@ def update_mergeable_check(commit: Commit, pr_info: PRInfo, check_name: str) -> def trigger_mergeable_check( commit: Commit, statuses: CommitStatuses, - hide_url: bool = False, set_if_green: bool = False, workflow_failed: bool = False, ) -> StatusType: @@ -484,9 +480,12 @@ def trigger_mergeable_check( success = [] fail = [] + pending = [] for status in required_checks: if status.state == SUCCESS: success.append(status.context) + elif status.state == PENDING: + pending.append(status.context) else: fail.append(status.context) @@ -503,6 +502,8 @@ def trigger_mergeable_check( elif workflow_failed: description = "check workflow failures" state = FAILURE + elif pending: + description = "pending: " + ", ".join(pending) description = format_description(description) if not set_if_green and state == SUCCESS: @@ -510,7 +511,7 @@ def trigger_mergeable_check( pass else: if mergeable_status is None or mergeable_status.description != description: - set_mergeable_check(commit, description, state, hide_url) + set_mergeable_check(commit, description, state) return state diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 1a7000f5353..130973ee8ff 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -67,7 +67,7 @@ def main(): if status.state == PENDING: post_commit_status( commit, - SUCCESS, + state, # map Mergeable Check status to CI Running status.target_url, "All checks finished", StatusNames.CI, diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 500de4eb718..e1c7bf94ff5 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -250,7 +250,6 @@ def main(): trigger_mergeable_check( commit, statuses, - hide_url=False, set_if_green=True, workflow_failed=(args.wf_status != "success"), ) From 22b441ed40034280d80506150f9f4969966a3f87 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 11:46:50 +0200 Subject: [PATCH 127/158] fix PR template --- .github/PULL_REQUEST_TEMPLATE.md | 64 +++++++++++++++----------------- tests/ci/ci.py | 3 +- 2 files changed, 31 insertions(+), 36 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 64dc9049bc2..663b464d002 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -46,42 +46,36 @@ At a minimum, the following information should be added (but add more as needed) **NOTE:** If your merge the PR with modified CI you **MUST KNOW** what you are doing **NOTE:** Checked options will be applied if set before CI RunConfig/PrepareRunConfig step - -#### Run these jobs only (required builds will be added automatically): -- [ ] Integration Tests -- [ ] Stateless tests -- [ ] Stateful tests -- [ ] Unit tests -- [ ] Performance tests -- [ ] All with aarch64 -- [ ] All with ASAN -- [ ] All with TSAN -- [ ] All with Analyzer -- [ ] All with Azure -- [ ] Add your option here - -#### Deny these jobs: -- [ ] Fast test -- [ ] Integration Tests -- [ ] Stateless tests -- [ ] Stateful tests -- [ ] Performance tests -- [ ] All with ASAN -- [ ] All with TSAN -- [ ] All with MSAN -- [ ] All with UBSAN -- [ ] All with Coverage -- [ ] All with Aarch64 - -#### Extra options: +--- +- [ ] Allow: Integration Tests +- [ ] Allow:: Stateless tests +- [ ] Allow: Stateful tests +- [ ] Allow: Unit tests +- [ ] Allow: Performance tests +- [ ] Allow: All with aarch64 +- [ ] Allow: All with ASAN +- [ ] Allow: All with TSAN +- [ ] Allow: All with Analyzer +- [ ] Allow: All with Azure +- [ ] Allow: Add your option here +--- +- [ ] Exclude: Fast test +- [ ] Exclude: Integration Tests +- [ ] Exclude: Stateless tests +- [ ] Exclude: Stateful tests +- [ ] Exclude: Performance tests +- [ ] Exclude: All with ASAN +- [ ] Exclude: All with TSAN +- [ ] Exclude: All with MSAN +- [ ] Exclude: All with UBSAN +- [ ] Exclude: All with Coverage +- [ ] Exclude: All with Aarch64 +--- - [ ] do not test (only style check) - [ ] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) - -#### Only specified batches in multi-batch jobs: -- [ ] 1 -- [ ] 2 -- [ ] 3 -- [ ] 4 - +- [ ] only batch 1 for multi-batch jobs +- [ ] only batch 2 for multi-batch jobs +- [ ] only batch 3 for multi-batch jobs +- [ ] only batch 4 for multi-batch jobs diff --git a/tests/ci/ci.py b/tests/ci/ci.py index fc25bee354d..c4e06ccd79a 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1938,6 +1938,7 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No def _set_pending_statuses(pr_info: PRInfo) -> None: commit = get_commit(GitHub(get_best_robot_token(), per_page=100), pr_info.sha) try: + print("Set SYNC status to pending") commit.create_status( state=PENDING, target_url="", @@ -2284,7 +2285,7 @@ def main() -> int: assert False, "BUG! Not supported scenario" ### SET PENDING STATUS - elif args.cancel_previous_run: + elif args.set_pending_status: if pr_info.is_pr: _set_pending_statuses(pr_info) else: From a725112c4c7e33ae23e970b2c50f762ca2edea96 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 May 2024 10:10:39 +0000 Subject: [PATCH 128/158] Fix different hashes for reading/writing from/to query cache --- src/Interpreters/executeQuery.cpp | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0b5f68f27f6..59d012a0a0e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1093,6 +1093,15 @@ static std::tuple executeQueryImpl( && (ast->as() || ast->as()); QueryCache::Usage query_cache_usage = QueryCache::Usage::None; + /// If the query runs with "use_query_cache = 1", we first probe if the query cache already contains the query result (if yes: + /// return result from cache). If doesn't, we execute the query normally and write the result into the query cache. Both steps use a + /// hash of the AST, the current database and the settings as cache key. Unfortunately, the settings are in some places internally + /// modified between steps 1 and 2 (= during query execution) - this is silly but hard to forbid. As a result, the hashes no longer + /// match and the cache is rendered ineffective. Therefore make a copy of the settings and use it for steps 1 and 2. + std::optional settings_copy; + if (can_use_query_cache) + settings_copy = settings; + if (!async_insert) { /// If it is a non-internal SELECT, and passive (read) use of the query cache is enabled, and the cache knows the query, then set @@ -1101,7 +1110,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getCurrentDatabase(), settings, context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), *settings_copy, context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1224,7 +1233,7 @@ static std::tuple executeQueryImpl( && (!ast_contains_system_tables || system_table_handling == QueryCacheSystemTableHandling::Save)) { QueryCache::Key key( - ast, context->getCurrentDatabase(), settings, res.pipeline.getHeader(), + ast, context->getCurrentDatabase(), *settings_copy, res.pipeline.getHeader(), context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), From 0e758722c6da7044fcb2c8958f175a8321c056a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 May 2024 10:19:01 +0000 Subject: [PATCH 129/158] Enable 02494_query_cache_nested_query_bug for Analyzer --- .../0_stateless/02494_query_cache_nested_query_bug.reference | 2 +- tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference index 389e2621455..b261da18d51 100644 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference @@ -1,2 +1,2 @@ -2 +1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 8712c7c84c6..15015761295 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" -SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS="SETTINGS use_query_cache=1, max_threads=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. From f1421c9e5c542ed529dd3b225fc06c696a054080 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 12:02:14 +0200 Subject: [PATCH 130/158] style fix --- .github/PULL_REQUEST_TEMPLATE.md | 11 +++++------ tests/ci/commit_status_helper.py | 4 +--- tests/ci/finish_check.py | 4 ++-- 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 663b464d002..f9765c1d57b 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -46,9 +46,8 @@ At a minimum, the following information should be added (but add more as needed) **NOTE:** If your merge the PR with modified CI you **MUST KNOW** what you are doing **NOTE:** Checked options will be applied if set before CI RunConfig/PrepareRunConfig step ---- - [ ] Allow: Integration Tests -- [ ] Allow:: Stateless tests +- [ ] Allow: Stateless tests - [ ] Allow: Stateful tests - [ ] Allow: Unit tests - [ ] Allow: Performance tests @@ -74,8 +73,8 @@ At a minimum, the following information should be added (but add more as needed) - [ ] do not test (only style check) - [ ] disable merge-commit (no merge from master before tests) - [ ] disable CI cache (job reuse) -- [ ] only batch 1 for multi-batch jobs -- [ ] only batch 2 for multi-batch jobs -- [ ] only batch 3 for multi-batch jobs -- [ ] only batch 4 for multi-batch jobs +- [ ] allow: batch 1 for multi-batch jobs +- [ ] allow: batch 2 +- [ ] allow: batch 3 +- [ ] allow: batch 4, 5 and 6 diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 22cc0085781..bdbb0e80653 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -20,7 +20,6 @@ from github.Repository import Repository from ci_config import CHECK_DESCRIPTIONS, CheckDescription, StatusNames, is_required from env_helper import ( GITHUB_REPOSITORY, - GITHUB_RUN_URL, GITHUB_UPSTREAM_REPOSITORY, TEMP_PATH, ) @@ -557,13 +556,12 @@ def update_upstream_sync_status( post_commit_status( last_synced_upstream_commit, sync_status, - "", # let's won't expose any urls from cloud + "", "", StatusNames.SYNC, ) trigger_mergeable_check( last_synced_upstream_commit, get_commit_filtered_statuses(last_synced_upstream_commit), - True, set_if_green=can_set_green_mergeable_status, ) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 130973ee8ff..269d5aa3175 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -15,7 +15,7 @@ from commit_status_helper import ( ) from get_robot_token import get_best_robot_token from pr_info import PRInfo -from report import PENDING, SUCCESS +from report import PENDING from synchronizer_utils import SYNC_BRANCH_PREFIX from env_helper import GITHUB_REPOSITORY, GITHUB_UPSTREAM_REPOSITORY @@ -67,7 +67,7 @@ def main(): if status.state == PENDING: post_commit_status( commit, - state, # map Mergeable Check status to CI Running + state, # map Mergeable Check status to CI Running status.target_url, "All checks finished", StatusNames.CI, From 1f1c2c21b19dc3d29b60f0508b79bceb425585e7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 May 2024 10:32:42 +0000 Subject: [PATCH 131/158] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 6df2e426561..6eae333681d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1617,6 +1617,8 @@ gcem generateRandom generateRandomStructure generateSeries +generateSnowflakeID +generateSnowflakeIDThreadMonotonic generateULID generateUUIDv geoDistance From 7ccb776ed93196e72485aa0219d7b281ea0f68de Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 12:39:35 +0200 Subject: [PATCH 132/158] mcheck fix --- tests/ci/commit_status_helper.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index bdbb0e80653..b17c189c405 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -490,11 +490,6 @@ def trigger_mergeable_check( state: StatusType = SUCCESS - if success: - description = ", ".join(success) - else: - description = "awaiting job statuses" - if fail: description = "failed: " + ", ".join(fail) state = FAILURE @@ -503,6 +498,11 @@ def trigger_mergeable_check( state = FAILURE elif pending: description = "pending: " + ", ".join(pending) + state = PENDING + else: + # all good + description = ", ".join(success) + description = format_description(description) if not set_if_green and state == SUCCESS: From a6e06b27d221cfd7f5b7987c2b642487b2a80d01 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 24 May 2024 14:17:37 +0200 Subject: [PATCH 133/158] Update description for settings cross_join_min_rows_to_compress and cross_join_min_bytes_to_compress --- src/Core/SettingsChangesHistory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 23f7810835c..0521f70a91b 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -89,8 +89,8 @@ static std::map sett {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, + {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, + {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, {"http_max_chunk_size", 0, 0, "Internal limitation"}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, From 3d207039584cb69d9fffe1b3ec923a31fab5f032 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 24 May 2024 12:27:19 +0000 Subject: [PATCH 134/158] Force-enable analyzer so that tests without Analyzer can no longer fail --- tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index 15015761295..a5339a098dc 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -15,7 +15,7 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" -SETTINGS="SETTINGS use_query_cache=1, max_threads=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. From aada1de796144829b2a6e334764923cef6da4fff Mon Sep 17 00:00:00 2001 From: TTPO100AJIEX Date: Fri, 24 May 2024 15:36:41 +0300 Subject: [PATCH 135/158] Rename function parameters, remove unnecessary virtual --- src/Server/ServersManager/IServersManager.cpp | 8 ++-- src/Server/ServersManager/IServersManager.h | 14 +++---- .../ServersManager/InterServersManager.cpp | 20 +++++----- .../ServersManager/InterServersManager.h | 1 - .../ServersManager/ProtocolServersManager.cpp | 40 +++++++++---------- 5 files changed, 41 insertions(+), 42 deletions(-) diff --git a/src/Server/ServersManager/IServersManager.cpp b/src/Server/ServersManager/IServersManager.cpp index c903d90f766..8b1eee94303 100644 --- a/src/Server/ServersManager/IServersManager.cpp +++ b/src/Server/ServersManager/IServersManager.cpp @@ -17,8 +17,8 @@ extern const int NETWORK_ERROR; extern const int INVALID_CONFIG_PARAMETER; } -IServersManager::IServersManager(ContextMutablePtr l_global_context, Poco::Logger * l_logger) - : global_context(l_global_context), logger(l_logger) +IServersManager::IServersManager(ContextMutablePtr global_context_, Poco::Logger * logger_) + : global_context(global_context_), logger(logger_) { } @@ -107,8 +107,8 @@ void IServersManager::createServer( const Poco::Util::AbstractConfiguration & config, const std::string & listen_host, const char * port_name, - CreateServerFunc && func, - bool start_server) + bool start_server, + CreateServerFunc && func) { /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. if (config.getString(port_name, "").empty()) diff --git a/src/Server/ServersManager/IServersManager.h b/src/Server/ServersManager/IServersManager.h index 5218ab63554..7e1d9d50d82 100644 --- a/src/Server/ServersManager/IServersManager.h +++ b/src/Server/ServersManager/IServersManager.h @@ -19,7 +19,7 @@ namespace DB class IServersManager { public: - IServersManager(ContextMutablePtr global_context, Poco::Logger * logger); + IServersManager(ContextMutablePtr global_context_, Poco::Logger * logger_); virtual ~IServersManager() = default; bool empty() const; @@ -35,9 +35,9 @@ public: const ServerType & server_type) = 0; - virtual void startServers(); + void startServers(); - virtual void stopServers(const ServerType & server_type); + void stopServers(const ServerType & server_type); virtual size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) = 0; virtual void updateServers( @@ -58,14 +58,14 @@ protected: const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port) const; using CreateServerFunc = std::function; - virtual void createServer( + void createServer( const Poco::Util::AbstractConfiguration & config, const std::string & listen_host, const char * port_name, - CreateServerFunc && func, - bool start_server); + bool start_server, + CreateServerFunc && func); - virtual void stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config); + void stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config); Strings getListenHosts(const Poco::Util::AbstractConfiguration & config) const; bool getListenTry(const Poco::Util::AbstractConfiguration & config) const; diff --git a/src/Server/ServersManager/InterServersManager.cpp b/src/Server/ServersManager/InterServersManager.cpp index 28491a4f4f4..4425d468248 100644 --- a/src/Server/ServersManager/InterServersManager.cpp +++ b/src/Server/ServersManager/InterServersManager.cpp @@ -71,6 +71,7 @@ void InterServersManager::createServers( config, listen_host, port_name, + /* start_server = */ false, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -92,14 +93,14 @@ void InterServersManager::createServers( false), server_pool, socket)); - }, - /* start_server = */ false); + }); constexpr auto secure_port_name = "keeper_server.tcp_port_secure"; createServer( config, listen_host, secure_port_name, + /* start_server = */ false, [&](UInt16 port) -> ProtocolServerAdapter { # if USE_SSL @@ -128,14 +129,14 @@ void InterServersManager::createServers( ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); # endif - }, - /* start_server: */ false); + }); /// HTTP control endpoints createServer( config, listen_host, /* port_name = */ "keeper_server.http_control.port", + /* start_server = */ false, [&](UInt16 port) -> ProtocolServerAdapter { auto http_context = std::make_shared(global_context); @@ -159,8 +160,7 @@ void InterServersManager::createServers( server_pool, socket, http_params)); - }, - /* start_server: */ false); + }); } #else throw Exception( @@ -264,6 +264,7 @@ void InterServersManager::createInterserverServers( config, interserver_listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -282,8 +283,7 @@ void InterServersManager::createInterserverServers( http_params, ProfileEvents::InterfaceInterserverReceiveBytes, ProfileEvents::InterfaceInterserverSendBytes)); - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) @@ -293,6 +293,7 @@ void InterServersManager::createInterserverServers( config, interserver_listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { #if USE_SSL @@ -318,8 +319,7 @@ void InterServersManager::createInterserverServers( ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }, - start_servers); + }); } } } diff --git a/src/Server/ServersManager/InterServersManager.h b/src/Server/ServersManager/InterServersManager.h index 2a389e28c22..8780eae18e0 100644 --- a/src/Server/ServersManager/InterServersManager.h +++ b/src/Server/ServersManager/InterServersManager.h @@ -19,7 +19,6 @@ public: bool start_servers, const ServerType & server_type) override; - using IServersManager::stopServers; size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) override; void updateServers( diff --git a/src/Server/ServersManager/ProtocolServersManager.cpp b/src/Server/ServersManager/ProtocolServersManager.cpp index 17b028eddbb..af57de3ac3c 100644 --- a/src/Server/ServersManager/ProtocolServersManager.cpp +++ b/src/Server/ServersManager/ProtocolServersManager.cpp @@ -99,6 +99,7 @@ void ProtocolServersManager::createServers( config, host, port_name.c_str(), + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -110,8 +111,7 @@ void ProtocolServersManager::createServers( port_name.c_str(), description + ": " + address.toString(), std::make_unique(stack.release(), server_pool, socket, new Poco::Net::TCPServerParams)); - }, - start_servers); + }); } } @@ -125,6 +125,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -143,8 +144,7 @@ void ProtocolServersManager::createServers( http_params, ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes)); - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::HTTPS)) @@ -155,6 +155,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { #if USE_SSL @@ -180,8 +181,7 @@ void ProtocolServersManager::createServers( ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); #endif - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::TCP)) @@ -192,6 +192,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -208,8 +209,7 @@ void ProtocolServersManager::createServers( server_pool, socket, new Poco::Net::TCPServerParams)); - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) @@ -220,6 +220,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -236,8 +237,7 @@ void ProtocolServersManager::createServers( server_pool, socket, new Poco::Net::TCPServerParams)); - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) @@ -248,6 +248,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { #if USE_SSL @@ -271,8 +272,7 @@ void ProtocolServersManager::createServers( ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); #endif - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::MYSQL)) @@ -282,6 +282,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -298,8 +299,7 @@ void ProtocolServersManager::createServers( server_pool, socket, new Poco::Net::TCPServerParams)); - }, - start_servers); + }); } if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) @@ -309,6 +309,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -325,8 +326,7 @@ void ProtocolServersManager::createServers( server_pool, socket, new Poco::Net::TCPServerParams)); - }, - start_servers); + }); } #if USE_GRPC @@ -337,6 +337,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::SocketAddress server_address(listen_host, port); @@ -345,8 +346,7 @@ void ProtocolServersManager::createServers( port_name, "gRPC protocol: " + server_address.toString(), std::make_unique(server, makeSocketAddress(listen_host, port, logger))); - }, - start_servers); + }); } #endif if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) @@ -357,6 +357,7 @@ void ProtocolServersManager::createServers( config, listen_host, port_name, + start_servers, [&](UInt16 port) -> ProtocolServerAdapter { Poco::Net::ServerSocket socket; @@ -375,8 +376,7 @@ void ProtocolServersManager::createServers( http_params, ProfileEvents::InterfacePrometheusReceiveBytes, ProfileEvents::InterfacePrometheusSendBytes)); - }, - start_servers); + }); } } } From 2cc1b27fb5f898a8c728dda03f4dea3941c653b4 Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 24 May 2024 14:41:04 +0200 Subject: [PATCH 136/158] Update docs for settings cross_join_min_rows_to_compress and cross_join_min_bytes_to_compress --- docs/en/operations/settings/settings.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2b5cd11819a..b2efe5d2af4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5468,3 +5468,15 @@ Defines how MySQL types are converted to corresponding ClickHouse types. A comma - `datetime64`: convert `DATETIME` and `TIMESTAMP` types to `DateTime64` instead of `DateTime` when precision is not `0`. - `date2Date32`: convert `DATE` to `Date32` instead of `Date`. Takes precedence over `date2String`. - `date2String`: convert `DATE` to `String` instead of `Date`. Overridden by `datetime64`. + +## cross_join_min_rows_to_compress + +Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. + +Default value: `10000000`. + +## cross_join_min_bytes_to_compress + +Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached. + +Default value: `1GiB`. From cb37b098ef23b0575b987edf35db2276bdb02a69 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 16:17:25 +0200 Subject: [PATCH 137/158] CI: add secrets to reusable stage wf yml --- .github/workflows/reusable_test_stage.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/reusable_test_stage.yml b/.github/workflows/reusable_test_stage.yml index d7bd55fab43..8926b43d372 100644 --- a/.github/workflows/reusable_test_stage.yml +++ b/.github/workflows/reusable_test_stage.yml @@ -10,6 +10,10 @@ name: StageWF description: ci data type: string required: true + secrets: + secret_envs: + description: if given, it's passed to the environments + required: false jobs: s: @@ -23,3 +27,5 @@ jobs: test_name: ${{ matrix.job_name_and_runner_type.job_name }} runner_type: ${{ matrix.job_name_and_runner_type.runner_type }} data: ${{ inputs.data }} + secrets: + secret_envs: ${{ secrets.secret_envs }} From b254be618087e8f949f420406e791b24d11c960a Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 16:57:08 +0200 Subject: [PATCH 138/158] CI: add secrets to reusable build stage wf yml --- .github/workflows/reusable_build_stage.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/workflows/reusable_build_stage.yml b/.github/workflows/reusable_build_stage.yml index 4463645880b..a8e84819c95 100644 --- a/.github/workflows/reusable_build_stage.yml +++ b/.github/workflows/reusable_build_stage.yml @@ -13,6 +13,10 @@ name: BuildStageWF description: ci data type: string required: true + secrets: + secret_envs: + description: if given, it's passed to the environments + required: false jobs: s: @@ -30,3 +34,5 @@ jobs: # for now let's do I deep checkout for builds checkout_depth: 0 data: ${{ inputs.data }} + secrets: + secret_envs: ${{ secrets.secret_envs }} From 4982d7c85cc7a71ddef773cd57df540e7b8cd33a Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 16:59:47 +0200 Subject: [PATCH 139/158] fix for mark release ready --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 7c55098bdfd..c2a893a8e99 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -136,7 +136,7 @@ jobs: MarkReleaseReady: if: ${{ !failure() && !cancelled() }} - needs: [RunConfig, Builds_1] + needs: [RunConfig, Builds_1, Builds_2] runs-on: [self-hosted, style-checker-aarch64] steps: - name: Debug From b3f836fbb1b451c08d57f4956c0a9c5137fe5ede Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:08:30 +0200 Subject: [PATCH 140/158] Run 03147_system_columns_access_checks only on release --- tests/queries/0_stateless/03147_system_columns_access_checks.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03147_system_columns_access_checks.sh b/tests/queries/0_stateless/03147_system_columns_access_checks.sh index 2bd7fb083ea..b027ea28504 100755 --- a/tests/queries/0_stateless/03147_system_columns_access_checks.sh +++ b/tests/queries/0_stateless/03147_system_columns_access_checks.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-ordinary-database, long +# Tags: no-fasttest, no-parallel, no-ordinary-database, long, no-debug, no-asan, no-tsan, no-msan CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d5b763d03d581b70b1243ab589223d85d231fe89 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:21:50 +0200 Subject: [PATCH 141/158] Limit max time for 01442_merge_detach_attach_long --- .../01442_merge_detach_attach_long.sh | 26 +++++++++++++------ 1 file changed, 18 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index acb2550d48c..e7c20158b5d 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel, no-debug +# Tags: long, no-parallel set -e @@ -11,14 +11,24 @@ CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE t (x Int8) ENGINE = MergeTree ORDER BY tuple()" -for _ in {1..100}; do - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" - ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & - ${CLICKHOUSE_CLIENT} --query="ALTER TABLE t DETACH PARTITION tuple()" - ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" -done +function thread_ops() +{ + local TIMELIMIT=$((SECONDS+$1)) + local it=0 + while [ $SECONDS -lt "$TIMELIMIT" ] && [ $it -lt 100 ]; + do + it=$((it+1)) + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="INSERT INTO t VALUES (0)" + ${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE t FINAL" 2>/dev/null & + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE t DETACH PARTITION tuple()" + ${CLICKHOUSE_CLIENT} --query="SELECT count() FROM t HAVING count() > 0" + done +} +export -f thread_ops +TIMEOUT=60 +thread_ops $TIMEOUT & wait $CLICKHOUSE_CLIENT -q "DROP TABLE t" From bd415cc83192a734dccb00bd004775e46bd74b7c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:27:47 +0200 Subject: [PATCH 142/158] Reduce 02228_merge_tree_insert_memory_usage partitions --- .../02228_merge_tree_insert_memory_usage.sql | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 8924627a717..26a201ec89f 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,16 +1,16 @@ -- Tags: long, no-parallel -SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections +SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 100 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush drop table if exists data_02228; -create table data_02228 (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 1024; -insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; -insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } +create table data_02228 (key1 UInt32, sign Int8, s UInt64) engine = CollapsingMergeTree(sign) order by (key1) partition by key1 % 100; +insert into data_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; +insert into data_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=1000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; drop table if exists data_rep_02228 SYNC; -create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; -insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; -insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } +create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 100; +insert into data_rep_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; +insert into data_rep_02228 select number, 1, number from numbers_mt(10_000) settings max_memory_usage='30Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=1000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_rep_02228 SYNC; From b396e63ea5721f72e0a1efb15e1c108c93dfad2e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:30:26 +0200 Subject: [PATCH 143/158] Reduce sizes in 02735_parquet_encoder --- tests/queries/0_stateless/02735_parquet_encoder.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02735_parquet_encoder.sql b/tests/queries/0_stateless/02735_parquet_encoder.sql index fe45a2a317d..9320d0e57c3 100644 --- a/tests/queries/0_stateless/02735_parquet_encoder.sql +++ b/tests/queries/0_stateless/02735_parquet_encoder.sql @@ -41,7 +41,7 @@ create temporary table basic_types_02735 as select * from generateRandom(' decimal128 Decimal128(20), decimal256 Decimal256(40), ipv4 IPv4, - ipv6 IPv6') limit 10101; + ipv6 IPv6') limit 1011; insert into function file(basic_types_02735.parquet) select * from basic_types_02735; desc file(basic_types_02735.parquet); select (select sum(cityHash64(*)) from basic_types_02735) - (select sum(cityHash64(*)) from file(basic_types_02735.parquet)); @@ -59,7 +59,7 @@ create temporary table nullables_02735 as select * from generateRandom(' fstr Nullable(FixedString(12)), i256 Nullable(Int256), decimal256 Nullable(Decimal256(40)), - ipv6 Nullable(IPv6)') limit 10000; + ipv6 Nullable(IPv6)') limit 1000; insert into function file(nullables_02735.parquet) select * from nullables_02735; select (select sum(cityHash64(*)) from nullables_02735) - (select sum(cityHash64(*)) from file(nullables_02735.parquet)); drop table nullables_02735; @@ -83,7 +83,7 @@ create table arrays_02735 engine = Memory as select * from generateRandom(' decimal64 Array(Decimal64(10)), ipv4 Array(IPv4), msi Map(String, Int16), - tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 10000; + tup Tuple(FixedString(3), Array(String), Map(Int8, Date))') limit 1000; insert into function file(arrays_02735.parquet) select * from arrays_02735; create temporary table arrays_out_02735 as arrays_02735; insert into arrays_out_02735 select * from file(arrays_02735.parquet); @@ -107,7 +107,7 @@ create temporary table madness_02735 as select * from generateRandom(' mln Map(LowCardinality(String), Nullable(Int8)), t Tuple(Map(FixedString(5), Tuple(Array(UInt16), Nullable(UInt16), Array(Tuple(Int8, Decimal64(10))))), Tuple(kitchen UInt64, sink String)), n Nested(hello UInt64, world Tuple(first String, second FixedString(1))) - ') limit 10000; + ') limit 1000; insert into function file(madness_02735.parquet) select * from madness_02735; insert into function file(a.csv) select * from madness_02735 order by tuple(*); insert into function file(b.csv) select aa, aaa, an, aan, l, ln, arrayMap(x->reinterpret(x, 'UInt128'), al) as al_, aaln, mln, t, n.hello, n.world from file(madness_02735.parquet) order by tuple(aa, aaa, an, aan, l, ln, al_, aaln, mln, t, n.hello, n.world); From 24797a093a216479d70b2b0e065d9f3850d484bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:31:39 +0200 Subject: [PATCH 144/158] Remove 02344_insert_profile_events_stress from sanitizer run as it's too slow --- .../queries/0_stateless/02344_insert_profile_events_stress.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql index f9fdd3b943f..e9a790bea5d 100644 --- a/tests/queries/0_stateless/02344_insert_profile_events_stress.sql +++ b/tests/queries/0_stateless/02344_insert_profile_events_stress.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel, long, no-debug, no-tsan +-- Tags: no-parallel, long, no-debug, no-tsan, no-msan, no-asan create table data_02344 (key Int) engine=Null; -- 3e9 rows is enough to fill the socket buffer and cause INSERT hung. From 049ca7c71e5c3543e4a63d22f075de2ff96373c9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 24 May 2024 17:34:48 +0200 Subject: [PATCH 145/158] Reduce 01396_inactive_replica_cleanup_nodes_zookeeper! --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 67a2a70b509..11102b128b2 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: replica, no-debug, no-parallel +# Tags: replica, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -11,7 +11,7 @@ REPLICA=$($CLICKHOUSE_CLIENT --query "Select getMacro('replica')") # Check that if we have one inactive replica and a huge number of INSERTs to active replicas, # the number of nodes in ZooKeeper does not grow unbounded. -SCALE=5000 +SCALE=1000 $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; From 7f9734d0cc9dc270ea129b75881234ace3cdf1fa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 24 May 2024 15:38:21 +0000 Subject: [PATCH 146/158] Fix Logical error: Bad cast for Buffer table with prewhere. --- src/Storages/StorageBuffer.cpp | 2 ++ .../0_stateless/00910_buffer_prewhere_different_types.sql | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index d9a0b2b4d59..a3f6b6afc5d 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -302,6 +302,8 @@ void StorageBuffer::read( auto src_table_query_info = query_info; if (src_table_query_info.prewhere_info) { + src_table_query_info.prewhere_info = src_table_query_info.prewhere_info->clone(); + auto actions_dag = ActionsDAG::makeConvertingActions( header_after_adding_defaults.getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), diff --git a/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql b/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql index 8f305914cb8..702d9bb3e6c 100644 --- a/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql +++ b/tests/queries/0_stateless/00910_buffer_prewhere_different_types.sql @@ -2,8 +2,14 @@ DROP TABLE IF EXISTS buffer_table1__fuzz_28; DROP TABLE IF EXISTS merge_tree_table1; CREATE TABLE merge_tree_table1 (`x` UInt32) ENGINE = MergeTree ORDER BY x; + +CREATE TABLE buffer_table1__fuzz_24 (`s` Nullable(Int128), `x` Nullable(FixedString(17))) ENGINE = Buffer(currentDatabase(), 'merge_tree_table1', 16, 10, 60, 10, 1000, 1048576, 2097152); +SELECT s FROM buffer_table1__fuzz_24 PREWHERE factorial(toNullable(10)); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } + INSERT INTO merge_tree_table1 VALUES (1), (2), (3), (4), (5), (6), (7), (8), (9), (10); +SELECT s FROM buffer_table1__fuzz_24 PREWHERE factorial(toNullable(10)); -- { serverError ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER } + SET send_logs_level='error'; CREATE TABLE buffer_table1__fuzz_28 (`x` Nullable(UInt32)) ENGINE = Buffer(currentDatabase(), 'merge_tree_table1', 16, 10, 60, 10, 1000, 1048576, 2097152); From 2669df7296a1b362807693d0cc41833ecf80a148 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 24 May 2024 17:30:36 +0200 Subject: [PATCH 147/158] add secrets to reusable build yml --- .github/workflows/reusable_build.yml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/.github/workflows/reusable_build.yml b/.github/workflows/reusable_build.yml index 80d78d93e1b..5e254d785ec 100644 --- a/.github/workflows/reusable_build.yml +++ b/.github/workflows/reusable_build.yml @@ -33,6 +33,10 @@ name: Build ClickHouse additional_envs: description: additional ENV variables to setup the job type: string + secrets: + secret_envs: + description: if given, it's passed to the environments + required: false jobs: Build: @@ -54,6 +58,7 @@ jobs: run: | cat >> "$GITHUB_ENV" << 'EOF' ${{inputs.additional_envs}} + ${{secrets.secret_envs}} DOCKER_TAG< Date: Fri, 24 May 2024 17:44:14 +0200 Subject: [PATCH 148/158] Restore tags --- .../01396_inactive_replica_cleanup_nodes_zookeeper.sh | 2 +- tests/queries/0_stateless/01442_merge_detach_attach_long.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index 11102b128b2..1c1eb4489ee 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: replica, no-parallel +# Tags: replica, no-debug, no-parallel CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh index e7c20158b5d..85fdf7ed764 100755 --- a/tests/queries/0_stateless/01442_merge_detach_attach_long.sh +++ b/tests/queries/0_stateless/01442_merge_detach_attach_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-parallel +# Tags: long, no-parallel, no-debug set -e From 772d38a0c139ca5ee76bd7886d70db874db503c0 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 24 May 2024 18:11:21 +0200 Subject: [PATCH 149/158] Update s3queue.md --- docs/en/engines/table-engines/integrations/s3queue.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 8ebab80423f..aa7fa512480 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -202,8 +202,7 @@ Example: CREATE TABLE s3queue_engine_table (name String, value UInt32) ENGINE=S3Queue('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/*', 'CSV', 'gzip') SETTINGS - mode = 'unordered', - keeper_path = '/clickhouse/s3queue/'; + mode = 'unordered'; CREATE TABLE stats (name String, value UInt32) ENGINE = MergeTree() ORDER BY name; From e59097274a72216e99dbec83cbbe4f5142463799 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 May 2024 13:56:16 -0300 Subject: [PATCH 150/158] test for #64211 --- ...uted_merge_global_in_primary_key.reference | 19 +++++ ...istributed_merge_global_in_primary_key.sql | 83 +++++++++++++++++++ 2 files changed, 102 insertions(+) create mode 100644 tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference create mode 100644 tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference new file mode 100644 index 00000000000..f572a3570f4 --- /dev/null +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.reference @@ -0,0 +1,19 @@ +------------------- Distributed ------------------ +1 +---------- merge() over distributed -------------- +2 +---------- merge() over local -------------------- +1 +1 +1 +---------- remote() over Merge ------------------- +2 +---------- Distributed over Merge ---------------- +1 +---------- remote() over Merge ------------------- +2 +---------- Merge over Distributed ----------------- +1 +1 +1 +2 diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql new file mode 100644 index 00000000000..78176e346f4 --- /dev/null +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql @@ -0,0 +1,83 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/64211 + +create database test; +use test; + +CREATE TABLE test_local (name String) +ENGINE = MergeTree +ORDER BY name as select 'x'; + +CREATE TABLE test_distributed as test_local +ENGINE = Distributed(default, currentDatabase(), test_local); + +CREATE TABLE test_merge as test_local +ENGINE = Merge(currentDatabase(), 'test_local'); + +CREATE TABLE test_merge_distributed as test_local +ENGINE = Distributed(default, currentDatabase(), test_merge); + +CREATE TABLE test_distributed_merge as test_local +ENGINE = Merge(currentDatabase(), 'test_distributed'); + +SELECT '------------------- Distributed ------------------'; +SELECT count() +FROM test_distributed +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- merge() over distributed --------------'; +SELECT count() +FROM merge(currentDatabase(), 'test_distributed') +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- merge() over local --------------------'; +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM merge(currentDatabase(), 'test_local')); + +SELECT count() +FROM merge(currentDatabase(), 'test_local') +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT '---------- remote() over Merge -------------------'; +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge) +WHERE name GLOBAL IN (SELECT name FROM test_distributed); + +SELECT '---------- Distributed over Merge ----------------'; +SELECT count() +FROM test_merge_distributed +WHERE name GLOBAL IN (SELECT name FROM test_merge_distributed); + +SELECT '---------- remote() over Merge -------------------'; +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge) +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT '---------- Merge over Distributed -----------------'; +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_distributed_merge)); + +SELECT count() +FROM test_distributed_merge +WHERE name GLOBAL IN (SELECT name FROM test_distributed_merge); + +SELECT count() +FROM remote('127.0.0.{1,2}', currentDatabase(), test_distributed_merge) +WHERE name GLOBAL IN (SELECT name FROM remote('127.0.0.{1,2}', currentDatabase(), test_merge)); + + +DROP TABLE test_merge; +DROP TABLE test_merge_distributed; +DROP TABLE test_distributed_merge; +DROP TABLE test_distributed; +DROP TABLE test_local; +drop database test; From 9a917db4b3eade94941225b4a792f4d2331459ba Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 May 2024 14:27:26 -0300 Subject: [PATCH 151/158] Update 01227_distributed_merge_global_in_primary_key.sql --- .../01227_distributed_merge_global_in_primary_key.sql | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql index 78176e346f4..e73d07c193f 100644 --- a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql @@ -1,8 +1,5 @@ -- https://github.com/ClickHouse/ClickHouse/issues/64211 -create database test; -use test; - CREATE TABLE test_local (name String) ENGINE = MergeTree ORDER BY name as select 'x'; @@ -80,4 +77,3 @@ DROP TABLE test_merge_distributed; DROP TABLE test_distributed_merge; DROP TABLE test_distributed; DROP TABLE test_local; -drop database test; From 91a84f8e17192a70b48d3152ad8b48107d60c117 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 May 2024 15:03:45 -0300 Subject: [PATCH 152/158] Update 01227_distributed_merge_global_in_primary_key.sql --- .../01227_distributed_merge_global_in_primary_key.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql index e73d07c193f..5cd4aaab1e6 100644 --- a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql @@ -5,13 +5,13 @@ ENGINE = MergeTree ORDER BY name as select 'x'; CREATE TABLE test_distributed as test_local -ENGINE = Distributed(default, currentDatabase(), test_local); +ENGINE = Distributed(test_shard_localhost, currentDatabase(), test_local); CREATE TABLE test_merge as test_local ENGINE = Merge(currentDatabase(), 'test_local'); CREATE TABLE test_merge_distributed as test_local -ENGINE = Distributed(default, currentDatabase(), test_merge); +ENGINE = Distributed(test_shard_localhost, currentDatabase(), test_merge); CREATE TABLE test_distributed_merge as test_local ENGINE = Merge(currentDatabase(), 'test_distributed'); From 3ed1ec2f63582819f005d591459f30cdbff0daff Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 24 May 2024 23:54:56 -0300 Subject: [PATCH 153/158] Update tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql Co-authored-by: Nikita Mikhaylov --- .../01227_distributed_merge_global_in_primary_key.sql | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql index 5cd4aaab1e6..6b0dd4c8747 100644 --- a/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql +++ b/tests/queries/0_stateless/01227_distributed_merge_global_in_primary_key.sql @@ -1,5 +1,9 @@ -- https://github.com/ClickHouse/ClickHouse/issues/64211 - +DROP TABLE IF EXISTS test_merge; +DROP TABLE IF EXISTS test_merge_distributed; +DROP TABLE IF EXISTS test_distributed_merge; +DROP TABLE IF EXISTS test_distributed; +DROP TABLE IF EXISTS test_local; CREATE TABLE test_local (name String) ENGINE = MergeTree ORDER BY name as select 'x'; From 031591f3dd5ae155e3a8d8cf061e2956a29e6a4a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 25 May 2024 15:48:45 +0200 Subject: [PATCH 154/158] Fix settings changes history --- src/Core/SettingsChangesHistory.h | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index a89516436e8..16f28d94640 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -85,6 +85,14 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"24.6", {{"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, + {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, + {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, + {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, + {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, + {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, + {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, + }}, {"24.5", {{"allow_deprecated_functions", true, false, "Allow usage of deprecated functions"}, {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, @@ -93,13 +101,6 @@ static std::map sett {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, {"http_max_chunk_size", 0, 0, "Internal limitation"}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, From 142d67d1b298478a0df46b2585d4719a9ef55f4e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 26 May 2024 11:16:48 +0200 Subject: [PATCH 155/158] Fix S3ObjectStorage::applyNewSettings --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 21 ++++++++----------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 5 +---- .../ObjectStorage/S3/Configuration.cpp | 2 +- 3 files changed, 11 insertions(+), 17 deletions(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index c07313b52db..69485bd4d01 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -575,24 +575,21 @@ void S3ObjectStorage::applyNewSettings( ContextPtr context, const ApplyNewSettingsOptions & options) { - auto new_s3_settings = getSettings(config, config_prefix, context, context->getSettingsRef().s3_validate_request_settings); - if (!static_headers.empty()) - { - new_s3_settings->auth_settings.headers.insert( - new_s3_settings->auth_settings.headers.end(), - static_headers.begin(), static_headers.end()); - } + auto settings_from_config = getSettings(config, config_prefix, context, context->getSettingsRef().s3_validate_request_settings); + auto modified_settings = std::make_unique(*s3_settings.get()); + modified_settings->auth_settings.updateFrom(settings_from_config->auth_settings); if (auto endpoint_settings = context->getStorageS3Settings().getSettings(uri.uri.toString(), context->getUserName())) - new_s3_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); + modified_settings->auth_settings.updateFrom(endpoint_settings->auth_settings); - auto current_s3_settings = s3_settings.get(); - if (options.allow_client_change && (current_s3_settings->auth_settings.hasUpdates(new_s3_settings->auth_settings) || for_disk_s3)) + auto current_settings = s3_settings.get(); + if (options.allow_client_change + && (current_settings->auth_settings.hasUpdates(modified_settings->auth_settings) || for_disk_s3)) { - auto new_client = getClient(config, config_prefix, context, *new_s3_settings, for_disk_s3, &uri); + auto new_client = getClient(config, config_prefix, context, *modified_settings, for_disk_s3, &uri); client.set(std::move(new_client)); } - s3_settings.set(std::move(new_s3_settings)); + s3_settings.set(std::move(modified_settings)); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 1fff6d67e23..062ddd4e2a2 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -54,8 +54,7 @@ private: const S3Capabilities & s3_capabilities_, ObjectStorageKeysGeneratorPtr key_generator_, const String & disk_name_, - bool for_disk_s3_ = true, - const HTTPHeaderEntries & static_headers_ = {}) + bool for_disk_s3_ = true) : uri(uri_) , disk_name(disk_name_) , client(std::move(client_)) @@ -64,7 +63,6 @@ private: , key_generator(std::move(key_generator_)) , log(getLogger(logger_name)) , for_disk_s3(for_disk_s3_) - , static_headers(static_headers_) { } @@ -189,7 +187,6 @@ private: LoggerPtr log; const bool for_disk_s3; - const HTTPHeaderEntries static_headers; }; } diff --git a/src/Storages/ObjectStorage/S3/Configuration.cpp b/src/Storages/ObjectStorage/S3/Configuration.cpp index 6b6cde0c431..4b217b94730 100644 --- a/src/Storages/ObjectStorage/S3/Configuration.cpp +++ b/src/Storages/ObjectStorage/S3/Configuration.cpp @@ -136,7 +136,7 @@ ObjectStoragePtr StorageS3Configuration::createObjectStorage(ContextPtr context, return std::make_shared( std::move(client), std::move(s3_settings), url, s3_capabilities, - key_generator, "StorageS3", false, headers_from_ast); + key_generator, "StorageS3", false); } void StorageS3Configuration::fromNamedCollection(const NamedCollection & collection) From 14f259d9d7a9d53ed8d1c64be36be20a622bf7ce Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 May 2024 13:54:35 +0000 Subject: [PATCH 156/158] Fix flaky test --- tests/queries/0_stateless/03130_generateSnowflakeId.reference | 2 -- tests/queries/0_stateless/03130_generateSnowflakeId.sql | 2 -- 2 files changed, 4 deletions(-) diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.reference b/tests/queries/0_stateless/03130_generateSnowflakeId.reference index 6ec0cafab16..f5b7872f81e 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.reference +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.reference @@ -1,11 +1,9 @@ -- generateSnowflakeID 1 -1 0 0 1 100 -- generateSnowflakeIDThreadMonotonic 1 -1 100 diff --git a/tests/queries/0_stateless/03130_generateSnowflakeId.sql b/tests/queries/0_stateless/03130_generateSnowflakeId.sql index 903be5b786c..57cdd21a9fe 100644 --- a/tests/queries/0_stateless/03130_generateSnowflakeId.sql +++ b/tests/queries/0_stateless/03130_generateSnowflakeId.sql @@ -1,6 +1,5 @@ SELECT '-- generateSnowflakeID'; -SELECT bitShiftLeft(toUInt64(generateSnowflakeID()), 52) = 0; -- check machine sequence number is zero SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeID()), 63), 1) = 0; -- check first bit is zero SELECT generateSnowflakeID(1) = generateSnowflakeID(2); -- disabled common subexpression elimination --> lhs != rhs @@ -18,7 +17,6 @@ FROM SELECT '-- generateSnowflakeIDThreadMonotonic'; -SELECT bitShiftLeft(toUInt64(generateSnowflakeIDThreadMonotonic()), 52) = 0; -- check machine sequence number is zero SELECT bitAnd(bitShiftRight(toUInt64(generateSnowflakeIDThreadMonotonic()), 63), 1) = 0; -- check first bit is zero SELECT generateSnowflakeIDThreadMonotonic(1, 2); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } From 8f4422d72917c1885a892200e267268f6b2e3b98 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 26 May 2024 14:07:50 +0000 Subject: [PATCH 157/158] Test analyzer and non-analyzer execution --- .../02494_query_cache_nested_query_bug.reference | 2 ++ .../02494_query_cache_nested_query_bug.sh | 12 +++++++++--- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference index b261da18d51..9ec033cefb1 100644 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.reference @@ -1,2 +1,4 @@ +2 +0 1 0 diff --git a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh index a5339a098dc..6bc3d03ac66 100755 --- a/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh +++ b/tests/queries/0_stateless/02494_query_cache_nested_query_bug.sh @@ -15,11 +15,17 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)" ${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)" -SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_NO_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" +SETTINGS_ANALYZER="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=1, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0" # Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated # with EXPLAIN PLAN, we need to check the logs. -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l -${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_NO_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l + +${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" + +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l +${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS_ANALYZER" 2>&1 | grep "Aggregated. " | wc -l ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE" From 3ee2307024c9a7b2c54247335f0fb0f0f54380e4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 27 May 2024 10:04:19 +0200 Subject: [PATCH 158/158] Revert "Refactoring of Server.h: Isolate server management from other logic" --- programs/server/Server.cpp | 987 +++++++++++++++++- programs/server/Server.h | 95 +- src/CMakeLists.txt | 1 - src/Server/ServersManager/IServersManager.cpp | 268 ----- src/Server/ServersManager/IServersManager.h | 74 -- .../ServersManager/InterServersManager.cpp | 327 ------ .../ServersManager/InterServersManager.h | 44 - .../ServersManager/ProtocolServersManager.cpp | 523 ---------- .../ServersManager/ProtocolServersManager.h | 37 - 9 files changed, 1032 insertions(+), 1324 deletions(-) delete mode 100644 src/Server/ServersManager/IServersManager.cpp delete mode 100644 src/Server/ServersManager/IServersManager.h delete mode 100644 src/Server/ServersManager/InterServersManager.cpp delete mode 100644 src/Server/ServersManager/InterServersManager.h delete mode 100644 src/Server/ServersManager/ProtocolServersManager.cpp delete mode 100644 src/Server/ServersManager/ProtocolServersManager.h diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b62ae40924c..223bc1f77e7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -42,9 +44,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -79,19 +83,29 @@ #include #include #include +#include #include "MetricsTransmitter.h" #include +#include +#include #include #include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include #include -#include -#include #include #include #include +#include #include "config.h" #include @@ -105,9 +119,19 @@ #endif #if USE_SSL +# include # include #endif +#if USE_GRPC +# include +#endif + +#if USE_NURAFT +# include +# include +#endif + #if USE_JEMALLOC # include #endif @@ -135,6 +159,18 @@ namespace ProfileEvents { extern const Event MainConfigLoads; extern const Event ServerStartupMilliseconds; + extern const Event InterfaceNativeSendBytes; + extern const Event InterfaceNativeReceiveBytes; + extern const Event InterfaceHTTPSendBytes; + extern const Event InterfaceHTTPReceiveBytes; + extern const Event InterfacePrometheusSendBytes; + extern const Event InterfacePrometheusReceiveBytes; + extern const Event InterfaceInterserverSendBytes; + extern const Event InterfaceInterserverReceiveBytes; + extern const Event InterfaceMySQLSendBytes; + extern const Event InterfaceMySQLReceiveBytes; + extern const Event InterfacePostgreSQLSendBytes; + extern const Event InterfacePostgreSQLReceiveBytes; } namespace fs = std::filesystem; @@ -202,9 +238,11 @@ namespace DB namespace ErrorCodes { extern const int NO_ELEMENTS_IN_CONFIG; + extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int EXCESSIVE_ELEMENT_IN_CONFIG; extern const int INVALID_CONFIG_PARAMETER; + extern const int NETWORK_ERROR; extern const int CORRUPTED_DATA; } @@ -219,6 +257,115 @@ static std::string getCanonicalPath(std::string && path) return std::move(path); } +Poco::Net::SocketAddress Server::socketBindListen( + const Poco::Util::AbstractConfiguration & config, + Poco::Net::ServerSocket & socket, + const std::string & host, + UInt16 port, + [[maybe_unused]] bool secure) const +{ + auto address = makeSocketAddress(host, port, &logger()); + socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config.getBool("listen_reuse_port", false)); + /// If caller requests any available port from the OS, discover it after binding. + if (port == 0) + { + address = socket.address(); + LOG_DEBUG(&logger(), "Requested any available port (port == 0), actual port is {:d}", address.port()); + } + + socket.listen(/* backlog = */ config.getUInt("listen_backlog", 4096)); + + return address; +} + +Strings getListenHosts(const Poco::Util::AbstractConfiguration & config) +{ + auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host"); + if (listen_hosts.empty()) + { + listen_hosts.emplace_back("::1"); + listen_hosts.emplace_back("127.0.0.1"); + } + return listen_hosts; +} + +Strings getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) +{ + auto interserver_listen_hosts = DB::getMultipleValuesFromConfig(config, "", "interserver_listen_host"); + if (!interserver_listen_hosts.empty()) + return interserver_listen_hosts; + + /// Use more general restriction in case of emptiness + return getListenHosts(config); +} + +bool getListenTry(const Poco::Util::AbstractConfiguration & config) +{ + bool listen_try = config.getBool("listen_try", false); + if (!listen_try) + { + Poco::Util::AbstractConfiguration::Keys protocols; + config.keys("protocols", protocols); + listen_try = + DB::getMultipleValuesFromConfig(config, "", "listen_host").empty() && + std::none_of(protocols.begin(), protocols.end(), [&](const auto & protocol) + { + return config.has("protocols." + protocol + ".host") && config.has("protocols." + protocol + ".port"); + }); + } + return listen_try; +} + + +void Server::createServer( + Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + bool listen_try, + bool start_server, + std::vector & servers, + CreateServerFunc && func) const +{ + /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. + if (config.getString(port_name, "").empty()) + return; + + /// If we already have an active server for this listen_host/port_name, don't create it again + for (const auto & server : servers) + { + if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name) + return; + } + + auto port = config.getInt(port_name); + try + { + servers.push_back(func(port)); + if (start_server) + { + servers.back().start(); + LOG_INFO(&logger(), "Listening for {}", servers.back().getDescription()); + } + global_context->registerServerPort(port_name, port); + } + catch (const Poco::Exception &) + { + if (listen_try) + { + LOG_WARNING(&logger(), "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " + "then consider to " + "specify not disabled IPv4 or IPv6 address to listen in element of configuration " + "file. Example for disabled IPv6: 0.0.0.0 ." + " Example for disabled IPv4: ::", + listen_host, port, getCurrentExceptionMessage(false)); + } + else + { + throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); + } + } +} + #if defined(OS_LINUX) namespace @@ -518,7 +665,6 @@ try ServerSettings server_settings; server_settings.loadSettingsFromConfig(config()); - Poco::ThreadPool server_pool(3, server_settings.max_connections); ASTAlterCommand::setFormatAlterCommandsWithParentheses(server_settings.format_alter_operations_with_parentheses); @@ -575,6 +721,11 @@ try CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::getVersionRevision()); CurrentMetrics::set(CurrentMetrics::VersionInteger, ClickHouseRevision::getVersionInteger()); + Poco::ThreadPool server_pool(3, server_settings.max_connections); + std::mutex servers_lock; + std::vector servers; + std::vector servers_to_start_before_tables; + /** Context contains all that query execution is dependent: * settings, available functions, data types, aggregate functions, databases, ... */ @@ -624,10 +775,6 @@ try bool will_have_trace_collector = hasPHDRCache() && config().has("trace_log"); - std::mutex servers_lock; - ProtocolServersManager servers(context(), &logger()); - InterServersManager servers_to_start_before_tables(context(), &logger()); - // Initialize global thread pool. Do it before we fetch configs from zookeeper // nodes (`from_zk`), because ZooKeeper interface uses the pool. We will // ignore `max_thread_pool_size` in configs we fetch from ZK, but oh well. @@ -659,7 +806,32 @@ try LOG_DEBUG(log, "Shut down storages."); - servers_to_start_before_tables.stopServers(server_settings, servers_lock); + if (!servers_to_start_before_tables.empty()) + { + LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish."); + size_t current_connections = 0; + { + std::lock_guard lock(servers_lock); + for (auto & server : servers_to_start_before_tables) + { + server.stop(); + current_connections += server.currentConnections(); + } + } + + if (current_connections) + LOG_INFO(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); + else + LOG_INFO(log, "Closed all listening sockets."); + + if (current_connections > 0) + current_connections = waitServersToFinish(servers_to_start_before_tables, servers_lock, server_settings.shutdown_wait_unfinished); + + if (current_connections) + LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections); + else + LOG_INFO(log, "Closed connections to servers for tables."); + } global_context->shutdownKeeperDispatcher(); @@ -756,13 +928,19 @@ try server_settings.asynchronous_heavy_metrics_update_period_s, [&]() -> std::vector { + std::vector metrics; + std::lock_guard lock(servers_lock); - std::vector metrics1 = servers_to_start_before_tables.getMetrics(); - std::vector metrics2 = servers.getMetrics(); - metrics1.reserve(metrics1.size() + metrics2.size()); - metrics1.insert(metrics1.end(), std::make_move_iterator(metrics2.begin()), std::make_move_iterator(metrics2.end())); - return metrics1; - }); + metrics.reserve(servers_to_start_before_tables.size() + servers.size()); + + for (const auto & server : servers_to_start_before_tables) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + + for (const auto & server : servers) + metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); + return metrics; + } + ); zkutil::validateZooKeeperConfig(config()); bool has_zookeeper = zkutil::hasZooKeeperConfig(config()); @@ -1410,8 +1588,7 @@ try if (global_context->isServerCompletelyStarted()) { std::lock_guard lock(servers_lock); - servers.updateServers(*config, *this, servers_lock, server_pool, async_metrics, latest_config); - servers_to_start_before_tables.updateServers(*config, *this, servers_lock, server_pool, async_metrics, latest_config); + updateServers(*config, server_pool, async_metrics, servers, servers_to_start_before_tables); } } @@ -1458,17 +1635,141 @@ try /// Must be the last. latest_config = config; }, - /* already_loaded = */ false); /// Reload it right now (initial loading) + /* already_loaded = */ false); /// Reload it right now (initial loading) - servers_to_start_before_tables.createServers( - config(), - *this, - servers_lock, - server_pool, - async_metrics, - /* start_servers= */ false, - ServerType(ServerType::Type::QUERIES_ALL) - ); + const auto listen_hosts = getListenHosts(config()); + const auto interserver_listen_hosts = getInterserverListenHosts(config()); + const auto listen_try = getListenTry(config()); + + if (config().has("keeper_server.server_id")) + { +#if USE_NURAFT + //// If we don't have configured connection probably someone trying to use clickhouse-server instead + //// of clickhouse-keeper, so start synchronously. + bool can_initialize_keeper_async = false; + + if (has_zookeeper) /// We have configured connection to some zookeeper cluster + { + /// If we cannot connect to some other node from our cluster then we have to wait our Keeper start + /// synchronously. + can_initialize_keeper_async = global_context->tryCheckClientConnectionToMyKeeperCluster(); + } + /// Initialize keeper RAFT. + global_context->initializeKeeperDispatcher(can_initialize_keeper_async); + FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); + + auto config_getter = [this] () -> const Poco::Util::AbstractConfiguration & + { + return global_context->getConfigRef(); + }; + + for (const auto & listen_host : listen_hosts) + { + /// TCP Keeper + const char * port_name = "keeper_server.tcp_port"; + createServer( + config(), listen_host, port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config(), socket, listen_host, port); + socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); + socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); + return ProtocolServerAdapter( + listen_host, + port_name, + "Keeper (tcp): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), + global_context->getSettingsRef().receive_timeout.totalSeconds(), + global_context->getSettingsRef().send_timeout.totalSeconds(), + false), server_pool, socket)); + }); + + const char * secure_port_name = "keeper_server.tcp_port_secure"; + createServer( + config(), listen_host, secure_port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config(), socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); + socket.setSendTimeout(Poco::Timespan(config().getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); + return ProtocolServerAdapter( + listen_host, + secure_port_name, + "Keeper with secure protocol (tcp_secure): " + address.toString(), + std::make_unique( + new KeeperTCPHandlerFactory( + config_getter, global_context->getKeeperDispatcher(), + global_context->getSettingsRef().receive_timeout.totalSeconds(), + global_context->getSettingsRef().send_timeout.totalSeconds(), true), server_pool, socket)); +#else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + }); + + /// HTTP control endpoints + port_name = "keeper_server.http_control.port"; + createServer(config(), listen_host, port_name, listen_try, /* start_server: */ false, + servers_to_start_before_tables, + [&](UInt16 port) -> ProtocolServerAdapter + { + auto http_context = httpContext(); + Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(http_context->getReceiveTimeout()); + http_params->setKeepAliveTimeout(keep_alive_timeout); + + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config(), socket, listen_host, port); + socket.setReceiveTimeout(http_context->getReceiveTimeout()); + socket.setSendTimeout(http_context->getSendTimeout()); + return ProtocolServerAdapter( + listen_host, + port_name, + "HTTP Control: http://" + address.toString(), + std::make_unique( + std::move(http_context), + createKeeperHTTPControlMainHandlerFactory( + config_getter(), + global_context->getKeeperDispatcher(), + "KeeperHTTPControlHandler-factory"), server_pool, socket, http_params)); + }); + } +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); +#endif + + } + + { + std::lock_guard lock(servers_lock); + /// We should start interserver communications before (and more important shutdown after) tables. + /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. + /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can + /// communicate with zookeeper, execute merges, etc. + createInterserverServers( + config(), + interserver_listen_hosts, + listen_try, + server_pool, + async_metrics, + servers_to_start_before_tables, + /* start_servers= */ false); + + + for (auto & server : servers_to_start_before_tables) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } + } /// Initialize access storages. auto & access_control = global_context->getAccessControl(); @@ -1498,18 +1799,19 @@ try global_context->setStopServersCallback([&](const ServerType & server_type) { std::lock_guard lock(servers_lock); - servers.stopServers(server_type); + stopServers(servers, server_type); }); global_context->setStartServersCallback([&](const ServerType & server_type) { std::lock_guard lock(servers_lock); - servers.createServers( + createServers( config(), - *this, - servers_lock, + listen_hosts, + listen_try, server_pool, async_metrics, + servers, /* start_servers= */ true, server_type); }); @@ -1722,21 +2024,18 @@ try { std::lock_guard lock(servers_lock); - servers.createServers( - config(), - *this, - servers_lock, - server_pool, - async_metrics, - false, - ServerType(ServerType::Type::QUERIES_ALL)); + createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers); if (servers.empty()) - throw Exception( - ErrorCodes::NO_ELEMENTS_IN_CONFIG, - "No servers started (add valid listen_host and 'tcp_port' " - "or 'http_port' to configuration file.)"); + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " + "to configuration file.)"); } + if (servers.empty()) + throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, + "No servers started (add valid listen_host and 'tcp_port' or 'http_port' " + "to configuration file.)"); + #if USE_SSL CertificateReloader::instance().tryLoad(config()); #endif @@ -1808,7 +2107,12 @@ try { std::lock_guard lock(servers_lock); - servers.startServers(); + for (auto & server : servers) + { + server.start(); + LOG_INFO(log, "Listening for {}", server.getDescription()); + } + global_context->setServerCompletelyStarted(); LOG_INFO(log, "Ready for connections."); } @@ -1844,10 +2148,46 @@ try access_control.stopPeriodicReloading(); is_cancelled = true; - const auto remaining_connections = servers.stopServers(server_settings, servers_lock); + + LOG_DEBUG(log, "Waiting for current connections to close."); + + size_t current_connections = 0; + { + std::lock_guard lock(servers_lock); + for (auto & server : servers) + { + server.stop(); + current_connections += server.currentConnections(); + } + } + + if (current_connections) + LOG_WARNING(log, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); + else + LOG_INFO(log, "Closed all listening sockets."); + + /// Wait for unfinished backups and restores. + /// This must be done after closing listening sockets (no more backups/restores) but before ProcessList::killAllQueries + /// (because killAllQueries() will cancel all running backups/restores). + if (server_settings.shutdown_wait_backups_and_restores) + global_context->waitAllBackupsAndRestores(); + + /// Killing remaining queries. + if (!server_settings.shutdown_wait_unfinished_queries) + global_context->getProcessList().killAllQueries(); + + if (current_connections) + current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); + + if (current_connections) + LOG_WARNING(log, "Closed connections. But {} remain." + " Tip: To increase wait time add to config: 60", current_connections); + else + LOG_INFO(log, "Closed connections."); + dns_cache_updater.reset(); - if (remaining_connections) + if (current_connections) { /// There is no better way to force connections to close in Poco. /// Otherwise connection handlers will continue to live @@ -1881,4 +2221,561 @@ catch (...) return code ? code : -1; } +std::unique_ptr Server::buildProtocolStackFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & protocol, + Poco::Net::HTTPServerParams::Ptr http_params, + AsynchronousMetrics & async_metrics, + bool & is_secure) +{ + auto create_factory = [&](const std::string & type, const std::string & conf_name) -> TCPServerConnectionFactory::Ptr + { + if (type == "tcp") + return TCPServerConnectionFactory::Ptr(new TCPHandlerFactory(*this, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes)); + + if (type == "tls") +#if USE_SSL + return TCPServerConnectionFactory::Ptr(new TLSHandlerFactory(*this, conf_name)); +#else + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + + if (type == "proxy1") + return TCPServerConnectionFactory::Ptr(new ProxyV1HandlerFactory(*this, conf_name)); + if (type == "mysql") + return TCPServerConnectionFactory::Ptr(new MySQLHandlerFactory(*this, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes)); + if (type == "postgres") + return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory(*this, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes)); + if (type == "http") + return TCPServerConnectionFactory::Ptr( + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes) + ); + if (type == "prometheus") + return TCPServerConnectionFactory::Ptr( + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), ProfileEvents::InterfacePrometheusReceiveBytes, ProfileEvents::InterfacePrometheusSendBytes) + ); + if (type == "interserver") + return TCPServerConnectionFactory::Ptr( + new HTTPServerConnectionFactory(httpContext(), http_params, createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), ProfileEvents::InterfaceInterserverReceiveBytes, ProfileEvents::InterfaceInterserverSendBytes) + ); + + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); + }; + + std::string conf_name = "protocols." + protocol; + std::string prefix = conf_name + "."; + std::unordered_set pset {conf_name}; + + auto stack = std::make_unique(*this, conf_name); + + while (true) + { + // if there is no "type" - it's a reference to another protocol and this is just an endpoint + if (config.has(prefix + "type")) + { + std::string type = config.getString(prefix + "type"); + if (type == "tls") + { + if (is_secure) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' contains more than one TLS layer", protocol); + is_secure = true; + } + + stack->append(create_factory(type, conf_name)); + } + + if (!config.has(prefix + "impl")) + break; + + conf_name = "protocols." + config.getString(prefix + "impl"); + prefix = conf_name + "."; + + if (!pset.insert(conf_name).second) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); + } + + return stack; +} + +HTTPContextPtr Server::httpContext() const +{ + return std::make_shared(context()); +} + +void Server::createServers( + Poco::Util::AbstractConfiguration & config, + const Strings & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers, + const ServerType & server_type) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + + Poco::Util::AbstractConfiguration::Keys protocols; + config.keys("protocols", protocols); + + for (const auto & protocol : protocols) + { + if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) + continue; + + std::string prefix = "protocols." + protocol + "."; + std::string port_name = prefix + "port"; + std::string description {" protocol"}; + if (config.has(prefix + "description")) + description = config.getString(prefix + "description"); + + if (!config.has(prefix + "port")) + continue; + + std::vector hosts; + if (config.has(prefix + "host")) + hosts.push_back(config.getString(prefix + "host")); + else + hosts = listen_hosts; + + for (const auto & host : hosts) + { + bool is_secure = false; + auto stack = buildProtocolStackFromConfig(config, protocol, http_params, async_metrics, is_secure); + + if (stack->empty()) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' stack empty", protocol); + + createServer(config, host, port_name.c_str(), listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, host, port, is_secure); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + + return ProtocolServerAdapter( + host, + port_name.c_str(), + description + ": " + address.toString(), + std::make_unique( + stack.release(), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } + } + + for (const auto & listen_host : listen_hosts) + { + const char * port_name; + + if (server_type.shouldStart(ServerType::Type::HTTP)) + { + /// HTTP + port_name = "http_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + + return ProtocolServerAdapter( + listen_host, + port_name, + "http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes)); + }); + } + + if (server_type.shouldStart(ServerType::Type::HTTPS)) + { + /// HTTPS + port_name = "https_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "https://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfaceHTTPReceiveBytes, ProfileEvents::InterfaceHTTPSendBytes)); +#else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "HTTPS protocol is disabled because Poco library was built without NetSSL support."); +#endif + }); + } + + if (server_type.shouldStart(ServerType::Type::TCP)) + { + /// TCP + port_name = "tcp_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } + + if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) + { + /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt + port_name = "tcp_with_proxy_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "native protocol (tcp) with PROXY: " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + }); + } + + if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) + { + /// TCP with SSL + port_name = "tcp_port_secure"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + #if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.receive_timeout); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "secure native protocol (tcp_secure): " + address.toString(), + std::make_unique( + new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), + server_pool, + socket, + new Poco::Net::TCPServerParams)); + #else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); + #endif + }); + } + + if (server_type.shouldStart(ServerType::Type::MYSQL)) + { + port_name = "mysql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "MySQL compatibility protocol: " + address.toString(), + std::make_unique(new MySQLHandlerFactory(*this, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } + + if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) + { + port_name = "postgresql_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(Poco::Timespan()); + socket.setSendTimeout(settings.send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "PostgreSQL compatibility protocol: " + address.toString(), + std::make_unique(new PostgreSQLHandlerFactory(*this, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes), server_pool, socket, new Poco::Net::TCPServerParams)); + }); + } + +#if USE_GRPC + if (server_type.shouldStart(ServerType::Type::GRPC)) + { + port_name = "grpc_port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::SocketAddress server_address(listen_host, port); + return ProtocolServerAdapter( + listen_host, + port_name, + "gRPC protocol: " + server_address.toString(), + std::make_unique(*this, makeSocketAddress(listen_host, port, &logger()))); + }); + } +#endif + if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) + { + /// Prometheus (if defined and not setup yet with http_port) + port_name = "prometheus.port"; + createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + listen_host, + port_name, + "Prometheus: http://" + address.toString(), + std::make_unique( + httpContext(), createHandlerFactory(*this, config, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params, ProfileEvents::InterfacePrometheusReceiveBytes, ProfileEvents::InterfacePrometheusSendBytes)); + }); + } + } +} + +void Server::createInterserverServers( + Poco::Util::AbstractConfiguration & config, + const Strings & interserver_listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers, + const ServerType & server_type) +{ + const Settings & settings = global_context->getSettingsRef(); + + Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; + http_params->setTimeout(settings.http_receive_timeout); + http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); + + /// Now iterate over interserver_listen_hosts + for (const auto & interserver_listen_host : interserver_listen_hosts) + { + const char * port_name; + + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) + { + /// Interserver IO HTTP + port_name = "interserver_http_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { + Poco::Net::ServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "replica communication (interserver): http://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceInterserverReceiveBytes, + ProfileEvents::InterfaceInterserverSendBytes)); + }); + } + + if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) + { + port_name = "interserver_https_port"; + createServer(config, interserver_listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter + { +#if USE_SSL + Poco::Net::SecureServerSocket socket; + auto address = socketBindListen(config, socket, interserver_listen_host, port, /* secure = */ true); + socket.setReceiveTimeout(settings.http_receive_timeout); + socket.setSendTimeout(settings.http_send_timeout); + return ProtocolServerAdapter( + interserver_listen_host, + port_name, + "secure replica communication (interserver): https://" + address.toString(), + std::make_unique( + httpContext(), + createHandlerFactory(*this, config, async_metrics, "InterserverIOHTTPSHandler-factory"), + server_pool, + socket, + http_params, + ProfileEvents::InterfaceInterserverReceiveBytes, + ProfileEvents::InterfaceInterserverSendBytes)); +#else + UNUSED(port); + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); +#endif + }); + } + } +} + +void Server::stopServers( + std::vector & servers, + const ServerType & server_type +) const +{ + LoggerRawPtr log = &logger(); + + /// Remove servers once all their connections are closed + auto check_server = [&log](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG(log, "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); + + for (auto & server : servers) + { + if (!server.isStopping()) + { + const std::string server_port_name = server.getPortName(); + + if (server_type.shouldStop(server_port_name)) + server.stop(); + } + } + + std::erase_if(servers, std::bind_front(check_server, "")); +} + +void Server::updateServers( + Poco::Util::AbstractConfiguration & config, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + std::vector & servers_to_start_before_tables) +{ + LoggerRawPtr log = &logger(); + + const auto listen_hosts = getListenHosts(config); + const auto interserver_listen_hosts = getInterserverListenHosts(config); + const auto listen_try = getListenTry(config); + + /// Remove servers once all their connections are closed + auto check_server = [&log](const char prefix[], auto & server) + { + if (!server.isStopping()) + return false; + size_t current_connections = server.currentConnections(); + LOG_DEBUG(log, "Server {}{}: {} ({} connections)", + server.getDescription(), + prefix, + !current_connections ? "finished" : "waiting", + current_connections); + return !current_connections; + }; + + std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); + + Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : this->config(); + + std::vector all_servers; + all_servers.reserve(servers.size() + servers_to_start_before_tables.size()); + for (auto & server : servers) + all_servers.push_back(&server); + + for (auto & server : servers_to_start_before_tables) + all_servers.push_back(&server); + + for (auto * server : all_servers) + { + if (!server->isStopping()) + { + std::string port_name = server->getPortName(); + bool has_host = false; + bool is_http = false; + if (port_name.starts_with("protocols.")) + { + std::string protocol = port_name.substr(0, port_name.find_last_of('.')); + has_host = config.has(protocol + ".host"); + + std::string conf_name = protocol; + std::string prefix = protocol + "."; + std::unordered_set pset {conf_name}; + while (true) + { + if (config.has(prefix + "type")) + { + std::string type = config.getString(prefix + "type"); + if (type == "http") + { + is_http = true; + break; + } + } + + if (!config.has(prefix + "impl")) + break; + + conf_name = "protocols." + config.getString(prefix + "impl"); + prefix = conf_name + "."; + + if (!pset.insert(conf_name).second) + throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); + } + } + else + { + /// NOTE: better to compare using getPortName() over using + /// dynamic_cast<> since HTTPServer is also used for prometheus and + /// internal replication communications. + is_http = server->getPortName() == "http_port" || server->getPortName() == "https_port"; + } + + if (!has_host) + has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server->getListenHost()) != listen_hosts.end(); + bool has_port = !config.getString(port_name, "").empty(); + bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); + if (force_restart) + LOG_TRACE(log, " had been changed, will reload {}", server->getDescription()); + + if (!has_host || !has_port || config.getInt(server->getPortName()) != server->portNumber() || force_restart) + { + server->stop(); + LOG_INFO(log, "Stopped listening for {}", server->getDescription()); + } + } + } + + createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers= */ true); + createInterserverServers(config, interserver_listen_hosts, listen_try, server_pool, async_metrics, servers_to_start_before_tables, /* start_servers= */ true); + + std::erase_if(servers, std::bind_front(check_server, "")); + std::erase_if(servers_to_start_before_tables, std::bind_front(check_server, "")); +} + } diff --git a/programs/server/Server.h b/programs/server/Server.h index b4931ce53d1..3f03dd137ef 100644 --- a/programs/server/Server.h +++ b/programs/server/Server.h @@ -1,10 +1,15 @@ #pragma once #include + #include +#include +#include +#include +#include /** Server provides three interfaces: - * 1. HTTP, GRPC - simple interfaces for any applications. + * 1. HTTP - simple interface for any applications. * 2. TCP - interface for native clickhouse-client and for server to server internal communications. * More rich and efficient, but less compatible * - data is transferred by columns; @@ -13,21 +18,43 @@ * 3. Interserver HTTP - for replication. */ +namespace Poco +{ + namespace Net + { + class ServerSocket; + } +} + namespace DB { +class AsynchronousMetrics; +class ProtocolServerAdapter; class Server : public BaseDaemon, public IServer { public: using ServerApplication::run; - Poco::Util::LayeredConfiguration & config() const override { return BaseDaemon::config(); } + Poco::Util::LayeredConfiguration & config() const override + { + return BaseDaemon::config(); + } - Poco::Logger & logger() const override { return BaseDaemon::logger(); } + Poco::Logger & logger() const override + { + return BaseDaemon::logger(); + } - ContextMutablePtr context() const override { return global_context; } + ContextMutablePtr context() const override + { + return global_context; + } - bool isCancelled() const override { return BaseDaemon::isCancelled(); } + bool isCancelled() const override + { + return BaseDaemon::isCancelled(); + } void defineOptions(Poco::Util::OptionSet & _options) override; @@ -46,6 +73,64 @@ private: ContextMutablePtr global_context; /// Updated/recent config, to compare http_handlers ConfigurationPtr latest_config; + + HTTPContextPtr httpContext() const; + + Poco::Net::SocketAddress socketBindListen( + const Poco::Util::AbstractConfiguration & config, + Poco::Net::ServerSocket & socket, + const std::string & host, + UInt16 port, + [[maybe_unused]] bool secure = false) const; + + std::unique_ptr buildProtocolStackFromConfig( + const Poco::Util::AbstractConfiguration & config, + const std::string & protocol, + Poco::Net::HTTPServerParams::Ptr http_params, + AsynchronousMetrics & async_metrics, + bool & is_secure); + + using CreateServerFunc = std::function; + void createServer( + Poco::Util::AbstractConfiguration & config, + const std::string & listen_host, + const char * port_name, + bool listen_try, + bool start_server, + std::vector & servers, + CreateServerFunc && func) const; + + void createServers( + Poco::Util::AbstractConfiguration & config, + const Strings & listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); + + void createInterserverServers( + Poco::Util::AbstractConfiguration & config, + const Strings & interserver_listen_hosts, + bool listen_try, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + bool start_servers = false, + const ServerType & server_type = ServerType(ServerType::Type::QUERIES_ALL)); + + void updateServers( + Poco::Util::AbstractConfiguration & config, + Poco::ThreadPool & server_pool, + AsynchronousMetrics & async_metrics, + std::vector & servers, + std::vector & servers_to_start_before_tables); + + void stopServers( + std::vector & servers, + const ServerType & server_type + ) const; }; } diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 33042fbc7fc..f2e10a27b75 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -236,7 +236,6 @@ add_object_library(clickhouse_client Client) add_object_library(clickhouse_bridge BridgeHelper) add_object_library(clickhouse_server Server) add_object_library(clickhouse_server_http Server/HTTP) -add_object_library(clickhouse_server_manager Server/ServersManager) add_object_library(clickhouse_formats Formats) add_object_library(clickhouse_processors Processors) add_object_library(clickhouse_processors_executors Processors/Executors) diff --git a/src/Server/ServersManager/IServersManager.cpp b/src/Server/ServersManager/IServersManager.cpp deleted file mode 100644 index 8b1eee94303..00000000000 --- a/src/Server/ServersManager/IServersManager.cpp +++ /dev/null @@ -1,268 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int NETWORK_ERROR; -extern const int INVALID_CONFIG_PARAMETER; -} - -IServersManager::IServersManager(ContextMutablePtr global_context_, Poco::Logger * logger_) - : global_context(global_context_), logger(logger_) -{ -} - - -bool IServersManager::empty() const -{ - return servers.empty(); -} - -std::vector IServersManager::getMetrics() const -{ - std::vector metrics; - metrics.reserve(servers.size()); - for (const auto & server : servers) - metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); - return metrics; -} - -void IServersManager::startServers() -{ - for (auto & server : servers) - { - server.start(); - LOG_INFO(logger, "Listening for {}", server.getDescription()); - } -} - -void IServersManager::stopServers(const ServerType & server_type) -{ - /// Remove servers once all their connections are closed - auto check_server = [&](const char prefix[], auto & server) - { - if (!server.isStopping()) - return false; - size_t current_connections = server.currentConnections(); - LOG_DEBUG( - logger, - "Server {}{}: {} ({} connections)", - server.getDescription(), - prefix, - !current_connections ? "finished" : "waiting", - current_connections); - return !current_connections; - }; - - std::erase_if(servers, std::bind_front(check_server, " (from one of previous remove)")); - - for (auto & server : servers) - { - if (!server.isStopping() && server_type.shouldStop(server.getPortName())) - server.stop(); - } - - std::erase_if(servers, std::bind_front(check_server, "")); -} - -void IServersManager::updateServers( - const Poco::Util::AbstractConfiguration & config, - IServer & iserver, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - ConfigurationPtr latest_config) -{ - stopServersForUpdate(config, latest_config); - createServers(config, iserver, servers_lock, server_pool, async_metrics, true, ServerType(ServerType::Type::QUERIES_ALL)); -} - -Poco::Net::SocketAddress IServersManager::socketBindListen( - const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port) const -{ - auto address = makeSocketAddress(host, port, logger); - socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config.getBool("listen_reuse_port", false)); - /// If caller requests any available port from the OS, discover it after binding. - if (port == 0) - { - address = socket.address(); - LOG_DEBUG(logger, "Requested any available port (port == 0), actual port is {:d}", address.port()); - } - - socket.listen(/* backlog = */ config.getUInt("listen_backlog", 4096)); - return address; -} - -void IServersManager::createServer( - const Poco::Util::AbstractConfiguration & config, - const std::string & listen_host, - const char * port_name, - bool start_server, - CreateServerFunc && func) -{ - /// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file. - if (config.getString(port_name, "").empty()) - return; - - /// If we already have an active server for this listen_host/port_name, don't create it again - for (const auto & server : servers) - { - if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name) - return; - } - - auto port = config.getInt(port_name); - try - { - servers.push_back(func(port)); - if (start_server) - { - servers.back().start(); - LOG_INFO(logger, "Listening for {}", servers.back().getDescription()); - } - global_context->registerServerPort(port_name, port); - } - catch (const Poco::Exception &) - { - if (!getListenTry(config)) - { - throw Exception(ErrorCodes::NETWORK_ERROR, "Listen [{}]:{} failed: {}", listen_host, port, getCurrentExceptionMessage(false)); - } - LOG_WARNING( - logger, - "Listen [{}]:{} failed: {}. If it is an IPv6 or IPv4 address and your host has disabled IPv6 or IPv4, " - "then consider to " - "specify not disabled IPv4 or IPv6 address to listen in element of configuration " - "file. Example for disabled IPv6: 0.0.0.0 ." - " Example for disabled IPv4: ::", - listen_host, - port, - getCurrentExceptionMessage(false)); - } -} - -void IServersManager::stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config) -{ - /// Remove servers once all their connections are closed - auto check_server = [&](const char prefix[], auto & server) - { - if (!server.isStopping()) - return false; - size_t current_connections = server.currentConnections(); - LOG_DEBUG( - logger, - "Server {}{}: {} ({} connections)", - server.getDescription(), - prefix, - !current_connections ? "finished" : "waiting", - current_connections); - return !current_connections; - }; - - std::erase_if(servers, std::bind_front(check_server, " (from one of previous reload)")); - - const auto listen_hosts = getListenHosts(config); - const Poco::Util::AbstractConfiguration & previous_config = latest_config ? *latest_config : config; - - for (auto & server : servers) - { - if (server.isStopping()) - return; - std::string port_name = server.getPortName(); - bool has_host = false; - bool is_http = false; - if (port_name.starts_with("protocols.")) - { - std::string protocol = port_name.substr(0, port_name.find_last_of('.')); - has_host = config.has(protocol + ".host"); - - std::string conf_name = protocol; - std::string prefix = protocol + "."; - std::unordered_set pset{conf_name}; - while (true) - { - if (config.has(prefix + "type")) - { - std::string type = config.getString(prefix + "type"); - if (type == "http") - { - is_http = true; - break; - } - } - - if (!config.has(prefix + "impl")) - break; - - conf_name = "protocols." + config.getString(prefix + "impl"); - prefix = conf_name + "."; - - if (!pset.insert(conf_name).second) - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); - } - } - else - { - /// NOTE: better to compare using getPortName() over using - /// dynamic_cast<> since HTTPServer is also used for prometheus and - /// internal replication communications. - is_http = server.getPortName() == "http_port" || server.getPortName() == "https_port"; - } - - if (!has_host) - has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end(); - bool has_port = !config.getString(port_name, "").empty(); - bool force_restart = is_http && !isSameConfiguration(previous_config, config, "http_handlers"); - if (force_restart) - LOG_TRACE(logger, " had been changed, will reload {}", server.getDescription()); - - if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber() || force_restart) - { - server.stop(); - LOG_INFO(logger, "Stopped listening for {}", server.getDescription()); - } - } - - std::erase_if(servers, std::bind_front(check_server, "")); -} - -Strings IServersManager::getListenHosts(const Poco::Util::AbstractConfiguration & config) const -{ - auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host"); - if (listen_hosts.empty()) - { - listen_hosts.emplace_back("::1"); - listen_hosts.emplace_back("127.0.0.1"); - } - return listen_hosts; -} - -bool IServersManager::getListenTry(const Poco::Util::AbstractConfiguration & config) const -{ - bool listen_try = config.getBool("listen_try", false); - if (!listen_try) - { - Poco::Util::AbstractConfiguration::Keys protocols; - config.keys("protocols", protocols); - listen_try = DB::getMultipleValuesFromConfig(config, "", "listen_host").empty() - && std::none_of( - protocols.begin(), - protocols.end(), - [&](const auto & protocol) - { return config.has("protocols." + protocol + ".host") && config.has("protocols." + protocol + ".port"); }); - } - return listen_try; -} - -} diff --git a/src/Server/ServersManager/IServersManager.h b/src/Server/ServersManager/IServersManager.h deleted file mode 100644 index 7e1d9d50d82..00000000000 --- a/src/Server/ServersManager/IServersManager.h +++ /dev/null @@ -1,74 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -class IServersManager -{ -public: - IServersManager(ContextMutablePtr global_context_, Poco::Logger * logger_); - virtual ~IServersManager() = default; - - bool empty() const; - std::vector getMetrics() const; - - virtual void createServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) - = 0; - - void startServers(); - - void stopServers(const ServerType & server_type); - virtual size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) = 0; - - virtual void updateServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - ConfigurationPtr latest_config); - -protected: - ContextMutablePtr global_context; - Poco::Logger * logger; - - std::vector servers; - - Poco::Net::SocketAddress socketBindListen( - const Poco::Util::AbstractConfiguration & config, Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port) const; - - using CreateServerFunc = std::function; - void createServer( - const Poco::Util::AbstractConfiguration & config, - const std::string & listen_host, - const char * port_name, - bool start_server, - CreateServerFunc && func); - - void stopServersForUpdate(const Poco::Util::AbstractConfiguration & config, ConfigurationPtr latest_config); - - Strings getListenHosts(const Poco::Util::AbstractConfiguration & config) const; - bool getListenTry(const Poco::Util::AbstractConfiguration & config) const; -}; - -} diff --git a/src/Server/ServersManager/InterServersManager.cpp b/src/Server/ServersManager/InterServersManager.cpp deleted file mode 100644 index 4425d468248..00000000000 --- a/src/Server/ServersManager/InterServersManager.cpp +++ /dev/null @@ -1,327 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if USE_SSL -# include -#endif - -#if USE_NURAFT -# include -# include -#endif - -namespace ProfileEvents -{ -extern const Event InterfaceInterserverSendBytes; -extern const Event InterfaceInterserverReceiveBytes; -} - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int SUPPORT_IS_DISABLED; -} - -void InterServersManager::createServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) -{ - if (config.has("keeper_server.server_id")) - { -#if USE_NURAFT - //// If we don't have configured connection probably someone trying to use clickhouse-server instead - //// of clickhouse-keeper, so start synchronously. - bool can_initialize_keeper_async = false; - - if (zkutil::hasZooKeeperConfig(config)) /// We have configured connection to some zookeeper cluster - { - /// If we cannot connect to some other node from our cluster then we have to wait our Keeper start - /// synchronously. - can_initialize_keeper_async = global_context->tryCheckClientConnectionToMyKeeperCluster(); - } - /// Initialize keeper RAFT. - global_context->initializeKeeperDispatcher(can_initialize_keeper_async); - FourLetterCommandFactory::registerCommands(*global_context->getKeeperDispatcher()); - - auto config_getter = [this]() -> const Poco::Util::AbstractConfiguration & { return global_context->getConfigRef(); }; - - for (const auto & listen_host : getListenHosts(config)) - { - /// TCP Keeper - constexpr auto port_name = "keeper_server.tcp_port"; - createServer( - config, - listen_host, - port_name, - /* start_server = */ false, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout( - Poco::Timespan(config.getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); - socket.setSendTimeout( - Poco::Timespan(config.getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); - return ProtocolServerAdapter( - listen_host, - port_name, - "Keeper (tcp): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, - global_context->getKeeperDispatcher(), - global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), - false), - server_pool, - socket)); - }); - - constexpr auto secure_port_name = "keeper_server.tcp_port_secure"; - createServer( - config, - listen_host, - secure_port_name, - /* start_server = */ false, - [&](UInt16 port) -> ProtocolServerAdapter - { -# if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout( - Poco::Timespan(config.getUInt64("keeper_server.socket_receive_timeout_sec", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0)); - socket.setSendTimeout( - Poco::Timespan(config.getUInt64("keeper_server.socket_send_timeout_sec", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0)); - return ProtocolServerAdapter( - listen_host, - secure_port_name, - "Keeper with secure protocol (tcp_secure): " + address.toString(), - std::make_unique( - new KeeperTCPHandlerFactory( - config_getter, - global_context->getKeeperDispatcher(), - global_context->getSettingsRef().receive_timeout.totalSeconds(), - global_context->getSettingsRef().send_timeout.totalSeconds(), - true), - server_pool, - socket)); -# else - UNUSED(port); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -# endif - }); - - /// HTTP control endpoints - createServer( - config, - listen_host, - /* port_name = */ "keeper_server.http_control.port", - /* start_server = */ false, - [&](UInt16 port) -> ProtocolServerAdapter - { - auto http_context = std::make_shared(global_context); - Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(http_context->getReceiveTimeout()); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(http_context->getReceiveTimeout()); - socket.setSendTimeout(http_context->getSendTimeout()); - return ProtocolServerAdapter( - listen_host, - port_name, - "HTTP Control: http://" + address.toString(), - std::make_unique( - std::move(http_context), - createKeeperHTTPControlMainHandlerFactory( - config_getter(), global_context->getKeeperDispatcher(), "KeeperHTTPControlHandler-factory"), - server_pool, - socket, - http_params)); - }); - } -#else - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination."); -#endif - } - - { - std::lock_guard lock(servers_lock); - /// We should start interserver communications before (and more important shutdown after) tables. - /// Because server can wait for a long-running queries (for example in tcp_handler) after interserver handler was already shut down. - /// In this case we will have replicated tables which are unable to send any parts to other replicas, but still can - /// communicate with zookeeper, execute merges, etc. - createInterserverServers(config, server, server_pool, async_metrics, start_servers, server_type); - startServers(); - } -} - -size_t InterServersManager::stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) -{ - if (servers.empty()) - { - return 0; - } - - LOG_DEBUG(logger, "Waiting for current connections to servers for tables to finish."); - - size_t current_connections = 0; - { - std::lock_guard lock(servers_lock); - for (auto & server : servers) - { - server.stop(); - current_connections += server.currentConnections(); - } - } - - if (current_connections) - LOG_INFO(logger, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); - else - LOG_INFO(logger, "Closed all listening sockets."); - - if (current_connections > 0) - current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); - - if (current_connections) - LOG_INFO( - logger, - "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections " - "after context shutdown.", - current_connections); - else - LOG_INFO(logger, "Closed connections to servers for tables."); - return current_connections; -} - -void InterServersManager::updateServers( - const Poco::Util::AbstractConfiguration & config, - IServer & iserver, - std::mutex & /*servers_lock*/, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - ConfigurationPtr latest_config) -{ - stopServersForUpdate(config, latest_config); - createInterserverServers(config, iserver, server_pool, async_metrics, true, ServerType(ServerType::Type::QUERIES_ALL)); -} - -Strings InterServersManager::getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) const -{ - auto interserver_listen_hosts = DB::getMultipleValuesFromConfig(config, "", "interserver_listen_host"); - if (!interserver_listen_hosts.empty()) - return interserver_listen_hosts; - - /// Use more general restriction in case of emptiness - return getListenHosts(config); -} - -void InterServersManager::createInterserverServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) -{ - const Settings & settings = global_context->getSettingsRef(); - - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); - - /// Now iterate over interserver_listen_hosts - for (const auto & interserver_listen_host : getInterserverListenHosts(config)) - { - if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTP)) - { - /// Interserver IO HTTP - constexpr auto port_name = "interserver_http_port"; - createServer( - config, - interserver_listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "replica communication (interserver): http://" + address.toString(), - std::make_unique( - std::make_shared(global_context), - createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceInterserverReceiveBytes, - ProfileEvents::InterfaceInterserverSendBytes)); - }); - } - - if (server_type.shouldStart(ServerType::Type::INTERSERVER_HTTPS)) - { - constexpr auto port_name = "interserver_https_port"; - createServer( - config, - interserver_listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, interserver_listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - interserver_listen_host, - port_name, - "secure replica communication (interserver): https://" + address.toString(), - std::make_unique( - std::make_shared(global_context), - createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPSHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceInterserverReceiveBytes, - ProfileEvents::InterfaceInterserverSendBytes)); -#else - UNUSED(port); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - } - } -} - -} diff --git a/src/Server/ServersManager/InterServersManager.h b/src/Server/ServersManager/InterServersManager.h deleted file mode 100644 index 8780eae18e0..00000000000 --- a/src/Server/ServersManager/InterServersManager.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -class InterServersManager : public IServersManager -{ -public: - using IServersManager::IServersManager; - - void createServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) override; - - size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) override; - - void updateServers( - const Poco::Util::AbstractConfiguration & config, - IServer & iserver, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - ConfigurationPtr latest_config) override; - -private: - Strings getInterserverListenHosts(const Poco::Util::AbstractConfiguration & config) const; - - void createInterserverServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type); -}; - -} diff --git a/src/Server/ServersManager/ProtocolServersManager.cpp b/src/Server/ServersManager/ProtocolServersManager.cpp deleted file mode 100644 index af57de3ac3c..00000000000 --- a/src/Server/ServersManager/ProtocolServersManager.cpp +++ /dev/null @@ -1,523 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#if USE_SSL -# include -#endif - -#if USE_GRPC -# include -#endif - -namespace ProfileEvents -{ -extern const Event InterfaceNativeSendBytes; -extern const Event InterfaceNativeReceiveBytes; -extern const Event InterfaceHTTPSendBytes; -extern const Event InterfaceHTTPReceiveBytes; -extern const Event InterfacePrometheusSendBytes; -extern const Event InterfacePrometheusReceiveBytes; -extern const Event InterfaceMySQLSendBytes; -extern const Event InterfaceMySQLReceiveBytes; -extern const Event InterfacePostgreSQLSendBytes; -extern const Event InterfacePostgreSQLReceiveBytes; -extern const Event InterfaceInterserverSendBytes; -extern const Event InterfaceInterserverReceiveBytes; -} - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int SUPPORT_IS_DISABLED; -extern const int INVALID_CONFIG_PARAMETER; -} - -void ProtocolServersManager::createServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & /*servers_lock*/, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) -{ - auto listen_hosts = getListenHosts(config); - const Settings & settings = global_context->getSettingsRef(); - - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(settings.http_receive_timeout); - http_params->setKeepAliveTimeout(global_context->getServerSettings().keep_alive_timeout); - - Poco::Util::AbstractConfiguration::Keys protocols; - config.keys("protocols", protocols); - - for (const auto & protocol : protocols) - { - if (!server_type.shouldStart(ServerType::Type::CUSTOM, protocol)) - continue; - - std::string prefix = "protocols." + protocol + "."; - std::string port_name = prefix + "port"; - std::string description{" protocol"}; - if (config.has(prefix + "description")) - description = config.getString(prefix + "description"); - - if (!config.has(prefix + "port")) - continue; - - std::vector hosts; - if (config.has(prefix + "host")) - hosts.push_back(config.getString(prefix + "host")); - else - hosts = listen_hosts; - - for (const auto & host : hosts) - { - bool is_secure = false; - auto stack = buildProtocolStackFromConfig(config, server, protocol, http_params, async_metrics, is_secure); - - if (stack->empty()) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' stack empty", protocol); - - createServer( - config, - host, - port_name.c_str(), - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - host, - port_name.c_str(), - description + ": " + address.toString(), - std::make_unique(stack.release(), server_pool, socket, new Poco::Net::TCPServerParams)); - }); - } - } - - for (const auto & listen_host : listen_hosts) - { - if (server_type.shouldStart(ServerType::Type::HTTP)) - { - /// HTTP - constexpr auto port_name = "http_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "http://" + address.toString(), - std::make_unique( - std::make_shared(global_context), - createHandlerFactory(server, config, async_metrics, "HTTPHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceHTTPReceiveBytes, - ProfileEvents::InterfaceHTTPSendBytes)); - }); - } - - if (server_type.shouldStart(ServerType::Type::HTTPS)) - { - /// HTTPS - constexpr auto port_name = "https_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "https://" + address.toString(), - std::make_unique( - std::make_shared(global_context), - createHandlerFactory(server, config, async_metrics, "HTTPSHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfaceHTTPReceiveBytes, - ProfileEvents::InterfaceHTTPSendBytes)); -#else - UNUSED(port); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "HTTPS protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP)) - { - /// TCP - constexpr auto port_name = "tcp_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp): " + address.toString(), - std::make_unique( - new TCPHandlerFactory( - server, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP_WITH_PROXY)) - { - /// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt - constexpr auto port_name = "tcp_with_proxy_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "native protocol (tcp) with PROXY: " + address.toString(), - std::make_unique( - new TCPHandlerFactory( - server, false, true, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::TCP_SECURE)) - { - /// TCP with SSL - constexpr auto port_name = "tcp_port_secure"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { -#if USE_SSL - Poco::Net::SecureServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.receive_timeout); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "secure native protocol (tcp_secure): " + address.toString(), - std::make_unique( - new TCPHandlerFactory( - server, true, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); -#else - UNUSED(port); - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - }); - } - - if (server_type.shouldStart(ServerType::Type::MYSQL)) - { - constexpr auto port_name = "mysql_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "MySQL compatibility protocol: " + address.toString(), - std::make_unique( - new MySQLHandlerFactory( - server, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - - if (server_type.shouldStart(ServerType::Type::POSTGRESQL)) - { - constexpr auto port_name = "postgresql_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(Poco::Timespan()); - socket.setSendTimeout(settings.send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "PostgreSQL compatibility protocol: " + address.toString(), - std::make_unique( - new PostgreSQLHandlerFactory( - server, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes), - server_pool, - socket, - new Poco::Net::TCPServerParams)); - }); - } - -#if USE_GRPC - if (server_type.shouldStart(ServerType::Type::GRPC)) - { - constexpr auto port_name = "grpc_port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::SocketAddress server_address(listen_host, port); - return ProtocolServerAdapter( - listen_host, - port_name, - "gRPC protocol: " + server_address.toString(), - std::make_unique(server, makeSocketAddress(listen_host, port, logger))); - }); - } -#endif - if (server_type.shouldStart(ServerType::Type::PROMETHEUS)) - { - /// Prometheus (if defined and not setup yet with http_port) - constexpr auto port_name = "prometheus.port"; - createServer( - config, - listen_host, - port_name, - start_servers, - [&](UInt16 port) -> ProtocolServerAdapter - { - Poco::Net::ServerSocket socket; - auto address = socketBindListen(config, socket, listen_host, port); - socket.setReceiveTimeout(settings.http_receive_timeout); - socket.setSendTimeout(settings.http_send_timeout); - return ProtocolServerAdapter( - listen_host, - port_name, - "Prometheus: http://" + address.toString(), - std::make_unique( - std::make_shared(global_context), - createHandlerFactory(server, config, async_metrics, "PrometheusHandler-factory"), - server_pool, - socket, - http_params, - ProfileEvents::InterfacePrometheusReceiveBytes, - ProfileEvents::InterfacePrometheusSendBytes)); - }); - } - } -} - -size_t ProtocolServersManager::stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) -{ - if (servers.empty()) - { - return 0; - } - - LOG_DEBUG(logger, "Waiting for current connections to close."); - - size_t current_connections = 0; - { - std::lock_guard lock(servers_lock); - for (auto & server : servers) - { - server.stop(); - current_connections += server.currentConnections(); - } - } - - if (current_connections) - LOG_WARNING(logger, "Closed all listening sockets. Waiting for {} outstanding connections.", current_connections); - else - LOG_INFO(logger, "Closed all listening sockets."); - - /// Wait for unfinished backups and restores. - /// This must be done after closing listening sockets (no more backups/restores) but before ProcessList::killAllQueries - /// (because killAllQueries() will cancel all running backups/restores). - if (server_settings.shutdown_wait_backups_and_restores) - global_context->waitAllBackupsAndRestores(); - /// Killing remaining queries. - if (!server_settings.shutdown_wait_unfinished_queries) - global_context->getProcessList().killAllQueries(); - - if (current_connections) - current_connections = waitServersToFinish(servers, servers_lock, server_settings.shutdown_wait_unfinished); - - if (current_connections) - LOG_WARNING( - logger, - "Closed connections. But {} remain." - " Tip: To increase wait time add to config: 60", - current_connections); - else - LOG_INFO(logger, "Closed connections."); - return current_connections; -} - -std::unique_ptr ProtocolServersManager::buildProtocolStackFromConfig( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - const std::string & protocol, - Poco::Net::HTTPServerParams::Ptr http_params, - AsynchronousMetrics & async_metrics, - bool & is_secure) const -{ - auto create_factory = [&](const std::string & type, const std::string & conf_name) -> TCPServerConnectionFactory::Ptr - { - if (type == "tcp") - return TCPServerConnectionFactory::Ptr(new TCPHandlerFactory( - server, false, false, ProfileEvents::InterfaceNativeReceiveBytes, ProfileEvents::InterfaceNativeSendBytes)); - - if (type == "tls") -#if USE_SSL - return TCPServerConnectionFactory::Ptr(new TLSHandlerFactory(server, conf_name)); -#else - throw Exception( - ErrorCodes::SUPPORT_IS_DISABLED, - "SSL support for TCP protocol is disabled because Poco library was built without NetSSL support."); -#endif - - if (type == "proxy1") - return TCPServerConnectionFactory::Ptr(new ProxyV1HandlerFactory(server, conf_name)); - if (type == "mysql") - return TCPServerConnectionFactory::Ptr( - new MySQLHandlerFactory(server, ProfileEvents::InterfaceMySQLReceiveBytes, ProfileEvents::InterfaceMySQLSendBytes)); - if (type == "postgres") - return TCPServerConnectionFactory::Ptr(new PostgreSQLHandlerFactory( - server, ProfileEvents::InterfacePostgreSQLReceiveBytes, ProfileEvents::InterfacePostgreSQLSendBytes)); - if (type == "http") - return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( - std::make_shared(global_context), - http_params, - createHandlerFactory(server, config, async_metrics, "HTTPHandler-factory"), - ProfileEvents::InterfaceHTTPReceiveBytes, - ProfileEvents::InterfaceHTTPSendBytes)); - if (type == "prometheus") - return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( - std::make_shared(global_context), - http_params, - createHandlerFactory(server, config, async_metrics, "PrometheusHandler-factory"), - ProfileEvents::InterfacePrometheusReceiveBytes, - ProfileEvents::InterfacePrometheusSendBytes)); - if (type == "interserver") - return TCPServerConnectionFactory::Ptr(new HTTPServerConnectionFactory( - std::make_shared(global_context), - http_params, - createHandlerFactory(server, config, async_metrics, "InterserverIOHTTPHandler-factory"), - ProfileEvents::InterfaceInterserverReceiveBytes, - ProfileEvents::InterfaceInterserverSendBytes)); - - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol configuration error, unknown protocol name '{}'", type); - }; - - std::string conf_name = "protocols." + protocol; - std::string prefix = conf_name + "."; - std::unordered_set pset{conf_name}; - - auto stack = std::make_unique(server, conf_name); - - while (true) - { - // if there is no "type" - it's a reference to another protocol and this is just an endpoint - if (config.has(prefix + "type")) - { - std::string type = config.getString(prefix + "type"); - if (type == "tls") - { - if (is_secure) - throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' contains more than one TLS layer", protocol); - is_secure = true; - } - - stack->append(create_factory(type, conf_name)); - } - - if (!config.has(prefix + "impl")) - break; - - conf_name = "protocols." + config.getString(prefix + "impl"); - prefix = conf_name + "."; - - if (!pset.insert(conf_name).second) - throw Exception( - ErrorCodes::INVALID_CONFIG_PARAMETER, "Protocol '{}' configuration contains a loop on '{}'", protocol, conf_name); - } - - return stack; -} - -} diff --git a/src/Server/ServersManager/ProtocolServersManager.h b/src/Server/ServersManager/ProtocolServersManager.h deleted file mode 100644 index e9eaaeb2184..00000000000 --- a/src/Server/ServersManager/ProtocolServersManager.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -class ProtocolServersManager : public IServersManager -{ -public: - using IServersManager::IServersManager; - - void createServers( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - std::mutex & servers_lock, - Poco::ThreadPool & server_pool, - AsynchronousMetrics & async_metrics, - bool start_servers, - const ServerType & server_type) override; - - using IServersManager::stopServers; - size_t stopServers(const ServerSettings & server_settings, std::mutex & servers_lock) override; - -private: - std::unique_ptr buildProtocolStackFromConfig( - const Poco::Util::AbstractConfiguration & config, - IServer & server, - const std::string & protocol, - Poco::Net::HTTPServerParams::Ptr http_params, - AsynchronousMetrics & async_metrics, - bool & is_secure) const; -}; - -}