mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #37139 from ClickHouse/i_object_storage
Separate object storage operations from disks
This commit is contained in:
commit
6d6779f17a
@ -87,6 +87,7 @@ add_headers_and_sources(clickhouse_common_io IO/S3)
|
||||
list (REMOVE_ITEM clickhouse_common_io_sources Common/malloc.cpp Common/new_delete.cpp)
|
||||
|
||||
add_headers_and_sources(dbms Disks/IO)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages)
|
||||
if (TARGET ch_contrib::sqlite)
|
||||
add_headers_and_sources(dbms Databases/SQLite)
|
||||
endif()
|
||||
@ -113,16 +114,16 @@ endif()
|
||||
|
||||
if (TARGET ch_contrib::aws_s3)
|
||||
add_headers_and_sources(dbms Common/S3)
|
||||
add_headers_and_sources(dbms Disks/S3)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/S3)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::azure_sdk)
|
||||
add_headers_and_sources(dbms Disks/AzureBlobStorage)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/AzureBlobStorage)
|
||||
endif()
|
||||
|
||||
if (TARGET ch_contrib::hdfs)
|
||||
add_headers_and_sources(dbms Storages/HDFS)
|
||||
add_headers_and_sources(dbms Disks/HDFS)
|
||||
add_headers_and_sources(dbms Disks/ObjectStorages/HDFS)
|
||||
endif()
|
||||
|
||||
add_headers_and_sources(dbms Storages/Cache)
|
||||
|
@ -628,6 +628,7 @@
|
||||
M(657, UNSUPPORTED_MEILISEARCH_TYPE) \
|
||||
M(658, MEILISEARCH_MISSING_SOME_COLUMNS) \
|
||||
M(659, UNKNOWN_STATUS_OF_TRANSACTION) \
|
||||
M(660, HDFS_ERROR) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -86,6 +86,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \
|
||||
M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \
|
||||
M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \
|
||||
M(Bool, enable_s3_requests_logging, false, "Enable very explicit logging of S3 requests. Makes sense for debug only.", 0) \
|
||||
M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \
|
||||
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) \
|
||||
|
@ -1,18 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <azure/storage/blobs.hpp>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,168 +0,0 @@
|
||||
#include <Disks/AzureBlobStorage/DiskAzureBlobStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AZURE_BLOB_STORAGE_ERROR;
|
||||
}
|
||||
|
||||
|
||||
DiskAzureBlobStorageSettings::DiskAzureBlobStorageSettings(
|
||||
UInt64 max_single_part_upload_size_,
|
||||
UInt64 min_bytes_for_seek_,
|
||||
int max_single_read_retries_,
|
||||
int max_single_download_retries_,
|
||||
int thread_pool_size_) :
|
||||
max_single_part_upload_size(max_single_part_upload_size_),
|
||||
min_bytes_for_seek(min_bytes_for_seek_),
|
||||
max_single_read_retries(max_single_read_retries_),
|
||||
max_single_download_retries(max_single_download_retries_),
|
||||
thread_pool_size(thread_pool_size_) {}
|
||||
|
||||
|
||||
DiskAzureBlobStorage::DiskAzureBlobStorage(
|
||||
const String & name_,
|
||||
DiskPtr metadata_disk_,
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_) :
|
||||
IDiskRemote(name_, "", metadata_disk_, nullptr, "DiskAzureBlobStorage", settings_->thread_pool_size),
|
||||
blob_container_client(blob_container_client_),
|
||||
current_settings(std::move(settings_)),
|
||||
settings_getter(settings_getter_) {}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskAzureBlobStorage::readFile(
|
||||
const String & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings = current_settings.get();
|
||||
auto metadata = readMetadata(path);
|
||||
|
||||
LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path));
|
||||
|
||||
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
|
||||
blob_container_client, metadata.remote_fs_root_path, metadata.remote_fs_objects,
|
||||
settings->max_single_read_retries, settings->max_single_download_retries, read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(reader_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(reader_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), current_settings.get()->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskAzureBlobStorage::writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings &)
|
||||
{
|
||||
auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name
|
||||
|
||||
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<WriteBufferFromAzureBlobStorage>(
|
||||
blob_container_client,
|
||||
blob_path,
|
||||
current_settings.get()->max_single_part_upload_size,
|
||||
buf_size);
|
||||
|
||||
auto create_metadata_callback = [this, path, mode, blob_path] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false, [blob_path, count] (Metadata & metadata) { metadata.addObject(blob_path, count); return true; });
|
||||
};
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(create_metadata_callback), blob_path);
|
||||
}
|
||||
|
||||
|
||||
DiskType DiskAzureBlobStorage::getType() const
|
||||
{
|
||||
return DiskType::AzureBlobStorage;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::isRemote() const
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::supportZeroCopyReplication() const
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool DiskAzureBlobStorage::checkUniqueId(const String & id) const
|
||||
{
|
||||
Azure::Storage::Blobs::ListBlobsOptions blobs_list_options;
|
||||
blobs_list_options.Prefix = id;
|
||||
blobs_list_options.PageSizeHint = 1;
|
||||
|
||||
auto blobs_list_response = blob_container_client->ListBlobs(blobs_list_options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
{
|
||||
if (id == blob.Name)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
void DiskAzureBlobStorage::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
for (const auto & path : paths)
|
||||
{
|
||||
try
|
||||
{
|
||||
auto delete_info = blob_container_client->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
catch (const Azure::Storage::StorageException & e)
|
||||
{
|
||||
LOG_INFO(log, "Caught an error while deleting file {} : {}", path, e.Message);
|
||||
throw;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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);
|
||||
|
||||
current_settings.set(std::move(new_settings));
|
||||
|
||||
if (AsyncExecutor * exec = dynamic_cast<AsyncExecutor*>(&getExecutor()))
|
||||
exec->setMaxThreads(current_settings.get()->thread_pool_size);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,86 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/ReadBufferFromAzureBlobStorage.h>
|
||||
#include <IO/WriteBufferFromAzureBlobStorage.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
|
||||
#include <azure/identity/managed_identity_credential.hpp>
|
||||
#include <azure/storage/blobs.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskAzureBlobStorageSettings final
|
||||
{
|
||||
DiskAzureBlobStorageSettings(
|
||||
UInt64 max_single_part_upload_size_,
|
||||
UInt64 min_bytes_for_seek_,
|
||||
int max_single_read_retries,
|
||||
int max_single_download_retries,
|
||||
int thread_pool_size_);
|
||||
|
||||
size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset
|
||||
UInt64 min_bytes_for_seek;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
size_t thread_pool_size;
|
||||
};
|
||||
|
||||
|
||||
class DiskAzureBlobStorage final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<DiskAzureBlobStorageSettings>;
|
||||
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextPtr)>;
|
||||
|
||||
DiskAzureBlobStorage(
|
||||
const String & name_,
|
||||
DiskPtr metadata_disk_,
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_);
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
DiskType getType() const override;
|
||||
|
||||
bool isRemote() const override;
|
||||
|
||||
bool supportZeroCopyReplication() const override;
|
||||
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths) override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override;
|
||||
|
||||
private:
|
||||
|
||||
/// client used to access the files in the Blob Storage cloud
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
|
||||
MultiVersion<DiskAzureBlobStorageSettings> current_settings;
|
||||
/// Gets disk settings from context.
|
||||
GetDiskSettings settings_getter;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -211,9 +211,9 @@ void DiskDecorator::shutdown()
|
||||
delegate->shutdown();
|
||||
}
|
||||
|
||||
void DiskDecorator::startup()
|
||||
void DiskDecorator::startup(ContextPtr context)
|
||||
{
|
||||
delegate->startup();
|
||||
delegate->startup(context);
|
||||
}
|
||||
|
||||
void DiskDecorator::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map)
|
||||
|
@ -71,7 +71,7 @@ public:
|
||||
void onFreeze(const String & path) override;
|
||||
SyncGuardPtr getDirectorySyncGuard(const String & path) const override;
|
||||
void shutdown() override;
|
||||
void startup() override;
|
||||
void startup(ContextPtr context) override;
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String & config_prefix, const DisksMap & map) override;
|
||||
String getCacheBasePath() const override { return delegate->getCacheBasePath(); }
|
||||
std::vector<String> getRemotePaths(const String & path) const override { return delegate->getRemotePaths(path); }
|
||||
|
@ -494,7 +494,7 @@ DiskLocal::DiskLocal(
|
||||
disk_checker = std::make_unique<DiskLocalCheckThread>(this, context, local_disk_check_period_ms);
|
||||
}
|
||||
|
||||
void DiskLocal::startup()
|
||||
void DiskLocal::startup(ContextPtr)
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -682,7 +682,7 @@ void registerDiskLocal(DiskFactory & factory)
|
||||
|
||||
std::shared_ptr<IDisk> disk
|
||||
= std::make_shared<DiskLocal>(name, path, keep_free_space_bytes, context, config.getUInt("local_disk_check_period_ms", 0));
|
||||
disk->startup();
|
||||
disk->startup(context);
|
||||
return std::make_shared<DiskRestartProxy>(disk);
|
||||
};
|
||||
factory.registerDiskType("local", creator);
|
||||
|
@ -110,7 +110,7 @@ public:
|
||||
|
||||
bool isBroken() const override { return broken; }
|
||||
|
||||
void startup() override;
|
||||
void startup(ContextPtr) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DEADLOCK_AVOIDED;
|
||||
@ -329,7 +330,7 @@ void DiskRestartProxy::getRemotePathsRecursive(const String & path, std::vector<
|
||||
return DiskDecorator::getRemotePathsRecursive(path, paths_map);
|
||||
}
|
||||
|
||||
void DiskRestartProxy::restart()
|
||||
void DiskRestartProxy::restart(ContextPtr context)
|
||||
{
|
||||
/// Speed up processing unhealthy requests.
|
||||
DiskDecorator::shutdown();
|
||||
@ -352,7 +353,7 @@ void DiskRestartProxy::restart()
|
||||
|
||||
LOG_INFO(log, "Restart lock acquired. Restarting disk {}", DiskDecorator::getName());
|
||||
|
||||
DiskDecorator::startup();
|
||||
DiskDecorator::startup(context);
|
||||
|
||||
LOG_INFO(log, "Disk restarted {}", DiskDecorator::getName());
|
||||
}
|
||||
|
@ -68,7 +68,7 @@ public:
|
||||
std::vector<String> getRemotePaths(const String & path) const override;
|
||||
void getRemotePathsRecursive(const String & path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
void restart();
|
||||
void restart(ContextPtr context);
|
||||
|
||||
private:
|
||||
friend class RestartAwareReadBuffer;
|
||||
|
@ -9,8 +9,12 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
@ -173,7 +177,7 @@ std::unique_ptr<ReadBufferFromFileBase> DiskWebServer::readFile(const String & p
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = IDiskRemote::getThreadPoolReader();
|
||||
auto reader = IObjectStorage::getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(web_impl), min_bytes_for_seek);
|
||||
}
|
||||
else
|
||||
|
@ -1,10 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <set>
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,143 +0,0 @@
|
||||
#include <Disks/HDFS/DiskHDFS.h>
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/FnTraits.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
DiskHDFS::DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
DiskPtr metadata_disk_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IDiskRemote(disk_name_, hdfs_root_path_, metadata_disk_, nullptr, "DiskHDFS", settings_->thread_pool_size)
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskHDFS::readFile(const String & path, const ReadSettings & read_settings, std::optional<size_t>, std::optional<size_t>) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
|
||||
LOG_TEST(log,
|
||||
"Read from file by path: {}. Existing HDFS objects: {}",
|
||||
backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size());
|
||||
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, remote_fs_root_path, remote_fs_root_path, metadata.remote_fs_objects, read_settings);
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings &)
|
||||
{
|
||||
/// Path to store new HDFS object.
|
||||
std::string file_name = getRandomName();
|
||||
std::string hdfs_path = fs::path(remote_fs_root_path) / file_name;
|
||||
|
||||
LOG_TRACE(log, "{} to file by path: {}. HDFS path: {}", mode == WriteMode::Rewrite ? "Write" : "Append",
|
||||
backQuote(metadata_disk->getPath() + path), hdfs_path);
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(hdfs_path,
|
||||
config, settings->replication, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
auto create_metadata_callback = [this, path, mode, file_name] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false, [file_name, count] (Metadata & metadata) { metadata.addObject(file_name, count); return true; });
|
||||
};
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(create_metadata_callback), hdfs_path);
|
||||
}
|
||||
|
||||
void DiskHDFS::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
{
|
||||
const size_t begin_of_path = hdfs_path.find('/', hdfs_path.find("//") + 2);
|
||||
|
||||
/// Add path from root to file name
|
||||
int res = hdfsDelete(hdfs_fs.get(), hdfs_path.substr(begin_of_path).c_str(), 0);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "HDFSDelete failed with path: " + hdfs_path);
|
||||
}
|
||||
}
|
||||
|
||||
bool DiskHDFS::checkUniqueId(const String & hdfs_uri) const
|
||||
{
|
||||
if (!boost::algorithm::starts_with(hdfs_uri, remote_fs_root_path))
|
||||
return false;
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String remote_fs_object_path = hdfs_uri.substr(begin_of_path);
|
||||
return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str()));
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
std::unique_ptr<DiskHDFSSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings)
|
||||
{
|
||||
return std::make_unique<DiskHDFSSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000),
|
||||
settings.hdfs_replication);
|
||||
}
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context_,
|
||||
const DisksMap & /*map*/) -> DiskPtr
|
||||
{
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
checkHDFSURL(uri);
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
auto metadata_disk = prepareForLocalMetadata(name, config, config_prefix, context_).second;
|
||||
|
||||
return std::make_shared<DiskHDFS>(
|
||||
name, uri,
|
||||
getSettings(config, config_prefix, context_->getSettingsRef()),
|
||||
metadata_disk, config);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
||||
#endif
|
@ -1,84 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct DiskHDFSSettings
|
||||
{
|
||||
size_t min_bytes_for_seek;
|
||||
int thread_pool_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
int replication;
|
||||
|
||||
DiskHDFSSettings(
|
||||
int min_bytes_for_seek_,
|
||||
int thread_pool_size_,
|
||||
int objects_chunk_size_to_delete_,
|
||||
int replication_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, thread_pool_size(thread_pool_size_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_)
|
||||
, replication(replication_) {}
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in HDFS and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains HDFS object key with actual data.
|
||||
*/
|
||||
class DiskHDFS final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using SettingsPtr = std::unique_ptr<DiskHDFSSettings>;
|
||||
|
||||
DiskHDFS(
|
||||
const String & disk_name_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
DiskPtr metadata_disk_,
|
||||
const Poco::Util::AbstractConfiguration & config_);
|
||||
|
||||
DiskType getType() const override { return DiskType::HDFS; }
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
bool supportZeroCopyReplication() const override { return true; }
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(const String & path, size_t buf_size, WriteMode mode, const WriteSettings & settings) override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths) override;
|
||||
|
||||
/// Check file exists and ClickHouse has an access to it
|
||||
/// Overrode in remote disk
|
||||
/// Required for remote disk to ensure that replica has access to data written by other node
|
||||
bool checkUniqueId(const String & hdfs_uri) const override;
|
||||
|
||||
private:
|
||||
String getRandomName() { return toString(UUIDHelpers::generateV4()); }
|
||||
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
};
|
||||
|
||||
}
|
||||
#endif
|
@ -10,6 +10,8 @@
|
||||
#include <Disks/DiskType.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
|
||||
#include <memory>
|
||||
#include <mutex>
|
||||
@ -48,14 +50,6 @@ class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
class MMappedFileCache;
|
||||
|
||||
/**
|
||||
* Mode of opening a file for write.
|
||||
*/
|
||||
enum class WriteMode
|
||||
{
|
||||
Rewrite,
|
||||
Append
|
||||
};
|
||||
|
||||
/**
|
||||
* Provide interface for reservation.
|
||||
@ -289,14 +283,14 @@ public:
|
||||
|
||||
virtual bool isReadOnly() const { return false; }
|
||||
|
||||
/// Check if disk is broken. Broken disks will have 0 space and not be used.
|
||||
/// Check if disk is broken. Broken disks will have 0 space and cannot be used.
|
||||
virtual bool isBroken() const { return false; }
|
||||
|
||||
/// Invoked when Global Context is shutdown.
|
||||
virtual void shutdown() {}
|
||||
|
||||
/// Performs action on disk startup.
|
||||
virtual void startup() {}
|
||||
virtual void startup(ContextPtr) {}
|
||||
|
||||
/// Return some uniq string for file, overrode for IDiskRemote
|
||||
/// Required for distinguish different copies of the same part on remote disk
|
||||
|
@ -1,708 +0,0 @@
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <Common/FileCache.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int BAD_FILE_TYPE;
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_)
|
||||
{
|
||||
Metadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync)
|
||||
{
|
||||
Metadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
Metadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
Metadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
updater(result);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
Metadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
metadata_disk_->removeFile(metadata_file_path_);
|
||||
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::Metadata::createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite)
|
||||
{
|
||||
if (overwrite || !metadata_disk_->exists(metadata_file_path_))
|
||||
{
|
||||
return createAndStoreMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_, sync);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto result = readMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
if (result.read_only)
|
||||
throw Exception("File is read-only: " + metadata_file_path_, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
void IDiskRemote::Metadata::load()
|
||||
{
|
||||
const ReadSettings read_settings;
|
||||
auto buf = metadata_disk->readFile(metadata_file_path, read_settings, 1024); /* reasonable buffer size for small file */
|
||||
|
||||
UInt32 version;
|
||||
readIntText(version, *buf);
|
||||
|
||||
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
|
||||
metadata_disk->getPath() + metadata_file_path, toString(version), toString(VERSION_READ_ONLY_FLAG));
|
||||
|
||||
assertChar('\n', *buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readIntText(total_size, *buf);
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
readIntText(remote_fs_object_size, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readEscapedString(remote_fs_object_path, *buf);
|
||||
if (version == VERSION_ABSOLUTE_PATHS)
|
||||
{
|
||||
if (!remote_fs_object_path.starts_with(remote_fs_root_path))
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Path in metadata does not correspond to root path. Path: {}, root path: {}, disk path: {}",
|
||||
remote_fs_object_path, remote_fs_root_path, metadata_disk->getPath());
|
||||
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects[i].relative_path = remote_fs_object_path;
|
||||
remote_fs_objects[i].bytes_size = remote_fs_object_size;
|
||||
}
|
||||
|
||||
readIntText(ref_count, *buf);
|
||||
assertChar('\n', *buf);
|
||||
|
||||
if (version >= VERSION_READ_ONLY_FLAG)
|
||||
{
|
||||
readBoolText(read_only, *buf);
|
||||
assertChar('\n', *buf);
|
||||
}
|
||||
}
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
IDiskRemote::Metadata::Metadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_)
|
||||
: remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_file_path(metadata_file_path_)
|
||||
, metadata_disk(metadata_disk_)
|
||||
{
|
||||
}
|
||||
|
||||
void IDiskRemote::Metadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::Metadata::saveToBuffer(WriteBuffer & buf, bool sync)
|
||||
{
|
||||
writeIntText(VERSION_RELATIVE_PATHS, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
writeEscapedString(remote_fs_object_path, buf);
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
|
||||
writeIntText(ref_count, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeBoolText(read_only, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
buf.finalize();
|
||||
if (sync)
|
||||
buf.sync();
|
||||
|
||||
}
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void IDiskRemote::Metadata::save(bool sync)
|
||||
{
|
||||
auto buf = metadata_disk->writeFile(metadata_file_path, 1024);
|
||||
saveToBuffer(*buf, sync);
|
||||
}
|
||||
|
||||
std::string IDiskRemote::Metadata::serializeToString()
|
||||
{
|
||||
WriteBufferFromOwnString write_buf;
|
||||
saveToBuffer(write_buf, false);
|
||||
return write_buf.str();
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const
|
||||
{
|
||||
return Metadata::readMetadata(remote_fs_root_path, metadata_disk, path);
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readMetadata(const String & path) const
|
||||
{
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
return readMetadataUnlocked(path, lock);
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readUpdateAndStoreMetadata(const String & path, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readUpdateStoreMetadataAndRemove(const String & path, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateStoreMetadataAndRemove(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
if (mode == WriteMode::Rewrite || !metadata_disk->exists(path))
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
else
|
||||
{
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::createAndStoreMetadata(const String & path, bool sync)
|
||||
{
|
||||
return Metadata::createAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync);
|
||||
}
|
||||
|
||||
IDiskRemote::Metadata IDiskRemote::createUpdateAndStoreMetadata(const String & path, bool sync, IDiskRemote::MetadataUpdater updater)
|
||||
{
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
|
||||
std::unordered_map<String, String> IDiskRemote::getSerializedMetadata(const std::vector<std::string> & file_paths) const
|
||||
{
|
||||
std::unordered_map<String, String> metadatas;
|
||||
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
|
||||
for (const auto & path : file_paths)
|
||||
{
|
||||
IDiskRemote::Metadata metadata = readMetadataUnlocked(path, lock);
|
||||
metadata.ref_count = 0;
|
||||
metadatas[path] = metadata.serializeToString();
|
||||
}
|
||||
|
||||
return metadatas;
|
||||
}
|
||||
|
||||
void IDiskRemote::removeMetadata(const String & path, std::vector<String> & paths_to_remove)
|
||||
{
|
||||
LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path));
|
||||
|
||||
if (!metadata_disk->exists(path))
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata path '{}' doesn't exist", path);
|
||||
|
||||
if (!metadata_disk->isFile(path))
|
||||
throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path '{}' is not a regular file", path);
|
||||
|
||||
try
|
||||
{
|
||||
auto metadata_updater = [&paths_to_remove, this] (Metadata & metadata)
|
||||
{
|
||||
if (metadata.ref_count == 0)
|
||||
{
|
||||
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
|
||||
{
|
||||
|
||||
paths_to_remove.push_back(remote_fs_root_path + remote_fs_object_path);
|
||||
|
||||
if (cache)
|
||||
{
|
||||
auto key = cache->hash(remote_fs_object_path);
|
||||
cache->remove(key);
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
else /// In other case decrement number of references, save metadata and delete hardlink.
|
||||
{
|
||||
--metadata.ref_count;
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
readUpdateStoreMetadataAndRemove(path, false, metadata_updater);
|
||||
/// If there is no references - delete content from remote FS.
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If it's impossible to read meta - just remove it from FS.
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
|
||||
backQuote(path), e.nested() ? e.nested()->message() : e.message());
|
||||
metadata_disk->removeFile(path);
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
if (metadata_disk->isFile(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove[path]);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
removeMetadataRecursive(it->path(), paths_to_remove);
|
||||
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<String> IDiskRemote::getRemotePaths(const String & local_path) const
|
||||
{
|
||||
auto metadata = readMetadata(local_path);
|
||||
|
||||
std::vector<String> remote_paths;
|
||||
for (const auto & [remote_path, _] : metadata.remote_fs_objects)
|
||||
remote_paths.push_back(fs::path(metadata.remote_fs_root_path) / remote_path);
|
||||
|
||||
return remote_paths;
|
||||
}
|
||||
|
||||
void IDiskRemote::getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map)
|
||||
{
|
||||
/// Protect against concurrent delition of files (for example because of a merge).
|
||||
if (metadata_disk->isFile(local_path))
|
||||
{
|
||||
try
|
||||
{
|
||||
paths_map.emplace_back(local_path, getRemotePaths(local_path));
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
DiskDirectoryIteratorPtr it;
|
||||
try
|
||||
{
|
||||
it = iterateDirectory(local_path);
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
{
|
||||
if (e.code() == std::errc::no_such_file_or_directory)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
|
||||
for (; it->isValid(); it->next())
|
||||
IDiskRemote::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map);
|
||||
}
|
||||
}
|
||||
|
||||
DiskPtr DiskRemoteReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX);
|
||||
return disk;
|
||||
}
|
||||
|
||||
void DiskRemoteReservation::update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
disk->reserved_bytes -= size;
|
||||
size = new_size;
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
|
||||
DiskRemoteReservation::~DiskRemoteReservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(disk->log, "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(disk->log, "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
IDiskRemote::IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
FileCachePtr cache_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size)
|
||||
: IDisk(std::make_unique<AsyncExecutor>(log_name_, thread_pool_size))
|
||||
, log(&Poco::Logger::get(log_name_))
|
||||
, name(name_)
|
||||
, remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_disk(metadata_disk_)
|
||||
, cache(cache_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
String IDiskRemote::getCacheBasePath() const
|
||||
{
|
||||
return cache ? cache->getBasePath() : "";
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::exists(const String & path) const
|
||||
{
|
||||
return metadata_disk->exists(path);
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isFile(const String & path) const
|
||||
{
|
||||
return metadata_disk->isFile(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createFile(const String & path)
|
||||
{
|
||||
createAndStoreMetadata(path, false);
|
||||
}
|
||||
|
||||
|
||||
size_t IDiskRemote::getFileSize(const String & path) const
|
||||
{
|
||||
return readMetadata(path).total_size;
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::moveFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
|
||||
metadata_disk->moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::replaceFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
{
|
||||
const String tmp_path = to_path + ".old";
|
||||
moveFile(to_path, tmp_path);
|
||||
moveFile(from_path, to_path);
|
||||
removeFile(tmp_path);
|
||||
}
|
||||
else
|
||||
moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
void IDiskRemote::removeSharedFile(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
removeMetadata(path, paths_to_remove);
|
||||
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
|
||||
void IDiskRemote::removeSharedFileIfExists(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
if (metadata_disk->exists(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove);
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
}
|
||||
|
||||
void IDiskRemote::removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only)
|
||||
{
|
||||
std::unordered_map<String, std::vector<String>> paths_to_remove;
|
||||
for (const auto & file : files)
|
||||
{
|
||||
bool skip = file.if_exists && !metadata_disk->exists(file.path);
|
||||
if (!skip)
|
||||
removeMetadata(file.path, paths_to_remove[file.path]);
|
||||
}
|
||||
|
||||
if (!keep_all_batch_data)
|
||||
{
|
||||
std::vector<String> remove_from_remote;
|
||||
for (auto && [path, remote_paths] : paths_to_remove)
|
||||
{
|
||||
if (!file_names_remove_metadata_only.contains(fs::path(path).filename()))
|
||||
remove_from_remote.insert(remove_from_remote.end(), remote_paths.begin(), remote_paths.end());
|
||||
}
|
||||
removeFromRemoteFS(remove_from_remote);
|
||||
}
|
||||
}
|
||||
|
||||
void IDiskRemote::removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only)
|
||||
{
|
||||
std::unordered_map<String, std::vector<String>> paths_to_remove;
|
||||
removeMetadataRecursive(path, paths_to_remove);
|
||||
|
||||
if (!keep_all_batch_data)
|
||||
{
|
||||
std::vector<String> remove_from_remote;
|
||||
for (auto && [local_path, remote_paths] : paths_to_remove)
|
||||
{
|
||||
if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename()))
|
||||
remove_from_remote.insert(remove_from_remote.end(), remote_paths.begin(), remote_paths.end());
|
||||
}
|
||||
removeFromRemoteFS(remove_from_remote);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setReadOnly(const String & path)
|
||||
{
|
||||
/// We should store read only flag inside metadata file (instead of using FS flag),
|
||||
/// because we modify metadata file when create hard-links from it.
|
||||
readUpdateAndStoreMetadata(path, false, [] (Metadata & metadata) { metadata.read_only = true; return true; });
|
||||
}
|
||||
|
||||
|
||||
bool IDiskRemote::isDirectory(const String & path) const
|
||||
{
|
||||
return metadata_disk->isDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createDirectories(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectories(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::clearDirectory(const String & path)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
if (isFile(it->path()))
|
||||
removeFile(it->path());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::removeDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
DiskDirectoryIteratorPtr IDiskRemote::iterateDirectory(const String & path)
|
||||
{
|
||||
return metadata_disk->iterateDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::listFiles(const String & path, std::vector<String> & file_names)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
file_names.push_back(it->name());
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::setLastModified(const String & path, const Poco::Timestamp & timestamp)
|
||||
{
|
||||
metadata_disk->setLastModified(path, timestamp);
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp IDiskRemote::getLastModified(const String & path)
|
||||
{
|
||||
return metadata_disk->getLastModified(path);
|
||||
}
|
||||
|
||||
|
||||
void IDiskRemote::createHardLink(const String & src_path, const String & dst_path)
|
||||
{
|
||||
readUpdateAndStoreMetadata(src_path, false, [] (Metadata & metadata) { metadata.ref_count++; return true; });
|
||||
|
||||
/// Create FS hardlink to metadata file.
|
||||
metadata_disk->createHardLink(src_path, dst_path);
|
||||
}
|
||||
|
||||
|
||||
ReservationPtr IDiskRemote::reserve(UInt64 bytes)
|
||||
{
|
||||
auto unreserved_space = tryReserve(bytes);
|
||||
if (!unreserved_space.has_value())
|
||||
return {};
|
||||
|
||||
return std::make_unique<DiskRemoteReservation>(
|
||||
std::static_pointer_cast<IDiskRemote>(shared_from_this()),
|
||||
bytes, unreserved_space.value());
|
||||
}
|
||||
|
||||
|
||||
std::optional<UInt64> IDiskRemote::tryReserve(UInt64 bytes)
|
||||
{
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
|
||||
auto available_space = getAvailableSpace();
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return {unreserved_space};
|
||||
}
|
||||
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return {unreserved_space - bytes};
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
String IDiskRemote::getUniqueId(const String & path) const
|
||||
{
|
||||
LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path);
|
||||
auto metadata = readMetadata(path);
|
||||
String id;
|
||||
if (!metadata.remote_fs_objects.empty())
|
||||
id = metadata.remote_fs_root_path + metadata.remote_fs_objects[0].relative_path;
|
||||
return id;
|
||||
}
|
||||
|
||||
|
||||
AsynchronousReaderPtr IDiskRemote::getThreadPoolReader()
|
||||
{
|
||||
constexpr size_t pool_size = 50;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolRemoteFSReader>(pool_size, queue_size);
|
||||
return reader;
|
||||
}
|
||||
|
||||
UInt32 IDiskRemote::getRefCount(const String & path) const
|
||||
{
|
||||
return readMetadata(path).ref_count;
|
||||
}
|
||||
|
||||
ThreadPool & IDiskRemote::getThreadPoolWriter()
|
||||
{
|
||||
constexpr size_t pool_size = 100;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static ThreadPool writer(pool_size, pool_size, queue_size);
|
||||
return writer;
|
||||
}
|
||||
|
||||
}
|
@ -1,325 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <atomic>
|
||||
#include <Common/FileCache_fwd.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <Disks/Executor.h>
|
||||
#include <utility>
|
||||
#include <mutex>
|
||||
#include <shared_mutex>
|
||||
#include <Common/MultiVersion.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <filesystem>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Path to blob with it's size
|
||||
struct BlobPathWithSize
|
||||
{
|
||||
std::string relative_path;
|
||||
uint64_t bytes_size;
|
||||
|
||||
BlobPathWithSize() = default;
|
||||
BlobPathWithSize(const BlobPathWithSize & other) = default;
|
||||
|
||||
BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_)
|
||||
: relative_path(relative_path_)
|
||||
, bytes_size(bytes_size_)
|
||||
{}
|
||||
};
|
||||
|
||||
/// List of blobs with their sizes
|
||||
using BlobsPathToSize = std::vector<BlobPathWithSize>;
|
||||
|
||||
class IAsynchronousReader;
|
||||
using AsynchronousReaderPtr = std::shared_ptr<IAsynchronousReader>;
|
||||
|
||||
|
||||
/// Base Disk class for remote FS's, which are not posix-compatible (e.g. DiskS3, DiskHDFS, DiskBlobStorage)
|
||||
class IDiskRemote : public IDisk
|
||||
{
|
||||
|
||||
friend class DiskRemoteReservation;
|
||||
|
||||
public:
|
||||
IDiskRemote(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
FileCachePtr cache_,
|
||||
const String & log_name_,
|
||||
size_t thread_pool_size);
|
||||
|
||||
struct Metadata;
|
||||
using MetadataUpdater = std::function<bool(Metadata & metadata)>;
|
||||
|
||||
const String & getName() const final override { return name; }
|
||||
|
||||
const String & getPath() const final override { return metadata_disk->getPath(); }
|
||||
|
||||
String getCacheBasePath() const final override;
|
||||
|
||||
std::vector<String> getRemotePaths(const String & local_path) const final override;
|
||||
|
||||
void getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
/// Methods for working with metadata. For some operations (like hardlink
|
||||
/// creation) metadata can be updated concurrently from multiple threads
|
||||
/// (file actually rewritten on disk). So additional RW lock is required for
|
||||
/// metadata read and write, but not for create new metadata.
|
||||
Metadata readMetadata(const String & path) const;
|
||||
Metadata readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const;
|
||||
Metadata readUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
Metadata readUpdateStoreMetadataAndRemove(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata createAndStoreMetadata(const String & path, bool sync);
|
||||
Metadata createUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
|
||||
void removeFileIfExists(const String & path) override { removeSharedFileIfExists(path, false); }
|
||||
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false, {}); }
|
||||
|
||||
|
||||
void removeSharedFile(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedFileIfExists(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
|
||||
|
||||
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
void removeDirectory(const String & path) override;
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
String getUniqueId(const String & path) const override;
|
||||
|
||||
bool checkUniqueId(const String & id) const override = 0;
|
||||
|
||||
virtual void removeFromRemoteFS(const std::vector<String> & paths) = 0;
|
||||
|
||||
static AsynchronousReaderPtr getThreadPoolReader();
|
||||
static ThreadPool & getThreadPoolWriter();
|
||||
|
||||
DiskPtr getMetadataDiskIfExistsOrSelf() override { return metadata_disk; }
|
||||
|
||||
UInt32 getRefCount(const String & path) const override;
|
||||
|
||||
/// Return metadata for each file path. Also, before serialization reset
|
||||
/// ref_count for each metadata to zero. This function used only for remote
|
||||
/// fetches/sends in replicated engines. That's why we reset ref_count to zero.
|
||||
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & file_paths) const override;
|
||||
protected:
|
||||
Poco::Logger * log;
|
||||
const String name;
|
||||
const String remote_fs_root_path;
|
||||
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
FileCachePtr cache;
|
||||
|
||||
private:
|
||||
void removeMetadata(const String & path, std::vector<String> & paths_to_remove);
|
||||
|
||||
void removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove);
|
||||
|
||||
std::optional<UInt64> tryReserve(UInt64 bytes);
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
mutable std::shared_mutex metadata_mutex;
|
||||
};
|
||||
|
||||
using RemoteDiskPtr = std::shared_ptr<IDiskRemote>;
|
||||
|
||||
/// Remote FS (S3, HDFS) metadata file layout:
|
||||
/// FS objects, their number and total size of all FS objects.
|
||||
/// Each FS object represents a file path in remote FS and its size.
|
||||
|
||||
struct IDiskRemote::Metadata
|
||||
{
|
||||
using Updater = std::function<bool(IDiskRemote::Metadata & metadata)>;
|
||||
/// Metadata file version.
|
||||
static constexpr UInt32 VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr UInt32 VERSION_RELATIVE_PATHS = 2;
|
||||
static constexpr UInt32 VERSION_READ_ONLY_FLAG = 3;
|
||||
|
||||
/// Remote FS objects paths and their sizes.
|
||||
std::vector<BlobPathWithSize> remote_fs_objects;
|
||||
|
||||
/// URI
|
||||
const String & remote_fs_root_path;
|
||||
|
||||
/// Relative path to metadata file on local FS.
|
||||
const String metadata_file_path;
|
||||
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
/// Total size of all remote FS (S3, HDFS) objects.
|
||||
size_t total_size = 0;
|
||||
|
||||
/// Number of references (hardlinks) to this metadata file.
|
||||
///
|
||||
/// FIXME: Why we are tracking it explicetly, without
|
||||
/// info from filesystem????
|
||||
UInt32 ref_count = 0;
|
||||
|
||||
/// Flag indicates that file is read only.
|
||||
bool read_only = false;
|
||||
|
||||
Metadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_);
|
||||
|
||||
void addObject(const String & path, size_t size);
|
||||
|
||||
static Metadata readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_);
|
||||
static Metadata readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static Metadata readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
|
||||
static Metadata createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync);
|
||||
static Metadata createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static Metadata createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite);
|
||||
|
||||
/// Serialize metadata to string (very same with saveToBuffer)
|
||||
std::string serializeToString();
|
||||
|
||||
private:
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void save(bool sync = false);
|
||||
void saveToBuffer(WriteBuffer & buffer, bool sync);
|
||||
void load();
|
||||
};
|
||||
|
||||
class DiskRemoteReservation final : public IReservation
|
||||
{
|
||||
public:
|
||||
DiskRemoteReservation(const RemoteDiskPtr & disk_, UInt64 size_, UInt64 unreserved_space_)
|
||||
: disk(disk_)
|
||||
, size(size_)
|
||||
, unreserved_space(unreserved_space_)
|
||||
, metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
|
||||
{
|
||||
}
|
||||
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return unreserved_space; }
|
||||
|
||||
DiskPtr getDisk(size_t i) const override;
|
||||
|
||||
Disks getDisks() const override { return {disk}; }
|
||||
|
||||
void update(UInt64 new_size) override;
|
||||
|
||||
~DiskRemoteReservation() override;
|
||||
|
||||
private:
|
||||
RemoteDiskPtr disk;
|
||||
UInt64 size;
|
||||
UInt64 unreserved_space;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
|
||||
/// Runs tasks asynchronously using thread pool.
|
||||
class AsyncExecutor : public Executor
|
||||
{
|
||||
public:
|
||||
explicit AsyncExecutor(const String & name_, int thread_pool_size)
|
||||
: name(name_)
|
||||
, pool(ThreadPool(thread_pool_size)) {}
|
||||
|
||||
std::future<void> execute(std::function<void()> task) override
|
||||
{
|
||||
auto promise = std::make_shared<std::promise<void>>();
|
||||
pool.scheduleOrThrowOnError(
|
||||
[promise, task]()
|
||||
{
|
||||
try
|
||||
{
|
||||
task();
|
||||
promise->set_value();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Failed to run async task");
|
||||
|
||||
try
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
});
|
||||
|
||||
return promise->get_future();
|
||||
}
|
||||
|
||||
void setMaxThreads(size_t threads)
|
||||
{
|
||||
pool.setMaxThreads(threads);
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
||||
}
|
@ -1,6 +1,5 @@
|
||||
#include "ReadBufferFromRemoteFSGather.h"
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Disks/IO/ReadBufferFromWebServer.h>
|
||||
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
#include <azure/storage/blobs.hpp>
|
||||
@ -114,7 +114,7 @@ class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather
|
||||
{
|
||||
public:
|
||||
ReadBufferFromS3Gather(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & version_id_,
|
||||
const std::string & common_path_prefix_,
|
||||
@ -132,7 +132,7 @@ public:
|
||||
SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr;
|
||||
String bucket;
|
||||
String version_id;
|
||||
UInt64 max_single_read_retries;
|
||||
@ -146,7 +146,7 @@ class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFS
|
||||
{
|
||||
public:
|
||||
ReadBufferFromAzureBlobStorageGather(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const std::string & common_path_prefix_,
|
||||
const BlobsPathToSize & blobs_to_read_,
|
||||
size_t max_single_read_retries_,
|
||||
@ -162,7 +162,7 @@ public:
|
||||
SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
};
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/config.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <utility>
|
||||
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -36,7 +36,8 @@ WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS()
|
||||
void WriteIndirectBufferFromRemoteFS::finalizeImpl()
|
||||
{
|
||||
WriteBufferFromFileDecorator::finalizeImpl();
|
||||
create_metadata_callback(count());
|
||||
if (create_metadata_callback)
|
||||
create_metadata_callback(count());
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteBufferFromFileDecorator.h>
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Disks/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
@ -66,27 +66,27 @@ AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::Abstr
|
||||
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> getClientWithConnectionString(const String & connection_str, const String & container_name) = delete;
|
||||
std::unique_ptr<T> getClientWithConnectionString(const String & connection_str, const String & container_name) = delete;
|
||||
|
||||
|
||||
template<>
|
||||
std::shared_ptr<BlobServiceClient> getClientWithConnectionString(
|
||||
std::unique_ptr<BlobServiceClient> getClientWithConnectionString(
|
||||
const String & connection_str, const String & /*container_name*/)
|
||||
{
|
||||
return std::make_shared<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(connection_str));
|
||||
return std::make_unique<BlobServiceClient>(BlobServiceClient::CreateFromConnectionString(connection_str));
|
||||
}
|
||||
|
||||
|
||||
template<>
|
||||
std::shared_ptr<BlobContainerClient> getClientWithConnectionString(
|
||||
std::unique_ptr<BlobContainerClient> getClientWithConnectionString(
|
||||
const String & connection_str, const String & container_name)
|
||||
{
|
||||
return std::make_shared<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(connection_str, container_name));
|
||||
return std::make_unique<BlobContainerClient>(BlobContainerClient::CreateFromConnectionString(connection_str, container_name));
|
||||
}
|
||||
|
||||
|
||||
template <class T>
|
||||
std::shared_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
std::unique_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
if (config.has(config_prefix + ".connection_string"))
|
||||
@ -101,15 +101,15 @@ std::shared_ptr<T> getAzureBlobStorageClientWithAuth(
|
||||
config.getString(config_prefix + ".account_name"),
|
||||
config.getString(config_prefix + ".account_key")
|
||||
);
|
||||
return std::make_shared<T>(url, storage_shared_key_credential);
|
||||
return std::make_unique<T>(url, storage_shared_key_credential);
|
||||
}
|
||||
|
||||
auto managed_identity_credential = std::make_shared<Azure::Identity::ManagedIdentityCredential>();
|
||||
return std::make_shared<T>(url, managed_identity_credential);
|
||||
return std::make_unique<T>(url, managed_identity_credential);
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
std::unique_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix)
|
||||
{
|
||||
auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix);
|
||||
@ -136,10 +136,20 @@ std::shared_ptr<BlobContainerClient> getAzureBlobContainerClient(
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<BlobContainerClient>(
|
||||
return std::make_unique<BlobContainerClient>(
|
||||
blob_service_client->CreateBlobContainer(container_name).Value);
|
||||
}
|
||||
|
||||
std::unique_ptr<AzureObjectStorageSettings> getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_unique<AzureObjectStorageSettings>(
|
||||
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),
|
||||
config.getInt(config_prefix + ".max_single_download_retries", 3)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -0,0 +1,20 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <azure/storage/blobs.hpp>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::unique_ptr<Azure::Storage::Blobs::BlobContainerClient> getAzureBlobContainerClient(
|
||||
const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
|
||||
std::unique_ptr<AzureObjectStorageSettings> getAzureBlobStorageSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
218
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
Normal file
218
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp
Normal file
@ -0,0 +1,218 @@
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <IO/ReadBufferFromAzureBlobStorage.h>
|
||||
#include <IO/WriteBufferFromAzureBlobStorage.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AZURE_BLOB_STORAGE_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
}
|
||||
|
||||
|
||||
AzureObjectStorage::AzureObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & name_,
|
||||
AzureClientPtr && client_,
|
||||
SettingsPtr && settings_)
|
||||
: IObjectStorage(std::move(cache_))
|
||||
, name(name_)
|
||||
, client(std::move(client_))
|
||||
, settings(std::move(settings_))
|
||||
{
|
||||
}
|
||||
|
||||
bool AzureObjectStorage::exists(const std::string & uri) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
/// What a shame, no Exists method...
|
||||
Azure::Storage::Blobs::ListBlobsOptions options;
|
||||
options.Prefix = uri;
|
||||
options.PageSizeHint = 1;
|
||||
|
||||
auto blobs_list_response = client_ptr->ListBlobs(options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
{
|
||||
if (uri == blob.Name)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> AzureObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = settings.get();
|
||||
|
||||
return std::make_unique<ReadBufferFromAzureBlobStorage>(
|
||||
client.get(), path, settings_ptr->max_single_read_retries,
|
||||
settings_ptr->max_single_download_retries, read_settings.remote_fs_buffer_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> AzureObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = settings.get();
|
||||
auto reader_impl = std::make_unique<ReadBufferFromAzureBlobStorageGather>(
|
||||
client.get(), common_path_prefix, blobs_to_read,
|
||||
settings_ptr->max_single_read_retries, settings_ptr->max_single_download_retries, read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, read_settings, std::move(reader_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(reader_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings_ptr->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> AzureObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes>,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings &)
|
||||
{
|
||||
if (mode != WriteMode::Rewrite)
|
||||
throw Exception("Azure storage doesn't support append", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
|
||||
auto buffer = std::make_unique<WriteBufferFromAzureBlobStorage>(
|
||||
client.get(),
|
||||
path,
|
||||
settings.get()->max_single_part_upload_size,
|
||||
buf_size);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
Azure::Storage::Blobs::ListBlobsOptions blobs_list_options;
|
||||
blobs_list_options.Prefix = path;
|
||||
|
||||
auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options);
|
||||
auto blobs_list = blobs_list_response.Blobs;
|
||||
|
||||
for (const auto & blob : blobs_list)
|
||||
children.emplace_back(blob.Name, blob.BlobSize);
|
||||
}
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void AzureObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
{
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
if (!delete_info.Value.Deleted)
|
||||
throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path);
|
||||
}
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
void AzureObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
for (const auto & path : paths)
|
||||
auto delete_info = client_ptr->DeleteBlob(path);
|
||||
}
|
||||
|
||||
|
||||
ObjectMetadata AzureObjectStorage::getObjectMetadata(const std::string & path) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto blob_client = client_ptr->GetBlobClient(path);
|
||||
auto properties = blob_client.GetProperties().Value;
|
||||
ObjectMetadata result;
|
||||
result.size_bytes = properties.BlobSize;
|
||||
if (!properties.Metadata.empty())
|
||||
{
|
||||
result.attributes.emplace();
|
||||
for (const auto & [key, value] : properties.Metadata)
|
||||
(*result.attributes)[key] = value;
|
||||
}
|
||||
result.last_modified.emplace(properties.LastModified.time_since_epoch().count());
|
||||
return result;
|
||||
}
|
||||
|
||||
void AzureObjectStorage::copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
auto dest_blob_client = client_ptr->GetBlobClient(object_to);
|
||||
auto source_blob_client = client_ptr->GetBlobClient(object_from);
|
||||
Azure::Storage::Blobs::CopyBlobFromUriOptions copy_options;
|
||||
if (object_to_attributes.has_value())
|
||||
{
|
||||
for (const auto & [key, value] : *object_to_attributes)
|
||||
copy_options.Metadata[key] = value;
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
auto new_settings = getAzureBlobStorageSettings(config, config_prefix, context);
|
||||
settings.set(std::move(new_settings));
|
||||
|
||||
/// We don't update client
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<IObjectStorage> AzureObjectStorage::cloneObjectStorage(const std::string &, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
return std::make_unique<AzureObjectStorage>(
|
||||
nullptr,
|
||||
name,
|
||||
getAzureBlobContainerClient(config, config_prefix),
|
||||
getAzureBlobStorageSettings(config, config_prefix, context)
|
||||
);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
113
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
Normal file
113
src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h
Normal file
@ -0,0 +1,113 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct AzureObjectStorageSettings
|
||||
{
|
||||
AzureObjectStorageSettings(
|
||||
uint64_t max_single_part_upload_size_,
|
||||
uint64_t min_bytes_for_seek_,
|
||||
int max_single_read_retries_,
|
||||
int max_single_download_retries_)
|
||||
: max_single_part_upload_size(max_single_part_upload_size_)
|
||||
, min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, max_single_read_retries(max_single_read_retries_)
|
||||
, max_single_download_retries(max_single_download_retries_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t max_single_part_upload_size; /// NOTE: on 32-bit machines it will be at most 4GB, but size_t is also used in BufferBase for offset
|
||||
uint64_t min_bytes_for_seek;
|
||||
size_t max_single_read_retries;
|
||||
size_t max_single_download_retries;
|
||||
};
|
||||
|
||||
using AzureClient = Azure::Storage::Blobs::BlobContainerClient;
|
||||
using AzureClientPtr = std::unique_ptr<Azure::Storage::Blobs::BlobContainerClient>;
|
||||
|
||||
class AzureObjectStorage : public IObjectStorage
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<AzureObjectStorageSettings>;
|
||||
|
||||
AzureObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & name_,
|
||||
AzureClientPtr && client_,
|
||||
SettingsPtr && settings_);
|
||||
|
||||
bool exists(const std::string & uri) const override;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void shutdown() override {}
|
||||
|
||||
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<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
private:
|
||||
const String name;
|
||||
/// client used to access the files in the Blob Storage cloud
|
||||
MultiVersion<Azure::Storage::Blobs::BlobContainerClient> client;
|
||||
MultiVersion<AzureObjectStorageSettings> settings;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -6,10 +6,11 @@
|
||||
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/AzureBlobStorage/DiskAzureBlobStorage.h>
|
||||
#include <Disks/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.h>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -19,18 +20,19 @@ namespace ErrorCodes
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
constexpr char test_file[] = "test.txt";
|
||||
constexpr char test_str[] = "test";
|
||||
constexpr size_t test_str_size = 4;
|
||||
|
||||
|
||||
void checkWriteAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.writeFile(test_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
|
||||
file->write(test_str, test_str_size);
|
||||
}
|
||||
|
||||
|
||||
void checkReadAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile(test_file);
|
||||
@ -40,7 +42,6 @@ void checkReadAccess(IDisk & disk)
|
||||
throw Exception("No read access to disk", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
|
||||
void checkReadWithOffset(IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile(test_file);
|
||||
@ -53,25 +54,13 @@ void checkReadWithOffset(IDisk & disk)
|
||||
throw Exception("Failed to read file with offset", ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
|
||||
void checkRemoveAccess(IDisk & disk)
|
||||
{
|
||||
disk.removeFile(test_file);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<DiskAzureBlobStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/)
|
||||
{
|
||||
return std::make_unique<DiskAzureBlobStorageSettings>(
|
||||
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),
|
||||
config.getInt(config_prefix + ".max_single_download_retries", 3),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16)
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](
|
||||
@ -83,12 +72,25 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
{
|
||||
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
|
||||
|
||||
std::shared_ptr<IDisk> azure_blob_storage_disk = std::make_shared<DiskAzureBlobStorage>(
|
||||
/// FIXME Cache currently unsupported :(
|
||||
ObjectStoragePtr azure_object_storage = std::make_unique<AzureObjectStorage>(
|
||||
nullptr,
|
||||
name,
|
||||
metadata_disk,
|
||||
getAzureBlobContainerClient(config, config_prefix),
|
||||
getSettings(config, config_prefix, context),
|
||||
getSettings
|
||||
getAzureBlobStorageSettings(config, config_prefix, context));
|
||||
|
||||
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
|
||||
bool send_metadata = config.getBool(config_prefix + ".send_metadata", false);
|
||||
|
||||
std::shared_ptr<IDisk> azure_blob_storage_disk = std::make_shared<DiskObjectStorage>(
|
||||
name,
|
||||
/* no namespaces */"",
|
||||
"DiskAzureBlobStorage",
|
||||
metadata_disk,
|
||||
std::move(azure_object_storage),
|
||||
DiskType::AzureBlobStorage,
|
||||
send_metadata,
|
||||
copy_thread_pool_size
|
||||
);
|
||||
|
||||
if (!config.getBool(config_prefix + ".skip_access_check", false))
|
||||
@ -99,9 +101,17 @@ void registerDiskAzureBlobStorage(DiskFactory & factory)
|
||||
checkRemoveAccess(*azure_blob_storage_disk);
|
||||
}
|
||||
|
||||
azure_blob_storage_disk->startup();
|
||||
#ifdef NDEBUG
|
||||
bool use_cache = true;
|
||||
#else
|
||||
/// Current cache implementation lead to allocations in destructor of
|
||||
/// read buffer.
|
||||
bool use_cache = false;
|
||||
#endif
|
||||
|
||||
if (config.getBool(config_prefix + ".cache_enabled", true))
|
||||
azure_blob_storage_disk->startup(context);
|
||||
|
||||
if (config.getBool(config_prefix + ".cache_enabled", use_cache))
|
||||
{
|
||||
String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/");
|
||||
azure_blob_storage_disk = wrapWithCache(azure_blob_storage_disk, "azure-blob-storage-cache", cache_path, metadata_path);
|
678
src/Disks/ObjectStorages/DiskObjectStorage.cpp
Normal file
678
src/Disks/ObjectStorages/DiskObjectStorage.cpp
Normal file
@ -0,0 +1,678 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/createHardLink.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <Common/filesystemHelpers.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <Common/FileCache.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int INCORRECT_DISK_INDEX;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
extern const int FILE_DOESNT_EXIST;
|
||||
extern const int BAD_FILE_TYPE;
|
||||
}
|
||||
|
||||
static String revisionToString(UInt64 revision)
|
||||
{
|
||||
return std::bitset<64>(revision).to_string();
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Runs tasks asynchronously using thread pool.
|
||||
class AsyncThreadPoolExecutor : public Executor
|
||||
{
|
||||
public:
|
||||
AsyncThreadPoolExecutor(const String & name_, int thread_pool_size)
|
||||
: name(name_)
|
||||
, pool(ThreadPool(thread_pool_size)) {}
|
||||
|
||||
std::future<void> execute(std::function<void()> task) override
|
||||
{
|
||||
auto promise = std::make_shared<std::promise<void>>();
|
||||
pool.scheduleOrThrowOnError(
|
||||
[promise, task]()
|
||||
{
|
||||
try
|
||||
{
|
||||
task();
|
||||
promise->set_value();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException("Failed to run async task");
|
||||
|
||||
try
|
||||
{
|
||||
promise->set_exception(std::current_exception());
|
||||
}
|
||||
catch (...) {}
|
||||
}
|
||||
});
|
||||
|
||||
return promise->get_future();
|
||||
}
|
||||
|
||||
void setMaxThreads(size_t threads)
|
||||
{
|
||||
pool.setMaxThreads(threads);
|
||||
}
|
||||
|
||||
private:
|
||||
String name;
|
||||
ThreadPool pool;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
DiskObjectStorage::DiskObjectStorage(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & log_name,
|
||||
DiskPtr metadata_disk_,
|
||||
ObjectStoragePtr && object_storage_,
|
||||
DiskType disk_type_,
|
||||
bool send_metadata_,
|
||||
uint64_t thread_pool_size)
|
||||
: IDisk(std::make_unique<AsyncThreadPoolExecutor>(log_name, thread_pool_size))
|
||||
, name(name_)
|
||||
, remote_fs_root_path(remote_fs_root_path_)
|
||||
, log (&Poco::Logger::get(log_name))
|
||||
, metadata_disk(metadata_disk_)
|
||||
, disk_type(disk_type_)
|
||||
, object_storage(std::move(object_storage_))
|
||||
, send_metadata(send_metadata_)
|
||||
, metadata_helper(std::make_unique<DiskObjectStorageMetadataHelper>(this, ReadSettings{}))
|
||||
{}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const
|
||||
{
|
||||
return Metadata::readMetadata(remote_fs_root_path, metadata_disk, path);
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readMetadata(const String & path) const
|
||||
{
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
return readMetadataUnlocked(path, lock);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readUpdateAndStoreMetadata(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readUpdateStoreMetadataAndRemove(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::readUpdateStoreMetadataAndRemove(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
if (mode == WriteMode::Rewrite || !metadata_disk->exists(path))
|
||||
{
|
||||
std::unique_lock lock(metadata_mutex);
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
else
|
||||
{
|
||||
return Metadata::readUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::createAndStoreMetadata(const String & path, bool sync)
|
||||
{
|
||||
return Metadata::createAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync);
|
||||
}
|
||||
|
||||
DiskObjectStorage::Metadata DiskObjectStorage::createUpdateAndStoreMetadata(const String & path, bool sync, DiskObjectStorage::MetadataUpdater updater)
|
||||
{
|
||||
return Metadata::createUpdateAndStoreMetadata(remote_fs_root_path, metadata_disk, path, sync, updater);
|
||||
}
|
||||
|
||||
std::vector<String> DiskObjectStorage::getRemotePaths(const String & local_path) const
|
||||
{
|
||||
auto metadata = readMetadata(local_path);
|
||||
|
||||
std::vector<String> remote_paths;
|
||||
for (const auto & [remote_path, _] : metadata.remote_fs_objects)
|
||||
remote_paths.push_back(fs::path(metadata.remote_fs_root_path) / remote_path);
|
||||
|
||||
return remote_paths;
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map)
|
||||
{
|
||||
/// Protect against concurrent delition of files (for example because of a merge).
|
||||
if (metadata_disk->isFile(local_path))
|
||||
{
|
||||
try
|
||||
{
|
||||
paths_map.emplace_back(local_path, getRemotePaths(local_path));
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
if (e.code() == ErrorCodes::FILE_DOESNT_EXIST)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
DiskDirectoryIteratorPtr it;
|
||||
try
|
||||
{
|
||||
it = iterateDirectory(local_path);
|
||||
}
|
||||
catch (const fs::filesystem_error & e)
|
||||
{
|
||||
if (e.code() == std::errc::no_such_file_or_directory)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
|
||||
for (; it->isValid(); it->next())
|
||||
DiskObjectStorage::getRemotePathsRecursive(fs::path(local_path) / it->name(), paths_map);
|
||||
}
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::exists(const String & path) const
|
||||
{
|
||||
return metadata_disk->exists(path);
|
||||
}
|
||||
|
||||
|
||||
bool DiskObjectStorage::isFile(const String & path) const
|
||||
{
|
||||
return metadata_disk->isFile(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createFile(const String & path)
|
||||
{
|
||||
createAndStoreMetadata(path, false);
|
||||
}
|
||||
|
||||
size_t DiskObjectStorage::getFileSize(const String & path) const
|
||||
{
|
||||
return readMetadata(path).total_size;
|
||||
}
|
||||
|
||||
void DiskObjectStorage::moveFile(const String & from_path, const String & to_path, bool should_send_metadata)
|
||||
{
|
||||
if (exists(to_path))
|
||||
throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
|
||||
if (should_send_metadata)
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter += 1;
|
||||
|
||||
const ObjectAttributes object_metadata {
|
||||
{"from_path", from_path},
|
||||
{"to_path", to_path}
|
||||
};
|
||||
metadata_helper->createFileOperationObject("rename", revision, object_metadata);
|
||||
}
|
||||
|
||||
metadata_disk->moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::moveFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
moveFile(from_path, to_path, send_metadata);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::replaceFile(const String & from_path, const String & to_path)
|
||||
{
|
||||
if (exists(to_path))
|
||||
{
|
||||
const String tmp_path = to_path + ".old";
|
||||
moveFile(to_path, tmp_path);
|
||||
moveFile(from_path, to_path);
|
||||
removeFile(tmp_path);
|
||||
}
|
||||
else
|
||||
moveFile(from_path, to_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedFile(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
removeMetadata(path, paths_to_remove);
|
||||
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeFromRemoteFS(const std::vector<String> & paths)
|
||||
{
|
||||
object_storage->removeObjects(paths);
|
||||
}
|
||||
|
||||
UInt32 DiskObjectStorage::getRefCount(const String & path) const
|
||||
{
|
||||
return readMetadata(path).ref_count;
|
||||
}
|
||||
|
||||
std::unordered_map<String, String> DiskObjectStorage::getSerializedMetadata(const std::vector<String> & file_paths) const
|
||||
{
|
||||
std::unordered_map<String, String> metadatas;
|
||||
|
||||
std::shared_lock lock(metadata_mutex);
|
||||
|
||||
for (const auto & path : file_paths)
|
||||
{
|
||||
DiskObjectStorage::Metadata metadata = readMetadataUnlocked(path, lock);
|
||||
metadata.ref_count = 0;
|
||||
metadatas[path] = metadata.serializeToString();
|
||||
}
|
||||
|
||||
return metadatas;
|
||||
}
|
||||
|
||||
String DiskObjectStorage::getUniqueId(const String & path) const
|
||||
{
|
||||
LOG_TRACE(log, "Remote path: {}, Path: {}", remote_fs_root_path, path);
|
||||
auto metadata = readMetadata(path);
|
||||
String id;
|
||||
if (!metadata.remote_fs_objects.empty())
|
||||
id = metadata.remote_fs_root_path + metadata.remote_fs_objects[0].relative_path;
|
||||
return id;
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::checkObjectExists(const String & path) const
|
||||
{
|
||||
if (!path.starts_with(remote_fs_root_path))
|
||||
return false;
|
||||
|
||||
return object_storage->exists(path);
|
||||
}
|
||||
|
||||
bool DiskObjectStorage::checkUniqueId(const String & id) const
|
||||
{
|
||||
return checkObjectExists(id);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata)
|
||||
{
|
||||
readUpdateAndStoreMetadata(src_path, false, [](Metadata & metadata) { metadata.ref_count++; return true; });
|
||||
|
||||
if (should_send_metadata && !dst_path.starts_with("shadow/"))
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter += 1;
|
||||
const ObjectAttributes object_metadata {
|
||||
{"src_path", src_path},
|
||||
{"dst_path", dst_path}
|
||||
};
|
||||
metadata_helper->createFileOperationObject("hardlink", revision, object_metadata);
|
||||
}
|
||||
|
||||
/// Create FS hardlink to metadata file.
|
||||
metadata_disk->createHardLink(src_path, dst_path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::createHardLink(const String & src_path, const String & dst_path)
|
||||
{
|
||||
createHardLink(src_path, dst_path, send_metadata);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::setReadOnly(const String & path)
|
||||
{
|
||||
/// We should store read only flag inside metadata file (instead of using FS flag),
|
||||
/// because we modify metadata file when create hard-links from it.
|
||||
readUpdateAndStoreMetadata(path, false, [](Metadata & metadata) { metadata.read_only = true; return true; });
|
||||
}
|
||||
|
||||
|
||||
bool DiskObjectStorage::isDirectory(const String & path) const
|
||||
{
|
||||
return metadata_disk->isDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::createDirectories(const String & path)
|
||||
{
|
||||
metadata_disk->createDirectories(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::clearDirectory(const String & path)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
if (isFile(it->path()))
|
||||
removeFile(it->path());
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::removeDirectory(const String & path)
|
||||
{
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
DiskDirectoryIteratorPtr DiskObjectStorage::iterateDirectory(const String & path)
|
||||
{
|
||||
return metadata_disk->iterateDirectory(path);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::listFiles(const String & path, std::vector<String> & file_names)
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
file_names.push_back(it->name());
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::setLastModified(const String & path, const Poco::Timestamp & timestamp)
|
||||
{
|
||||
metadata_disk->setLastModified(path, timestamp);
|
||||
}
|
||||
|
||||
|
||||
Poco::Timestamp DiskObjectStorage::getLastModified(const String & path)
|
||||
{
|
||||
return metadata_disk->getLastModified(path);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeMetadata(const String & path, std::vector<String> & paths_to_remove)
|
||||
{
|
||||
LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path));
|
||||
|
||||
if (!metadata_disk->exists(path))
|
||||
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata path '{}' doesn't exist", path);
|
||||
|
||||
if (!metadata_disk->isFile(path))
|
||||
throw Exception(ErrorCodes::BAD_FILE_TYPE, "Path '{}' is not a regular file", path);
|
||||
|
||||
try
|
||||
{
|
||||
auto metadata_updater = [&paths_to_remove, this] (Metadata & metadata)
|
||||
{
|
||||
if (metadata.ref_count == 0)
|
||||
{
|
||||
for (const auto & [remote_fs_object_path, _] : metadata.remote_fs_objects)
|
||||
{
|
||||
String object_path = fs::path(remote_fs_root_path) / remote_fs_object_path;
|
||||
paths_to_remove.push_back(object_path);
|
||||
object_storage->removeFromCache(object_path);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
else /// In other case decrement number of references, save metadata and delete hardlink.
|
||||
{
|
||||
--metadata.ref_count;
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
readUpdateStoreMetadataAndRemove(path, false, metadata_updater);
|
||||
/// If there is no references - delete content from remote FS.
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If it's impossible to read meta - just remove it from FS.
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
{
|
||||
LOG_WARNING(log,
|
||||
"Metadata file {} can't be read by reason: {}. Removing it forcibly.",
|
||||
backQuote(path), e.nested() ? e.nested()->message() : e.message());
|
||||
metadata_disk->removeFile(path);
|
||||
}
|
||||
else
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
if (metadata_disk->isFile(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove[path]);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = iterateDirectory(path); it->isValid(); it->next())
|
||||
removeMetadataRecursive(it->path(), paths_to_remove);
|
||||
|
||||
metadata_disk->removeDirectory(path);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::shutdown()
|
||||
{
|
||||
LOG_INFO(log, "Shutting down disk {}", name);
|
||||
object_storage->shutdown();
|
||||
LOG_INFO(log, "Disk {} shut down", name);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::startup(ContextPtr context)
|
||||
{
|
||||
|
||||
LOG_INFO(log, "Starting up disk {}", name);
|
||||
object_storage->startup();
|
||||
|
||||
restoreMetadataIfNeeded(context->getConfigRef(), "storage_configuration.disks." + name, context);
|
||||
|
||||
LOG_INFO(log, "Disk {} started up", name);
|
||||
}
|
||||
|
||||
ReservationPtr DiskObjectStorage::reserve(UInt64 bytes)
|
||||
{
|
||||
if (!tryReserve(bytes))
|
||||
return {};
|
||||
|
||||
return std::make_unique<DiskObjectStorageReservation>(std::static_pointer_cast<DiskObjectStorage>(shared_from_this()), bytes);
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedFileIfExists(const String & path, bool delete_metadata_only)
|
||||
{
|
||||
std::vector<String> paths_to_remove;
|
||||
if (metadata_disk->exists(path))
|
||||
{
|
||||
removeMetadata(path, paths_to_remove);
|
||||
if (!delete_metadata_only)
|
||||
removeFromRemoteFS(paths_to_remove);
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorage::removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only)
|
||||
{
|
||||
std::unordered_map<String, std::vector<String>> paths_to_remove;
|
||||
removeMetadataRecursive(path, paths_to_remove);
|
||||
|
||||
if (!keep_all_batch_data)
|
||||
{
|
||||
std::vector<String> remove_from_remote;
|
||||
for (auto && [local_path, remote_paths] : paths_to_remove)
|
||||
{
|
||||
if (!file_names_remove_metadata_only.contains(fs::path(local_path).filename()))
|
||||
{
|
||||
remove_from_remote.insert(remove_from_remote.end(), remote_paths.begin(), remote_paths.end());
|
||||
}
|
||||
}
|
||||
removeFromRemoteFS(remove_from_remote);
|
||||
}
|
||||
}
|
||||
|
||||
std::optional<UInt64> DiskObjectStorage::tryReserve(UInt64 bytes)
|
||||
{
|
||||
std::lock_guard lock(reservation_mutex);
|
||||
|
||||
auto available_space = getAvailableSpace();
|
||||
UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes);
|
||||
|
||||
if (bytes == 0)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name));
|
||||
++reservation_count;
|
||||
return {unreserved_space};
|
||||
}
|
||||
|
||||
if (unreserved_space >= bytes)
|
||||
{
|
||||
LOG_TRACE(log, "Reserving {} on disk {}, having unreserved {}.",
|
||||
ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space));
|
||||
++reservation_count;
|
||||
reserved_bytes += bytes;
|
||||
return {unreserved_space - bytes};
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> DiskObjectStorage::readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const
|
||||
{
|
||||
auto metadata = readMetadata(path);
|
||||
return object_storage->readObjects(remote_fs_root_path, metadata.remote_fs_objects, settings, read_hint, file_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorage::writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings)
|
||||
{
|
||||
auto blob_name = getRandomASCIIString();
|
||||
|
||||
std::optional<ObjectAttributes> object_attributes;
|
||||
if (send_metadata)
|
||||
{
|
||||
auto revision = metadata_helper->revision_counter + 1;
|
||||
metadata_helper->revision_counter++;
|
||||
object_attributes = {
|
||||
{"path", path}
|
||||
};
|
||||
blob_name = "r" + revisionToString(revision) + "-file-" + blob_name;
|
||||
}
|
||||
|
||||
auto create_metadata_callback = [this, path, blob_name, mode] (size_t count)
|
||||
{
|
||||
readOrCreateUpdateAndStoreMetadata(path, mode, false,
|
||||
[blob_name, count] (DiskObjectStorage::Metadata & metadata) { metadata.addObject(blob_name, count); return true; });
|
||||
};
|
||||
|
||||
/// We always use mode Rewrite because we simulate append using metadata and different files
|
||||
return object_storage->writeObject(
|
||||
fs::path(remote_fs_root_path) / blob_name, WriteMode::Rewrite, object_attributes,
|
||||
std::move(create_metadata_callback),
|
||||
buf_size, settings);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &)
|
||||
{
|
||||
const auto config_prefix = "storage_configuration.disks." + name;
|
||||
object_storage->applyNewSettings(config, config_prefix, context_);
|
||||
|
||||
if (AsyncThreadPoolExecutor * exec = dynamic_cast<AsyncThreadPoolExecutor *>(&getExecutor()))
|
||||
exec->setMaxThreads(config.getInt(config_prefix + ".thread_pool_size", 16));
|
||||
}
|
||||
|
||||
void DiskObjectStorage::restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
if (send_metadata)
|
||||
{
|
||||
metadata_helper->restore(config, config_prefix, context);
|
||||
|
||||
if (metadata_helper->readSchemaVersion(object_storage.get(), remote_fs_root_path) < DiskObjectStorageMetadataHelper::RESTORABLE_SCHEMA_VERSION)
|
||||
metadata_helper->migrateToRestorableSchema();
|
||||
|
||||
metadata_helper->findLastRevision();
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorage::syncRevision(UInt64 revision)
|
||||
{
|
||||
metadata_helper->syncRevision(revision);
|
||||
}
|
||||
|
||||
UInt64 DiskObjectStorage::getRevision() const
|
||||
{
|
||||
return metadata_helper->getRevision();
|
||||
}
|
||||
|
||||
|
||||
DiskPtr DiskObjectStorageReservation::getDisk(size_t i) const
|
||||
{
|
||||
if (i != 0)
|
||||
throw Exception("Can't use i != 0 with single disk reservation", ErrorCodes::INCORRECT_DISK_INDEX);
|
||||
return disk;
|
||||
}
|
||||
|
||||
void DiskObjectStorageReservation::update(UInt64 new_size)
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
disk->reserved_bytes -= size;
|
||||
size = new_size;
|
||||
disk->reserved_bytes += size;
|
||||
}
|
||||
|
||||
DiskObjectStorageReservation::~DiskObjectStorageReservation()
|
||||
{
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(disk->reservation_mutex);
|
||||
if (disk->reserved_bytes < size)
|
||||
{
|
||||
disk->reserved_bytes = 0;
|
||||
LOG_ERROR(disk->log, "Unbalanced reservations size for disk '{}'.", disk->getName());
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->reserved_bytes -= size;
|
||||
}
|
||||
|
||||
if (disk->reservation_count == 0)
|
||||
LOG_ERROR(disk->log, "Unbalanced reservation count for disk '{}'.", disk->getName());
|
||||
else
|
||||
--disk->reservation_count;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
232
src/Disks/ObjectStorages/DiskObjectStorage.h
Normal file
232
src/Disks/ObjectStorages/DiskObjectStorage.h
Normal file
@ -0,0 +1,232 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadata.h>
|
||||
#include <re2/re2.h>
|
||||
|
||||
namespace CurrentMetrics
|
||||
{
|
||||
extern const Metric DiskSpaceReservedForMerge;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Disk build on top of IObjectStorage. Use additional disk (local for example)
|
||||
/// for metadata storage. Metadata is a small files with mapping from local paths to
|
||||
/// objects in object storage, like:
|
||||
/// "/var/lib/clickhouse/data/db/table/all_0_0_0/columns.txt" -> /xxxxxxxxxxxxxxxxxxxx
|
||||
/// -> /yyyyyyyyyyyyyyyyyyyy
|
||||
class DiskObjectStorage : public IDisk
|
||||
{
|
||||
|
||||
friend class DiskObjectStorageReservation;
|
||||
friend class DiskObjectStorageMetadataHelper;
|
||||
|
||||
public:
|
||||
DiskObjectStorage(
|
||||
const String & name_,
|
||||
const String & remote_fs_root_path_,
|
||||
const String & log_name,
|
||||
DiskPtr metadata_disk_,
|
||||
ObjectStoragePtr && object_storage_,
|
||||
DiskType disk_type_,
|
||||
bool send_metadata_,
|
||||
uint64_t thread_pool_size);
|
||||
|
||||
DiskType getType() const override { return disk_type; }
|
||||
|
||||
bool supportZeroCopyReplication() const override { return true; }
|
||||
|
||||
bool supportParallelWrite() const override { return true; }
|
||||
|
||||
using Metadata = DiskObjectStorageMetadata;
|
||||
using MetadataUpdater = std::function<bool(Metadata & metadata)>;
|
||||
|
||||
const String & getName() const override { return name; }
|
||||
|
||||
const String & getPath() const override { return metadata_disk->getPath(); }
|
||||
|
||||
std::vector<String> getRemotePaths(const String & local_path) const override;
|
||||
|
||||
void getRemotePathsRecursive(const String & local_path, std::vector<LocalPathWithRemotePaths> & paths_map) override;
|
||||
|
||||
std::string getCacheBasePath() const override
|
||||
{
|
||||
return object_storage->getCacheBasePath();
|
||||
}
|
||||
|
||||
/// Methods for working with metadata. For some operations (like hardlink
|
||||
/// creation) metadata can be updated concurrently from multiple threads
|
||||
/// (file actually rewritten on disk). So additional RW lock is required for
|
||||
/// metadata read and write, but not for create new metadata.
|
||||
Metadata readMetadata(const String & path) const;
|
||||
Metadata readMetadataUnlocked(const String & path, std::shared_lock<std::shared_mutex> &) const;
|
||||
Metadata readUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
Metadata readUpdateStoreMetadataAndRemove(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata readOrCreateUpdateAndStoreMetadata(const String & path, WriteMode mode, bool sync, MetadataUpdater updater);
|
||||
|
||||
Metadata createAndStoreMetadata(const String & path, bool sync);
|
||||
Metadata createUpdateAndStoreMetadata(const String & path, bool sync, MetadataUpdater updater);
|
||||
|
||||
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
|
||||
|
||||
UInt64 getKeepingFreeSpace() const override { return 0; }
|
||||
|
||||
bool exists(const String & path) const override;
|
||||
|
||||
bool isFile(const String & path) const override;
|
||||
|
||||
void createFile(const String & path) override;
|
||||
|
||||
size_t getFileSize(const String & path) const override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path, bool should_send_metadata);
|
||||
|
||||
void replaceFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void removeFile(const String & path) override { removeSharedFile(path, false); }
|
||||
|
||||
void removeFileIfExists(const String & path) override { removeSharedFileIfExists(path, false); }
|
||||
|
||||
void removeRecursive(const String & path) override { removeSharedRecursive(path, false, {}); }
|
||||
|
||||
void removeSharedFile(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedFileIfExists(const String & path, bool delete_metadata_only) override;
|
||||
|
||||
void removeSharedRecursive(const String & path, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths);
|
||||
|
||||
DiskPtr getMetadataDiskIfExistsOrSelf() override { return metadata_disk; }
|
||||
|
||||
UInt32 getRefCount(const String & path) const override;
|
||||
|
||||
/// Return metadata for each file path. Also, before serialization reset
|
||||
/// ref_count for each metadata to zero. This function used only for remote
|
||||
/// fetches/sends in replicated engines. That's why we reset ref_count to zero.
|
||||
std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & file_paths) const override;
|
||||
|
||||
String getUniqueId(const String & path) const override;
|
||||
|
||||
bool checkObjectExists(const String & path) const;
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
void createHardLink(const String & src_path, const String & dst_path, bool should_send_metadata);
|
||||
|
||||
void listFiles(const String & path, std::vector<String> & file_names) override;
|
||||
|
||||
void setReadOnly(const String & path) override;
|
||||
|
||||
bool isDirectory(const String & path) const override;
|
||||
|
||||
void createDirectory(const String & path) override;
|
||||
|
||||
void createDirectories(const String & path) override;
|
||||
|
||||
void clearDirectory(const String & path) override;
|
||||
|
||||
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
|
||||
|
||||
void removeDirectory(const String & path) override;
|
||||
|
||||
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
|
||||
|
||||
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
|
||||
|
||||
Poco::Timestamp getLastModified(const String & path) override;
|
||||
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void startup(ContextPtr context) override;
|
||||
|
||||
ReservationPtr reserve(UInt64 bytes) override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context_, const String &, const DisksMap &) override;
|
||||
|
||||
void restoreMetadataIfNeeded(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context);
|
||||
|
||||
void onFreeze(const String & path) override;
|
||||
|
||||
void syncRevision(UInt64 revision) override;
|
||||
|
||||
UInt64 getRevision() const override;
|
||||
private:
|
||||
const String name;
|
||||
const String remote_fs_root_path;
|
||||
Poco::Logger * log;
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
const DiskType disk_type;
|
||||
ObjectStoragePtr object_storage;
|
||||
|
||||
UInt64 reserved_bytes = 0;
|
||||
UInt64 reservation_count = 0;
|
||||
std::mutex reservation_mutex;
|
||||
|
||||
mutable std::shared_mutex metadata_mutex;
|
||||
void removeMetadata(const String & path, std::vector<String> & paths_to_remove);
|
||||
|
||||
void removeMetadataRecursive(const String & path, std::unordered_map<String, std::vector<String>> & paths_to_remove);
|
||||
|
||||
std::optional<UInt64> tryReserve(UInt64 bytes);
|
||||
|
||||
bool send_metadata;
|
||||
|
||||
std::unique_ptr<DiskObjectStorageMetadataHelper> metadata_helper;
|
||||
};
|
||||
|
||||
class DiskObjectStorageReservation final : public IReservation
|
||||
{
|
||||
public:
|
||||
DiskObjectStorageReservation(const std::shared_ptr<DiskObjectStorage> & disk_, UInt64 size_)
|
||||
: disk(disk_)
|
||||
, size(size_)
|
||||
, metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
|
||||
{}
|
||||
|
||||
UInt64 getSize() const override { return size; }
|
||||
|
||||
UInt64 getUnreservedSpace() const override { return unreserved_space; }
|
||||
|
||||
DiskPtr getDisk(size_t i) const override;
|
||||
|
||||
Disks getDisks() const override { return {disk}; }
|
||||
|
||||
void update(UInt64 new_size) override;
|
||||
|
||||
~DiskObjectStorageReservation() override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<DiskObjectStorage> disk;
|
||||
UInt64 size;
|
||||
UInt64 unreserved_space;
|
||||
CurrentMetrics::Increment metric_increment;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
#include <Common/FileCache.h>
|
||||
@ -8,7 +8,8 @@ namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{extern const int BAD_ARGUMENTS;
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
@ -2,18 +2,22 @@
|
||||
|
||||
#include <random>
|
||||
#include <utility>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::shared_ptr<DiskCacheWrapper> wrapWithCache(
|
||||
std::shared_ptr<IDisk> disk, String cache_name, String cache_path, String metadata_path);
|
||||
std::shared_ptr<IDisk> disk,
|
||||
String cache_name,
|
||||
String cache_path,
|
||||
String metadata_path);
|
||||
|
||||
std::pair<String, DiskPtr> prepareForLocalMetadata(
|
||||
const String & name,
|
208
src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp
Normal file
208
src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp
Normal file
@ -0,0 +1,208 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadata.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
extern const int MEMORY_LIMIT_EXCEEDED;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
return result;
|
||||
}
|
||||
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
updater(result);
|
||||
result.save(sync);
|
||||
return result;
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, DiskObjectStorageMetadataUpdater updater)
|
||||
{
|
||||
DiskObjectStorageMetadata result(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
result.load();
|
||||
if (updater(result))
|
||||
result.save(sync);
|
||||
metadata_disk_->removeFile(metadata_file_path_);
|
||||
|
||||
return result;
|
||||
|
||||
}
|
||||
|
||||
DiskObjectStorageMetadata DiskObjectStorageMetadata::createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite)
|
||||
{
|
||||
if (overwrite || !metadata_disk_->exists(metadata_file_path_))
|
||||
{
|
||||
return createAndStoreMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_, sync);
|
||||
}
|
||||
else
|
||||
{
|
||||
auto result = readMetadata(remote_fs_root_path_, metadata_disk_, metadata_file_path_);
|
||||
if (result.read_only)
|
||||
throw Exception("File is read-only: " + metadata_file_path_, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadata::load()
|
||||
{
|
||||
try
|
||||
{
|
||||
const ReadSettings read_settings;
|
||||
auto buf = metadata_disk->readFile(metadata_file_path, read_settings, 1024); /* reasonable buffer size for small file */
|
||||
|
||||
UInt32 version;
|
||||
readIntText(version, *buf);
|
||||
|
||||
if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG)
|
||||
throw Exception(
|
||||
ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}",
|
||||
metadata_disk->getPath() + metadata_file_path, toString(version), toString(VERSION_READ_ONLY_FLAG));
|
||||
|
||||
assertChar('\n', *buf);
|
||||
|
||||
UInt32 remote_fs_objects_count;
|
||||
readIntText(remote_fs_objects_count, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readIntText(total_size, *buf);
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects.resize(remote_fs_objects_count);
|
||||
|
||||
for (size_t i = 0; i < remote_fs_objects_count; ++i)
|
||||
{
|
||||
String remote_fs_object_path;
|
||||
size_t remote_fs_object_size;
|
||||
readIntText(remote_fs_object_size, *buf);
|
||||
assertChar('\t', *buf);
|
||||
readEscapedString(remote_fs_object_path, *buf);
|
||||
if (version == VERSION_ABSOLUTE_PATHS)
|
||||
{
|
||||
if (!remote_fs_object_path.starts_with(remote_fs_root_path))
|
||||
throw Exception(ErrorCodes::UNKNOWN_FORMAT,
|
||||
"Path in metadata does not correspond to root path. Path: {}, root path: {}, disk path: {}",
|
||||
remote_fs_object_path, remote_fs_root_path, metadata_disk->getPath());
|
||||
|
||||
remote_fs_object_path = remote_fs_object_path.substr(remote_fs_root_path.size());
|
||||
}
|
||||
assertChar('\n', *buf);
|
||||
remote_fs_objects[i].relative_path = remote_fs_object_path;
|
||||
remote_fs_objects[i].bytes_size = remote_fs_object_size;
|
||||
}
|
||||
|
||||
readIntText(ref_count, *buf);
|
||||
assertChar('\n', *buf);
|
||||
|
||||
if (version >= VERSION_READ_ONLY_FLAG)
|
||||
{
|
||||
readBoolText(read_only, *buf);
|
||||
assertChar('\n', *buf);
|
||||
}
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
|
||||
if (e.code() == ErrorCodes::UNKNOWN_FORMAT)
|
||||
throw;
|
||||
|
||||
if (e.code() == ErrorCodes::MEMORY_LIMIT_EXCEEDED)
|
||||
throw;
|
||||
|
||||
throw Exception("Failed to read metadata file: " + metadata_file_path, ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
|
||||
/// Load metadata by path or create empty if `create` flag is set.
|
||||
DiskObjectStorageMetadata::DiskObjectStorageMetadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_)
|
||||
: remote_fs_root_path(remote_fs_root_path_)
|
||||
, metadata_file_path(metadata_file_path_)
|
||||
, metadata_disk(metadata_disk_)
|
||||
, total_size(0), ref_count(0)
|
||||
{
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadata::addObject(const String & path, size_t size)
|
||||
{
|
||||
total_size += size;
|
||||
remote_fs_objects.emplace_back(path, size);
|
||||
}
|
||||
|
||||
|
||||
void DiskObjectStorageMetadata::saveToBuffer(WriteBuffer & buf, bool sync)
|
||||
{
|
||||
writeIntText(VERSION_RELATIVE_PATHS, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeIntText(remote_fs_objects.size(), buf);
|
||||
writeChar('\t', buf);
|
||||
writeIntText(total_size, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
for (const auto & [remote_fs_object_path, remote_fs_object_size] : remote_fs_objects)
|
||||
{
|
||||
writeIntText(remote_fs_object_size, buf);
|
||||
writeChar('\t', buf);
|
||||
writeEscapedString(remote_fs_object_path, buf);
|
||||
writeChar('\n', buf);
|
||||
}
|
||||
|
||||
writeIntText(ref_count, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
writeBoolText(read_only, buf);
|
||||
writeChar('\n', buf);
|
||||
|
||||
buf.finalize();
|
||||
if (sync)
|
||||
buf.sync();
|
||||
|
||||
}
|
||||
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void DiskObjectStorageMetadata::save(bool sync)
|
||||
{
|
||||
auto buf = metadata_disk->writeFile(metadata_file_path, 1024);
|
||||
saveToBuffer(*buf, sync);
|
||||
}
|
||||
|
||||
std::string DiskObjectStorageMetadata::serializeToString()
|
||||
{
|
||||
WriteBufferFromOwnString write_buf;
|
||||
saveToBuffer(write_buf, false);
|
||||
return write_buf.str();
|
||||
}
|
||||
|
||||
|
||||
}
|
68
src/Disks/ObjectStorages/DiskObjectStorageMetadata.h
Normal file
68
src/Disks/ObjectStorages/DiskObjectStorageMetadata.h
Normal file
@ -0,0 +1,68 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Metadata for DiskObjectStorage, stored on local disk
|
||||
struct DiskObjectStorageMetadata
|
||||
{
|
||||
using Updater = std::function<bool(DiskObjectStorageMetadata & metadata)>;
|
||||
/// Metadata file version.
|
||||
static constexpr UInt32 VERSION_ABSOLUTE_PATHS = 1;
|
||||
static constexpr UInt32 VERSION_RELATIVE_PATHS = 2;
|
||||
static constexpr UInt32 VERSION_READ_ONLY_FLAG = 3;
|
||||
|
||||
/// Remote FS objects paths and their sizes.
|
||||
std::vector<BlobPathWithSize> remote_fs_objects;
|
||||
|
||||
/// URI
|
||||
const String & remote_fs_root_path;
|
||||
|
||||
/// Relative path to metadata file on local FS.
|
||||
const String metadata_file_path;
|
||||
|
||||
DiskPtr metadata_disk;
|
||||
|
||||
/// Total size of all remote FS (S3, HDFS) objects.
|
||||
size_t total_size = 0;
|
||||
|
||||
/// Number of references (hardlinks) to this metadata file.
|
||||
///
|
||||
/// FIXME: Why we are tracking it explicetly, without
|
||||
/// info from filesystem????
|
||||
UInt32 ref_count = 0;
|
||||
|
||||
/// Flag indicates that file is read only.
|
||||
bool read_only = false;
|
||||
|
||||
DiskObjectStorageMetadata(
|
||||
const String & remote_fs_root_path_,
|
||||
DiskPtr metadata_disk_,
|
||||
const String & metadata_file_path_);
|
||||
|
||||
void addObject(const String & path, size_t size);
|
||||
|
||||
static DiskObjectStorageMetadata readMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_);
|
||||
static DiskObjectStorageMetadata readUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static DiskObjectStorageMetadata readUpdateStoreMetadataAndRemove(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
|
||||
static DiskObjectStorageMetadata createAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync);
|
||||
static DiskObjectStorageMetadata createUpdateAndStoreMetadata(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, Updater updater);
|
||||
static DiskObjectStorageMetadata createAndStoreMetadataIfNotExists(const String & remote_fs_root_path_, DiskPtr metadata_disk_, const String & metadata_file_path_, bool sync, bool overwrite);
|
||||
|
||||
/// Serialize metadata to string (very same with saveToBuffer)
|
||||
std::string serializeToString();
|
||||
|
||||
private:
|
||||
/// Fsync metadata file if 'sync' flag is set.
|
||||
void save(bool sync = false);
|
||||
void saveToBuffer(WriteBuffer & buffer, bool sync);
|
||||
void load();
|
||||
};
|
||||
|
||||
using DiskObjectStorageMetadataUpdater = std::function<bool(DiskObjectStorageMetadata & metadata)>;
|
||||
|
||||
}
|
571
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.cpp
Normal file
571
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.cpp
Normal file
@ -0,0 +1,571 @@
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
static String revisionToString(UInt64 revision)
|
||||
{
|
||||
return std::bitset<64>(revision).to_string();
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const
|
||||
{
|
||||
const String path = disk->remote_fs_root_path + "operations/r" + revisionToString(revision) + operation_log_suffix + "-" + operation_name;
|
||||
auto buf = disk->object_storage->writeObject(path, WriteMode::Rewrite, metadata);
|
||||
buf->write('0');
|
||||
buf->finalize();
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::findLastRevision()
|
||||
{
|
||||
/// Construct revision number from high to low bits.
|
||||
String revision;
|
||||
revision.reserve(64);
|
||||
for (int bit = 0; bit < 64; ++bit)
|
||||
{
|
||||
auto revision_prefix = revision + "1";
|
||||
|
||||
LOG_TRACE(disk->log, "Check object exists with revision prefix {}", revision_prefix);
|
||||
|
||||
/// Check file or operation with such revision prefix exists.
|
||||
if (disk->object_storage->exists(disk->remote_fs_root_path + "r" + revision_prefix)
|
||||
|| disk->object_storage->exists(disk->remote_fs_root_path + "operations/r" + revision_prefix))
|
||||
revision += "1";
|
||||
else
|
||||
revision += "0";
|
||||
}
|
||||
revision_counter = static_cast<UInt64>(std::bitset<64>(revision).to_ullong());
|
||||
LOG_INFO(disk->log, "Found last revision number {} for disk {}", revision_counter, disk->name);
|
||||
}
|
||||
|
||||
int DiskObjectStorageMetadataHelper::readSchemaVersion(IObjectStorage * object_storage, const String & source_path)
|
||||
{
|
||||
const std::string path = source_path + SCHEMA_VERSION_OBJECT;
|
||||
int version = 0;
|
||||
if (!object_storage->exists(path))
|
||||
return version;
|
||||
|
||||
auto buf = object_storage->readObject(path);
|
||||
readIntText(version, *buf);
|
||||
|
||||
return version;
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::saveSchemaVersion(const int & version) const
|
||||
{
|
||||
auto path = disk->remote_fs_root_path + SCHEMA_VERSION_OBJECT;
|
||||
|
||||
auto buf = disk->object_storage->writeObject(path, WriteMode::Rewrite);
|
||||
writeIntText(version, *buf);
|
||||
buf->finalize();
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const
|
||||
{
|
||||
disk->object_storage->copyObject(key, key, metadata);
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::migrateFileToRestorableSchema(const String & path) const
|
||||
{
|
||||
LOG_TRACE(disk->log, "Migrate file {} to restorable schema", disk->metadata_disk->getPath() + path);
|
||||
|
||||
auto meta = disk->readMetadata(path);
|
||||
|
||||
for (const auto & [key, _] : meta.remote_fs_objects)
|
||||
{
|
||||
ObjectAttributes metadata {
|
||||
{"path", path}
|
||||
};
|
||||
updateObjectMetadata(disk->remote_fs_root_path + key, metadata);
|
||||
}
|
||||
}
|
||||
void DiskObjectStorageMetadataHelper::migrateToRestorableSchemaRecursive(const String & path, Futures & results)
|
||||
{
|
||||
checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks.
|
||||
|
||||
LOG_TRACE(disk->log, "Migrate directory {} to restorable schema", disk->metadata_disk->getPath() + path);
|
||||
|
||||
bool dir_contains_only_files = true;
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
{
|
||||
if (disk->isDirectory(it->path()))
|
||||
{
|
||||
dir_contains_only_files = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// The whole directory can be migrated asynchronously.
|
||||
if (dir_contains_only_files)
|
||||
{
|
||||
auto result = disk->getExecutor().execute([this, path]
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
migrateFileToRestorableSchema(it->path());
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
else
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(path); it->isValid(); it->next())
|
||||
if (!disk->isDirectory(it->path()))
|
||||
{
|
||||
auto source_path = it->path();
|
||||
auto result = disk->getExecutor().execute([this, source_path]
|
||||
{
|
||||
migrateFileToRestorableSchema(source_path);
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
else
|
||||
migrateToRestorableSchemaRecursive(it->path(), results);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::migrateToRestorableSchema()
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_INFO(disk->log, "Start migration to restorable schema for disk {}", disk->name);
|
||||
|
||||
Futures results;
|
||||
|
||||
for (const auto & root : data_roots)
|
||||
if (disk->exists(root))
|
||||
migrateToRestorableSchemaRecursive(root + '/', results);
|
||||
|
||||
for (auto & result : results)
|
||||
result.wait();
|
||||
for (auto & result : results)
|
||||
result.get();
|
||||
|
||||
saveSchemaVersion(RESTORABLE_SCHEMA_VERSION);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, fmt::format("Failed to migrate to restorable schema for disk {}", disk->name));
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
LOG_INFO(disk->log, "Restore operation for disk {} called", disk->name);
|
||||
|
||||
if (!disk->exists(RESTORE_FILE_NAME))
|
||||
{
|
||||
LOG_INFO(disk->log, "No restore file '{}' exists, finishing restore", RESTORE_FILE_NAME);
|
||||
return;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
RestoreInformation information;
|
||||
information.source_path = disk->remote_fs_root_path;
|
||||
information.source_namespace = disk->object_storage->getObjectsNamespace();
|
||||
|
||||
readRestoreInformation(information);
|
||||
if (information.revision == 0)
|
||||
information.revision = LATEST_REVISION;
|
||||
if (!information.source_path.ends_with('/'))
|
||||
information.source_path += '/';
|
||||
|
||||
IObjectStorage * source_object_storage = disk->object_storage.get();
|
||||
if (information.source_namespace == disk->object_storage->getObjectsNamespace())
|
||||
{
|
||||
/// In this case we need to additionally cleanup S3 from objects with later revision.
|
||||
/// Will be simply just restore to different path.
|
||||
if (information.source_path == disk->remote_fs_root_path && information.revision != LATEST_REVISION)
|
||||
throw Exception("Restoring to the same bucket and path is allowed if revision is latest (0)", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
/// This case complicates S3 cleanup in case of unsuccessful restore.
|
||||
if (information.source_path != disk->remote_fs_root_path && disk->remote_fs_root_path.starts_with(information.source_path))
|
||||
throw Exception("Restoring to the same bucket is allowed only if source path is not a sub-path of configured path in S3 disk", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else
|
||||
{
|
||||
object_storage_from_another_namespace = disk->object_storage->cloneObjectStorage(information.source_namespace, config, config_prefix, context);
|
||||
source_object_storage = object_storage_from_another_namespace.get();
|
||||
}
|
||||
|
||||
LOG_INFO(disk->log, "Starting to restore disk {}. Revision: {}, Source path: {}",
|
||||
disk->name, information.revision, information.source_path);
|
||||
|
||||
if (readSchemaVersion(source_object_storage, information.source_path) < RESTORABLE_SCHEMA_VERSION)
|
||||
throw Exception("Source bucket doesn't have restorable schema.", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
LOG_INFO(disk->log, "Removing old metadata...");
|
||||
|
||||
bool cleanup_s3 = information.source_path != disk->remote_fs_root_path;
|
||||
for (const auto & root : data_roots)
|
||||
if (disk->exists(root))
|
||||
disk->removeSharedRecursive(root + '/', !cleanup_s3, {});
|
||||
|
||||
LOG_INFO(disk->log, "Old metadata removed, restoring new one");
|
||||
restoreFiles(source_object_storage, information);
|
||||
restoreFileOperations(source_object_storage, information);
|
||||
|
||||
disk->metadata_disk->removeFile(RESTORE_FILE_NAME);
|
||||
|
||||
saveSchemaVersion(RESTORABLE_SCHEMA_VERSION);
|
||||
|
||||
LOG_INFO(disk->log, "Restore disk {} finished", disk->name);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, fmt::format("Failed to restore disk {}", disk->name));
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::readRestoreInformation(RestoreInformation & restore_information) /// NOLINT
|
||||
{
|
||||
auto buffer = disk->metadata_disk->readFile(RESTORE_FILE_NAME, ReadSettings{}, 512);
|
||||
buffer->next();
|
||||
|
||||
try
|
||||
{
|
||||
std::map<String, String> properties;
|
||||
|
||||
while (buffer->hasPendingData())
|
||||
{
|
||||
String property;
|
||||
readText(property, *buffer);
|
||||
assertChar('\n', *buffer);
|
||||
|
||||
auto pos = property.find('=');
|
||||
if (pos == std::string::npos || pos == 0 || pos == property.length())
|
||||
throw Exception(fmt::format("Invalid property {} in restore file", property), ErrorCodes::UNKNOWN_FORMAT);
|
||||
|
||||
auto key = property.substr(0, pos);
|
||||
auto value = property.substr(pos + 1);
|
||||
|
||||
auto it = properties.find(key);
|
||||
if (it != properties.end())
|
||||
throw Exception(fmt::format("Property key duplication {} in restore file", key), ErrorCodes::UNKNOWN_FORMAT);
|
||||
|
||||
properties[key] = value;
|
||||
}
|
||||
|
||||
for (const auto & [key, value] : properties)
|
||||
{
|
||||
ReadBufferFromString value_buffer(value);
|
||||
|
||||
if (key == "revision")
|
||||
readIntText(restore_information.revision, value_buffer);
|
||||
else if (key == "source_bucket" || key == "source_namespace")
|
||||
readText(restore_information.source_namespace, value_buffer);
|
||||
else if (key == "source_path")
|
||||
readText(restore_information.source_path, value_buffer);
|
||||
else if (key == "detached")
|
||||
readBoolTextWord(restore_information.detached, value_buffer);
|
||||
else
|
||||
throw Exception(fmt::format("Unknown key {} in restore file", key), ErrorCodes::UNKNOWN_FORMAT);
|
||||
}
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
tryLogCurrentException(disk->log, "Failed to read restore information");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
static String shrinkKey(const String & path, const String & key)
|
||||
{
|
||||
if (!key.starts_with(path))
|
||||
throw Exception("The key " + key + " prefix mismatch with given " + path, ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return key.substr(path.length());
|
||||
}
|
||||
|
||||
static std::tuple<UInt64, String> extractRevisionAndOperationFromKey(const String & key)
|
||||
{
|
||||
String revision_str;
|
||||
String suffix;
|
||||
String operation;
|
||||
/// Key has format: ../../r{revision}(-{hostname})-{operation}
|
||||
static const re2::RE2 key_regexp{R"(.*/r(\d+)(-[\w\d\-\.]+)?-(\w+)$)"};
|
||||
|
||||
re2::RE2::FullMatch(key, key_regexp, &revision_str, &suffix, &operation);
|
||||
|
||||
return {(revision_str.empty() ? 0 : static_cast<UInt64>(std::bitset<64>(revision_str).to_ullong())), operation};
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata)
|
||||
{
|
||||
if (disk->exists(to_path))
|
||||
{
|
||||
if (send_metadata)
|
||||
{
|
||||
auto revision = ++revision_counter;
|
||||
const ObjectAttributes object_metadata {
|
||||
{"from_path", from_path},
|
||||
{"to_path", to_path}
|
||||
};
|
||||
createFileOperationObject("rename", revision, object_metadata);
|
||||
}
|
||||
if (disk->isDirectory(from_path))
|
||||
{
|
||||
for (auto it = disk->iterateDirectory(from_path); it->isValid(); it->next())
|
||||
moveRecursiveOrRemove(it->path(), fs::path(to_path) / it->name(), false);
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->removeFile(from_path);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
disk->moveFile(from_path, to_path, send_metadata);
|
||||
}
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information)
|
||||
{
|
||||
LOG_INFO(disk->log, "Starting restore files for disk {}", disk->name);
|
||||
|
||||
std::vector<std::future<void>> results;
|
||||
auto restore_files = [this, &source_object_storage, &restore_information, &results](const BlobsPathToSize & keys)
|
||||
{
|
||||
std::vector<String> keys_names;
|
||||
for (const auto & [key, size] : keys)
|
||||
{
|
||||
|
||||
LOG_INFO(disk->log, "Calling restore for key for disk {}", key);
|
||||
|
||||
/// Skip file operations objects. They will be processed separately.
|
||||
if (key.find("/operations/") != String::npos)
|
||||
continue;
|
||||
|
||||
const auto [revision, _] = extractRevisionAndOperationFromKey(key);
|
||||
/// Filter early if it's possible to get revision from key.
|
||||
if (revision > restore_information.revision)
|
||||
continue;
|
||||
|
||||
keys_names.push_back(key);
|
||||
}
|
||||
|
||||
if (!keys_names.empty())
|
||||
{
|
||||
auto result = disk->getExecutor().execute([this, &source_object_storage, &restore_information, keys_names]()
|
||||
{
|
||||
processRestoreFiles(source_object_storage, restore_information.source_path, keys_names);
|
||||
});
|
||||
|
||||
results.push_back(std::move(result));
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path, children);
|
||||
|
||||
restore_files(children);
|
||||
|
||||
for (auto & result : results)
|
||||
result.wait();
|
||||
for (auto & result : results)
|
||||
result.get();
|
||||
|
||||
LOG_INFO(disk->log, "Files are restored for disk {}", disk->name);
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::processRestoreFiles(IObjectStorage * source_object_storage, const String & source_path, const std::vector<String> & keys) const
|
||||
{
|
||||
for (const auto & key : keys)
|
||||
{
|
||||
auto meta = source_object_storage->getObjectMetadata(key);
|
||||
auto object_attributes = meta.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())
|
||||
{
|
||||
/// 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
|
||||
continue;
|
||||
|
||||
disk->createDirectories(directoryPath(path));
|
||||
auto relative_key = shrinkKey(source_path, key);
|
||||
|
||||
/// Copy object if we restore to different bucket / path.
|
||||
if (source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->remote_fs_root_path != source_path)
|
||||
source_object_storage->copyObjectToAnotherObjectStorage(key, disk->remote_fs_root_path + relative_key, *disk->object_storage);
|
||||
|
||||
auto updater = [relative_key, meta] (DiskObjectStorage::Metadata & metadata)
|
||||
{
|
||||
metadata.addObject(relative_key, meta.size_bytes);
|
||||
return true;
|
||||
};
|
||||
|
||||
disk->createUpdateAndStoreMetadata(path, false, updater);
|
||||
|
||||
LOG_TRACE(disk->log, "Restored file {}", path);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void DiskObjectStorage::onFreeze(const String & path)
|
||||
{
|
||||
createDirectories(path);
|
||||
auto revision_file_buf = metadata_disk->writeFile(path + "revision.txt", 32);
|
||||
writeIntText(metadata_helper->revision_counter.load(), *revision_file_buf);
|
||||
revision_file_buf->finalize();
|
||||
}
|
||||
|
||||
static String pathToDetached(const String & source_path)
|
||||
{
|
||||
if (source_path.ends_with('/'))
|
||||
return fs::path(source_path).parent_path().parent_path() / "detached/";
|
||||
return fs::path(source_path).parent_path() / "detached/";
|
||||
}
|
||||
|
||||
void DiskObjectStorageMetadataHelper::restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information)
|
||||
{
|
||||
/// Enable recording file operations if we restore to different bucket / path.
|
||||
bool send_metadata = source_object_storage->getObjectsNamespace() != disk->object_storage->getObjectsNamespace() || disk->remote_fs_root_path != restore_information.source_path;
|
||||
|
||||
std::set<String> renames;
|
||||
auto restore_file_operations = [this, &source_object_storage, &restore_information, &renames, &send_metadata](const BlobsPathToSize & keys)
|
||||
{
|
||||
const String rename = "rename";
|
||||
const String hardlink = "hardlink";
|
||||
|
||||
for (const auto & [key, _]: keys)
|
||||
{
|
||||
const auto [revision, operation] = extractRevisionAndOperationFromKey(key);
|
||||
if (revision == UNKNOWN_REVISION)
|
||||
{
|
||||
LOG_WARNING(disk->log, "Skip key {} with unknown revision", key);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// S3 ensures that keys will be listed in ascending UTF-8 bytes order (revision order).
|
||||
/// We can stop processing if revision of the object is already more than required.
|
||||
if (revision > restore_information.revision)
|
||||
return false;
|
||||
|
||||
/// Keep original revision if restore to different bucket / path.
|
||||
if (send_metadata)
|
||||
revision_counter = revision - 1;
|
||||
|
||||
auto object_attributes = *(source_object_storage->getObjectMetadata(key).attributes);
|
||||
if (operation == rename)
|
||||
{
|
||||
auto from_path = object_attributes["from_path"];
|
||||
auto to_path = object_attributes["to_path"];
|
||||
if (disk->exists(from_path))
|
||||
{
|
||||
moveRecursiveOrRemove(from_path, to_path, send_metadata);
|
||||
|
||||
LOG_TRACE(disk->log, "Revision {}. Restored rename {} -> {}", revision, from_path, to_path);
|
||||
|
||||
if (restore_information.detached && disk->isDirectory(to_path))
|
||||
{
|
||||
/// Sometimes directory paths are passed without trailing '/'. We should keep them in one consistent way.
|
||||
if (!from_path.ends_with('/'))
|
||||
from_path += '/';
|
||||
if (!to_path.ends_with('/'))
|
||||
to_path += '/';
|
||||
|
||||
/// Always keep latest actual directory path to avoid 'detaching' not existing paths.
|
||||
auto it = renames.find(from_path);
|
||||
if (it != renames.end())
|
||||
renames.erase(it);
|
||||
|
||||
renames.insert(to_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (operation == hardlink)
|
||||
{
|
||||
auto src_path = object_attributes["src_path"];
|
||||
auto dst_path = object_attributes["dst_path"];
|
||||
if (disk->exists(src_path))
|
||||
{
|
||||
disk->createDirectories(directoryPath(dst_path));
|
||||
disk->createHardLink(src_path, dst_path, send_metadata);
|
||||
LOG_TRACE(disk->log, "Revision {}. Restored hardlink {} -> {}", revision, src_path, dst_path);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
};
|
||||
|
||||
BlobsPathToSize children;
|
||||
source_object_storage->listPrefix(restore_information.source_path + "operations/", children);
|
||||
restore_file_operations(children);
|
||||
|
||||
if (restore_information.detached)
|
||||
{
|
||||
Strings not_finished_prefixes{"tmp_", "delete_tmp_", "attaching_", "deleting_"};
|
||||
|
||||
for (const auto & path : renames)
|
||||
{
|
||||
/// Skip already detached parts.
|
||||
if (path.find("/detached/") != std::string::npos)
|
||||
continue;
|
||||
|
||||
/// Skip not finished parts. They shouldn't be in 'detached' directory, because CH wouldn't be able to finish processing them.
|
||||
fs::path directory_path(path);
|
||||
auto directory_name = directory_path.parent_path().filename().string();
|
||||
|
||||
auto predicate = [&directory_name](String & prefix) { return directory_name.starts_with(prefix); };
|
||||
if (std::any_of(not_finished_prefixes.begin(), not_finished_prefixes.end(), predicate))
|
||||
continue;
|
||||
|
||||
auto detached_path = pathToDetached(path);
|
||||
|
||||
LOG_TRACE(disk->log, "Move directory to 'detached' {} -> {}", path, detached_path);
|
||||
|
||||
fs::path from_path = fs::path(path);
|
||||
fs::path to_path = fs::path(detached_path);
|
||||
if (path.ends_with('/'))
|
||||
to_path /= from_path.parent_path().filename();
|
||||
else
|
||||
to_path /= from_path.filename();
|
||||
|
||||
/// to_path may exist and non-empty in case for example abrupt restart, so remove it before rename
|
||||
if (disk->metadata_disk->exists(to_path))
|
||||
disk->metadata_disk->removeRecursive(to_path);
|
||||
|
||||
disk->createDirectories(directoryPath(to_path));
|
||||
disk->metadata_disk->moveDirectory(from_path, to_path);
|
||||
}
|
||||
}
|
||||
|
||||
LOG_INFO(disk->log, "File operations restored for disk {}", disk->name);
|
||||
}
|
||||
|
||||
}
|
100
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h
Normal file
100
src/Disks/ObjectStorages/DiskObjectStorageMetadataHelper.h
Normal file
@ -0,0 +1,100 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DiskObjectStorage;
|
||||
|
||||
/// Class implements storage of ObjectStorage metadata inside object storage itself,
|
||||
/// so it's possible to recover from this remote information in case of local disk loss.
|
||||
///
|
||||
/// This mechanism can be enabled with `<send_metadata>true</send_metadata>` option inside
|
||||
/// disk configuration. Implemented only for S3 and Azure Blob storage. Other object storages
|
||||
/// don't support metadata for blobs.
|
||||
///
|
||||
/// FIXME: this class is very intrusive and use a lot of DiskObjectStorage internals.
|
||||
/// FIXME: it's very complex and unreliable, need to implement something better.
|
||||
class DiskObjectStorageMetadataHelper
|
||||
{
|
||||
public:
|
||||
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
|
||||
static constexpr UInt64 UNKNOWN_REVISION = 0;
|
||||
|
||||
DiskObjectStorageMetadataHelper(DiskObjectStorage * disk_, ReadSettings read_settings_)
|
||||
: disk(disk_)
|
||||
, read_settings(std::move(read_settings_))
|
||||
, operation_log_suffix("-" + getFQDNOrHostName())
|
||||
{
|
||||
}
|
||||
|
||||
/// Most important method, called on DiskObjectStorage startup
|
||||
void restore(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context);
|
||||
|
||||
void syncRevision(UInt64 revision)
|
||||
{
|
||||
UInt64 local_revision = revision_counter.load();
|
||||
while ((revision > local_revision) && revision_counter.compare_exchange_weak(local_revision, revision));
|
||||
}
|
||||
|
||||
UInt64 getRevision() const
|
||||
{
|
||||
return revision_counter.load();
|
||||
}
|
||||
|
||||
static int readSchemaVersion(IObjectStorage * object_storage, const String & source_path);
|
||||
|
||||
void migrateToRestorableSchema();
|
||||
|
||||
void findLastRevision();
|
||||
|
||||
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectAttributes & metadata) const;
|
||||
|
||||
/// Version with possibility to backup-restore metadata.
|
||||
static constexpr int RESTORABLE_SCHEMA_VERSION = 1;
|
||||
|
||||
std::atomic<UInt64> revision_counter = 0;
|
||||
private:
|
||||
struct RestoreInformation
|
||||
{
|
||||
UInt64 revision = LATEST_REVISION;
|
||||
String source_namespace;
|
||||
String source_path;
|
||||
bool detached = false;
|
||||
};
|
||||
|
||||
using Futures = std::vector<std::future<void>>;
|
||||
|
||||
/// Move file or files in directory when possible and remove files in other case
|
||||
/// to restore by S3 operation log with same operations from different replicas
|
||||
void moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata);
|
||||
|
||||
void saveSchemaVersion(const int & version) const;
|
||||
void updateObjectMetadata(const String & key, const ObjectAttributes & metadata) const;
|
||||
void migrateFileToRestorableSchema(const String & path) const;
|
||||
void migrateToRestorableSchemaRecursive(const String & path, Futures & results);
|
||||
|
||||
void readRestoreInformation(RestoreInformation & restore_information);
|
||||
void restoreFiles(IObjectStorage * source_object_storage, const RestoreInformation & restore_information);
|
||||
void processRestoreFiles(IObjectStorage * source_object_storage, const String & source_path, const std::vector<String> & keys) const;
|
||||
void restoreFileOperations(IObjectStorage * source_object_storage, const RestoreInformation & restore_information);
|
||||
|
||||
inline static const String RESTORE_FILE_NAME = "restore";
|
||||
|
||||
/// Object contains information about schema version.
|
||||
inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION";
|
||||
/// Directories with data.
|
||||
const std::vector<String> data_roots {"data", "store"};
|
||||
|
||||
DiskObjectStorage * disk;
|
||||
|
||||
ObjectStoragePtr object_storage_from_another_namespace;
|
||||
|
||||
ReadSettings read_settings;
|
||||
|
||||
String operation_log_suffix;
|
||||
};
|
||||
|
||||
}
|
156
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp
Normal file
156
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp
Normal file
@ -0,0 +1,156 @@
|
||||
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
||||
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
#include <Storages/HDFS/WriteBufferFromHDFS.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
#include <Storages/HDFS/ReadBufferFromHDFS.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int HDFS_ERROR;
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::shutdown()
|
||||
{
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::startup()
|
||||
{
|
||||
}
|
||||
|
||||
bool HDFSObjectStorage::exists(const std::string & hdfs_uri) const
|
||||
{
|
||||
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
|
||||
const String remote_fs_object_path = hdfs_uri.substr(begin_of_path);
|
||||
return (0 == hdfsExists(hdfs_fs.get(), remote_fs_object_path.c_str()));
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> HDFSObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
return std::make_unique<ReadBufferFromHDFS>(path, path, config, read_settings.remote_fs_buffer_size);
|
||||
}
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> HDFSObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto hdfs_impl = std::make_unique<ReadBufferFromHDFSGather>(config, common_path_prefix, common_path_prefix, blobs_to_read, read_settings);
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(hdfs_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings->min_bytes_for_seek);
|
||||
}
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> HDFSObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings &)
|
||||
{
|
||||
if (attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
/// Single O_WRONLY in libhdfs adds O_TRUNC
|
||||
auto hdfs_buffer = std::make_unique<WriteBufferFromHDFS>(
|
||||
path, config, settings->replication, buf_size,
|
||||
mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(hdfs_buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
|
||||
void HDFSObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
const size_t begin_of_path = path.find('/', path.find("//") + 2);
|
||||
int32_t num_entries;
|
||||
auto * files_list = hdfsListDirectory(hdfs_fs.get(), path.substr(begin_of_path).c_str(), &num_entries);
|
||||
if (num_entries == -1)
|
||||
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path);
|
||||
|
||||
for (int32_t i = 0; i < num_entries; ++i)
|
||||
children.emplace_back(files_list[i].mName, files_list[i].mSize);
|
||||
}
|
||||
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void HDFSObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
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);
|
||||
if (res == -1)
|
||||
throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path);
|
||||
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObject(hdfs_path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
if (exists(path))
|
||||
removeObject(path);
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
for (const auto & hdfs_path : paths)
|
||||
removeObjectIfExists(hdfs_path);
|
||||
}
|
||||
|
||||
ObjectMetadata HDFSObjectStorage::getObjectMetadata(const std::string &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
}
|
||||
|
||||
void HDFSObjectStorage::copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes)
|
||||
{
|
||||
if (object_to_attributes.has_value())
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "HDFS API doesn't support custom attributes/metadata for stored objects");
|
||||
|
||||
auto in = readObject(object_from);
|
||||
auto out = writeObject(object_to, WriteMode::Rewrite);
|
||||
copyData(*in, *out);
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
|
||||
void HDFSObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration &, const std::string &, ContextPtr)
|
||||
{
|
||||
}
|
||||
|
||||
std::unique_ptr<IObjectStorage> 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");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
119
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
Normal file
119
src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h
Normal file
@ -0,0 +1,119 @@
|
||||
#pragma once
|
||||
#include <Common/config.h>
|
||||
|
||||
|
||||
#if USE_HDFS
|
||||
|
||||
#include <Disks/IDisk.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
#include <Core/UUID.h>
|
||||
#include <memory>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
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_)
|
||||
: min_bytes_for_seek(min_bytes_for_seek_)
|
||||
, objects_chunk_size_to_delete(objects_chunk_size_to_delete_)
|
||||
, replication(replication_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
class HDFSObjectStorage : public IObjectStorage
|
||||
{
|
||||
public:
|
||||
|
||||
using SettingsPtr = std::unique_ptr<HDFSObjectStorageSettings>;
|
||||
|
||||
HDFSObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
const String & hdfs_root_path_,
|
||||
SettingsPtr settings_,
|
||||
const Poco::Util::AbstractConfiguration & config_)
|
||||
: IObjectStorage(std::move(cache_))
|
||||
, config(config_)
|
||||
, hdfs_builder(createHDFSBuilder(hdfs_root_path_, config))
|
||||
, hdfs_fs(createHDFSFS(hdfs_builder.get()))
|
||||
, settings(std::move(settings_))
|
||||
{}
|
||||
|
||||
bool exists(const std::string & hdfs_uri) const override;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
/// Remove file. Throws exception if file doesn't exists or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
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<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
private:
|
||||
const Poco::Util::AbstractConfiguration & config;
|
||||
|
||||
HDFSBuilderWrapper hdfs_builder;
|
||||
HDFSFSPtr hdfs_fs;
|
||||
|
||||
SettingsPtr settings;
|
||||
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
54
src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp
Normal file
54
src/Disks/ObjectStorages/HDFS/registerDiskHDFS.cpp
Normal file
@ -0,0 +1,54 @@
|
||||
#include <Disks/ObjectStorages/HDFS/HDFSObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
#include <Storages/HDFS/HDFSCommon.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
void registerDiskHDFS(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context_,
|
||||
const DisksMap & /*map*/) -> DiskPtr
|
||||
{
|
||||
String uri{config.getString(config_prefix + ".endpoint")};
|
||||
checkHDFSURL(uri);
|
||||
|
||||
if (uri.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri);
|
||||
|
||||
std::unique_ptr<HDFSObjectStorageSettings> settings = std::make_unique<HDFSObjectStorageSettings>(
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000),
|
||||
context_->getSettingsRef().hdfs_replication
|
||||
);
|
||||
/// FIXME Cache currently unsupported :(
|
||||
ObjectStoragePtr hdfs_storage = std::make_unique<HDFSObjectStorage>(nullptr, uri, std::move(settings), config);
|
||||
|
||||
auto metadata_disk = prepareForLocalMetadata(name, config, config_prefix, context_).second;
|
||||
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
|
||||
|
||||
return std::make_shared<DiskObjectStorage>(
|
||||
name,
|
||||
uri,
|
||||
"DiskHDFS",
|
||||
metadata_disk,
|
||||
std::move(hdfs_storage),
|
||||
DiskType::HDFS,
|
||||
/* send_metadata = */ false,
|
||||
copy_thread_pool_size);
|
||||
};
|
||||
|
||||
factory.registerDiskType("hdfs", creator);
|
||||
}
|
||||
|
||||
}
|
48
src/Disks/ObjectStorages/IObjectStorage.cpp
Normal file
48
src/Disks/ObjectStorages/IObjectStorage.cpp
Normal file
@ -0,0 +1,48 @@
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <IO/copyData.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
AsynchronousReaderPtr IObjectStorage::getThreadPoolReader()
|
||||
{
|
||||
constexpr size_t pool_size = 50;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static AsynchronousReaderPtr reader = std::make_shared<ThreadPoolRemoteFSReader>(pool_size, queue_size);
|
||||
return reader;
|
||||
}
|
||||
|
||||
ThreadPool & IObjectStorage::getThreadPoolWriter()
|
||||
{
|
||||
constexpr size_t pool_size = 100;
|
||||
constexpr size_t queue_size = 1000000;
|
||||
static ThreadPool writer(pool_size, pool_size, queue_size);
|
||||
return writer;
|
||||
}
|
||||
|
||||
std::string IObjectStorage::getCacheBasePath() const
|
||||
{
|
||||
return cache ? cache->getBasePath() : "";
|
||||
}
|
||||
|
||||
void IObjectStorage::removeFromCache(const std::string & path)
|
||||
{
|
||||
if (cache)
|
||||
{
|
||||
auto key = cache->hash(path);
|
||||
cache->remove(key);
|
||||
}
|
||||
}
|
||||
|
||||
void IObjectStorage::copyObjectToAnotherObjectStorage(const std::string & object_from, const std::string & object_to, IObjectStorage & object_storage_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
if (&object_storage_to == this)
|
||||
copyObject(object_from, object_to, object_to_attributes);
|
||||
|
||||
auto in = readObject(object_from);
|
||||
auto out = object_storage_to.writeObject(object_to, WriteMode::Rewrite);
|
||||
copyData(*in, *out);
|
||||
out->finalize();
|
||||
}
|
||||
|
||||
}
|
159
src/Disks/ObjectStorages/IObjectStorage.h
Normal file
159
src/Disks/ObjectStorages/IObjectStorage.h
Normal file
@ -0,0 +1,159 @@
|
||||
#pragma once
|
||||
|
||||
#include <filesystem>
|
||||
#include <string>
|
||||
#include <map>
|
||||
#include <optional>
|
||||
|
||||
#include <Poco/Timestamp.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <IO/ReadSettings.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/FileCache.h>
|
||||
#include <Disks/WriteMode.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ReadBufferFromFileBase;
|
||||
class WriteBufferFromFileBase;
|
||||
|
||||
using ObjectAttributes = std::map<std::string, std::string>;
|
||||
|
||||
/// Path to blob with it's size
|
||||
struct BlobPathWithSize
|
||||
{
|
||||
std::string relative_path;
|
||||
uint64_t bytes_size;
|
||||
|
||||
BlobPathWithSize() = default;
|
||||
BlobPathWithSize(const BlobPathWithSize & other) = default;
|
||||
|
||||
BlobPathWithSize(const std::string & relative_path_, uint64_t bytes_size_)
|
||||
: relative_path(relative_path_)
|
||||
, bytes_size(bytes_size_)
|
||||
{}
|
||||
};
|
||||
|
||||
/// List of blobs with their sizes
|
||||
using BlobsPathToSize = std::vector<BlobPathWithSize>;
|
||||
|
||||
struct ObjectMetadata
|
||||
{
|
||||
uint64_t size_bytes;
|
||||
std::optional<Poco::Timestamp> last_modified;
|
||||
std::optional<ObjectAttributes> attributes;
|
||||
};
|
||||
|
||||
using FinalizeCallback = std::function<void(size_t bytes_count)>;
|
||||
|
||||
/// Base class for all object storages which implement some subset of ordinary filesystem operations.
|
||||
///
|
||||
/// Examples of object storages are S3, Azure Blob Storage, HDFS.
|
||||
class IObjectStorage
|
||||
{
|
||||
public:
|
||||
explicit IObjectStorage(FileCachePtr && cache_)
|
||||
: cache(std::move(cache_))
|
||||
{}
|
||||
|
||||
/// Path exists or not
|
||||
virtual bool exists(const std::string & path) const = 0;
|
||||
|
||||
/// List on prefix, return children with their sizes.
|
||||
virtual void listPrefix(const std::string & path, BlobsPathToSize & children) const = 0;
|
||||
|
||||
/// Get object metadata if supported. It should be possible to receive
|
||||
/// at least size of object
|
||||
virtual ObjectMetadata getObjectMetadata(const std::string & path) const = 0;
|
||||
|
||||
/// Read single path from object storage
|
||||
virtual std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const = 0;
|
||||
|
||||
/// Read multiple objects with common prefix
|
||||
virtual std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const = 0;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
virtual std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) = 0;
|
||||
|
||||
/// Remove object. Throws exception if object doesn't exists.
|
||||
virtual void removeObject(const std::string & path) = 0;
|
||||
|
||||
/// Remove multiple objects. Some object storages can do batch remove in a more
|
||||
/// optimal way.
|
||||
virtual void removeObjects(const std::vector<std::string> & paths) = 0;
|
||||
|
||||
/// Remove object on path if exists
|
||||
virtual void removeObjectIfExists(const std::string & path) = 0;
|
||||
|
||||
/// Remove objects on path if exists
|
||||
virtual void removeObjectsIfExist(const std::vector<std::string> & paths) = 0;
|
||||
|
||||
/// Copy object with different attributes if required
|
||||
virtual void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) = 0;
|
||||
|
||||
/// Copy object to another instance of object storage
|
||||
/// by default just read the object from source object storage and write
|
||||
/// to destination through buffers.
|
||||
virtual void copyObjectToAnotherObjectStorage( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
IObjectStorage & object_storage_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {});
|
||||
|
||||
virtual ~IObjectStorage() = default;
|
||||
|
||||
/// Path to directory with objects cache
|
||||
std::string getCacheBasePath() const;
|
||||
|
||||
static AsynchronousReaderPtr getThreadPoolReader();
|
||||
|
||||
static ThreadPool & getThreadPoolWriter();
|
||||
|
||||
virtual void shutdown() = 0;
|
||||
|
||||
virtual void startup() = 0;
|
||||
|
||||
void removeFromCache(const std::string & path);
|
||||
|
||||
/// Apply new settings, in most cases reiniatilize client and some other staff
|
||||
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 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 getObjectsNamespace() const = 0;
|
||||
|
||||
/// FIXME: confusing function required for a very specific case. Create new instance of object storage
|
||||
/// in different namespace.
|
||||
virtual std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) = 0;
|
||||
|
||||
protected:
|
||||
FileCachePtr cache;
|
||||
};
|
||||
|
||||
using ObjectStoragePtr = std::unique_ptr<IObjectStorage>;
|
||||
|
||||
}
|
502
src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
Normal file
502
src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp
Normal file
@ -0,0 +1,502 @@
|
||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/IO/AsynchronousReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ReadIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/WriteIndirectBufferFromRemoteFS.h>
|
||||
#include <Disks/IO/ThreadPoolRemoteFSReader.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/SeekAvoidingReadBuffer.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
||||
#include <aws/s3/model/CopyObjectRequest.h>
|
||||
#include <aws/s3/model/ListObjectsV2Request.h>
|
||||
#include <aws/s3/model/HeadObjectRequest.h>
|
||||
#include <aws/s3/model/DeleteObjectsRequest.h>
|
||||
#include <aws/s3/model/CreateMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/CompleteMultipartUploadRequest.h>
|
||||
#include <aws/s3/model/UploadPartCopyRequest.h>
|
||||
#include <aws/s3/model/AbortMultipartUploadRequest.h>
|
||||
|
||||
#include <Common/FileCache.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int S3_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void throwIfError(Aws::Utils::Outcome<Result, Error> & response)
|
||||
{
|
||||
if (!response.IsSuccess())
|
||||
{
|
||||
const auto & err = response.GetError();
|
||||
throw Exception(std::to_string(static_cast<int>(err.GetErrorType())) + ": " + err.GetMessage(), ErrorCodes::S3_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void throwIfError(const Aws::Utils::Outcome<Result, Error> & response)
|
||||
{
|
||||
if (!response.IsSuccess())
|
||||
{
|
||||
const auto & err = response.GetError();
|
||||
throw Exception(err.GetMessage(), static_cast<int>(err.GetErrorType()));
|
||||
}
|
||||
}
|
||||
|
||||
template <typename Result, typename Error>
|
||||
void logIfError(const Aws::Utils::Outcome<Result, Error> & response, std::function<String()> && msg)
|
||||
{
|
||||
try
|
||||
{
|
||||
throwIfError(response);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__, msg());
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
Aws::S3::Model::HeadObjectOutcome S3ObjectStorage::requestObjectHeadData(const std::string & bucket_from, const std::string & key) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::HeadObjectRequest request;
|
||||
request.SetBucket(bucket_from);
|
||||
request.SetKey(key);
|
||||
|
||||
return client_ptr->HeadObject(request);
|
||||
}
|
||||
|
||||
bool S3ObjectStorage::exists(const std::string & path) const
|
||||
{
|
||||
auto object_head = requestObjectHeadData(bucket, path);
|
||||
if (!object_head.IsSuccess())
|
||||
{
|
||||
if (object_head.GetError().GetErrorType() == Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
return false;
|
||||
|
||||
throwIfError(object_head);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> S3ObjectStorage::readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
|
||||
ReadSettings disk_read_settings{read_settings};
|
||||
if (cache)
|
||||
{
|
||||
if (IFileCache::isReadOnly())
|
||||
disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
|
||||
|
||||
disk_read_settings.remote_fs_cache = cache;
|
||||
}
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
auto s3_impl = std::make_unique<ReadBufferFromS3Gather>(
|
||||
client.get(), bucket, version_id, common_path_prefix, blobs_to_read,
|
||||
settings_ptr->s3_settings.max_single_read_retries, disk_read_settings);
|
||||
|
||||
if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool)
|
||||
{
|
||||
auto reader = getThreadPoolReader();
|
||||
return std::make_unique<AsynchronousReadIndirectBufferFromRemoteFS>(reader, disk_read_settings, std::move(s3_impl));
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buf = std::make_unique<ReadIndirectBufferFromRemoteFS>(std::move(s3_impl));
|
||||
return std::make_unique<SeekAvoidingReadBuffer>(std::move(buf), settings_ptr->min_bytes_for_seek);
|
||||
}
|
||||
}
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> S3ObjectStorage::readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings,
|
||||
std::optional<size_t>,
|
||||
std::optional<size_t>) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
ReadSettings disk_read_settings{read_settings};
|
||||
if (cache)
|
||||
{
|
||||
if (IFileCache::isReadOnly())
|
||||
disk_read_settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache = true;
|
||||
|
||||
disk_read_settings.remote_fs_cache = cache;
|
||||
}
|
||||
|
||||
return std::make_unique<ReadBufferFromS3>(client.get(), bucket, path, version_id, settings_ptr->s3_settings.max_single_read_retries, disk_read_settings);
|
||||
}
|
||||
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> S3ObjectStorage::writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode, // S3 doesn't support append, only rewrite
|
||||
std::optional<ObjectAttributes> attributes,
|
||||
FinalizeCallback && finalize_callback,
|
||||
size_t buf_size,
|
||||
const WriteSettings & write_settings)
|
||||
{
|
||||
if (mode != WriteMode::Rewrite)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 doesn't support append to files");
|
||||
|
||||
bool cache_on_write = cache
|
||||
&& fs::path(path).extension() != ".tmp"
|
||||
&& write_settings.enable_filesystem_cache_on_write_operations
|
||||
&& FileCacheFactory::instance().getSettings(getCacheBasePath()).cache_on_write_operations;
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto s3_buffer = std::make_unique<WriteBufferFromS3>(
|
||||
client.get(),
|
||||
bucket,
|
||||
path,
|
||||
settings_ptr->s3_settings,
|
||||
attributes,
|
||||
buf_size, threadPoolCallbackRunner(getThreadPoolWriter()),
|
||||
cache_on_write ? cache : nullptr);
|
||||
|
||||
return std::make_unique<WriteIndirectBufferFromRemoteFS>(std::move(s3_buffer), std::move(finalize_callback), path);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::listPrefix(const std::string & path, BlobsPathToSize & children) const
|
||||
{
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
|
||||
Aws::S3::Model::ListObjectsV2Request request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetPrefix(path);
|
||||
request.SetMaxKeys(settings_ptr->list_object_keys_size);
|
||||
|
||||
Aws::S3::Model::ListObjectsV2Outcome outcome;
|
||||
do
|
||||
{
|
||||
outcome = client_ptr->ListObjectsV2(request);
|
||||
throwIfError(outcome);
|
||||
|
||||
auto result = outcome.GetResult();
|
||||
auto objects = result.GetContents();
|
||||
|
||||
if (objects.empty())
|
||||
break;
|
||||
|
||||
for (const auto & object : objects)
|
||||
children.emplace_back(object.GetKey(), object.GetSize());
|
||||
|
||||
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
|
||||
} while (outcome.GetResult().GetIsTruncated());
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObject(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(path);
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects({obj});
|
||||
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjects(const std::vector<std::string> & paths)
|
||||
{
|
||||
if (paths.empty())
|
||||
return;
|
||||
|
||||
auto client_ptr = client.get();
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete;
|
||||
size_t current_position = 0;
|
||||
|
||||
while (current_position < paths.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String keys;
|
||||
for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(paths[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += paths[current_position];
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectIfExists(const std::string & path)
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(path);
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects({obj});
|
||||
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetErrorType() != Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::removeObjectsIfExist(const std::vector<std::string> & paths)
|
||||
{
|
||||
if (paths.empty())
|
||||
return;
|
||||
|
||||
auto client_ptr = client.get();
|
||||
auto settings_ptr = s3_settings.get();
|
||||
|
||||
|
||||
size_t chunk_size_limit = settings_ptr->objects_chunk_size_to_delete;
|
||||
size_t current_position = 0;
|
||||
|
||||
while (current_position < paths.size())
|
||||
{
|
||||
std::vector<Aws::S3::Model::ObjectIdentifier> current_chunk;
|
||||
String keys;
|
||||
for (; current_position < paths.size() && current_chunk.size() < chunk_size_limit; ++current_position)
|
||||
{
|
||||
Aws::S3::Model::ObjectIdentifier obj;
|
||||
obj.SetKey(paths[current_position]);
|
||||
current_chunk.push_back(obj);
|
||||
|
||||
if (!keys.empty())
|
||||
keys += ", ";
|
||||
keys += paths[current_position];
|
||||
}
|
||||
|
||||
Aws::S3::Model::Delete delkeys;
|
||||
delkeys.SetObjects(current_chunk);
|
||||
Aws::S3::Model::DeleteObjectsRequest request;
|
||||
request.SetBucket(bucket);
|
||||
request.SetDelete(delkeys);
|
||||
auto outcome = client_ptr->DeleteObjects(request);
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetErrorType() != Aws::S3::S3Errors::RESOURCE_NOT_FOUND)
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
ObjectMetadata S3ObjectStorage::getObjectMetadata(const std::string & path) const
|
||||
{
|
||||
ObjectMetadata result;
|
||||
|
||||
auto object_head = requestObjectHeadData(bucket, path);
|
||||
throwIfError(object_head);
|
||||
|
||||
auto & object_head_result = object_head.GetResult();
|
||||
result.size_bytes = object_head_result.GetContentLength();
|
||||
result.last_modified = object_head_result.GetLastModified().Millis();
|
||||
result.attributes = object_head_result.GetMetadata();
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectToAnotherObjectStorage(const std::string & object_from, const std::string & object_to, IObjectStorage & object_storage_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
/// Shortcut for S3
|
||||
if (auto * dest_s3 = dynamic_cast<S3ObjectStorage * >(&object_storage_to); dest_s3 != nullptr)
|
||||
copyObjectImpl(bucket, object_from, dest_s3->bucket, object_to, {}, object_to_attributes);
|
||||
else
|
||||
IObjectStorage::copyObjectToAnotherObjectStorage(object_from, object_to, object_storage_to, object_to_attributes);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
Aws::S3::Model::CopyObjectRequest request;
|
||||
request.SetCopySource(src_bucket + "/" + src_key);
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
{
|
||||
request.SetMetadata(*metadata);
|
||||
request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE);
|
||||
}
|
||||
|
||||
auto outcome = client_ptr->CopyObject(request);
|
||||
|
||||
if (!outcome.IsSuccess() && outcome.GetError().GetExceptionName() == "EntityTooLarge")
|
||||
{ // Can't come here with MinIO, MinIO allows single part upload for large objects.
|
||||
copyObjectMultipartImpl(src_bucket, src_key, dst_bucket, dst_key, head, metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObjectMultipartImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head,
|
||||
std::optional<ObjectAttributes> metadata) const
|
||||
{
|
||||
if (!head)
|
||||
head = requestObjectHeadData(src_bucket, src_key).GetResult();
|
||||
|
||||
auto settings_ptr = s3_settings.get();
|
||||
auto client_ptr = client.get();
|
||||
size_t size = head->GetContentLength();
|
||||
|
||||
String multipart_upload_id;
|
||||
|
||||
{
|
||||
Aws::S3::Model::CreateMultipartUploadRequest request;
|
||||
request.SetBucket(dst_bucket);
|
||||
request.SetKey(dst_key);
|
||||
if (metadata)
|
||||
request.SetMetadata(*metadata);
|
||||
|
||||
auto outcome = client_ptr->CreateMultipartUpload(request);
|
||||
|
||||
throwIfError(outcome);
|
||||
|
||||
multipart_upload_id = outcome.GetResult().GetUploadId();
|
||||
}
|
||||
|
||||
std::vector<String> part_tags;
|
||||
|
||||
size_t upload_part_size = settings_ptr->s3_settings.min_upload_part_size;
|
||||
for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size)
|
||||
{
|
||||
Aws::S3::Model::UploadPartCopyRequest part_request;
|
||||
part_request.SetCopySource(src_bucket + "/" + src_key);
|
||||
part_request.SetBucket(dst_bucket);
|
||||
part_request.SetKey(dst_key);
|
||||
part_request.SetUploadId(multipart_upload_id);
|
||||
part_request.SetPartNumber(part_number);
|
||||
part_request.SetCopySourceRange(fmt::format("bytes={}-{}", position, std::min(size, position + upload_part_size) - 1));
|
||||
|
||||
auto outcome = client_ptr->UploadPartCopy(part_request);
|
||||
if (!outcome.IsSuccess())
|
||||
{
|
||||
Aws::S3::Model::AbortMultipartUploadRequest abort_request;
|
||||
abort_request.SetBucket(dst_bucket);
|
||||
abort_request.SetKey(dst_key);
|
||||
abort_request.SetUploadId(multipart_upload_id);
|
||||
client_ptr->AbortMultipartUpload(abort_request);
|
||||
// In error case we throw exception later with first error from UploadPartCopy
|
||||
}
|
||||
throwIfError(outcome);
|
||||
|
||||
auto etag = outcome.GetResult().GetCopyPartResult().GetETag();
|
||||
part_tags.push_back(etag);
|
||||
}
|
||||
|
||||
{
|
||||
Aws::S3::Model::CompleteMultipartUploadRequest req;
|
||||
req.SetBucket(dst_bucket);
|
||||
req.SetKey(dst_key);
|
||||
req.SetUploadId(multipart_upload_id);
|
||||
|
||||
Aws::S3::Model::CompletedMultipartUpload multipart_upload;
|
||||
for (size_t i = 0; i < part_tags.size(); ++i)
|
||||
{
|
||||
Aws::S3::Model::CompletedPart part;
|
||||
multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1));
|
||||
}
|
||||
|
||||
req.SetMultipartUpload(multipart_upload);
|
||||
|
||||
auto outcome = client_ptr->CompleteMultipartUpload(req);
|
||||
|
||||
throwIfError(outcome);
|
||||
}
|
||||
}
|
||||
|
||||
void S3ObjectStorage::copyObject(const std::string & object_from, const std::string & object_to, std::optional<ObjectAttributes> object_to_attributes) // NOLINT
|
||||
{
|
||||
auto head = requestObjectHeadData(bucket, object_from).GetResult();
|
||||
if (head.GetContentLength() >= static_cast<int64_t>(5UL * 1024 * 1024 * 1024))
|
||||
copyObjectMultipartImpl(bucket, object_from, bucket, object_to, head, object_to_attributes);
|
||||
else
|
||||
copyObjectImpl(bucket, object_from, bucket, object_to, head, object_to_attributes);
|
||||
}
|
||||
|
||||
void S3ObjectStorage::setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_)
|
||||
{
|
||||
s3_settings.set(std::move(s3_settings_));
|
||||
}
|
||||
|
||||
void S3ObjectStorage::setNewClient(std::unique_ptr<Aws::S3::S3Client> && client_)
|
||||
{
|
||||
client.set(std::move(client_));
|
||||
}
|
||||
|
||||
void S3ObjectStorage::shutdown()
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
/// This call stops any next retry attempts for ongoing S3 requests.
|
||||
/// If S3 request is failed and the method below is executed S3 client immediately returns the last failed S3 request outcome.
|
||||
/// If S3 is healthy nothing wrong will be happened and S3 requests will be processed in a regular way without errors.
|
||||
/// This should significantly speed up shutdown process if S3 is unhealthy.
|
||||
const_cast<Aws::S3::S3Client &>(*client_ptr).DisableRequestProcessing();
|
||||
}
|
||||
|
||||
void S3ObjectStorage::startup()
|
||||
{
|
||||
auto client_ptr = client.get();
|
||||
|
||||
/// Need to be enabled if it was disabled during shutdown() call.
|
||||
const_cast<Aws::S3::S3Client &>(*client_ptr).EnableRequestProcessing();
|
||||
}
|
||||
|
||||
void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
s3_settings.set(getSettings(config, config_prefix, context));
|
||||
client.set(getClient(config, config_prefix, context));
|
||||
}
|
||||
|
||||
std::unique_ptr<IObjectStorage> S3ObjectStorage::cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context)
|
||||
{
|
||||
return std::make_unique<S3ObjectStorage>(
|
||||
nullptr, getClient(config, config_prefix, context),
|
||||
getSettings(config, config_prefix, context),
|
||||
version_id, new_namespace);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
#endif
|
138
src/Disks/ObjectStorages/S3/S3ObjectStorage.h
Normal file
138
src/Disks/ObjectStorages/S3/S3ObjectStorage.h
Normal file
@ -0,0 +1,138 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <memory>
|
||||
#include <aws/s3/S3Client.h>
|
||||
#include <aws/s3/model/HeadObjectResult.h>
|
||||
#include <aws/s3/model/ListObjectsV2Result.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct S3ObjectStorageSettings
|
||||
{
|
||||
S3ObjectStorageSettings() = default;
|
||||
|
||||
S3ObjectStorageSettings(
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
uint64_t min_bytes_for_seek_,
|
||||
int32_t list_object_keys_size_,
|
||||
int32_t objects_chunk_size_to_delete_)
|
||||
: s3_settings(s3_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_)
|
||||
{}
|
||||
|
||||
S3Settings::ReadWriteSettings s3_settings;
|
||||
|
||||
uint64_t min_bytes_for_seek;
|
||||
int32_t list_object_keys_size;
|
||||
int32_t objects_chunk_size_to_delete;
|
||||
};
|
||||
|
||||
|
||||
class S3ObjectStorage : public IObjectStorage
|
||||
{
|
||||
public:
|
||||
S3ObjectStorage(
|
||||
FileCachePtr && cache_,
|
||||
std::unique_ptr<Aws::S3::S3Client> && client_,
|
||||
std::unique_ptr<S3ObjectStorageSettings> && s3_settings_,
|
||||
String version_id_,
|
||||
String bucket_)
|
||||
: IObjectStorage(std::move(cache_))
|
||||
, bucket(bucket_)
|
||||
, client(std::move(client_))
|
||||
, s3_settings(std::move(s3_settings_))
|
||||
, version_id(std::move(version_id_))
|
||||
{}
|
||||
|
||||
bool exists(const std::string & path) const override;
|
||||
|
||||
std::unique_ptr<SeekableReadBuffer> readObject( /// NOLINT
|
||||
const std::string & path,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readObjects( /// NOLINT
|
||||
const std::string & common_path_prefix,
|
||||
const BlobsPathToSize & blobs_to_read,
|
||||
const ReadSettings & read_settings = ReadSettings{},
|
||||
std::optional<size_t> read_hint = {},
|
||||
std::optional<size_t> file_size = {}) const override;
|
||||
|
||||
/// Open the file for write and return WriteBufferFromFileBase object.
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeObject( /// NOLINT
|
||||
const std::string & path,
|
||||
WriteMode mode,
|
||||
std::optional<ObjectAttributes> attributes = {},
|
||||
FinalizeCallback && finalize_callback = {},
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
const WriteSettings & write_settings = {}) override;
|
||||
|
||||
void listPrefix(const std::string & path, BlobsPathToSize & children) const override;
|
||||
/// Remove file. Throws exception if file doesn't exist or it's a directory.
|
||||
void removeObject(const std::string & path) override;
|
||||
|
||||
void removeObjects(const std::vector<std::string> & paths) override;
|
||||
|
||||
void removeObjectIfExists(const std::string & path) override;
|
||||
|
||||
void removeObjectsIfExist(const std::vector<std::string> & paths) override;
|
||||
|
||||
ObjectMetadata getObjectMetadata(const std::string & path) const override;
|
||||
|
||||
void copyObject( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void copyObjectToAnotherObjectStorage( /// NOLINT
|
||||
const std::string & object_from,
|
||||
const std::string & object_to,
|
||||
IObjectStorage & object_storage_to,
|
||||
std::optional<ObjectAttributes> object_to_attributes = {}) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void startup() override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
|
||||
String getObjectsNamespace() const override { return bucket; }
|
||||
|
||||
std::unique_ptr<IObjectStorage> cloneObjectStorage(const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) override;
|
||||
private:
|
||||
void setNewSettings(std::unique_ptr<S3ObjectStorageSettings> && s3_settings_);
|
||||
|
||||
void setNewClient(std::unique_ptr<Aws::S3::S3Client> && client_);
|
||||
|
||||
void copyObjectImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
void copyObjectMultipartImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<ObjectAttributes> metadata = std::nullopt) const;
|
||||
|
||||
Aws::S3::Model::HeadObjectOutcome requestObjectHeadData(const std::string & bucket_from, const std::string & key) const;
|
||||
|
||||
std::string bucket;
|
||||
|
||||
MultiVersion<Aws::S3::S3Client> client;
|
||||
MultiVersion<S3ObjectStorageSettings> s3_settings;
|
||||
|
||||
const String version_id;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
@ -1,54 +1,50 @@
|
||||
#include <Common/config.h>
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include "Disks/DiskFactory.h"
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <aws/core/client/DefaultRetryStrategy.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include "DiskS3.h"
|
||||
#include "Disks/DiskCacheWrapper.h"
|
||||
#include "Storages/StorageS3Settings.h"
|
||||
#include "ProxyConfiguration.h"
|
||||
#include "ProxyListConfiguration.h"
|
||||
#include "ProxyResolverConfiguration.h"
|
||||
#include "Disks/DiskRestartProxy.h"
|
||||
#include "Disks/DiskLocal.h"
|
||||
#include "Disks/RemoteDisksCommon.h"
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyListConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyResolverConfiguration.h>
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
std::unique_ptr<S3ObjectStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context)
|
||||
{
|
||||
void checkWriteAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.writeFile("test_acl", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
|
||||
file->write("test", 4);
|
||||
}
|
||||
S3Settings::ReadWriteSettings rw_settings;
|
||||
rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries);
|
||||
rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size);
|
||||
rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", context->getSettingsRef().s3_upload_part_size_multiply_factor);
|
||||
rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold);
|
||||
rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size);
|
||||
|
||||
void checkReadAccess(const String & disk_name, IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile("test_acl");
|
||||
String buf(4, '0');
|
||||
file->readStrict(buf.data(), 4);
|
||||
if (buf != "test")
|
||||
throw Exception("No read access to S3 bucket in disk " + disk_name, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
return std::make_unique<S3ObjectStorageSettings>(
|
||||
rw_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));
|
||||
}
|
||||
|
||||
void checkRemoveAccess(IDisk & disk) { disk.removeFile("test_acl"); }
|
||||
|
||||
std::shared_ptr<S3::ProxyResolverConfiguration> getProxyResolverConfiguration(
|
||||
const String & prefix, const Poco::Util::AbstractConfiguration & proxy_resolver_config)
|
||||
{
|
||||
@ -112,12 +108,13 @@ std::shared_ptr<S3::ProxyConfiguration> getProxyConfiguration(const String & pre
|
||||
return getProxyListConfiguration(prefix + ".proxy", config);
|
||||
}
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client>
|
||||
getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context)
|
||||
|
||||
std::unique_ptr<Aws::S3::S3Client> getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context)
|
||||
{
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
config.getString(config_prefix + ".region", ""),
|
||||
context->getRemoteHostFilter(), context->getGlobalContext()->getSettingsRef().s3_max_redirects);
|
||||
context->getRemoteHostFilter(), context->getGlobalContext()->getSettingsRef().s3_max_redirects,
|
||||
context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging);
|
||||
|
||||
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
|
||||
if (uri.key.back() != '/')
|
||||
@ -151,93 +148,6 @@ getClient(const Poco::Util::AbstractConfiguration & config, const String & confi
|
||||
config.getBool(config_prefix + ".use_insecure_imds_request", config.getBool("s3.use_insecure_imds_request", false)));
|
||||
}
|
||||
|
||||
std::unique_ptr<DiskS3Settings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context)
|
||||
{
|
||||
S3Settings::ReadWriteSettings rw_settings;
|
||||
rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries);
|
||||
rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size);
|
||||
rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", context->getSettingsRef().s3_upload_part_size_multiply_factor);
|
||||
rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold);
|
||||
rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size);
|
||||
|
||||
return std::make_unique<DiskS3Settings>(
|
||||
getClient(config, config_prefix, context),
|
||||
rw_settings,
|
||||
config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024),
|
||||
config.getBool(config_prefix + ".send_metadata", false),
|
||||
config.getInt(config_prefix + ".thread_pool_size", 16),
|
||||
config.getInt(config_prefix + ".list_object_keys_size", 1000),
|
||||
config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
void registerDiskS3(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context,
|
||||
const DisksMap & /*map*/) -> DiskPtr {
|
||||
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
|
||||
|
||||
if (uri.key.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No key in S3 uri: {}", uri.uri.toString());
|
||||
|
||||
if (uri.key.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 path must ends with '/', but '{}' doesn't.", uri.key);
|
||||
|
||||
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
|
||||
|
||||
FileCachePtr cache = getCachePtrForDisk(name, config, config_prefix, context);
|
||||
|
||||
std::shared_ptr<IDisk> s3disk = std::make_shared<DiskS3>(
|
||||
name,
|
||||
uri.bucket,
|
||||
uri.key,
|
||||
uri.version_id,
|
||||
metadata_disk,
|
||||
std::move(cache),
|
||||
context,
|
||||
getSettings(config, config_prefix, context),
|
||||
getSettings,
|
||||
"-" + getFQDNOrHostName());
|
||||
|
||||
/// This code is used only to check access to the corresponding disk.
|
||||
if (!config.getBool(config_prefix + ".skip_access_check", false))
|
||||
{
|
||||
checkWriteAccess(*s3disk);
|
||||
checkReadAccess(name, *s3disk);
|
||||
checkRemoveAccess(*s3disk);
|
||||
}
|
||||
|
||||
s3disk->startup();
|
||||
|
||||
|
||||
#ifdef NDEBUG
|
||||
bool use_cache = true;
|
||||
#else
|
||||
/// Current S3 cache implementation lead to allocations in destructor of
|
||||
/// read buffer.
|
||||
bool use_cache = false;
|
||||
#endif
|
||||
|
||||
if (config.getBool(config_prefix + ".cache_enabled", use_cache))
|
||||
{
|
||||
String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/");
|
||||
s3disk = wrapWithCache(s3disk, "s3-cache", cache_path, metadata_path);
|
||||
}
|
||||
|
||||
return std::make_shared<DiskRestartProxy>(s3disk);
|
||||
};
|
||||
factory.registerDiskType("s3", creator);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#else
|
||||
|
||||
void registerDiskS3(DiskFactory &) {}
|
||||
|
||||
#endif
|
29
src/Disks/ObjectStorages/S3/diskSettings.h
Normal file
29
src/Disks/ObjectStorages/S3/diskSettings.h
Normal file
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <aws/core/client/DefaultRetryStrategy.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyListConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyResolverConfiguration.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
std::unique_ptr<S3ObjectStorageSettings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context);
|
||||
|
||||
std::unique_ptr<Aws::S3::S3Client> getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context);
|
||||
|
||||
}
|
||||
|
||||
#endif
|
29
src/Disks/ObjectStorages/S3/parseConfig.h
Normal file
29
src/Disks/ObjectStorages/S3/parseConfig.h
Normal file
@ -0,0 +1,29 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <aws/core/client/DefaultRetryStrategy.h>
|
||||
#include <IO/S3Common.h>
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyListConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyResolverConfiguration.h>
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
std::unique_ptr<DiskS3Settings> getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context);
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context);
|
||||
|
||||
|
||||
}
|
145
src/Disks/ObjectStorages/S3/registerDiskS3.cpp
Normal file
145
src/Disks/ObjectStorages/S3/registerDiskS3.cpp
Normal file
@ -0,0 +1,145 @@
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Disks/DiskFactory.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <aws/core/client/DefaultRetryStrategy.h>
|
||||
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Common/FileCacheFactory.h>
|
||||
|
||||
#include <Disks/DiskCacheWrapper.h>
|
||||
#include <Disks/DiskRestartProxy.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
|
||||
#include <Disks/ObjectStorages/DiskObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/DiskObjectStorageCommon.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyListConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/ProxyResolverConfiguration.h>
|
||||
#include <Disks/ObjectStorages/S3/S3ObjectStorage.h>
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
||||
#include <IO/S3Common.h>
|
||||
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int PATH_ACCESS_DENIED;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void checkWriteAccess(IDisk & disk)
|
||||
{
|
||||
auto file = disk.writeFile("test_acl", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite);
|
||||
try
|
||||
{
|
||||
file->write("test", 4);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
file->finalize();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void checkReadAccess(const String & disk_name, IDisk & disk)
|
||||
{
|
||||
auto file = disk.readFile("test_acl");
|
||||
String buf(4, '0');
|
||||
file->readStrict(buf.data(), 4);
|
||||
if (buf != "test")
|
||||
throw Exception("No read access to S3 bucket in disk " + disk_name, ErrorCodes::PATH_ACCESS_DENIED);
|
||||
}
|
||||
|
||||
void checkRemoveAccess(IDisk & disk) { disk.removeFile("test_acl"); }
|
||||
|
||||
}
|
||||
|
||||
void registerDiskS3(DiskFactory & factory)
|
||||
{
|
||||
auto creator = [](const String & name,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const String & config_prefix,
|
||||
ContextPtr context,
|
||||
const DisksMap & /*map*/) -> DiskPtr {
|
||||
S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint")));
|
||||
|
||||
if (uri.key.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "No key in S3 uri: {}", uri.uri.toString());
|
||||
|
||||
if (uri.key.back() != '/')
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "S3 path must ends with '/', but '{}' doesn't.", uri.key);
|
||||
|
||||
auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context);
|
||||
|
||||
FileCachePtr cache = getCachePtrForDisk(name, config, config_prefix, context);
|
||||
|
||||
ObjectStoragePtr s3_storage = std::make_unique<S3ObjectStorage>(
|
||||
std::move(cache), getClient(config, config_prefix, context),
|
||||
getSettings(config, config_prefix, context),
|
||||
uri.version_id, uri.bucket);
|
||||
|
||||
bool send_metadata = config.getBool(config_prefix + ".send_metadata", false);
|
||||
uint64_t copy_thread_pool_size = config.getUInt(config_prefix + ".thread_pool_size", 16);
|
||||
|
||||
std::shared_ptr<DiskObjectStorage> s3disk = std::make_shared<DiskObjectStorage>(
|
||||
name,
|
||||
uri.key,
|
||||
"DiskS3",
|
||||
metadata_disk,
|
||||
std::move(s3_storage),
|
||||
DiskType::S3,
|
||||
send_metadata,
|
||||
copy_thread_pool_size);
|
||||
|
||||
/// This code is used only to check access to the corresponding disk.
|
||||
if (!config.getBool(config_prefix + ".skip_access_check", false))
|
||||
{
|
||||
checkWriteAccess(*s3disk);
|
||||
checkReadAccess(name, *s3disk);
|
||||
checkRemoveAccess(*s3disk);
|
||||
}
|
||||
|
||||
s3disk->startup(context);
|
||||
|
||||
std::shared_ptr<IDisk> disk_result = s3disk;
|
||||
|
||||
#ifdef NDEBUG
|
||||
bool use_cache = true;
|
||||
#else
|
||||
/// Current S3 cache implementation lead to allocations in destructor of
|
||||
/// read buffer.
|
||||
bool use_cache = false;
|
||||
#endif
|
||||
|
||||
if (config.getBool(config_prefix + ".cache_enabled", use_cache))
|
||||
{
|
||||
String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/");
|
||||
disk_result = wrapWithCache(disk_result, "s3-cache", cache_path, metadata_path);
|
||||
}
|
||||
|
||||
return std::make_shared<DiskRestartProxy>(disk_result);
|
||||
};
|
||||
factory.registerDiskType("s3", creator);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#else
|
||||
|
||||
void registerDiskS3(DiskFactory &) {}
|
||||
|
||||
#endif
|
File diff suppressed because it is too large
Load Diff
@ -1,199 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <atomic>
|
||||
#include <optional>
|
||||
#include <Common/logger_useful.h>
|
||||
#include "Disks/DiskFactory.h"
|
||||
#include "Disks/Executor.h"
|
||||
|
||||
#include <aws/s3/S3Client.h>
|
||||
#include <aws/s3/model/HeadObjectResult.h>
|
||||
#include <aws/s3/model/ListObjectsV2Result.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <re2/re2.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Common/FileCache_fwd.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Settings for DiskS3 that can be changed in runtime.
|
||||
struct DiskS3Settings
|
||||
{
|
||||
DiskS3Settings(
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
size_t min_bytes_for_seek_,
|
||||
bool send_metadata_,
|
||||
int thread_pool_size_,
|
||||
int list_object_keys_size_,
|
||||
int objects_chunk_size_to_delete_);
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
S3Settings::ReadWriteSettings s3_settings;
|
||||
size_t min_bytes_for_seek;
|
||||
bool send_metadata;
|
||||
int thread_pool_size;
|
||||
int list_object_keys_size;
|
||||
int objects_chunk_size_to_delete;
|
||||
};
|
||||
|
||||
|
||||
/**
|
||||
* Storage for persisting data in S3 and metadata on the local disk.
|
||||
* Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file)
|
||||
* that contains S3 object key with actual data.
|
||||
*/
|
||||
class DiskS3 final : public IDiskRemote
|
||||
{
|
||||
public:
|
||||
using ObjectMetadata = std::map<std::string, std::string>;
|
||||
using Futures = std::vector<std::future<void>>;
|
||||
|
||||
using SettingsPtr = std::unique_ptr<DiskS3Settings>;
|
||||
using GetDiskSettings = std::function<SettingsPtr(const Poco::Util::AbstractConfiguration &, const String, ContextPtr)>;
|
||||
|
||||
struct RestoreInformation;
|
||||
|
||||
DiskS3(
|
||||
String name_,
|
||||
String bucket_,
|
||||
String s3_root_path_,
|
||||
String version_id_,
|
||||
DiskPtr metadata_disk_,
|
||||
FileCachePtr cache_,
|
||||
ContextPtr context_,
|
||||
SettingsPtr settings_,
|
||||
GetDiskSettings settings_getter_,
|
||||
String operation_log_suffix_);
|
||||
|
||||
std::unique_ptr<ReadBufferFromFileBase> readFile(
|
||||
const String & path,
|
||||
const ReadSettings & settings,
|
||||
std::optional<size_t> read_hint,
|
||||
std::optional<size_t> file_size) const override;
|
||||
|
||||
std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
||||
const String & path,
|
||||
size_t buf_size,
|
||||
WriteMode mode,
|
||||
const WriteSettings & settings) override;
|
||||
|
||||
void removeFromRemoteFS(const std::vector<String> & paths) override;
|
||||
|
||||
void moveFile(const String & from_path, const String & to_path, bool send_metadata);
|
||||
void moveFile(const String & from_path, const String & to_path) override;
|
||||
|
||||
void createHardLink(const String & src_path, const String & dst_path) override;
|
||||
void createHardLink(const String & src_path, const String & dst_path, bool send_metadata);
|
||||
|
||||
DiskType getType() const override { return DiskType::S3; }
|
||||
bool isRemote() const override { return true; }
|
||||
|
||||
bool supportZeroCopyReplication() const override { return true; }
|
||||
|
||||
bool supportParallelWrite() const override { return true; }
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void startup() override;
|
||||
|
||||
/// Check file exists and ClickHouse has an access to it
|
||||
/// Overrode in remote disk
|
||||
/// Required for remote disk to ensure that replica has access to data written by other node
|
||||
bool checkUniqueId(const String & id) const override;
|
||||
|
||||
/// Dumps current revision counter into file 'revision.txt' at given path.
|
||||
void onFreeze(const String & path) override;
|
||||
|
||||
void applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) override;
|
||||
|
||||
void syncRevision(UInt64 revision) override;
|
||||
UInt64 getRevision() const override;
|
||||
|
||||
private:
|
||||
void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata);
|
||||
/// Converts revision to binary string with leading zeroes (64 bit).
|
||||
static String revisionToString(UInt64 revision);
|
||||
|
||||
bool checkObjectExists(const String & source_bucket, const String & prefix) const;
|
||||
void findLastRevision();
|
||||
|
||||
int readSchemaVersion(const String & source_bucket, const String & source_path);
|
||||
void saveSchemaVersion(const int & version);
|
||||
void updateObjectMetadata(const String & key, const ObjectMetadata & metadata);
|
||||
void migrateFileToRestorableSchema(const String & path);
|
||||
void migrateToRestorableSchemaRecursive(const String & path, Futures & results);
|
||||
void migrateToRestorableSchema();
|
||||
|
||||
Aws::S3::Model::HeadObjectResult headObject(const String & source_bucket, const String & key) const;
|
||||
void listObjects(const String & source_bucket, const String & source_path, std::function<bool(const Aws::S3::Model::ListObjectsV2Result &)> callback) const;
|
||||
void copyObject(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt) const;
|
||||
|
||||
void copyObjectImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<std::reference_wrapper<const ObjectMetadata>> metadata = std::nullopt) const;
|
||||
void copyObjectMultipartImpl(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key,
|
||||
std::optional<Aws::S3::Model::HeadObjectResult> head = std::nullopt,
|
||||
std::optional<std::reference_wrapper<const ObjectMetadata>> metadata = std::nullopt) const;
|
||||
|
||||
/// Restore S3 metadata files on file system.
|
||||
void restore();
|
||||
void readRestoreInformation(RestoreInformation & restore_information);
|
||||
void restoreFiles(const RestoreInformation & restore_information);
|
||||
void processRestoreFiles(const String & source_bucket, const String & source_path, std::vector<String> keys);
|
||||
void restoreFileOperations(const RestoreInformation & restore_information);
|
||||
|
||||
/// Remove 'path' prefix from 'key' to get relative key.
|
||||
/// It's needed to store keys to metadata files in RELATIVE_PATHS version.
|
||||
static String shrinkKey(const String & path, const String & key);
|
||||
std::tuple<UInt64, String> extractRevisionAndOperationFromKey(const String & key);
|
||||
|
||||
/// Forms detached path '../../detached/part_name/' from '../../part_name/'
|
||||
static String pathToDetached(const String & source_path);
|
||||
|
||||
/// Move file or files in directory when possible and remove files in other case
|
||||
/// to restore by S3 operation log with same operations from different replicas
|
||||
void moveRecursiveOrRemove(const String & from_path, const String & to_path, bool send_metadata);
|
||||
|
||||
const String bucket;
|
||||
|
||||
const String version_id;
|
||||
|
||||
MultiVersion<DiskS3Settings> current_settings;
|
||||
/// Gets disk settings from context.
|
||||
GetDiskSettings settings_getter;
|
||||
|
||||
std::atomic<UInt64> revision_counter = 0;
|
||||
static constexpr UInt64 LATEST_REVISION = std::numeric_limits<UInt64>::max();
|
||||
static constexpr UInt64 UNKNOWN_REVISION = 0;
|
||||
|
||||
/// File at path {metadata_path}/restore contains metadata restore information
|
||||
inline static const String RESTORE_FILE_NAME = "restore";
|
||||
|
||||
/// Key has format: ../../r{revision}(-{hostname})-{operation}
|
||||
const re2::RE2 key_regexp {".*/r(\\d+)(-[\\w\\d\\-\\.]+)?-(\\w+)$"};
|
||||
|
||||
/// Object contains information about schema version.
|
||||
inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION";
|
||||
/// Version with possibility to backup-restore metadata.
|
||||
static constexpr int RESTORABLE_SCHEMA_VERSION = 1;
|
||||
/// Directories with data.
|
||||
const std::vector<String> data_roots {"data", "store"};
|
||||
|
||||
ContextPtr context;
|
||||
|
||||
String operation_log_suffix;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
#endif
|
15
src/Disks/WriteMode.h
Normal file
15
src/Disks/WriteMode.h
Normal file
@ -0,0 +1,15 @@
|
||||
#pragma once
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/**
|
||||
* Mode of opening a file for write.
|
||||
*/
|
||||
enum class WriteMode
|
||||
{
|
||||
Rewrite,
|
||||
Append
|
||||
};
|
||||
|
||||
}
|
@ -21,7 +21,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const String & path_,
|
||||
size_t max_single_read_retries_,
|
||||
size_t max_single_download_retries_,
|
||||
|
@ -17,17 +17,17 @@ class ReadBufferFromAzureBlobStorage : public SeekableReadBuffer, public WithFil
|
||||
{
|
||||
public:
|
||||
|
||||
explicit ReadBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
ReadBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const String & path_,
|
||||
size_t max_single_read_retries_,
|
||||
size_t max_single_download_retries_,
|
||||
size_t tmp_buffer_size_,
|
||||
bool use_external_buffer_ = false,
|
||||
size_t read_until_position_ = 0
|
||||
);
|
||||
size_t read_until_position_ = 0);
|
||||
|
||||
off_t seek(off_t off, int whence) override;
|
||||
|
||||
off_t getPosition() override;
|
||||
|
||||
bool nextImpl() override;
|
||||
@ -41,7 +41,7 @@ private:
|
||||
void initialize();
|
||||
|
||||
std::unique_ptr<Azure::Core::IO::BodyStream> data_stream;
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
std::unique_ptr<Azure::Storage::Blobs::BlobClient> blob_client;
|
||||
|
||||
const String path;
|
||||
|
@ -37,7 +37,7 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
ReadBufferFromS3::ReadBufferFromS3(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
const String & version_id_,
|
||||
|
@ -29,7 +29,7 @@ namespace DB
|
||||
class ReadBufferFromS3 : public SeekableReadBuffer, public WithFileName, public WithFileSize
|
||||
{
|
||||
private:
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr;
|
||||
String bucket;
|
||||
String key;
|
||||
String version_id;
|
||||
@ -48,7 +48,7 @@ private:
|
||||
|
||||
public:
|
||||
ReadBufferFromS3(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
const String & version_id_,
|
||||
@ -94,7 +94,7 @@ class ReadBufferS3Factory : public ParallelReadBuffer::ReadBufferFactory, public
|
||||
{
|
||||
public:
|
||||
explicit ReadBufferS3Factory(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
const String & version_id_,
|
||||
@ -125,7 +125,7 @@ public:
|
||||
String getFileName() const override { return bucket + "/" + key; }
|
||||
|
||||
private:
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr;
|
||||
const String bucket;
|
||||
const String key;
|
||||
const String version_id;
|
||||
|
@ -54,10 +54,12 @@ namespace DB::S3
|
||||
PocoHTTPClientConfiguration::PocoHTTPClientConfiguration(
|
||||
const String & force_region_,
|
||||
const RemoteHostFilter & remote_host_filter_,
|
||||
unsigned int s3_max_redirects_)
|
||||
unsigned int s3_max_redirects_,
|
||||
bool enable_s3_requests_logging_)
|
||||
: force_region(force_region_)
|
||||
, remote_host_filter(remote_host_filter_)
|
||||
, s3_max_redirects(s3_max_redirects_)
|
||||
, enable_s3_requests_logging(enable_s3_requests_logging_)
|
||||
{
|
||||
}
|
||||
|
||||
@ -92,16 +94,17 @@ void PocoHTTPClientConfiguration::updateSchemeAndRegion()
|
||||
}
|
||||
|
||||
|
||||
PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration)
|
||||
: per_request_configuration(clientConfiguration.perRequestConfiguration)
|
||||
, error_report(clientConfiguration.error_report)
|
||||
PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration)
|
||||
: per_request_configuration(client_configuration.perRequestConfiguration)
|
||||
, error_report(client_configuration.error_report)
|
||||
, timeouts(ConnectionTimeouts(
|
||||
Poco::Timespan(clientConfiguration.connectTimeoutMs * 1000), /// connection timeout.
|
||||
Poco::Timespan(clientConfiguration.requestTimeoutMs * 1000), /// send timeout.
|
||||
Poco::Timespan(clientConfiguration.requestTimeoutMs * 1000) /// receive timeout.
|
||||
Poco::Timespan(client_configuration.connectTimeoutMs * 1000), /// connection timeout.
|
||||
Poco::Timespan(client_configuration.requestTimeoutMs * 1000), /// send timeout.
|
||||
Poco::Timespan(client_configuration.requestTimeoutMs * 1000) /// receive timeout.
|
||||
))
|
||||
, remote_host_filter(clientConfiguration.remote_host_filter)
|
||||
, s3_max_redirects(clientConfiguration.s3_max_redirects)
|
||||
, remote_host_filter(client_configuration.remote_host_filter)
|
||||
, s3_max_redirects(client_configuration.s3_max_redirects)
|
||||
, enable_s3_requests_logging(client_configuration.enable_s3_requests_logging)
|
||||
{
|
||||
}
|
||||
|
||||
@ -124,7 +127,8 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
Poco::Logger * log = &Poco::Logger::get("AWSClient");
|
||||
|
||||
auto uri = request.GetUri().GetURIString();
|
||||
LOG_TEST(log, "Make request to: {}", uri);
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Make request to: {}", uri);
|
||||
|
||||
enum class S3MetricType
|
||||
{
|
||||
@ -250,7 +254,8 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
|
||||
if (request.GetContentBody())
|
||||
{
|
||||
LOG_TEST(log, "Writing request body.");
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Writing request body.");
|
||||
|
||||
if (attempt > 0) /// rewind content body buffer.
|
||||
{
|
||||
@ -258,24 +263,28 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
request.GetContentBody()->seekg(0);
|
||||
}
|
||||
auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream);
|
||||
LOG_TEST(log, "Written {} bytes to request body", size);
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Written {} bytes to request body", size);
|
||||
}
|
||||
|
||||
LOG_TEST(log, "Receiving response...");
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Receiving response...");
|
||||
auto & response_body_stream = session->receiveResponse(poco_response);
|
||||
|
||||
watch.stop();
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds());
|
||||
|
||||
int status_code = static_cast<int>(poco_response.getStatus());
|
||||
LOG_TEST(log, "Response status: {}, {}", status_code, poco_response.getReason());
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Response status: {}, {}", status_code, poco_response.getReason());
|
||||
|
||||
if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT)
|
||||
{
|
||||
auto location = poco_response.get("location");
|
||||
remote_host_filter.checkURL(Poco::URI(location));
|
||||
uri = location;
|
||||
LOG_TEST(log, "Redirecting request to new location: {}", location);
|
||||
if (enable_s3_requests_logging)
|
||||
LOG_TEST(log, "Redirecting request to new location: {}", location);
|
||||
|
||||
ProfileEvents::increment(select_metric(S3MetricType::Redirects));
|
||||
|
||||
@ -285,13 +294,21 @@ void PocoHTTPClient::makeRequestInternal(
|
||||
response->SetResponseCode(static_cast<Aws::Http::HttpResponseCode>(status_code));
|
||||
response->SetContentType(poco_response.getContentType());
|
||||
|
||||
WriteBufferFromOwnString headers_ss;
|
||||
for (const auto & [header_name, header_value] : poco_response)
|
||||
if (enable_s3_requests_logging)
|
||||
{
|
||||
response->AddHeader(header_name, header_value);
|
||||
headers_ss << header_name << ": " << header_value << "; ";
|
||||
WriteBufferFromOwnString headers_ss;
|
||||
for (const auto & [header_name, header_value] : poco_response)
|
||||
{
|
||||
response->AddHeader(header_name, header_value);
|
||||
headers_ss << header_name << ": " << header_value << "; ";
|
||||
}
|
||||
LOG_TEST(log, "Received headers: {}", headers_ss.str());
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & [header_name, header_value] : poco_response)
|
||||
response->AddHeader(header_name, header_value);
|
||||
}
|
||||
LOG_TEST(log, "Received headers: {}", headers_ss.str());
|
||||
|
||||
if (status_code == 429 || status_code == 503)
|
||||
{ // API throttling
|
||||
|
@ -32,13 +32,14 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration
|
||||
String force_region;
|
||||
const RemoteHostFilter & remote_host_filter;
|
||||
unsigned int s3_max_redirects;
|
||||
bool enable_s3_requests_logging;
|
||||
|
||||
void updateSchemeAndRegion();
|
||||
|
||||
std::function<void(const Aws::Client::ClientConfigurationPerRequest &)> error_report;
|
||||
|
||||
private:
|
||||
PocoHTTPClientConfiguration(const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_);
|
||||
PocoHTTPClientConfiguration(const String & force_region_, const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, bool enable_s3_requests_logging_);
|
||||
|
||||
/// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization.
|
||||
friend ClientFactory;
|
||||
@ -79,7 +80,7 @@ private:
|
||||
class PocoHTTPClient : public Aws::Http::HttpClient
|
||||
{
|
||||
public:
|
||||
explicit PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfiguration);
|
||||
explicit PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration);
|
||||
~PocoHTTPClient() override = default;
|
||||
|
||||
std::shared_ptr<Aws::Http::HttpResponse> MakeRequest(
|
||||
@ -99,6 +100,7 @@ private:
|
||||
ConnectionTimeouts timeouts;
|
||||
const RemoteHostFilter & remote_host_filter;
|
||||
unsigned int s3_max_redirects;
|
||||
bool enable_s3_requests_logging;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -61,7 +61,8 @@ const std::pair<DB::LogsLevel, Poco::Message::Priority> & convertLogLevel(Aws::U
|
||||
class AWSLogger final : public Aws::Utils::Logging::LogSystemInterface
|
||||
{
|
||||
public:
|
||||
AWSLogger()
|
||||
explicit AWSLogger(bool enable_s3_requests_logging_)
|
||||
:enable_s3_requests_logging(enable_s3_requests_logging_)
|
||||
{
|
||||
for (auto [tag, name] : S3_LOGGER_TAG_NAMES)
|
||||
tag_loggers[tag] = &Poco::Logger::get(name);
|
||||
@ -71,7 +72,13 @@ public:
|
||||
|
||||
~AWSLogger() final = default;
|
||||
|
||||
Aws::Utils::Logging::LogLevel GetLogLevel() const final { return Aws::Utils::Logging::LogLevel::Trace; }
|
||||
Aws::Utils::Logging::LogLevel GetLogLevel() const final
|
||||
{
|
||||
if (enable_s3_requests_logging)
|
||||
return Aws::Utils::Logging::LogLevel::Trace;
|
||||
else
|
||||
return Aws::Utils::Logging::LogLevel::Info;
|
||||
}
|
||||
|
||||
void Log(Aws::Utils::Logging::LogLevel log_level, const char * tag, const char * format_str, ...) final // NOLINT
|
||||
{
|
||||
@ -100,6 +107,7 @@ public:
|
||||
|
||||
private:
|
||||
Poco::Logger * default_logger;
|
||||
bool enable_s3_requests_logging;
|
||||
std::unordered_map<String, Poco::Logger *> tag_loggers;
|
||||
};
|
||||
|
||||
@ -535,7 +543,7 @@ public:
|
||||
/// AWS API tries credentials providers one by one. Some of providers (like ProfileConfigFileAWSCredentialsProvider) can be
|
||||
/// quite verbose even if nobody configured them. So we use our provider first and only after it use default providers.
|
||||
{
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects);
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging);
|
||||
AddProvider(std::make_shared<AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider>(aws_client_configuration));
|
||||
}
|
||||
|
||||
@ -572,7 +580,7 @@ public:
|
||||
}
|
||||
else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true")
|
||||
{
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects);
|
||||
DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging);
|
||||
|
||||
/// See MakeDefaultHttpResourceClientConfiguration().
|
||||
/// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside
|
||||
@ -692,7 +700,7 @@ namespace S3
|
||||
{
|
||||
aws_options = Aws::SDKOptions{};
|
||||
Aws::InitAPI(aws_options);
|
||||
Aws::Utils::Logging::InitializeAWSLogging(std::make_shared<AWSLogger>());
|
||||
Aws::Utils::Logging::InitializeAWSLogging(std::make_shared<AWSLogger>(false));
|
||||
Aws::Http::SetHttpClientFactory(std::make_shared<PocoHTTPClientFactory>());
|
||||
}
|
||||
|
||||
@ -708,7 +716,7 @@ namespace S3
|
||||
return ret;
|
||||
}
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> ClientFactory::create( // NOLINT
|
||||
std::unique_ptr<Aws::S3::S3Client> ClientFactory::create( // NOLINT
|
||||
const PocoHTTPClientConfiguration & cfg_,
|
||||
bool is_virtual_hosted_style,
|
||||
const String & access_key_id,
|
||||
@ -746,7 +754,7 @@ namespace S3
|
||||
use_environment_credentials,
|
||||
use_insecure_imds_request);
|
||||
|
||||
return std::make_shared<Aws::S3::S3Client>(
|
||||
return std::make_unique<Aws::S3::S3Client>(
|
||||
std::move(auth_signer),
|
||||
std::move(client_configuration), // Client configuration.
|
||||
is_virtual_hosted_style || client_configuration.endpointOverride.empty() // Use virtual addressing only if endpoint is not specified.
|
||||
@ -756,9 +764,10 @@ namespace S3
|
||||
PocoHTTPClientConfiguration ClientFactory::createClientConfiguration( // NOLINT
|
||||
const String & force_region,
|
||||
const RemoteHostFilter & remote_host_filter,
|
||||
unsigned int s3_max_redirects)
|
||||
unsigned int s3_max_redirects,
|
||||
bool enable_s3_requestrs_logging)
|
||||
{
|
||||
return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects);
|
||||
return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requestrs_logging);
|
||||
}
|
||||
|
||||
URI::URI(const Poco::URI & uri_)
|
||||
@ -856,7 +865,7 @@ namespace S3
|
||||
quoteString(bucket), !uri.empty() ? " (" + uri.toString() + ")" : "");
|
||||
}
|
||||
|
||||
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id, bool throw_on_error)
|
||||
size_t getObjectSize(std::shared_ptr<const Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id, bool throw_on_error)
|
||||
{
|
||||
Aws::S3::Model::HeadObjectRequest req;
|
||||
req.SetBucket(bucket);
|
||||
|
@ -31,7 +31,7 @@ public:
|
||||
|
||||
static ClientFactory & instance();
|
||||
|
||||
std::shared_ptr<Aws::S3::S3Client> create(
|
||||
std::unique_ptr<Aws::S3::S3Client> create(
|
||||
const PocoHTTPClientConfiguration & cfg,
|
||||
bool is_virtual_hosted_style,
|
||||
const String & access_key_id,
|
||||
@ -44,12 +44,14 @@ public:
|
||||
PocoHTTPClientConfiguration createClientConfiguration(
|
||||
const String & force_region,
|
||||
const RemoteHostFilter & remote_host_filter,
|
||||
unsigned int s3_max_redirects);
|
||||
unsigned int s3_max_redirects,
|
||||
bool enable_s3_requestrs_logging);
|
||||
|
||||
private:
|
||||
ClientFactory();
|
||||
|
||||
Aws::SDKOptions aws_options;
|
||||
std::atomic<bool> s3_requests_logging_enabled;
|
||||
};
|
||||
|
||||
/**
|
||||
@ -76,7 +78,7 @@ struct URI
|
||||
static void validateBucket(const String & bucket, const Poco::URI & uri);
|
||||
};
|
||||
|
||||
size_t getObjectSize(std::shared_ptr<Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id = {}, bool throw_on_error = true);
|
||||
size_t getObjectSize(std::shared_ptr<const Aws::S3::S3Client> client_ptr, const String & bucket, const String & key, const String & version_id = {}, bool throw_on_error = true);
|
||||
|
||||
}
|
||||
|
||||
|
@ -3,7 +3,6 @@
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <IO/WriteBufferFromAzureBlobStorage.h>
|
||||
#include <Disks/RemoteDisksCommon.h>
|
||||
#include <Common/getRandomASCIIString.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
@ -12,14 +11,18 @@ namespace DB
|
||||
{
|
||||
|
||||
WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const String & blob_path_,
|
||||
size_t max_single_part_upload_size_,
|
||||
size_t buf_size_) :
|
||||
BufferWithOwnMemory<WriteBuffer>(buf_size_, nullptr, 0),
|
||||
blob_container_client(blob_container_client_),
|
||||
max_single_part_upload_size(max_single_part_upload_size_),
|
||||
blob_path(blob_path_) {}
|
||||
size_t buf_size_,
|
||||
std::optional<std::map<std::string, std::string>> attributes_)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size_, nullptr, 0)
|
||||
, blob_container_client(blob_container_client_)
|
||||
, max_single_part_upload_size(max_single_part_upload_size_)
|
||||
, blob_path(blob_path_)
|
||||
, attributes(attributes_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage()
|
||||
@ -29,6 +32,15 @@ WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage()
|
||||
|
||||
void WriteBufferFromAzureBlobStorage::finalizeImpl()
|
||||
{
|
||||
if (attributes.has_value())
|
||||
{
|
||||
auto blob_client = blob_container_client->GetBlobClient(blob_path);
|
||||
Azure::Storage::Metadata metadata;
|
||||
for (const auto & [key, value] : *attributes)
|
||||
metadata[key] = value;
|
||||
blob_client.SetMetadata(metadata);
|
||||
}
|
||||
|
||||
const size_t max_tries = 3;
|
||||
for (size_t i = 0; i < max_tries; ++i)
|
||||
{
|
||||
|
@ -19,11 +19,12 @@ class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
|
||||
explicit WriteBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
WriteBufferFromAzureBlobStorage(
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client_,
|
||||
const String & blob_path_,
|
||||
size_t max_single_part_upload_size_,
|
||||
size_t buf_size_);
|
||||
size_t buf_size_,
|
||||
std::optional<std::map<std::string, std::string>> attributes_ = {});
|
||||
|
||||
~WriteBufferFromAzureBlobStorage() override;
|
||||
|
||||
@ -32,9 +33,10 @@ public:
|
||||
private:
|
||||
void finalizeImpl() override;
|
||||
|
||||
std::shared_ptr<Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
std::shared_ptr<const Azure::Storage::Blobs::BlobContainerClient> blob_container_client;
|
||||
size_t max_single_part_upload_size;
|
||||
const String blob_path;
|
||||
std::optional<std::map<std::string, std::string>> attributes;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ struct WriteBufferFromS3::PutObjectTask
|
||||
};
|
||||
|
||||
WriteBufferFromS3::WriteBufferFromS3(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
@ -65,10 +65,10 @@ WriteBufferFromS3::WriteBufferFromS3(
|
||||
: BufferWithOwnMemory<WriteBuffer>(buffer_size_, nullptr, 0)
|
||||
, bucket(bucket_)
|
||||
, key(key_)
|
||||
, object_metadata(std::move(object_metadata_))
|
||||
, client_ptr(std::move(client_ptr_))
|
||||
, upload_part_size(s3_settings_.min_upload_part_size)
|
||||
, s3_settings(s3_settings_)
|
||||
, object_metadata(std::move(object_metadata_))
|
||||
, schedule(std::move(schedule_))
|
||||
, cache(cache_)
|
||||
{
|
||||
|
@ -48,7 +48,7 @@ class WriteBufferFromS3 final : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
WriteBufferFromS3(
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr_,
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr_,
|
||||
const String & bucket_,
|
||||
const String & key_,
|
||||
const S3Settings::ReadWriteSettings & s3_settings_,
|
||||
@ -90,10 +90,11 @@ private:
|
||||
|
||||
String bucket;
|
||||
String key;
|
||||
std::optional<std::map<String, String>> object_metadata;
|
||||
std::shared_ptr<Aws::S3::S3Client> client_ptr;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client_ptr;
|
||||
size_t upload_part_size = 0;
|
||||
S3Settings::ReadWriteSettings s3_settings;
|
||||
std::optional<std::map<String, String>> object_metadata;
|
||||
|
||||
/// Buffer to accumulate data.
|
||||
std::shared_ptr<Aws::StringStream> temporary_buffer;
|
||||
size_t last_part_size = 0;
|
||||
|
@ -29,7 +29,7 @@
|
||||
#include <Storages/CompressionCodecSelector.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Disks/ObjectStorages/IObjectStorage.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
#include <Interpreters/ActionLocksManager.h>
|
||||
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
|
||||
@ -313,7 +313,7 @@ struct ContextSharedPart
|
||||
/// since it may use per-user MemoryTracker which will be destroyed here.
|
||||
try
|
||||
{
|
||||
IDiskRemote::getThreadPoolWriter().wait();
|
||||
IObjectStorage::getThreadPoolWriter().wait();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
|
@ -791,7 +791,7 @@ void InterpreterSystemQuery::restartDisk(String & name)
|
||||
auto disk = getContext()->getDisk(name);
|
||||
|
||||
if (DiskRestartProxy * restart_proxy = dynamic_cast<DiskRestartProxy*>(disk.get()))
|
||||
restart_proxy->restart();
|
||||
restart_proxy->restart(getContext());
|
||||
else
|
||||
throw Exception("Disk " + name + " doesn't have possibility to restart", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Storages/MergeTree/DataPartsExchange.h>
|
||||
|
||||
#include <Formats/NativeWriter.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
#include <Disks/SingleDiskVolume.h>
|
||||
#include <Disks/createVolume.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
|
@ -184,7 +184,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
|
||||
if (storage_settings_ptr->allow_remote_fs_zero_copy_replication)
|
||||
{
|
||||
if (auto disk = reserved_space->getDisk(); disk->getType() == DB::DiskType::S3)
|
||||
if (auto disk = reserved_space->getDisk(); disk->supportZeroCopyReplication())
|
||||
{
|
||||
String dummy;
|
||||
if (!storage.findReplicaHavingCoveringPart(entry.new_part_name, true, dummy).empty())
|
||||
|
@ -110,7 +110,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare()
|
||||
|
||||
if (storage_settings_ptr->allow_remote_fs_zero_copy_replication)
|
||||
{
|
||||
if (auto disk = reserved_space->getDisk(); disk->getType() == DB::DiskType::S3)
|
||||
if (auto disk = reserved_space->getDisk(); disk->supportZeroCopyReplication())
|
||||
{
|
||||
String dummy;
|
||||
if (!storage.findReplicaHavingCoveringPart(entry.new_part_name, true, dummy).empty())
|
||||
|
@ -1219,7 +1219,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
|
||||
auto disks = storage.getDisks();
|
||||
bool only_s3_storage = true;
|
||||
for (const auto & disk : disks)
|
||||
if (disk->getType() != DB::DiskType::S3)
|
||||
if (!disk->supportZeroCopyReplication())
|
||||
only_s3_storage = false;
|
||||
|
||||
if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0]))
|
||||
|
@ -7758,7 +7758,8 @@ String StorageReplicatedMergeTree::getSharedDataReplica(
|
||||
}
|
||||
|
||||
|
||||
Strings StorageReplicatedMergeTree::getZeroCopyPartPath(const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid,
|
||||
Strings StorageReplicatedMergeTree::getZeroCopyPartPath(
|
||||
const MergeTreeSettings & settings, DiskType disk_type, const String & table_uuid,
|
||||
const String & part_name, const String & zookeeper_path_old)
|
||||
{
|
||||
Strings res;
|
||||
|
@ -86,8 +86,17 @@ class StorageS3Source::DisclosedGlobIterator::Impl : WithContext
|
||||
{
|
||||
|
||||
public:
|
||||
Impl(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_, ASTPtr & query_, const Block & virtual_header_, ContextPtr context_)
|
||||
: WithContext(context_), client(client_), globbed_uri(globbed_uri_), query(query_), virtual_header(virtual_header_)
|
||||
Impl(
|
||||
const Aws::S3::S3Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr & query_,
|
||||
const Block & virtual_header_,
|
||||
ContextPtr context_)
|
||||
: WithContext(context_)
|
||||
, client(client_)
|
||||
, globbed_uri(globbed_uri_)
|
||||
, query(query_)
|
||||
, virtual_header(virtual_header_)
|
||||
{
|
||||
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
|
||||
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
@ -230,7 +239,11 @@ private:
|
||||
};
|
||||
|
||||
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(
|
||||
Aws::S3::S3Client & client_, const S3::URI & globbed_uri_, ASTPtr query, const Block & virtual_header, ContextPtr context)
|
||||
const Aws::S3::S3Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const Block & virtual_header,
|
||||
ContextPtr context)
|
||||
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_, query, virtual_header, context))
|
||||
{
|
||||
}
|
||||
@ -379,7 +392,7 @@ StorageS3Source::StorageS3Source(
|
||||
UInt64 max_block_size_,
|
||||
UInt64 max_single_read_retries_,
|
||||
String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & client_,
|
||||
const String & bucket_,
|
||||
const String & version_id_,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
@ -516,7 +529,7 @@ Chunk StorageS3Source::generate()
|
||||
return {};
|
||||
}
|
||||
|
||||
static bool checkIfObjectExists(const std::shared_ptr<Aws::S3::S3Client> & client, const String & bucket, const String & key)
|
||||
static bool checkIfObjectExists(const std::shared_ptr<const Aws::S3::S3Client> & client, const String & bucket, const String & key)
|
||||
{
|
||||
bool is_finished = false;
|
||||
Aws::S3::Model::ListObjectsV2Request request;
|
||||
@ -667,7 +680,6 @@ private:
|
||||
const Block sample_block;
|
||||
ContextPtr context;
|
||||
const CompressionMethod compression_method;
|
||||
|
||||
const StorageS3::S3Configuration & s3_configuration;
|
||||
const String bucket;
|
||||
const String key;
|
||||
@ -1010,7 +1022,8 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration
|
||||
|
||||
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
|
||||
settings.auth_settings.region,
|
||||
ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects);
|
||||
ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects,
|
||||
ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging);
|
||||
|
||||
client_configuration.endpointOverride = upd.uri.endpoint;
|
||||
client_configuration.maxConnections = upd.rw_settings.max_connections;
|
||||
|
@ -36,7 +36,12 @@ public:
|
||||
{
|
||||
public:
|
||||
DisclosedGlobIterator(
|
||||
Aws::S3::S3Client & client_, const S3::URI & globbed_uri_, ASTPtr query, const Block & virtual_header, ContextPtr context);
|
||||
const Aws::S3::S3Client & client_,
|
||||
const S3::URI & globbed_uri_,
|
||||
ASTPtr query,
|
||||
const Block & virtual_header,
|
||||
ContextPtr context);
|
||||
|
||||
String next();
|
||||
|
||||
private:
|
||||
@ -85,7 +90,7 @@ public:
|
||||
UInt64 max_block_size_,
|
||||
UInt64 max_single_read_retries_,
|
||||
String compression_hint_,
|
||||
const std::shared_ptr<Aws::S3::S3Client> & client_,
|
||||
const std::shared_ptr<const Aws::S3::S3Client> & client_,
|
||||
const String & bucket,
|
||||
const String & version_id,
|
||||
std::shared_ptr<IteratorWrapper> file_iterator_,
|
||||
@ -107,7 +112,7 @@ private:
|
||||
UInt64 max_block_size;
|
||||
UInt64 max_single_read_retries;
|
||||
String compression_hint;
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client;
|
||||
Block sample_block;
|
||||
std::optional<FormatSettings> format_settings;
|
||||
|
||||
@ -194,7 +199,7 @@ public:
|
||||
const S3::URI uri;
|
||||
const String access_key_id;
|
||||
const String secret_access_key;
|
||||
std::shared_ptr<Aws::S3::S3Client> client;
|
||||
std::shared_ptr<const Aws::S3::S3Client> client;
|
||||
S3Settings::AuthSettings auth_settings;
|
||||
S3Settings::ReadWriteSettings rw_settings;
|
||||
};
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Storages/System/StorageSystemDisks.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Disks/IDiskRemote.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -214,8 +214,9 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
# Wait for merges and old parts deletion
|
||||
for attempt in range(0, 10):
|
||||
parts_count = node.query(
|
||||
"SELECT COUNT(*) FROM system.parts WHERE table = 's3_test' FORMAT Values"
|
||||
"SELECT COUNT(*) FROM system.parts WHERE table = 's3_test' and active = 1 FORMAT Values"
|
||||
)
|
||||
|
||||
if parts_count == "(1)":
|
||||
break
|
||||
|
||||
@ -228,7 +229,9 @@ def test_insert_same_partition_and_merge(cluster, merge_vertical, node_name):
|
||||
assert (
|
||||
node.query("SELECT count(distinct(id)) FROM s3_test FORMAT Values") == "(8192)"
|
||||
)
|
||||
wait_for_delete_s3_objects(cluster, FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD)
|
||||
wait_for_delete_s3_objects(
|
||||
cluster, FILES_OVERHEAD_PER_PART_WIDE + FILES_OVERHEAD, timeout=45
|
||||
)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("node_name", ["node"])
|
||||
|
@ -88,6 +88,8 @@ def generate_values(date_str, count, sign=1):
|
||||
def create_table(
|
||||
node, table_name, attach=False, replicated=False, db_atomic=False, uuid=""
|
||||
):
|
||||
node.query("DROP DATABASE IF EXISTS s3")
|
||||
|
||||
node.query(
|
||||
"CREATE DATABASE IF NOT EXISTS s3 ENGINE = {engine}".format(
|
||||
engine="Atomic" if db_atomic else "Ordinary"
|
||||
@ -501,8 +503,8 @@ def test_restore_mutations(cluster, db_atomic):
|
||||
) == "({})".format(4096)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("db_atomic", [False, True])
|
||||
def test_migrate_to_restorable_schema(cluster, db_atomic):
|
||||
def test_migrate_to_restorable_schema(cluster):
|
||||
db_atomic = True
|
||||
node = cluster.instances["node_not_restorable"]
|
||||
|
||||
create_table(node, "test", db_atomic=db_atomic)
|
||||
|
@ -4,6 +4,7 @@ import time
|
||||
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.test_tools import assert_eq_with_retry
|
||||
|
||||
from pyhdfs import HdfsClient
|
||||
|
||||
@ -77,7 +78,7 @@ def test_hdfs_zero_copy_replication_insert(cluster):
|
||||
)
|
||||
|
||||
node1.query("INSERT INTO hdfs_test VALUES (now() - INTERVAL 3 DAY, 10)")
|
||||
node2.query("SYSTEM SYNC REPLICA hdfs_test")
|
||||
node2.query("SYSTEM SYNC REPLICA hdfs_test", timeout=30)
|
||||
assert node1.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)"
|
||||
assert node2.query("SELECT count() FROM hdfs_test FORMAT Values") == "(1)"
|
||||
assert (
|
||||
@ -192,7 +193,7 @@ def test_hdfs_zero_copy_replication_move(cluster, storage_policy, init_objects):
|
||||
node1.query(
|
||||
"INSERT INTO move_test VALUES (now() - INTERVAL 3 DAY, 10), (now() - INTERVAL 1 DAY, 11)"
|
||||
)
|
||||
node2.query("SYSTEM SYNC REPLICA move_test")
|
||||
node2.query("SYSTEM SYNC REPLICA move_test", timeout=30)
|
||||
|
||||
assert (
|
||||
node1.query("SELECT id FROM move_test ORDER BY dt FORMAT Values")
|
||||
@ -262,10 +263,10 @@ def test_hdfs_zero_copy_with_ttl_move(cluster, storage_policy):
|
||||
node1.query("INSERT INTO ttl_move_test VALUES (now() - INTERVAL 1 DAY, 11)")
|
||||
|
||||
node1.query("OPTIMIZE TABLE ttl_move_test FINAL")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_move_test")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_move_test", timeout=30)
|
||||
|
||||
assert node1.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)"
|
||||
assert node2.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)"
|
||||
assert_eq_with_retry(node1, "SELECT count() FROM ttl_move_test", "2")
|
||||
assert_eq_with_retry(node2, "SELECT count() FROM ttl_move_test", "2")
|
||||
assert (
|
||||
node1.query("SELECT id FROM ttl_move_test ORDER BY id FORMAT Values")
|
||||
== "(10),(11)"
|
||||
@ -297,10 +298,11 @@ def test_hdfs_zero_copy_with_ttl_delete(cluster):
|
||||
node1.query("INSERT INTO ttl_delete_test VALUES (now() - INTERVAL 1 DAY, 11)")
|
||||
|
||||
node1.query("OPTIMIZE TABLE ttl_delete_test FINAL")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_delete_test")
|
||||
node2.query("SYSTEM SYNC REPLICA ttl_delete_test", timeout=30)
|
||||
|
||||
assert_eq_with_retry(node1, "SELECT count() FROM ttl_delete_test", "1")
|
||||
assert_eq_with_retry(node2, "SELECT count() FROM ttl_delete_test", "1")
|
||||
|
||||
assert node1.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)"
|
||||
assert node2.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)"
|
||||
assert (
|
||||
node1.query("SELECT id FROM ttl_delete_test ORDER BY id FORMAT Values")
|
||||
== "(11)"
|
||||
|
@ -3,14 +3,43 @@
|
||||
SET enable_filesystem_cache_on_write_operations=1;
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
SYSTEM STOP MERGES test;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
0
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
0
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
Row 1:
|
||||
──────
|
||||
file_segment_range_begin: 0
|
||||
@ -33,7 +62,21 @@ SELECT count() size FROM system.filesystem_cache;
|
||||
7
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200);
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
Row 1:
|
||||
──────
|
||||
file_segment_range_begin: 0
|
||||
@ -53,6 +96,7 @@ INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000);
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
21
|
||||
SYSTEM START MERGES test;
|
||||
OPTIMIZE TABLE test FINAL;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
27
|
||||
@ -62,12 +106,17 @@ SELECT count() FROM system.filesystem_cache;
|
||||
28
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(5000000);
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read
|
||||
FROM system.query_log
|
||||
WHERE query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%'
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = currentDatabase()
|
||||
ORDER BY query_start_time DESC
|
||||
SELECT
|
||||
query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read
|
||||
FROM
|
||||
system.query_log
|
||||
WHERE
|
||||
query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%'
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = currentDatabase()
|
||||
ORDER BY
|
||||
query_start_time
|
||||
DESC
|
||||
LIMIT 1;
|
||||
SELECT count() FROM test;
|
||||
5010500
|
||||
|
@ -6,16 +6,45 @@ SET enable_filesystem_cache_on_write_operations=1;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (key UInt32, value String) Engine=MergeTree() ORDER BY key SETTINGS storage_policy='s3_cache', min_bytes_for_wide_part = 10485760;
|
||||
|
||||
SYSTEM STOP MERGES test;
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
@ -33,7 +62,22 @@ SYSTEM DROP FILESYSTEM CACHE;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100, 200);
|
||||
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state FROM (SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path) WHERE endsWith(local_path, 'data.bin') FORMAT Vertical;
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state
|
||||
FROM
|
||||
(
|
||||
SELECT file_segment_range_begin, file_segment_range_end, size, state, local_path
|
||||
FROM
|
||||
(
|
||||
SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path
|
||||
FROM system.remote_data_paths
|
||||
) AS data_paths
|
||||
INNER JOIN
|
||||
system.filesystem_cache AS caches
|
||||
ON data_paths.cache_path = caches.cache_path
|
||||
)
|
||||
WHERE endsWith(local_path, 'data.bin')
|
||||
FORMAT Vertical;
|
||||
|
||||
SELECT count() FROM (SELECT arrayJoin(cache_paths) AS cache_path, local_path, remote_path FROM system.remote_data_paths ) AS data_paths INNER JOIN system.filesystem_cache AS caches ON data_paths.cache_path = caches.cache_path;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
@ -44,6 +88,9 @@ SELECT count() FROM system.filesystem_cache;
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(100);
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(300, 10000);
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
SYSTEM START MERGES test;
|
||||
|
||||
OPTIMIZE TABLE test FINAL;
|
||||
SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
@ -53,12 +100,18 @@ SELECT count() FROM system.filesystem_cache;
|
||||
|
||||
INSERT INTO test SELECT number, toString(number) FROM numbers(5000000);
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read
|
||||
FROM system.query_log
|
||||
WHERE query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%'
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = currentDatabase()
|
||||
ORDER BY query_start_time DESC
|
||||
SELECT
|
||||
query, ProfileEvents['RemoteFSReadBytes'] > 0 as remote_fs_read
|
||||
FROM
|
||||
system.query_log
|
||||
WHERE
|
||||
query LIKE 'SELECT number, toString(number) FROM numbers(5000000)%'
|
||||
AND type = 'QueryFinish'
|
||||
AND current_database = currentDatabase()
|
||||
ORDER BY
|
||||
query_start_time
|
||||
DESC
|
||||
LIMIT 1;
|
||||
|
||||
SELECT count() FROM test;
|
||||
SELECT count() FROM test WHERE value LIKE '%010%';
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
SET max_memory_usage='20G';
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT count() FROM test.hits_s3;
|
||||
8873898
|
||||
SELECT count() FROM test.hits_s3 WHERE AdvEngineID != 0;
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
SET enable_filesystem_cache_on_write_operations=0;
|
||||
SET max_memory_usage='20G';
|
||||
SYSTEM DROP FILESYSTEM CACHE;
|
||||
SELECT count() FROM test.hits_s3;
|
||||
SELECT count() FROM test.hits_s3 WHERE AdvEngineID != 0;
|
||||
SELECT sum(AdvEngineID), count(), avg(ResolutionWidth) FROM test.hits_s3 ;
|
||||
|
Loading…
Reference in New Issue
Block a user