ClickHouse/src/Disks/IDisk.h

428 lines
15 KiB
C++
Raw Normal View History

2019-11-27 09:39:44 +00:00
#pragma once
2021-04-20 18:29:03 +00:00
#include <Interpreters/Context_fwd.h>
#include <Interpreters/Context.h>
#include <Core/Defines.h>
2021-10-02 07:13:14 +00:00
#include <base/types.h>
2019-11-27 09:39:44 +00:00
#include <Common/CurrentMetrics.h>
#include <Common/Exception.h>
#include <Disks/Executor.h>
2021-02-26 09:48:57 +00:00
#include <Disks/DiskType.h>
2021-08-16 00:00:32 +00:00
#include <IO/ReadSettings.h>
2022-03-14 18:40:53 +00:00
#include <IO/WriteSettings.h>
2019-11-27 09:39:44 +00:00
#include <memory>
#include <mutex>
2019-11-27 09:39:44 +00:00
#include <utility>
#include <boost/noncopyable.hpp>
#include <Poco/Timestamp.h>
2021-04-30 15:13:19 +00:00
#include <filesystem>
2019-11-27 09:39:44 +00:00
2021-08-16 00:00:32 +00:00
2021-04-30 15:13:19 +00:00
namespace fs = std::filesystem;
2019-11-27 09:39:44 +00:00
2021-08-16 00:00:32 +00:00
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
2019-11-27 09:39:44 +00:00
namespace DB
{
2021-08-16 00:00:32 +00:00
2022-03-23 12:01:18 +00:00
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
2019-12-03 13:37:40 +00:00
class IDiskDirectoryIterator;
using DiskDirectoryIteratorPtr = std::unique_ptr<IDiskDirectoryIterator>;
2019-11-27 09:39:44 +00:00
class IReservation;
2020-05-16 20:31:17 +00:00
using ReservationPtr = std::unique_ptr<IReservation>;
using Reservations = std::vector<ReservationPtr>;
2019-11-27 09:39:44 +00:00
2020-02-14 14:28:33 +00:00
class ReadBufferFromFileBase;
class WriteBufferFromFileBase;
2021-03-28 19:24:28 +00:00
class MMappedFileCache;
2019-11-27 09:39:44 +00:00
2019-12-26 14:28:22 +00:00
/**
* Mode of opening a file for write.
*/
enum class WriteMode
{
Rewrite,
Append
};
2019-11-27 09:39:44 +00:00
/**
2019-11-27 15:49:15 +00:00
* Provide interface for reservation.
2019-11-27 09:39:44 +00:00
*/
class Space : public std::enable_shared_from_this<Space>
{
public:
2019-12-03 13:37:40 +00:00
/// Return the name of the space object.
2019-11-27 09:39:44 +00:00
virtual const String & getName() const = 0;
2019-12-03 13:37:40 +00:00
/// Reserve the specified number of bytes.
virtual ReservationPtr reserve(UInt64 bytes) = 0;
2019-11-27 09:39:44 +00:00
virtual ~Space() = default;
};
2019-12-03 13:37:40 +00:00
using SpacePtr = std::shared_ptr<Space>;
2019-11-27 09:39:44 +00:00
2021-01-26 13:29:45 +00:00
/**
2021-01-27 00:32:18 +00:00
* A guard, that should synchronize file's or directory's state
2021-01-26 13:29:45 +00:00
* with storage device (e.g. fsync in POSIX) in its destructor.
*/
class ISyncGuard
{
public:
ISyncGuard() = default;
virtual ~ISyncGuard() = default;
};
using SyncGuardPtr = std::unique_ptr<ISyncGuard>;
/**
* A unit of storage persisting data and metadata.
* Abstract underlying storage technology.
* Responsible for:
* - file management;
* - space accounting and reservation.
*/
2019-11-27 09:39:44 +00:00
class IDisk : public Space
{
public:
/// Default constructor.
explicit IDisk(std::unique_ptr<Executor> executor_ = std::make_unique<SyncExecutor>()) : executor(std::move(executor_)) { }
2019-12-03 13:37:40 +00:00
/// Root path for all files stored on the disk.
/// It's not required to be a local filesystem path.
2019-11-27 09:39:44 +00:00
virtual const String & getPath() const = 0;
2019-12-03 13:37:40 +00:00
/// Total available space on the disk.
2019-11-27 09:39:44 +00:00
virtual UInt64 getTotalSpace() const = 0;
2019-12-03 13:37:40 +00:00
/// Space currently available on the disk.
2019-11-27 09:39:44 +00:00
virtual UInt64 getAvailableSpace() const = 0;
2019-12-03 13:37:40 +00:00
/// Space available for reservation (available space minus reserved space).
2019-11-27 09:39:44 +00:00
virtual UInt64 getUnreservedSpace() const = 0;
2019-12-03 13:37:40 +00:00
/// Amount of bytes which should be kept free on the disk.
2019-11-27 09:39:44 +00:00
virtual UInt64 getKeepingFreeSpace() const { return 0; }
2019-12-03 13:37:40 +00:00
/// Return `true` if the specified file exists.
virtual bool exists(const String & path) const = 0;
2019-11-27 09:39:44 +00:00
2019-12-03 13:37:40 +00:00
/// Return `true` if the specified file exists and it's a regular file (not a directory or special file type).
virtual bool isFile(const String & path) const = 0;
2019-11-27 09:39:44 +00:00
2019-12-03 13:37:40 +00:00
/// Return `true` if the specified file exists and it's a directory.
virtual bool isDirectory(const String & path) const = 0;
2019-11-27 09:39:44 +00:00
/// Return size of the specified file.
virtual size_t getFileSize(const String & path) const = 0;
2019-12-03 13:37:40 +00:00
/// Create directory.
virtual void createDirectory(const String & path) = 0;
2019-11-27 09:39:44 +00:00
2019-12-03 13:37:40 +00:00
/// Create directory and all parent directories if necessary.
virtual void createDirectories(const String & path) = 0;
2019-11-27 09:39:44 +00:00
2020-01-18 23:18:23 +00:00
/// Remove all files from the directory. Directories are not removed.
virtual void clearDirectory(const String & path) = 0;
/// Move directory from `from_path` to `to_path`.
virtual void moveDirectory(const String & from_path, const String & to_path) = 0;
2019-12-03 13:37:40 +00:00
/// Return iterator to the contents of the specified directory.
virtual DiskDirectoryIteratorPtr iterateDirectory(const String & path) = 0;
2019-11-27 09:39:44 +00:00
/// Return `true` if the specified directory is empty.
bool isDirectoryEmpty(const String & path);
/// Create empty file at `path`.
virtual void createFile(const String & path) = 0;
2019-12-03 13:37:40 +00:00
/// Move the file from `from_path` to `to_path`.
2020-01-02 14:37:31 +00:00
/// If a file with `to_path` path already exists, an exception will be thrown .
2019-12-03 13:37:40 +00:00
virtual void moveFile(const String & from_path, const String & to_path) = 0;
2019-11-27 09:39:44 +00:00
2020-01-02 14:37:31 +00:00
/// Move the file from `from_path` to `to_path`.
/// If a file with `to_path` path already exists, it will be replaced.
virtual void replaceFile(const String & from_path, const String & to_path) = 0;
/// Recursively copy data containing at `from_path` to `to_path` located at `to_disk`.
virtual void copy(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path);
2020-02-28 17:14:55 +00:00
/// List files at `path` and add their names to `file_names`
virtual void listFiles(const String & path, std::vector<String> & file_names) = 0;
/// Open the file for read and return ReadBufferFromFileBase object.
virtual std::unique_ptr<ReadBufferFromFileBase> readFile( /// NOLINT
const String & path,
2021-08-16 00:00:32 +00:00
const ReadSettings & 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> writeFile( /// NOLINT
const String & path,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
2022-03-14 18:40:53 +00:00
WriteMode mode = WriteMode::Rewrite,
const WriteSettings & settings = {}) = 0;
/// Remove file. Throws exception if file doesn't exists or it's a directory.
virtual void removeFile(const String & path) = 0;
/// Remove file if it exists.
virtual void removeFileIfExists(const String & path) = 0;
/// Remove directory. Throws exception if it's not a directory or if directory is not empty.
virtual void removeDirectory(const String & path) = 0;
2020-01-18 23:18:23 +00:00
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
virtual void removeRecursive(const String & path) = 0;
2021-01-20 09:23:03 +00:00
/// Remove file. Throws exception if file doesn't exists or if directory is not empty.
2021-04-20 11:41:32 +00:00
/// Differs from removeFile for S3/HDFS disks
2021-02-26 09:48:57 +00:00
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
2021-01-20 09:23:03 +00:00
virtual void removeSharedFile(const String & path, bool) { removeFile(path); }
/// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists.
2021-04-20 11:41:32 +00:00
/// Differs from removeRecursive for S3/HDFS disks
2021-02-26 09:48:57 +00:00
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
virtual void removeSharedRecursive(const String & path, bool) { removeRecursive(path); }
/// Remove file or directory if it exists.
2021-04-20 11:41:32 +00:00
/// Differs from removeFileIfExists for S3/HDFS disks
2021-02-26 09:48:57 +00:00
/// Second bool param is a flag to remove (true) or keep (false) shared data on S3
2021-01-20 09:23:03 +00:00
virtual void removeSharedFileIfExists(const String & path, bool) { removeFileIfExists(path); }
2022-03-23 12:01:18 +00:00
virtual String getCacheBasePath() const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getCacheBasePath() not implemented fro disk: {}`", getType());
}
/// Returnes a list of paths because for Log family engines
/// there might be multiple files in remote fs for single clickhouse file.
virtual std::vector<String> getRemotePaths(const String &) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePaths() not implemented fro disk: {}`", getType());
}
/// For one local path there might be multiple remote paths in case of Log family engines.
using LocalPathWithRemotePaths = std::pair<String, std::vector<String>>;
virtual void getRemotePathsRecursive(const String &, std::vector<LocalPathWithRemotePaths> &)
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method `getRemotePathsRecursive() not implemented fro disk: {}`", getType());
}
struct RemoveRequest
{
String path;
bool if_exists = false;
explicit RemoveRequest(String path_, bool if_exists_ = false)
: path(std::move(path_)), if_exists(std::move(if_exists_))
{
}
};
using RemoveBatchRequest = std::vector<RemoveRequest>;
/// Batch request to remove multiple files.
/// May be much faster for blob storage.
virtual void removeSharedFiles(const RemoveBatchRequest & files, bool keep_in_remote_fs)
{
for (const auto & file : files)
{
if (file.if_exists)
removeSharedFileIfExists(file.path, keep_in_remote_fs);
else
removeSharedFile(file.path, keep_in_remote_fs);
}
}
/// Set last modified time to file or directory at `path`.
virtual void setLastModified(const String & path, const Poco::Timestamp & timestamp) = 0;
/// Get last modified time of file or directory at `path`.
virtual Poco::Timestamp getLastModified(const String & path) = 0;
/// Set file at `path` as read-only.
virtual void setReadOnly(const String & path) = 0;
/// Create hardlink from `src_path` to `dst_path`.
virtual void createHardLink(const String & src_path, const String & dst_path) = 0;
/// Truncate file to specified size.
virtual void truncateFile(const String & path, size_t size);
2020-07-03 12:45:01 +00:00
/// Return disk type - "local", "s3", etc.
2021-08-24 22:24:47 +00:00
virtual DiskType getType() const = 0;
2021-08-24 23:27:20 +00:00
/// Involves network interaction.
virtual bool isRemote() const = 0;
2021-07-05 03:32:56 +00:00
/// Whether this disk support zero-copy replication.
/// Overrode in remote fs disks.
virtual bool supportZeroCopyReplication() const = 0;
/// Whether this disk support parallel write
/// Overrode in remote fs disks.
virtual bool supportParallelWrite() const { return false; }
2021-08-23 11:26:54 +00:00
virtual bool isReadOnly() const { return false; }
/// Check if disk is broken. Broken disks will have 0 space and not be used.
virtual bool isBroken() const { return false; }
/// Invoked when Global Context is shutdown.
virtual void shutdown() {}
2021-04-20 18:29:03 +00:00
/// Performs action on disk startup.
virtual void startup() {}
2021-07-05 03:32:56 +00:00
/// Return some uniq string for file, overrode for IDiskRemote
2021-06-24 08:25:05 +00:00
/// Required for distinguish different copies of the same part on remote disk
2020-10-22 09:32:05 +00:00
virtual String getUniqueId(const String & path) const { return path; }
2021-02-26 09:48:57 +00:00
/// Check file exists and ClickHouse has an access to it
2021-07-05 03:32:56 +00:00
/// Overrode in remote FS disks (s3/hdfs)
2021-06-24 08:25:05 +00:00
/// Required for remote disk to ensure that replica has access to data written by other node
virtual bool checkUniqueId(const String & id) const { return exists(id); }
/// Invoked on partitions freeze query.
virtual void onFreeze(const String &) { }
2021-01-26 13:29:45 +00:00
/// Returns guard, that insures synchronization of directory metadata with storage device.
virtual SyncGuardPtr getDirectorySyncGuard(const String & path) const;
2021-04-29 20:32:19 +00:00
/// Applies new settings for disk in runtime.
2021-08-16 00:00:32 +00:00
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr, const String &, const DisksMap &) {}
2021-04-29 20:32:19 +00:00
2022-02-15 09:11:50 +00:00
/// Quite leaky abstraction. Some disks can use additional disk to store
/// some parts of metadata. In general case we have only one disk itself and
/// return pointer to it.
///
/// Actually it's a part of IDiskRemote implementation but we have so
/// complex hierarchy of disks (with decorators), so we cannot even
/// dynamic_cast some pointer to IDisk to pointer to IDiskRemote.
virtual std::shared_ptr<IDisk> getMetadataDiskIfExistsOrSelf() { return std::static_pointer_cast<IDisk>(shared_from_this()); }
2022-02-15 09:11:50 +00:00
/// Very similar case as for getMetadataDiskIfExistsOrSelf(). If disk has "metadata"
/// it will return mapping for each required path: path -> metadata as string.
/// Only for IDiskRemote.
virtual std::unordered_map<String, String> getSerializedMetadata(const std::vector<String> & /* paths */) const { return {}; }
2022-02-14 19:19:49 +00:00
/// Return reference count for remote FS.
/// You can ask -- why we have zero and what does it mean? For some unknown reason
/// the decision was made to take 0 as "no references exist", but only file itself left.
/// With normal file system we will get 1 in this case:
/// $ stat clickhouse
/// File: clickhouse
/// Size: 3014014920 Blocks: 5886760 IO Block: 4096 regular file
/// Device: 10301h/66305d Inode: 3109907 Links: 1
/// Why we have always zero by default? Because normal filesystem
/// manages hardlinks by itself. So you can always remove hardlink and all
/// other alive harlinks will not be removed.
virtual UInt32 getRefCount(const String &) const { return 0; }
2022-02-14 19:19:49 +00:00
2021-04-20 18:29:03 +00:00
protected:
friend class DiskDecorator;
/// Returns executor to perform asynchronous operations.
virtual Executor & getExecutor() { return *executor; }
/// Base implementation of the function copy().
/// It just opens two files, reads data by portions from the first file, and writes it to the second one.
/// A derived class may override copy() to provide a faster implementation.
void copyThroughBuffers(const String & from_path, const std::shared_ptr<IDisk> & to_disk, const String & to_path);
private:
std::unique_ptr<Executor> executor;
2019-11-27 09:39:44 +00:00
};
2019-12-03 13:37:40 +00:00
using DiskPtr = std::shared_ptr<IDisk>;
using Disks = std::vector<DiskPtr>;
/**
* Iterator of directory contents on particular disk.
*/
class IDiskDirectoryIterator
2019-11-27 09:39:44 +00:00
{
public:
2019-12-03 13:37:40 +00:00
/// Iterate to the next file.
2019-11-27 09:39:44 +00:00
virtual void next() = 0;
2019-12-03 13:37:40 +00:00
/// Return `true` if the iterator points to a valid element.
2019-11-27 09:39:44 +00:00
virtual bool isValid() const = 0;
2020-01-10 21:42:26 +00:00
/// Path to the file that the iterator currently points to.
virtual String path() const = 0;
2019-11-27 09:39:44 +00:00
/// Name of the file that the iterator currently points to.
virtual String name() const = 0;
2019-12-03 13:37:40 +00:00
virtual ~IDiskDirectoryIterator() = default;
2019-11-27 09:39:44 +00:00
};
/**
2019-11-27 15:49:15 +00:00
* Information about reserved size on particular disk.
2019-11-27 09:39:44 +00:00
*/
class IReservation : boost::noncopyable
2019-11-27 09:39:44 +00:00
{
public:
/// Get reservation size.
2019-12-03 13:37:40 +00:00
virtual UInt64 getSize() const = 0;
2019-11-27 09:39:44 +00:00
/// Get i-th disk where reservation take place.
virtual DiskPtr getDisk(size_t i = 0) const = 0; /// NOLINT
/// Get all disks, used in reservation
virtual Disks getDisks() const = 0;
2019-11-27 09:39:44 +00:00
/// Changes amount of reserved space.
virtual void update(UInt64 new_size) = 0;
/// Unreserves reserved space.
virtual ~IReservation() = default;
};
2019-12-03 13:37:40 +00:00
/// Return full path to a file on disk.
inline String fullPath(const DiskPtr & disk, const String & path)
{
return fs::path(disk->getPath()) / path;
2019-12-03 13:37:40 +00:00
}
/// Return parent path for the specified path.
inline String parentPath(const String & path)
{
2021-05-14 07:06:38 +00:00
if (path.ends_with('/'))
return fs::path(path).parent_path().parent_path() / "";
2021-05-13 09:32:52 +00:00
return fs::path(path).parent_path() / "";
}
/// Return file name for the specified path.
inline String fileName(const String & path)
{
2021-04-30 15:13:19 +00:00
return fs::path(path).filename();
}
/// Return directory path for the specified path.
inline String directoryPath(const String & path)
{
2021-05-15 06:20:26 +00:00
return fs::path(path).parent_path() / "";
}
2019-11-27 09:39:44 +00:00
}