This commit is contained in:
AlexNsf 2024-09-19 20:22:29 +08:00 committed by GitHub
commit e4b2493855
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 661 additions and 0 deletions

75
src/Disks/DiskBackup.cpp Normal file
View File

@ -0,0 +1,75 @@
#include "DiskBackup.h"
#include <Disks/DiskFactory.h>
#include "Disks/IDisk.h"
#include <Disks/ObjectStorages/MetadataStorageFactory.h>
namespace DB
{
DiskBackup::DiskBackup(const String & name_, DiskPtr delegate_, MetadataStoragePtr metadata_) : IDisk(name_), delegate(delegate_), metadata(metadata_)
{
}
DiskBackup::DiskBackup(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_) : IDisk(name_)
{
String disk_delegate_name = config_.getString(config_prefix_ + ".disk_delegate");
String metadata_name = config_.getString(config_prefix_ + ".metadata");
delegate = map_.at(disk_delegate_name);
metadata = MetadataStorageFactory::instance().create(metadata_name, config_, config_prefix_, nullptr, "");
}
bool DiskBackup::exists(const String & path) const
{
return metadata->exists(fs::path(getPath()) / path);
}
bool DiskBackup::isFile(const String & path) const
{
return metadata->isFile(fs::path(getPath()) / path);
}
bool DiskBackup::isDirectory(const String & path) const
{
return metadata->isDirectory(fs::path(getPath()) / path);
}
size_t DiskBackup::getFileSize(const String & path) const
{
return metadata->getFileSize(fs::path(getPath()) / path);
}
DirectoryIteratorPtr DiskBackup::iterateDirectory(const String & path) const
{
return metadata->iterateDirectory(fs::path(getPath()) / path);
}
void DiskBackup::listFiles(const String & path, std::vector<String> & file_names) const
{
file_names = metadata->listDirectory(fs::path(getPath()) / path);
}
void registerDiskBackup(DiskFactory & factory, bool global_skip_access_check)
{
auto creator = [global_skip_access_check](
const String & name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
ContextPtr context,
const DisksMap & map,
bool, bool) -> DiskPtr
{
std::shared_ptr<IDisk> disk = std::make_shared<DiskBackup>(name, config, config_prefix, map);
bool skip_access_check = global_skip_access_check || config.getBool(config_prefix + ".skip_access_check", false);
disk->startup(context, skip_access_check);
return disk;
};
factory.registerDiskType("backup", creator);
}
}

206
src/Disks/DiskBackup.h Normal file
View File

