2019-11-27 09:39:44 +00:00
|
|
|
#pragma once
|
|
|
|
|
2021-04-20 18:29:03 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2019-12-12 08:57:25 +00:00
|
|
|
#include <Core/Defines.h>
|
2020-09-15 09:55:57 +00:00
|
|
|
#include <common/types.h>
|
2019-11-27 09:39:44 +00:00
|
|
|
#include <Common/CurrentMetrics.h>
|
|
|
|
#include <Common/Exception.h>
|
2020-08-11 19:08:32 +00:00
|
|
|
#include <Disks/Executor.h>
|
2021-02-26 09:48:57 +00:00
|
|
|
#include <Disks/DiskType.h>
|
2019-11-27 09:39:44 +00:00
|
|
|
|
|
|
|
#include <memory>
|
2019-12-17 13:45:53 +00:00
|
|
|
#include <mutex>
|
2019-11-27 09:39:44 +00:00
|
|
|
#include <utility>
|
|
|
|
#include <boost/noncopyable.hpp>
|
2020-03-05 14:02:15 +00:00
|
|
|
#include <Poco/Timestamp.h>
|
2021-04-30 15:13:19 +00:00
|
|
|
#include <filesystem>
|
2021-04-29 20:32:19 +00:00
|
|
|
#include "Poco/Util/AbstractConfiguration.h"
|
2019-11-27 09:39:44 +00:00
|
|
|
|
2021-04-30 15:13:19 +00:00
|
|
|
namespace fs = std::filesystem;
|
2019-11-27 09:39:44 +00:00
|
|
|
|
|
|
|
namespace CurrentMetrics
|
|
|
|
{
|
|
|
|
extern const Metric DiskSpaceReservedForMerge;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
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>;
|
2020-05-24 16:32:58 +00:00
|
|
|
using Reservations = std::vector<ReservationPtr>;
|
2019-11-27 09:39:44 +00:00
|
|
|
|
2020-02-14 14:28:33 +00:00
|
|
|
class ReadBufferFromFileBase;
|
2020-02-20 16:39:32 +00:00
|
|
|
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>;
|
|
|
|
|
2019-11-28 19:11:49 +00:00
|
|
|
/**
|
|
|
|
* 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:
|
2020-08-11 19:08:32 +00:00
|
|
|
/// 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
|
|
|
|
2019-12-12 08:57:25 +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.
|
2020-10-23 12:01:50 +00:00
|
|
|
virtual void clearDirectory(const String & path) = 0;
|
2019-12-12 08:57:25 +00:00
|
|
|
|
|
|
|
/// 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
|
|
|
|
2019-12-12 08:57:25 +00:00
|
|
|
/// Return `true` if the specified directory is empty.
|
|
|
|
bool isDirectoryEmpty(const String & path);
|
|
|
|
|
2020-03-19 16:37:55 +00:00
|
|
|
/// 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;
|
|
|
|
|
2020-03-19 16:37:55 +00:00
|
|
|
/// 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;
|
|
|
|
|
2020-02-20 16:39:32 +00:00
|
|
|
/// Open the file for read and return ReadBufferFromFileBase object.
|
|
|
|
virtual std::unique_ptr<ReadBufferFromFileBase> readFile(
|
|
|
|
const String & path,
|
|
|
|
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
|
|
|
size_t estimated_size = 0,
|
2021-07-05 20:02:24 +00:00
|
|
|
size_t direct_io_threshold = 0,
|
2021-03-28 01:10:30 +00:00
|
|
|
size_t mmap_threshold = 0,
|
2021-03-28 19:24:28 +00:00
|
|
|
MMappedFileCache * mmap_cache = nullptr) const = 0;
|
2020-02-20 16:39:32 +00:00
|
|
|
|
|
|
|
/// Open the file for write and return WriteBufferFromFileBase object.
|
|
|
|
virtual std::unique_ptr<WriteBufferFromFileBase> writeFile(
|
|
|
|
const String & path,
|
|
|
|
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
2021-01-10 00:28:59 +00:00
|
|
|
WriteMode mode = WriteMode::Rewrite) = 0;
|
2019-12-17 13:45:53 +00:00
|
|
|
|
2021-01-14 16:24:13 +00:00
|
|
|
/// 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;
|
2019-12-17 13:45:53 +00:00
|
|
|
|
2021-01-14 16:24:13 +00:00
|
|
|
/// 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.
|
2020-10-23 12:01:50 +00:00
|
|
|
virtual void removeRecursive(const String & path) = 0;
|
2020-03-05 14:02:15 +00:00
|
|
|
|
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); }
|
2020-10-23 12:01:50 +00:00
|
|
|
|
|
|
|
/// 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
|
2020-10-23 12:01:50 +00:00
|
|
|
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); }
|
2020-03-19 16:37:55 +00:00
|
|
|
|
2020-03-05 14:02:15 +00:00
|
|
|
/// 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;
|
2020-03-19 16:37:55 +00:00
|
|
|
|
|
|
|
/// 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;
|
2020-06-26 21:55:48 +00:00
|
|
|
|
2020-07-12 02:31:58 +00:00
|
|
|
/// 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-02-02 14:09:43 +00:00
|
|
|
virtual DiskType::Type getType() const = 0;
|
2020-08-07 11:40:19 +00:00
|
|
|
|
2020-09-04 14:17:27 +00:00
|
|
|
/// Invoked when Global Context is shutdown.
|
2021-05-05 15:10:14 +00:00
|
|
|
virtual void shutdown() {}
|
2020-09-04 14:17:27 +00:00
|
|
|
|
2021-04-20 18:29:03 +00:00
|
|
|
/// Performs action on disk startup.
|
2021-05-05 15:10:14 +00:00
|
|
|
virtual void startup() {}
|
2021-04-20 18:29:03 +00:00
|
|
|
|
2021-01-20 09:48:22 +00:00
|
|
|
/// Return some uniq string for file, overrode for S3
|
2021-02-26 09:48:57 +00:00
|
|
|
/// Required for distinguish different copies of the same part on S3
|
2020-10-22 09:32:05 +00:00
|
|
|
virtual String getUniqueId(const String & path) const { return path; }
|
2020-10-08 15:45:10 +00:00
|
|
|
|
2021-02-26 09:48:57 +00:00
|
|
|
/// Check file exists and ClickHouse has an access to it
|
|
|
|
/// Overrode in DiskS3
|
2021-04-20 18:29:03 +00:00
|
|
|
/// Required for S3 to ensure that replica has access to data written by other node
|
2020-11-03 08:58:26 +00:00
|
|
|
virtual bool checkUniqueId(const String & id) const { return exists(id); }
|
2020-10-14 15:05:59 +00:00
|
|
|
|
2021-01-13 12:05:32 +00:00
|
|
|
/// Invoked on partitions freeze query.
|
2021-01-11 17:37:08 +00:00
|
|
|
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-06-01 12:20:52 +00:00
|
|
|
virtual void applyNewSettings(const Poco::Util::AbstractConfiguration &, ContextPtr) {}
|
2021-04-29 20:32:19 +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; }
|
|
|
|
|
2020-10-07 11:35:28 +00:00
|
|
|
private:
|
2020-08-11 19:08:32 +00:00
|
|
|
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
|
|
|
|
2020-02-27 16:47:40 +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
|
|
|
*/
|
2019-12-17 13:45:53 +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
|
|
|
|
2020-05-09 16:03:22 +00:00
|
|
|
/// Get i-th disk where reservation take place.
|
|
|
|
virtual DiskPtr getDisk(size_t i = 0) const = 0;
|
|
|
|
|
|
|
|
/// 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)
|
|
|
|
{
|
2021-05-05 15:10:14 +00:00
|
|
|
return fs::path(disk->getPath()) / path;
|
2019-12-03 13:37:40 +00:00
|
|
|
}
|
2019-12-27 10:26:23 +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() / "";
|
2019-12-27 10:26:23 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// 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();
|
2019-12-27 10:26:23 +00:00
|
|
|
}
|
2020-12-23 12:35:52 +00:00
|
|
|
|
|
|
|
/// 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() / "";
|
2020-12-23 12:35:52 +00:00
|
|
|
}
|
|
|
|
|
2019-11-27 09:39:44 +00:00
|
|
|
}
|