diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index bca9bd9d280..7124961821e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,7 +107,7 @@ if (USE_AWS_S3) endif() if (USE_AZURE_BLOB_STORAGE) - add_headers_and_sources(dbms Disks/BlobStorage) + add_headers_and_sources(dbms Disks/AzureBlobStorage) endif() if (USE_HDFS) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9b0e2cdc494..70d85433513 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -470,7 +470,7 @@ M(497, ACCESS_DENIED) \ M(498, LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED) \ M(499, S3_ERROR) \ - M(500, BLOB_STORAGE_ERROR) \ + M(500, AZURE_BLOB_STORAGE_ERROR) \ M(501, CANNOT_CREATE_DATABASE) \ M(502, CANNOT_SIGQUEUE) \ M(503, AGGREGATE_FUNCTION_THROW) \ diff --git a/src/Disks/BlobStorage/BlobStorageAuth.cpp b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp similarity index 80% rename from src/Disks/BlobStorage/BlobStorageAuth.cpp rename to src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp index 2378ffe12a7..94553ba04e9 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.cpp +++ b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE @@ -17,7 +17,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct BlobStorageEndpoint +struct AzureBlobStorageEndpoint { const String storage_account_url; const String container_name; @@ -41,18 +41,18 @@ void validateContainerName(const String & container_name) auto len = container_name.length(); if (len < 3 || len > 64) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Blob Storage container name is not valid, should have length between 3 and 64, but has length: {}", len); + "AzureBlob Storage container name is not valid, should have length between 3 and 64, but has length: {}", len); const auto * container_name_pattern_str = R"([a-z][a-z0-9-]+)"; static const RE2 container_name_pattern(container_name_pattern_str); if (!re2::RE2::FullMatch(container_name, container_name_pattern)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Blob Storage container name is not valid, should follow the format: {}, got: {}", container_name_pattern_str, container_name); + "AzureBlob Storage container name is not valid, should follow the format: {}, got: {}", container_name_pattern_str, container_name); } -BlobStorageEndpoint processBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { String storage_account_url = config.getString(config_prefix + ".storage_account_url"); validateStorageAccountUrl(storage_account_url); @@ -86,7 +86,7 @@ std::shared_ptr getClientWithConnectionString( template -std::shared_ptr getBlobStorageClientWithAuth( +std::shared_ptr getAzureBlobStorageClientWithAuth( const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { if (config.has(config_prefix + ".connection_string")) @@ -109,19 +109,19 @@ std::shared_ptr getBlobStorageClientWithAuth( } -std::shared_ptr getBlobContainerClient( +std::shared_ptr getAzureBlobContainerClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - auto endpoint = processBlobStorageEndpoint(config, config_prefix); + auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix); auto container_name = endpoint.container_name; auto final_url = endpoint.storage_account_url + (endpoint.storage_account_url.back() == '/' ? "" : "/") + container_name; if (endpoint.container_already_exists.value_or(false)) - return getBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); + return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); - auto blob_service_client = getBlobStorageClientWithAuth(endpoint.storage_account_url, container_name, config, config_prefix); + auto blob_service_client = getAzureBlobStorageClientWithAuth(endpoint.storage_account_url, container_name, config, config_prefix); if (!endpoint.container_already_exists.has_value()) { @@ -132,7 +132,7 @@ std::shared_ptr getBlobContainerClient( for (const auto & blob_container : blob_containers) { if (blob_container.Name == endpoint.container_name) - return getBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); + return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); } } diff --git a/src/Disks/BlobStorage/BlobStorageAuth.h b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h similarity index 76% rename from src/Disks/BlobStorage/BlobStorageAuth.h rename to src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h index 16a088fc960..1cef6105d41 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.h +++ b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h @@ -12,7 +12,7 @@ namespace DB { -std::shared_ptr getBlobContainerClient( +std::shared_ptr getAzureBlobContainerClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix); } diff --git a/src/Disks/BlobStorage/DiskBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp similarity index 71% rename from src/Disks/BlobStorage/DiskBlobStorage.cpp rename to src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index f33e698a50d..7883bc0b537 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE @@ -15,11 +15,11 @@ namespace DB namespace ErrorCodes { - extern const int BLOB_STORAGE_ERROR; + extern const int AZURE_BLOB_STORAGE_ERROR; } -DiskBlobStorageSettings::DiskBlobStorageSettings( +DiskAzureBlobStorageSettings::DiskAzureBlobStorageSettings( UInt64 max_single_part_upload_size_, UInt64 min_bytes_for_seek_, int max_single_read_retries_, @@ -32,11 +32,11 @@ DiskBlobStorageSettings::DiskBlobStorageSettings( thread_pool_size(thread_pool_size_) {} -class BlobStoragePathKeeper : public RemoteFSPathKeeper +class AzureBlobStoragePathKeeper : public RemoteFSPathKeeper { public: /// RemoteFSPathKeeper constructed with a placeholder argument for chunk_limit, it is unused in this class - BlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {} + AzureBlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {} void addPath(const String & path) override { @@ -47,19 +47,19 @@ public: }; -DiskBlobStorage::DiskBlobStorage( +DiskAzureBlobStorage::DiskAzureBlobStorage( const String & name_, DiskPtr metadata_disk_, std::shared_ptr blob_container_client_, SettingsPtr settings_, GetDiskSettings settings_getter_) : - IDiskRemote(name_, "", metadata_disk_, "DiskBlobStorage", settings_->thread_pool_size), + IDiskRemote(name_, "", metadata_disk_, "DiskAzureBlobStorage", settings_->thread_pool_size), blob_container_client(blob_container_client_), current_settings(std::move(settings_)), settings_getter(settings_getter_) {} -std::unique_ptr DiskBlobStorage::readFile( +std::unique_ptr DiskAzureBlobStorage::readFile( const String & path, const ReadSettings & read_settings, std::optional /*estimated_size*/) const @@ -71,7 +71,7 @@ std::unique_ptr DiskBlobStorage::readFile( bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; - auto reader_impl = std::make_unique( + auto reader_impl = std::make_unique( path, blob_container_client, metadata, settings->max_single_read_retries, settings->max_single_download_retries, read_settings, threadpool_read); @@ -88,7 +88,7 @@ std::unique_ptr DiskBlobStorage::readFile( } -std::unique_ptr DiskBlobStorage::writeFile( +std::unique_ptr DiskAzureBlobStorage::writeFile( const String & path, size_t buf_size, WriteMode mode) @@ -96,38 +96,38 @@ std::unique_ptr DiskBlobStorage::writeFile( auto metadata = readOrCreateMetaForWriting(path, mode); auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name - LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}", + LOG_TRACE(log, "{} to file by path: {}. AzureBlob Storage path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path); - auto buffer = std::make_unique( + auto buffer = std::make_unique( blob_container_client, blob_path, current_settings.get()->max_single_part_upload_size, buf_size); - return std::make_unique>(std::move(buffer), std::move(metadata), blob_path); + return std::make_unique>(std::move(buffer), std::move(metadata), blob_path); } -DiskType DiskBlobStorage::getType() const +DiskType DiskAzureBlobStorage::getType() const { - return DiskType::BlobStorage; + return DiskType::AzureBlobStorage; } -bool DiskBlobStorage::isRemote() const +bool DiskAzureBlobStorage::isRemote() const { return true; } -bool DiskBlobStorage::supportZeroCopyReplication() const +bool DiskAzureBlobStorage::supportZeroCopyReplication() const { return true; } -bool DiskBlobStorage::checkUniqueId(const String & id) const +bool DiskAzureBlobStorage::checkUniqueId(const String & id) const { Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; blobs_list_options.Prefix = id; @@ -146,9 +146,9 @@ bool DiskBlobStorage::checkUniqueId(const String & id) const } -void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) +void DiskAzureBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) { - auto * paths_keeper = dynamic_cast(fs_paths_keeper.get()); + auto * paths_keeper = dynamic_cast(fs_paths_keeper.get()); if (paths_keeper) { @@ -158,7 +158,7 @@ void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) { auto delete_info = blob_container_client->DeleteBlob(path); if (!delete_info.Value.Deleted) - throw Exception(ErrorCodes::BLOB_STORAGE_ERROR, "Failed to delete file in Blob Storage: {}", path); + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path); } catch (const Azure::Storage::StorageException& e) { @@ -170,13 +170,13 @@ void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) } -RemoteFSPathKeeperPtr DiskBlobStorage::createFSPathKeeper() const +RemoteFSPathKeeperPtr DiskAzureBlobStorage::createFSPathKeeper() const { - return std::make_shared(); + return std::make_shared(); } -void DiskBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) +void DiskAzureBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) { auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context); diff --git a/src/Disks/BlobStorage/DiskBlobStorage.h b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h similarity index 84% rename from src/Disks/BlobStorage/DiskBlobStorage.h rename to src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h index 16ba216df96..f90ede1add9 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.h +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h @@ -1,14 +1,12 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE #include -#include -#include +#include +#include #include #include @@ -18,9 +16,9 @@ namespace DB { -struct DiskBlobStorageSettings final +struct DiskAzureBlobStorageSettings final { - DiskBlobStorageSettings( + DiskAzureBlobStorageSettings( UInt64 max_single_part_upload_size_, UInt64 min_bytes_for_seek_, int max_single_read_retries, @@ -35,14 +33,14 @@ struct DiskBlobStorageSettings final }; -class DiskBlobStorage final : public IDiskRemote +class DiskAzureBlobStorage final : public IDiskRemote { public: - using SettingsPtr = std::unique_ptr; + using SettingsPtr = std::unique_ptr; using GetDiskSettings = std::function; - DiskBlobStorage( + DiskAzureBlobStorage( const String & name_, DiskPtr metadata_disk_, std::shared_ptr blob_container_client_, @@ -78,7 +76,7 @@ private: /// client used to access the files in the Blob Storage cloud std::shared_ptr blob_container_client; - MultiVersion current_settings; + MultiVersion current_settings; /// Gets disk settings from context. GetDiskSettings settings_getter; }; diff --git a/src/Disks/BlobStorage/registerDiskBlobStorage.cpp b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp similarity index 69% rename from src/Disks/BlobStorage/registerDiskBlobStorage.cpp rename to src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 58b84a3f972..243452353d3 100644 --- a/src/Disks/BlobStorage/registerDiskBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include namespace DB @@ -62,9 +62,9 @@ void checkRemoveAccess(IDisk & disk) } -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*/) { - return std::make_unique( + return std::make_unique( config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".max_single_read_retries", 3), @@ -74,7 +74,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractC } -void registerDiskBlobStorage(DiskFactory & factory) +void registerDiskAzureBlobStorage(DiskFactory & factory) { auto creator = []( const String & name, @@ -85,33 +85,33 @@ void registerDiskBlobStorage(DiskFactory & factory) { auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); - std::shared_ptr blob_storage_disk = std::make_shared( + std::shared_ptr azure_blob_storage_disk = std::make_shared( name, metadata_disk, - getBlobContainerClient(config, config_prefix), + getAzureBlobContainerClient(config, config_prefix), getSettings(config, config_prefix, context), getSettings ); if (!config.getBool(config_prefix + ".skip_access_check", false)) { - checkWriteAccess(*blob_storage_disk); - checkReadAccess(*blob_storage_disk); - checkReadWithOffset(*blob_storage_disk); - checkRemoveAccess(*blob_storage_disk); + checkWriteAccess(*azure_blob_storage_disk); + checkReadAccess(*azure_blob_storage_disk); + checkReadWithOffset(*azure_blob_storage_disk); + checkRemoveAccess(*azure_blob_storage_disk); } - blob_storage_disk->startup(); + azure_blob_storage_disk->startup(); if (config.getBool(config_prefix + ".cache_enabled", true)) { String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); - blob_storage_disk = wrapWithCache(blob_storage_disk, "blob-storage-cache", cache_path, metadata_path); + azure_blob_storage_disk = wrapWithCache(azure_blob_storage_disk, "azure-blob-storage-cache", cache_path, metadata_path); } - return std::make_shared(blob_storage_disk); + return std::make_shared(azure_blob_storage_disk); }; - factory.registerDiskType("blob_storage", creator); + factory.registerDiskType("azure_blob_storage", creator); } } @@ -121,7 +121,7 @@ void registerDiskBlobStorage(DiskFactory & factory) namespace DB { -void registerDiskBlobStorage(DiskFactory &) {} +void registerDiskAzureBlobStorage(DiskFactory &) {} } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index a1dd276c51f..435f427b05a 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -10,10 +10,10 @@ enum class DiskType Local, RAM, S3, - BlobStorage, HDFS, Encrypted, WebServer, + AzureBlobStorage, }; inline String toString(DiskType disk_type) @@ -26,14 +26,14 @@ inline String toString(DiskType disk_type) return "memory"; case DiskType::S3: return "s3"; - case DiskType::BlobStorage: - return "blob_storage"; case DiskType::HDFS: return "hdfs"; case DiskType::Encrypted: return "encrypted"; case DiskType::WebServer: return "web"; + case DiskType::AzureBlobStorage: + return "azure_blob_storage"; } __builtin_unreachable(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 534258eaca6..be63223b313 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -9,7 +9,7 @@ #endif #if USE_AZURE_BLOB_STORAGE -#include +#include #endif #if USE_HDFS @@ -35,9 +35,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S #if USE_AZURE_BLOB_STORAGE -SeekableReadBufferPtr ReadBufferFromBlobStorageGather::createImplementationBuffer(const String & path, size_t read_until_position_) const +SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementationBuffer(const String & path, size_t read_until_position_) const { - return std::make_unique(blob_container_client, path, max_single_read_retries, + return std::make_unique(blob_container_client, path, max_single_read_retries, max_single_download_retries, settings.remote_fs_buffer_size, threadpool_read, read_until_position_); } #endif diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 045ab43850d..9dd5d79d883 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -102,11 +102,11 @@ private: #if USE_AZURE_BLOB_STORAGE -/// Reads data from Blob Storage using paths stored in metadata. -class ReadBufferFromBlobStorageGather final : public ReadBufferFromRemoteFSGather +/// Reads data from AzureBlob Storage using paths stored in metadata. +class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFSGather { public: - ReadBufferFromBlobStorageGather( + ReadBufferFromAzureBlobStorageGather( const String & path_, std::shared_ptr blob_container_client_, IDiskRemote::Metadata metadata_, diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp index 6e10aefc7a2..87453440693 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp @@ -1,7 +1,7 @@ #include "WriteIndirectBufferFromRemoteFS.h" #include -#include +#include #include #include @@ -60,7 +60,7 @@ class WriteIndirectBufferFromRemoteFS; #if USE_AZURE_BLOB_STORAGE template -class WriteIndirectBufferFromRemoteFS; +class WriteIndirectBufferFromRemoteFS; #endif #if USE_HDFS diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 44522f56130..88c3fdde1e0 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -15,7 +15,7 @@ void registerDiskS3(DiskFactory & factory); #endif #if USE_AZURE_BLOB_STORAGE -void registerDiskBlobStorage(DiskFactory & factory); +void registerDiskAzureBlobStorage(DiskFactory & factory); #endif #if USE_SSL @@ -41,7 +41,7 @@ void registerDisks() #endif #if USE_AZURE_BLOB_STORAGE - registerDiskBlobStorage(factory); + registerDiskAzureBlobStorage(factory); #endif #if USE_SSL diff --git a/src/IO/ReadBufferFromBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp similarity index 93% rename from src/IO/ReadBufferFromBlobStorage.cpp rename to src/IO/ReadBufferFromAzureBlobStorage.cpp index ada462f0b87..d4762ecc044 100644 --- a/src/IO/ReadBufferFromBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -4,7 +4,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include #include @@ -22,7 +22,7 @@ namespace ErrorCodes } -ReadBufferFromBlobStorage::ReadBufferFromBlobStorage( +ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & path_, size_t max_single_read_retries_, @@ -48,7 +48,7 @@ ReadBufferFromBlobStorage::ReadBufferFromBlobStorage( } -bool ReadBufferFromBlobStorage::nextImpl() +bool ReadBufferFromAzureBlobStorage::nextImpl() { if (read_until_position) { @@ -102,7 +102,7 @@ bool ReadBufferFromBlobStorage::nextImpl() } -off_t ReadBufferFromBlobStorage::seek(off_t offset_, int whence) +off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) { if (initialized) throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); @@ -119,13 +119,13 @@ off_t ReadBufferFromBlobStorage::seek(off_t offset_, int whence) } -off_t ReadBufferFromBlobStorage::getPosition() +off_t ReadBufferFromAzureBlobStorage::getPosition() { return offset - available(); } -void ReadBufferFromBlobStorage::initialize() +void ReadBufferFromAzureBlobStorage::initialize() { if (initialized) return; diff --git a/src/IO/ReadBufferFromBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h similarity index 87% rename from src/IO/ReadBufferFromBlobStorage.h rename to src/IO/ReadBufferFromAzureBlobStorage.h index cd66e897e25..53749ad3199 100644 --- a/src/IO/ReadBufferFromBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -14,11 +14,11 @@ namespace DB { -class ReadBufferFromBlobStorage : public SeekableReadBuffer +class ReadBufferFromAzureBlobStorage : public SeekableReadBuffer { public: - explicit ReadBufferFromBlobStorage( + explicit ReadBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & path_, size_t max_single_read_retries_, @@ -55,7 +55,7 @@ private: char * data_ptr; size_t data_capacity; - Poco::Logger * log = &Poco::Logger::get("ReadBufferFromBlobStorage"); + Poco::Logger * log = &Poco::Logger::get("ReadBufferFromAzureBlobStorage"); }; } diff --git a/src/IO/WriteBufferFromBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp similarity index 83% rename from src/IO/WriteBufferFromBlobStorage.cpp rename to src/IO/WriteBufferFromAzureBlobStorage.cpp index 2e63c8c4565..5fa312d973a 100644 --- a/src/IO/WriteBufferFromBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -4,7 +4,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include @@ -12,7 +12,7 @@ namespace DB { -WriteBufferFromBlobStorage::WriteBufferFromBlobStorage( +WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, @@ -23,13 +23,13 @@ WriteBufferFromBlobStorage::WriteBufferFromBlobStorage( blob_path(blob_path_) {} -WriteBufferFromBlobStorage::~WriteBufferFromBlobStorage() +WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { finalize(); } -void WriteBufferFromBlobStorage::nextImpl() +void WriteBufferFromAzureBlobStorage::nextImpl() { if (!offset()) return; @@ -54,7 +54,7 @@ void WriteBufferFromBlobStorage::nextImpl() } -void WriteBufferFromBlobStorage::finalizeImpl() +void WriteBufferFromAzureBlobStorage::finalizeImpl() { next(); diff --git a/src/IO/WriteBufferFromBlobStorage.h b/src/IO/WriteBufferFromAzureBlobStorage.h similarity index 81% rename from src/IO/WriteBufferFromBlobStorage.h rename to src/IO/WriteBufferFromAzureBlobStorage.h index 5f8eaba3c0c..78e0c490b39 100644 --- a/src/IO/WriteBufferFromBlobStorage.h +++ b/src/IO/WriteBufferFromAzureBlobStorage.h @@ -17,17 +17,17 @@ namespace DB { -class WriteBufferFromBlobStorage : public BufferWithOwnMemory +class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory { public: - explicit WriteBufferFromBlobStorage( + explicit WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, size_t buf_size_); - ~WriteBufferFromBlobStorage() override; + ~WriteBufferFromAzureBlobStorage() override; void nextImpl() override; diff --git a/tests/integration/test_blob_storage_zero_copy_replication/__init__.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/__init__.py similarity index 100% rename from tests/integration/test_blob_storage_zero_copy_replication/__init__.py rename to tests/integration/test_azure_blob_storage_zero_copy_replication/__init__.py diff --git a/tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml similarity index 96% rename from tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml rename to tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml index 73eae98a80d..4235083f5ca 100644 --- a/tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml @@ -2,7 +2,7 @@ - blob_storage + azure_blob_storage http://azurite1:10000/devstoreaccount1 cont false diff --git a/tests/integration/test_blob_storage_zero_copy_replication/test.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py similarity index 100% rename from tests/integration/test_blob_storage_zero_copy_replication/test.py rename to tests/integration/test_azure_blob_storage_zero_copy_replication/test.py diff --git a/tests/integration/test_merge_tree_blob_storage/__init__.py b/tests/integration/test_merge_tree_azure_blob_storage/__init__.py similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/__init__.py rename to tests/integration/test_merge_tree_azure_blob_storage/__init__.py diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.d/bg_processing_pool_conf.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/bg_processing_pool_conf.xml similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/configs/config.d/bg_processing_pool_conf.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/bg_processing_pool_conf.xml diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml similarity index 96% rename from tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml index c4f381db941..09fa0d6c767 100644 --- a/tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml @@ -2,7 +2,7 @@ - blob_storage + azure_blob_storage http://azurite1:10000/devstoreaccount1 cont false diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/configs/config.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml diff --git a/tests/integration/test_merge_tree_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py similarity index 98% rename from tests/integration/test_merge_tree_blob_storage/test.py rename to tests/integration/test_merge_tree_azure_blob_storage/test.py index 2a38ac128ac..a1f1eeb928d 100644 --- a/tests/integration/test_merge_tree_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -12,7 +12,7 @@ CONFIG_PATH = os.path.join(SCRIPT_DIR, './{}/node/configs/config.d/storage_conf. NODE_NAME = "node" TABLE_NAME = "blob_storage_table" -BLOB_STORAGE_DISK = "blob_storage_disk" +AZURE_BLOB_STORAGE_DISK = "blob_storage_disk" LOCAL_DISK = "hdd" CONTAINER_NAME = "cont" @@ -196,7 +196,7 @@ def test_move_partition_to_another_disk(cluster): node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{LOCAL_DISK}'") assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" - node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{BLOB_STORAGE_DISK}'") + node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{AZURE_BLOB_STORAGE_DISK}'") assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" @@ -327,7 +327,7 @@ def test_restart_during_load(cluster): def restart_disk(): for iii in range(0, 2): logging.info(f"Restarting disk, attempt {iii}") - node.query(f"SYSTEM RESTART DISK {BLOB_STORAGE_DISK}") + node.query(f"SYSTEM RESTART DISK {AZURE_BLOB_STORAGE_DISK}") logging.info(f"Disk restarted, attempt {iii}") time.sleep(0.5)