@ -0,0 +1,206 @@
#pragma once
#include <Disks/IDisk.h>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
#include "Common/Exception.h"
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class DiskBackup : public IDisk
{
private:
[[noreturn]] void throwNotAllowed() const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only operations are supported in DiskBackup");
}
public:
DiskBackup(const String & name_, DiskPtr delegate_, MetadataStoragePtr metadata_);
DiskBackup(const String & name_, const Poco::Util::AbstractConfiguration & config_, const String & config_prefix_, const DisksMap & map_);
bool isReadOnly() const override
{
return true;
}
const String & getPath() const override
{
return delegate->getPath();
}
ReservationPtr reserve(UInt64 /* bytes */) override
{
throwNotAllowed();
}
std::optional<UInt64> getTotalSpace() const override
{
return delegate->getTotalSpace();
}
std::optional<UInt64> getAvailableSpace() const override
{
return {};
}
std::optional<UInt64> getUnreservedSpace() const override
{
return {};
}
UInt64 getKeepingFreeSpace() const override
{
throwNotAllowed();
}
bool exists(const String & path) const override;
bool isFile(const String & path) const override;
bool isDirectory(const String & path) const override;
size_t getFileSize(const String & path) const override;
void createDirectory(const String & /* path */) override
{
throwNotAllowed();
}
void createDirectories(const String & /* path */) override
{
throwNotAllowed();
}
void clearDirectory(const String & /* path */) override
{
throwNotAllowed();
}
void moveDirectory(const String & /* from_path */, const String & /* to_path */) override
{
throwNotAllowed();
}
DirectoryIteratorPtr iterateDirectory(const String & path) const override;
void createFile(const String & /* path */) override
{
throwNotAllowed();
}
void moveFile(const String & /* from_path */, const String & /* to_path */) override
{
throwNotAllowed();
}
void replaceFile(const String & /* from_path */, const String & /* to_path */) override
{
throwNotAllowed();
}
void listFiles(const String & path, std::vector<String> & file_names) const 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
{
return delegate->readFile(path, settings, read_hint, file_size);
}
std::unique_ptr<WriteBufferFromFileBase> writeFile(
const String & /* path */,
size_t /* buf_size */,
WriteMode /* mode */,
const WriteSettings & /* settings */) override
{
throwNotAllowed();
}
Strings getBlobPath(const String & path) const override
{
return delegate->getBlobPath(path);
}
void writeFileUsingBlobWritingFunction(const String & /*path*/, WriteMode /*mode*/, WriteBlobFunction && /*write_blob_function*/) override
{
throwNotAllowed();
}
void removeFile(const String & /* path */) override
{
throwNotAllowed();
}
void removeFileIfExists(const String & /* path */) override
{
throwNotAllowed();
}
void removeDirectory(const String & /* path */) override
{
throwNotAllowed();
}
void removeRecursive(const String & /* path */) override
{
throwNotAllowed();
}
void setLastModified(const String & /*path*/, const Poco::Timestamp & /*timestamp*/) override
{
throwNotAllowed();
}
Poco::Timestamp getLastModified(const String & path) const override
{
return delegate->getLastModified(path);
}
time_t getLastChanged(const String & path) const override
{
return delegate->getLastChanged(path);
}
void setReadOnly(const String & /*path*/) override
{
throwNotAllowed();
}
void createHardLink(const String & /*src_path*/, const String & /*dst_path*/) override
{
throwNotAllowed();
}
DataSourceDescription getDataSourceDescription() const override
{
return delegate->getDataSourceDescription();
}
/// Involves network interaction.
bool isRemote() const override
{
return delegate->isRemote();
}
/// Whether this disk support zero-copy replication.
/// Overrode in remote fs disks.
bool supportZeroCopyReplication() const override
{
return delegate->supportZeroCopyReplication();
}
private:
DiskPtr delegate;
MetadataStoragePtr metadata;
};
};

View File

@ -30,6 +30,7 @@ enum class MetadataStorageType : uint8_t
Plain,
PlainRewritable,
StaticWeb,
Backup,
};
MetadataStorageType metadataTypeFromString(const String & type);

View File

