Merge branch 'master' into turn_on_s3_tests

This commit is contained in:
alesapin 2022-05-27 14:27:24 +02:00
commit e5d7eeb5f9
86 changed files with 4018 additions and 3136 deletions

View File

@ -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)

View File

@ -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) \

View File

@ -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) \

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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)

View File

@ -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); }

View File

@ -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);

View File

@ -110,7 +110,7 @@ public:
bool isBroken() const override { return broken; }
void startup() override;
void startup(ContextPtr) override;
void shutdown() override;

View File

@ -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());
}

View File

@ -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;

View File

@ -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

View File

@ -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
{

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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;
}
}

View File

@ -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;
};
}

View File

@ -1,6 +1,5 @@
#include "ReadBufferFromRemoteFSGather.h"
#include <Disks/IDiskRemote.h>
#include <IO/SeekableReadBuffer.h>
#include <Disks/IO/ReadBufferFromWebServer.h>

View File

@ -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;
};

View File

@ -2,7 +2,6 @@
#include <Common/config.h>
#include <IO/ReadBufferFromFile.h>
#include <Disks/IDiskRemote.h>
#include <utility>

View File

@ -4,7 +4,6 @@
#include <IO/SeekableReadBuffer.h>
#include <Common/ThreadPool.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IDiskRemote.h>
namespace DB

View File

@ -36,7 +36,8 @@ WriteIndirectBufferFromRemoteFS::~WriteIndirectBufferFromRemoteFS()
void WriteIndirectBufferFromRemoteFS::finalizeImpl()
{
WriteBufferFromFileDecorator::finalizeImpl();
create_metadata_callback(count());
if (create_metadata_callback)
create_metadata_callback(count());
}

View File

@ -2,7 +2,6 @@
#include <Common/config.h>
#include <Disks/IDiskRemote.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromFileDecorator.h>

View File

@ -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

View File

@ -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

View 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

View 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

View File

@ -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);

View 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__);
}
}
}

View 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;
};
}

View File

@ -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(

View File

@ -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,

View 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();
}
}

View 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)>;
}

View 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);
}
}

View 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;
};
}

View 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

View 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

View 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);
}
}

View 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();
}
}

View 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>;
}

View 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

View 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

View File

@ -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

View 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

View 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);
}

View 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

View File

@ -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
View File

@ -0,0 +1,15 @@
#pragma once
namespace DB
{
/**
* Mode of opening a file for write.
*/
enum class WriteMode
{
Rewrite,
Append
};
}

View File

@ -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_,

View File

@ -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;

View File

@ -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_,

View File

@ -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;

View File

@ -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

View File

@ -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;
};
}

View File

@ -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);

View File

@ -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);
}

View File

@ -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)
{

View File

@ -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;
};
}

View File

@ -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_)
{

View File

@ -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;

View File

@ -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 (...)
{

View File

@ -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);
}

View File

@ -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>

View File

@ -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())

View File

@ -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())

View File

@ -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]))

View File

@ -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;

View File

@ -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;

View File

@ -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;
};

View File

@ -1,7 +1,6 @@
#include <Storages/System/StorageSystemDisks.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Interpreters/Context.h>
#include <Disks/IDiskRemote.h>
namespace DB
{

View File

@ -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"])

View File

@ -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)

View File

@ -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)"

View File

@ -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

View File

@ -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%';

View File

@ -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;

View File

@ -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 ;