ClickHouse/src/Disks/IDiskRemote.h

284 lines
7.9 KiB
C++
Raw Normal View History

2021-04-16 00:11:47 +00:00
#pragma once
2021-06-11 11:26:33 +00:00
2021-05-20 13:02:26 +00:00
#include <Common/config.h>
2021-04-16 00:11:47 +00:00
#include <atomic>
2021-09-17 15:27:43 +00:00
#include <Disks/DiskFactory.h>
#include <Disks/Executor.h>
2021-05-19 12:03:00 +00:00
#include <utility>
#include <Common/MultiVersion.h>
2021-05-25 11:29:46 +00:00
#include <Common/ThreadPool.h>
2021-05-26 21:02:24 +00:00
#include <filesystem>
2021-04-16 00:11:47 +00:00
2021-05-26 21:02:24 +00:00
namespace fs = std::filesystem;
2021-04-16 00:11:47 +00:00
namespace CurrentMetrics
{
extern const Metric DiskSpaceReservedForMerge;
}
2021-04-16 00:11:47 +00:00
namespace DB
{
2021-05-25 11:29:46 +00:00
/// Helper class to collect paths into chunks of maximum size.
2021-06-18 11:07:41 +00:00
/// For s3 it is Aws::vector<ObjectIdentifier>, for hdfs it is std::vector<std::string>.
2021-05-25 11:29:46 +00:00
class RemoteFSPathKeeper
2021-05-19 12:03:00 +00:00
{
public:
2021-05-25 11:29:46 +00:00
RemoteFSPathKeeper(size_t chunk_limit_) : chunk_limit(chunk_limit_) {}
2021-05-19 12:03:00 +00:00
2021-05-25 11:29:46 +00:00
virtual ~RemoteFSPathKeeper() = default;
virtual void addPath(const String & path) = 0;
protected:
size_t chunk_limit;
2021-05-19 12:03:00 +00:00
};
2021-04-16 00:11:47 +00:00
2021-05-25 11:29:46 +00:00
using RemoteFSPathKeeperPtr = std::shared_ptr<RemoteFSPathKeeper>;
2021-09-21 07:00:56 +00:00
class IAsynchronousReader;
using AsynchronousReaderPtr = std::shared_ptr<IAsynchronousReader>;
2021-09-28 14:37:38 +00:00
/// Base Disk class for remote FS's, which are not posix-compatible (e.g. DiskS3, DiskHDFS, DiskBlobStorage)
2021-04-16 00:11:47 +00:00
class IDiskRemote : public IDisk
{
2021-06-18 11:07:41 +00:00
2021-04-19 08:14:48 +00:00
friend class DiskRemoteReservation;
2021-04-16 00:11:47 +00:00
public:
IDiskRemote(
2021-05-11 15:29:37 +00:00
const String & name_,
2021-04-16 00:11:47 +00:00
const String & remote_fs_root_path_,
DiskPtr metadata_disk_,
2021-04-19 08:14:48 +00:00
const String & log_name_,
2021-05-25 11:29:46 +00:00
size_t thread_pool_size);
2021-04-16 00:11:47 +00:00
2021-06-13 14:02:08 +00:00
struct Metadata;
2021-04-19 08:14:48 +00:00
2021-06-18 11:07:41 +00:00
const String & getName() const final override { return name; }
const String & getPath() const final override { return metadata_disk->getPath(); }
2021-06-18 11:07:41 +00:00
2021-04-16 00:11:47 +00:00
Metadata readMeta(const String & path) const;
Metadata createMeta(const String & path) const;
2021-05-11 15:29:37 +00:00
Metadata readOrCreateMetaForWriting(const String & path, WriteMode mode);
2021-06-18 11:07:41 +00:00
UInt64 getTotalSpace() const override { return std::numeric_limits<UInt64>::max(); }
2021-06-13 14:02:08 +00:00
2021-06-18 11:07:41 +00:00
UInt64 getAvailableSpace() const override { return std::numeric_limits<UInt64>::max(); }
2021-06-13 14:02:08 +00:00
2021-06-18 11:07:41 +00:00
UInt64 getUnreservedSpace() const override { return std::numeric_limits<UInt64>::max(); }
2021-04-16 00:11:47 +00:00
2021-06-18 11:07:41 +00:00
UInt64 getKeepingFreeSpace() const override { return 0; }
2021-04-16 00:11:47 +00:00
2021-06-13 14:02:08 +00:00
bool exists(const String & path) const override;
2021-04-16 00:11:47 +00:00
2021-06-13 14:02:08 +00:00
bool isFile(const String & path) const override;
2021-04-16 00:11:47 +00:00
2021-06-18 11:07:41 +00:00
void createFile(const String & path) override;
2021-06-12 14:29:16 +00:00
2021-06-18 11:07:41 +00:00
size_t getFileSize(const String & path) const override;
2021-04-16 00:11:47 +00:00
2021-04-20 11:41:32 +00:00
void moveFile(const String & from_path, const String & to_path) override;
2021-04-16 00:11:47 +00:00
void replaceFile(const String & from_path, const String & to_path) override;
2021-05-19 12:03:00 +00:00
void removeFile(const String & path) override { removeSharedFile(path, false); }
void removeFileIfExists(const String & path) override { removeSharedFileIfExists(path, false); }
2021-05-19 12:03:00 +00:00
void removeRecursive(const String & path) override { removeSharedRecursive(path, false); }
void removeSharedFile(const String & path, bool keep_in_remote_fs) override;
void removeSharedFileIfExists(const String & path, bool keep_in_remote_fs) override;
2021-05-19 12:03:00 +00:00
void removeSharedRecursive(const String & path, bool keep_in_remote_fs) override;
2021-06-18 11:07:41 +00:00
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;
2021-04-16 00:11:47 +00:00
void clearDirectory(const String & path) override;
2021-04-19 15:50:46 +00:00
void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); }
2021-04-16 00:11:47 +00:00
void removeDirectory(const String & path) override;
2021-06-18 11:07:41 +00:00
DiskDirectoryIteratorPtr iterateDirectory(const String & path) override;
2021-04-16 00:11:47 +00:00
2021-06-18 11:07:41 +00:00
void setLastModified(const String & path, const Poco::Timestamp & timestamp) override;
2021-04-20 11:41:32 +00:00
2021-06-18 11:07:41 +00:00
Poco::Timestamp getLastModified(const String & path) override;
2021-06-12 14:29:16 +00:00
2021-06-18 11:07:41 +00:00
void createHardLink(const String & src_path, const String & dst_path) override;
2021-06-12 14:29:16 +00:00
2021-06-18 11:07:41 +00:00
ReservationPtr reserve(UInt64 bytes) override;
2021-04-20 11:41:32 +00:00
2021-07-05 03:32:56 +00:00
String getUniqueId(const String & path) const override;
bool checkUniqueId(const String & id) const override = 0;
2021-06-18 11:07:41 +00:00
virtual void removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) = 0;
2021-05-25 11:29:46 +00:00
2021-06-18 11:07:41 +00:00
virtual RemoteFSPathKeeperPtr createFSPathKeeper() const = 0;
2021-05-19 12:03:00 +00:00
2021-09-21 07:00:56 +00:00
static AsynchronousReaderPtr getThreadPoolReader();
2021-04-16 00:11:47 +00:00
protected:
2021-05-19 12:03:00 +00:00
Poco::Logger * log;
2021-05-11 15:29:37 +00:00
const String name;
2021-04-16 00:11:47 +00:00
const String remote_fs_root_path;
2021-05-19 12:03:00 +00:00
DiskPtr metadata_disk;
2021-04-19 08:14:48 +00:00
2021-04-20 11:41:32 +00:00
private:
2021-05-25 11:29:46 +00:00
void removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
2021-05-19 12:03:00 +00:00
2021-05-25 11:29:46 +00:00
void removeMetaRecursive(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper);
2021-05-19 12:03:00 +00:00
2021-04-20 11:41:32 +00:00
bool tryReserve(UInt64 bytes);
2021-04-19 08:14:48 +00:00
UInt64 reserved_bytes = 0;
UInt64 reservation_count = 0;
std::mutex reservation_mutex;
2021-04-16 00:11:47 +00:00
};
2021-04-20 11:41:32 +00:00
using RemoteDiskPtr = std::shared_ptr<IDiskRemote>;
2021-04-19 08:14:48 +00:00
2021-06-12 14:29:16 +00:00
2021-06-18 11:07:41 +00:00
/// Minimum info, required to be passed to ReadIndirectBufferFromRemoteFS<T>
struct RemoteMetadata
{
using PathAndSize = std::pair<String, size_t>;
/// Remote FS objects paths and their sizes.
std::vector<PathAndSize> remote_fs_objects;
/// URI
const String & remote_fs_root_path;
/// Relative path to metadata file on local FS.
2021-06-18 14:32:08 +00:00
const String metadata_file_path;
2021-06-18 11:07:41 +00:00
RemoteMetadata(const String & remote_fs_root_path_, const String & metadata_file_path_)
: remote_fs_root_path(remote_fs_root_path_), metadata_file_path(metadata_file_path_) {}
};
/// Remote FS (S3, HDFS) metadata file layout:
2021-06-13 19:30:59 +00:00
/// FS objects, their number and total size of all FS objects.
/// Each FS object represents a file path in remote FS and its size.
2021-04-16 00:11:47 +00:00
2021-06-18 11:07:41 +00:00
struct IDiskRemote::Metadata : RemoteMetadata
2021-04-16 00:11:47 +00:00
{
/// 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;
DiskPtr metadata_disk;
2021-04-16 00:11:47 +00:00
2021-06-18 11:07:41 +00:00
/// Total size of all remote FS (S3, HDFS) objects.
2021-05-21 08:35:13 +00:00
size_t total_size = 0;
2021-04-16 00:11:47 +00:00
/// Number of references (hardlinks) to this metadata file.
2021-05-21 08:35:13 +00:00
UInt32 ref_count = 0;
2021-04-16 00:11:47 +00:00
/// Flag indicates that file is read only.
bool read_only = false;
/// Load metadata by path or create empty if `create` flag is set.
Metadata(const String & remote_fs_root_path_,
DiskPtr metadata_disk_,
2021-04-16 00:11:47 +00:00
const String & metadata_file_path_,
bool create = false);
void addObject(const String & path, size_t size);
/// Fsync metadata file if 'sync' flag is set.
void save(bool sync = false);
};
2021-04-19 08:14:48 +00:00
class DiskRemoteReservation final : public IReservation
{
public:
2021-04-20 11:41:32 +00:00
DiskRemoteReservation(const RemoteDiskPtr & disk_, UInt64 size_)
2021-04-19 08:14:48 +00:00
: disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_)
{
}
UInt64 getSize() const override { return size; }
DiskPtr getDisk(size_t i) const override;
Disks getDisks() const override { return {disk}; }
void update(UInt64 new_size) override;
~DiskRemoteReservation() override;
private:
2021-04-20 11:41:32 +00:00
RemoteDiskPtr disk;
2021-04-19 08:14:48 +00:00
UInt64 size;
CurrentMetrics::Increment metric_increment;
};
2021-05-20 13:02:26 +00:00
2021-05-25 11:29:46 +00:00
/// 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;
};
}