@ -0,0 +1,162 @@
#include <filesystem>
#include <Common/XMLUtils.h>
#include <Disks/IDisk.h>
#include <Disks/ObjectStorages/StaticDirectoryIterator.h>
#include <Disks/DiskType.h>
#include <IO/ReadBufferFromFile.h>
#include "IO/ReadHelpers.h"
#include <Poco/DOM/DOMParser.h>
#include <Poco/DOM/Node.h>
#include <Poco/Util/XMLConfiguration.h>
#include "MetadataStorageFromBackupFile.h"
namespace DB
{
MetadataStorageFromBackupFile::MetadataStorageFromBackupFile(const String & path_to_backup_file)
{
using namespace XMLUtils;
ReadBufferFromFile buf(path_to_backup_file);
String str;
readStringUntilEOF(str, buf);
Poco::XML::DOMParser dom_parser;
Poco::AutoPtr<Poco::XML::Document> config = dom_parser.parseMemory(str.data(), str.size());
const Poco::XML::Node * config_root = getRootNode(config);
const auto * contents = config_root->getNodeByPath("contents");
for (const Poco::XML::Node * child = contents->firstChild(); child; child = child->nextSibling())
{
if (child->nodeName() == "file")
{
const Poco::XML::Node * file_config = child;
fs::path file_path = getString(file_config, "name");
if (!file_path.string().starts_with("/"))
{
file_path = "" / file_path;
}
uint64_t file_size = getUInt64(file_config, "size");
nodes[file_path] =
{
file_path.filename(),
file_path.string(),
file_size,
true,
false,
};
while (file_path.has_relative_path())
{
fs::path parent_path = file_path.parent_path();
auto current_node = nodes.at(file_path);
if (!nodes.contains(parent_path))
{
nodes[parent_path] =
{parent_path.filename(),
parent_path.string(),
current_node.file_size,
false,
true,
{{file_path.string()}}
};
}
else
{
nodes[parent_path].file_size += current_node.file_size;
nodes[parent_path].children.push_back(file_path.string());
}
file_path = parent_path;
}
}
}
}
MetadataTransactionPtr MetadataStorageFromBackupFile::createTransaction()
{
return std::make_shared<MetadataStorageFromBackupFileTransaction>(*this);
}
const String & MetadataStorageFromBackupFile::getPath() const
{
static const String no_root;
return no_root;
}
MetadataStorageType MetadataStorageFromBackupFile::getType() const
{
return MetadataStorageType::Backup;
}
bool MetadataStorageFromBackupFile::exists(const String & path) const
{
return nodes.contains(path);
}
bool MetadataStorageFromBackupFile::isFile(const String & path) const
{
return exists(path) && nodes.at(path).is_file;
}
bool MetadataStorageFromBackupFile::isDirectory(const String & path) const
{
return exists(path) && nodes.at(path).is_directory;
}
uint64_t MetadataStorageFromBackupFile::getFileSize(const String & path) const
{
return nodes.at(path).file_size;
}
std::vector<String> MetadataStorageFromBackupFile::listDirectory(const String & path) const
{
if (!isDirectory(path))
{
return {};
}
return nodes.at(path).children;
}
DirectoryIteratorPtr MetadataStorageFromBackupFile::iterateDirectory(const String & path) const
{
std::vector<fs::path> dir_file_paths;
if (!exists(path))
{
return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths));
}
for (const auto& listed_path: nodes.at(path).children)
{
dir_file_paths.push_back(fs::path(listed_path));
}
return std::make_unique<StaticDirectoryIterator>(std::move(dir_file_paths));
}
StoredObjects MetadataStorageFromBackupFile::getStorageObjects(const String & path) const
{
if (!exists(path))
{
return {};
}
auto node = nodes.at(path);
// size_t object_size = getFileSize(path);
// auto object_key = object_storage->generateObjectKeyForPath(path);
return {StoredObject(path, path, node.file_size)};
}
const IMetadataStorage & MetadataStorageFromBackupFileTransaction::getStorageForNonTransactionalReads() const
{
return metadata_storage;
}
}

View File

@ -0,0 +1,109 @@
#pragma once
#include <unordered_map>
#include <vector>
#include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Disks/IDisk.h>
namespace DB
{
class MetadataStorageFromBackupFile final : public IMetadataStorage
{
private:
friend class MetadataStorageFromBackupFileTransaction;
struct MetadataStorageFromBackupFilePseudoFileSystemNode
{
public:
String name;
String path;
uint64_t file_size = 0;
bool is_file = false;
bool is_directory = false;
std::vector<String> children = {};
};
std::unordered_map<String /* path */, MetadataStorageFromBackupFilePseudoFileSystemNode> nodes;
public:
explicit MetadataStorageFromBackupFile(const String & path_to_backup_file);
MetadataTransactionPtr createTransaction() override;
const String & getPath() const override;
MetadataStorageType getType() const override;
bool exists(const String & path) const override;
bool isFile(const String & path) const override;
bool isDirectory(const String & path) const override;
uint64_t getFileSize(const String & path) const override;
std::vector<String> listDirectory(const String & path) const override;
DirectoryIteratorPtr iterateDirectory(const String & path) const override;
StoredObjects getStorageObjects(const String & path) const override;
struct stat stat(const String & /* path */) const override { return {}; }
Poco::Timestamp getLastModified(const String & /* path */) const override
{
/// Required by MergeTree
return {};
}
uint32_t getHardlinkCount(const String & /* path */) const override
{
return 1;
}
bool supportsChmod() const override { return false; }
bool supportsStat() const override { return false; }
};
class MetadataStorageFromBackupFileTransaction final : public IMetadataTransaction
{
private:
DiskPtr disk;
const MetadataStorageFromBackupFile & metadata_storage;
public:
explicit MetadataStorageFromBackupFileTransaction(
const MetadataStorageFromBackupFile & metadata_storage_)
: metadata_storage(metadata_storage_)
{}
const IMetadataStorage & getStorageForNonTransactionalReads() const override;
void createEmptyMetadataFile(const String & /* path */) override
{
/// No metadata, no need to create anything.
}
void createMetadataFile(const String & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override
{
/// Noop
}
void createDirectory(const String & /* path */) override
{
/// Noop
}
void createDirectoryRecursive(const String & /* path */) override
{
/// Noop
}
void commit() override
{
/// Nothing to commit.
}
bool supportsChmod() const override { return false; }
};
}

View File

@ -16,6 +16,7 @@ void registerDiskEncrypted(DiskFactory & factory, bool global_skip_access_check)
void registerDiskCache(DiskFactory & factory, bool global_skip_access_check);
void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check);
void registerDiskBackup(DiskFactory & factory, bool global_skip_access_check);
void registerDisks(bool global_skip_access_check)
{
@ -30,6 +31,8 @@ void registerDisks(bool global_skip_access_check)
registerDiskCache(factory, global_skip_access_check);
registerDiskObjectStorage(factory, global_skip_access_check);
registerDiskBackup(factory, global_skip_access_check);
}
}

View File

@ -0,0 +1,105 @@
#include <filesystem>
#include <fstream>
#include <memory>
#include <Disks/DiskLocal.h>
#include <Disks/DiskBackup.h>
#include <Disks/IDisk.h>
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
#include <gtest/gtest.h>
#include <Disks/ObjectStorages/IObjectStorage_fwd.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <Core/Defines.h>
#include <Disks/WriteMode.h>
#include "Disks/MetadataStorageFromBackupFile.h"
#include "base/types.h"
#include <iostream>
using DB::DiskPtr, DB::MetadataStoragePtr;
DB::DiskPtr createLocalDisk(const std::string & path)
{
fs::create_directory(path);
return std::make_shared<DB::DiskLocal>("local_disk", path);
}
class DiskBackupTest : public testing::Test {
public:
DiskPtr delegate, backup;
MetadataStoragePtr meta;
void SetUp() override {
fs::create_directories("tmp/test_backup");
delegate = createLocalDisk("tmp/test_backup/delegate");
}
void TearDown() override {
fs::remove_all("tmp/test_backup");
}
void createBackupMetadataFile(const char* data) const {
Coordination::WriteBufferFromFile write_buffer("tmp/test_backup/.backup");
write_buffer.write(data, strlen(data));
write_buffer.finalize();
}
void createDiskBackup() {
meta = std::make_shared<DB::MetadataStorageFromBackupFile>("tmp/test_backup/.backup");
backup = std::make_shared<DB::DiskBackup>("disk_backup", delegate, meta);
}
};
TEST_F(DiskBackupTest, checkFileExists)
{
const char* data =
"<config>"
" <version>1</version>"
" <deduplicate_files>1</deduplicate_files>"
" <timestamp>2023-08-28 10:07:20</timestamp>"
" <uuid>b70413e4-f421-4b81-9069-5c72018699f7</uuid>"
" <contents>"
" <file>"
" <name>tmp/test_backup/delegate/test.txt</name>"
" <size>210</size>"
" <checksum>f62085ffac70227a8f3e7f11b8448f8d</checksum>"
" </file>"
" </contents>"
"</config>";
createBackupMetadataFile(data);
createDiskBackup();
delegate->createFile("test.txt");
EXPECT_EQ(backup->exists("test.txt"), true);
}
TEST_F(DiskBackupTest, getObjectType)
{
const char* data =
"<config>"
" <version>1</version>"
" <deduplicate_files>1</deduplicate_files>"
" <timestamp>2023-08-28 10:07:20</timestamp>"
" <uuid>b70413e4-f421-4b81-9069-5c72018699f7</uuid>"
" <contents>"
" <file>"
" <name>tmp/test_backup/delegate/test_folder/test.txt</name>"
" <size>210</size>"
" <checksum>f62085ffac70227a8f3e7f11b8448f8d</checksum>"
" </file>"
" </contents>"
"</config>";
createBackupMetadataFile(data);
createDiskBackup();
delegate->createDirectory("test_folder");
delegate->createFile("test_folder/test.txt");
EXPECT_EQ(backup->isFile("test_folder/test.txt"), true);
EXPECT_EQ(backup->isDirectory("test_folder"), true);
}