ClickHouse/src/Storages/MergeTree/DataPartStorageOnDisk.cpp

931 lines
32 KiB
C++
Raw Normal View History

2022-04-05 19:12:48 +00:00
#include <Storages/MergeTree/DataPartStorageOnDisk.h>
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Disks/IVolume.h>
2022-04-19 19:34:41 +00:00
#include <Disks/TemporaryFileOnDisk.h>
2022-04-05 19:12:48 +00:00
#include <IO/WriteBufferFromFileBase.h>
2022-04-22 16:58:09 +00:00
#include <IO/ReadBufferFromFileBase.h>
2022-05-03 15:48:05 +00:00
#include <IO/ReadHelpers.h>
2022-05-03 17:58:47 +00:00
#include <Common/logger_useful.h>
2022-04-07 11:58:38 +00:00
#include <Disks/IStoragePolicy.h>
2022-04-19 19:34:41 +00:00
#include <Backups/BackupEntryFromSmallFile.h>
#include <Backups/BackupEntryFromImmutableFile.h>
#include <Storages/MergeTree/localBackup.h>
#include <Disks/SingleDiskVolume.h>
2022-05-03 15:48:05 +00:00
#include <Interpreters/TransactionVersionMetadata.h>
2022-04-05 19:12:48 +00:00
namespace DB
{
2022-04-07 11:58:38 +00:00
namespace ErrorCodes
{
extern const int DIRECTORY_ALREADY_EXISTS;
2022-04-12 18:59:49 +00:00
extern const int NOT_ENOUGH_SPACE;
2022-05-05 09:23:23 +00:00
extern const int LOGICAL_ERROR;
2022-04-07 11:58:38 +00:00
}
2022-04-12 18:59:49 +00:00
DataPartStorageOnDisk::DataPartStorageOnDisk(VolumePtr volume_, std::string root_path_, std::string part_dir_)
: volume(std::move(volume_)), root_path(std::move(root_path_)), part_dir(std::move(part_dir_))
2022-05-05 09:19:12 +00:00
{
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageOnDisk::getFullPath() const
2022-04-19 19:34:41 +00:00
{
2022-06-20 18:18:17 +00:00
return fs::path(volume->getDisk()->getPath()) / root_path / part_dir / "";
2022-04-19 19:34:41 +00:00
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageOnDisk::getRelativePath() const
2022-04-12 18:59:49 +00:00
{
2022-04-22 20:38:18 +00:00
return fs::path(root_path) / part_dir / "";
2022-04-12 18:59:49 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageOnDisk::setRelativePath(const std::string & path)
2022-04-07 11:58:38 +00:00
{
2022-06-20 18:18:17 +00:00
part_dir = path;
2022-04-07 11:58:38 +00:00
}
2022-04-19 19:34:41 +00:00
std::string DataPartStorageOnDisk::getFullRootPath() const
{
2022-04-22 20:38:18 +00:00
return fs::path(volume->getDisk()->getPath()) / root_path / "";
2022-04-19 19:34:41 +00:00
}
2022-06-20 18:18:17 +00:00
DataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name) const
2022-04-22 16:58:09 +00:00
{
2022-06-20 18:18:17 +00:00
return std::make_shared<DataPartStorageOnDisk>(volume, std::string(fs::path(root_path) / part_dir), name);
2022-04-22 16:58:09 +00:00
}
2022-04-05 19:12:48 +00:00
bool DataPartStorageOnDisk::exists() const
{
2022-04-12 18:59:49 +00:00
return volume->getDisk()->exists(fs::path(root_path) / part_dir);
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
bool DataPartStorageOnDisk::exists(const std::string & name) const
2022-04-08 18:56:08 +00:00
{
2022-06-20 18:18:17 +00:00
return volume->getDisk()->exists(fs::path(root_path) / part_dir / name);
2022-04-08 18:56:08 +00:00
}
2022-06-20 18:18:17 +00:00
bool DataPartStorageOnDisk::isDirectory(const std::string & name) const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume->getDisk()->isDirectory(fs::path(root_path) / part_dir / name);
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
Poco::Timestamp DataPartStorageOnDisk::getLastModified() const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume->getDisk()->getLastModified(fs::path(root_path) / part_dir);
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
class DataPartStorageIteratorOnDisk final : public IDataPartStorageIterator
2022-04-05 19:12:48 +00:00
{
2022-04-21 19:19:13 +00:00
public:
2022-06-15 16:32:22 +00:00
DataPartStorageIteratorOnDisk(DiskPtr disk_, DirectoryIteratorPtr it_)
2022-04-21 19:19:13 +00:00
: disk(std::move(disk_)), it(std::move(it_))
{
}
void next() override { it->next(); }
bool isValid() const override { return it->isValid(); }
bool isFile() const override { return isValid() && disk->isFile(it->path()); }
std::string name() const override { return it->name(); }
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
private:
DiskPtr disk;
2022-06-15 16:32:22 +00:00
DirectoryIteratorPtr it;
2022-04-21 19:19:13 +00:00
};
2022-04-19 19:34:41 +00:00
2022-04-21 19:19:13 +00:00
DataPartStorageIteratorPtr DataPartStorageOnDisk::iterate() const
{
return std::make_unique<DataPartStorageIteratorOnDisk>(
volume->getDisk(),
volume->getDisk()->iterateDirectory(fs::path(root_path) / part_dir));
}
2022-06-20 18:18:17 +00:00
size_t DataPartStorageOnDisk::getFileSize(const String & file_name) const
2022-04-21 19:19:13 +00:00
{
2022-06-20 18:18:17 +00:00
return volume->getDisk()->getFileSize(fs::path(root_path) / part_dir / file_name);
}
UInt32 DataPartStorageOnDisk::getRefCount(const String & file_name) const
{
return volume->getDisk()->getRefCount(fs::path(root_path) / part_dir / file_name);
}
static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from)
{
if (disk->isFile(from))
return disk->getFileSize(from);
std::vector<std::string> files;
disk->listFiles(from, files);
UInt64 res = 0;
for (const auto & file : files)
res += calculateTotalSizeOnDiskImpl(disk, fs::path(from) / file);
return res;
}
UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const
{
return calculateTotalSizeOnDiskImpl(volume->getDisk(), fs::path(root_path) / part_dir);
}
std::unique_ptr<ReadBufferFromFileBase> DataPartStorageOnDisk::readFile(
const std::string & name,
const ReadSettings & settings,
std::optional<size_t> read_hint,
std::optional<size_t> file_size) const
{
return volume->getDisk()->readFile(fs::path(root_path) / part_dir / name, settings, read_hint, file_size);
}
static std::unique_ptr<ReadBufferFromFileBase> openForReading(const DiskPtr & disk, const String & path)
{
size_t file_size = disk->getFileSize(path);
return disk->readFile(path, ReadSettings().adjustBufferSize(file_size), file_size);
}
void DataPartStorageOnDisk::loadVersionMetadata(VersionMetadata & version, Poco::Logger * log) const
{
std::string version_file_name = fs::path(root_path) / part_dir / "txn_version.txt";
String tmp_version_file_name = version_file_name + ".tmp";
DiskPtr disk = volume->getDisk();
auto remove_tmp_file = [&]()
{
auto last_modified = disk->getLastModified(tmp_version_file_name);
auto buf = openForReading(disk, tmp_version_file_name);
String content;
readStringUntilEOF(content, *buf);
LOG_WARNING(log, "Found file {} that was last modified on {}, has size {} and the following content: {}",
tmp_version_file_name, last_modified.epochTime(), content.size(), content);
disk->removeFile(tmp_version_file_name);
};
if (disk->exists(version_file_name))
{
auto buf = openForReading(disk, version_file_name);
version.read(*buf);
if (disk->exists(tmp_version_file_name))
remove_tmp_file();
return;
}
/// Four (?) cases are possible:
/// 1. Part was created without transactions.
/// 2. Version metadata file was not renamed from *.tmp on part creation.
/// 3. Version metadata were written to *.tmp file, but hard restart happened before fsync.
/// 4. Fsyncs in storeVersionMetadata() work incorrectly.
if (!disk->exists(tmp_version_file_name))
{
/// Case 1.
/// We do not have version metadata and transactions history for old parts,
/// so let's consider that such parts were created by some ancient transaction
/// and were committed with some prehistoric CSN.
/// NOTE It might be Case 3, but version metadata file is written on part creation before other files,
/// so it's not Case 3 if part is not broken.
version.setCreationTID(Tx::PrehistoricTID, nullptr);
version.creation_csn = Tx::PrehistoricCSN;
return;
}
/// Case 2.
/// Content of *.tmp file may be broken, just use fake TID.
/// Transaction was not committed if *.tmp file was not renamed, so we should complete rollback by removing part.
version.setCreationTID(Tx::DummyTID, nullptr);
version.creation_csn = Tx::RolledBackCSN;
remove_tmp_file();
}
void DataPartStorageOnDisk::checkConsistency(const MergeTreeDataPartChecksums & checksums) const
{
checksums.checkSizes(volume->getDisk(), getRelativePath());
2022-04-21 19:19:13 +00:00
}
2022-04-19 19:34:41 +00:00
2022-06-27 19:41:29 +00:00
DataPartStorageBuilderPtr DataPartStorageOnDisk::getBuilder() const
{
return std::make_shared<DataPartStorageBuilderOnDisk>(volume, root_path, part_dir);
}
2022-04-19 19:34:41 +00:00
void DataPartStorageOnDisk::remove(
2022-05-03 15:48:05 +00:00
bool can_remove_shared_data,
const NameSet & names_not_to_remove,
2022-05-05 09:19:12 +00:00
const MergeTreeDataPartChecksums & checksums,
2022-04-19 19:34:41 +00:00
std::list<ProjectionChecksums> projections,
2022-08-12 10:47:25 +00:00
bool is_temp,
MergeTreeDataPartState state,
2022-04-19 19:34:41 +00:00
Poco::Logger * log) const
{
/// NOTE We rename part to delete_tmp_<relative_path> instead of delete_tmp_<name> to avoid race condition
/// when we try to remove two parts with the same name, but different relative paths,
/// for example all_1_2_1 (in Deleting state) and tmp_merge_all_1_2_1 (in Temporary state).
fs::path from = fs::path(root_path) / part_dir;
// fs::path to = fs::path(root_path) / ("delete_tmp_" + part_dir);
2022-04-19 19:34:41 +00:00
// TODO directory delete_tmp_<name> is never removed if server crashes before returning from this function
/// Cut last "/" if it exists (it shouldn't). Otherwise fs::path behave differently.
fs::path part_dir_without_slash = part_dir.ends_with("/") ? part_dir.substr(0, part_dir.size() - 1) : part_dir;
/// NOTE relative_path can contain not only part name itself, but also some prefix like
/// "moving/all_1_1_1" or "detached/all_2_3_5". We should handle this case more properly.
if (part_dir_without_slash.has_parent_path())
{
auto parent_path = part_dir_without_slash.parent_path();
if (parent_path == "detached")
2022-06-15 16:32:22 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to remove detached part {} with path {} in remove function. It shouldn't happen", part_dir, root_path);
part_dir_without_slash = parent_path / ("delete_tmp_" + std::string{part_dir_without_slash.filename()});
}
else
{
part_dir_without_slash = ("delete_tmp_" + std::string{part_dir_without_slash.filename()});
}
fs::path to = fs::path(root_path) / part_dir_without_slash;
2022-04-19 19:34:41 +00:00
auto disk = volume->getDisk();
if (disk->exists(to))
{
LOG_WARNING(log, "Directory {} (to which part must be renamed before removing) already exists. Most likely this is due to unclean restart or race condition. Removing it.", fullPath(disk, to));
try
{
2022-05-03 15:48:05 +00:00
disk->removeSharedRecursive(fs::path(to) / "", !can_remove_shared_data, names_not_to_remove);
2022-04-19 19:34:41 +00:00
}
catch (...)
{
LOG_ERROR(log, "Cannot recursively remove directory {}. Exception: {}", fullPath(disk, to), getCurrentExceptionMessage(false));
throw;
}
}
try
{
disk->moveDirectory(from, to);
}
catch (const fs::filesystem_error & e)
{
if (e.code() == std::errc::no_such_file_or_directory)
{
LOG_ERROR(log, "Directory {} (part to remove) doesn't exist or one of nested files has gone. Most likely this is due to manual removing. This should be discouraged. Ignoring.", fullPath(disk, to));
return;
}
throw;
}
// Record existing projection directories so we don't remove them twice
std::unordered_set<String> projection_directories;
std::string proj_suffix = ".proj";
2022-04-19 19:34:41 +00:00
for (const auto & projection : projections)
{
std::string proj_dir_name = projection.name + proj_suffix;
2022-04-19 19:34:41 +00:00
projection_directories.emplace(proj_dir_name);
2022-05-05 09:19:12 +00:00
clearDirectory(
fs::path(to) / proj_dir_name,
2022-08-12 10:47:25 +00:00
can_remove_shared_data, names_not_to_remove, projection.checksums, {}, is_temp, state, log, true);
2022-04-19 19:34:41 +00:00
}
/// It is possible that we are removing the part which have a written but not loaded projection.
/// Such a part can appear server was restarted after DROP PROJECTION but before old part was removed.
/// In this case, the old part will load only projections from metadata.
/// See test 01701_clear_projection_and_part.
for (const auto & [name, _] : checksums.files)
{
if (endsWith(name, proj_suffix) && !projection_directories.contains(name) && disk->isDirectory(fs::path(to) / name))
{
/// If we have a directory with suffix '.proj' it is likely a projection.
2022-07-12 09:37:00 +00:00
/// Try to load checksums for it (to avoid recursive removing fallback).
std::string checksum_path = fs::path(to) / name / "checksums.txt";
if (disk->exists(checksum_path))
{
try
{
MergeTreeDataPartChecksums tmp_checksums;
auto in = disk->readFile(checksum_path, {});
tmp_checksums.read(*in);
projection_directories.emplace(name);
clearDirectory(
fs::path(to) / name,
2022-08-12 10:47:25 +00:00
can_remove_shared_data, names_not_to_remove, tmp_checksums, {}, is_temp, state, log, true);
}
catch (...)
{
LOG_ERROR(log, "Cannot load checksums from {}", checksum_path);
}
}
}
}
2022-08-12 10:47:25 +00:00
clearDirectory(to, can_remove_shared_data, names_not_to_remove, checksums, projection_directories, is_temp, state, log, false);
2022-04-19 19:34:41 +00:00
}
void DataPartStorageOnDisk::clearDirectory(
const std::string & dir,
2022-05-03 15:48:05 +00:00
bool can_remove_shared_data,
const NameSet & names_not_to_remove,
2022-05-05 09:19:12 +00:00
const MergeTreeDataPartChecksums & checksums,
const std::unordered_set<String> & skip_directories,
2022-08-12 10:47:25 +00:00
bool is_temp,
MergeTreeDataPartState state,
2022-04-19 19:34:41 +00:00
Poco::Logger * log,
bool is_projection) const
{
auto disk = volume->getDisk();
2022-08-12 10:47:25 +00:00
/// It does not make sense to try fast path for incomplete temporary parts, because some files are probably absent.
/// Sometimes we add something to checksums.files before actually writing checksums and columns on disk.
/// Also sometimes we write checksums.txt and columns.txt in arbitrary order, so this check becomes complex...
2022-08-15 14:03:03 +00:00
bool is_temporary_part = is_temp || state == MergeTreeDataPartState::Temporary;
bool incomplete_temporary_part = is_temporary_part && (!disk->exists(fs::path(dir) / "checksums.txt") || !disk->exists(fs::path(dir) / "columns.txt"));
2022-08-12 10:47:25 +00:00
if (checksums.empty() || incomplete_temporary_part)
2022-04-19 19:34:41 +00:00
{
/// If the part is not completely written, we cannot use fast path by listing files.
2022-05-03 15:48:05 +00:00
disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove);
2022-04-19 19:34:41 +00:00
return;
}
try
{
/// Remove each expected file in directory, then remove directory itself.
2022-06-21 11:15:06 +00:00
RemoveBatchRequest request;
2022-04-19 19:34:41 +00:00
#if !defined(__clang__)
# pragma GCC diagnostic push
# pragma GCC diagnostic ignored "-Wunused-variable"
#endif
for (const auto & [file, _] : checksums.files)
{
if (skip_directories.find(file) == skip_directories.end())
request.emplace_back(fs::path(dir) / file);
}
#if !defined(__clang__)
# pragma GCC diagnostic pop
#endif
for (const auto & file : {"checksums.txt", "columns.txt"})
request.emplace_back(fs::path(dir) / file);
request.emplace_back(fs::path(dir) / "default_compression_codec.txt", true);
request.emplace_back(fs::path(dir) / "delete-on-destroy.txt", true);
if (!is_projection)
request.emplace_back(fs::path(dir) / "txn_version.txt", true);
2022-05-03 15:48:05 +00:00
disk->removeSharedFiles(request, !can_remove_shared_data, names_not_to_remove);
2022-04-19 19:34:41 +00:00
disk->removeDirectory(dir);
}
catch (...)
{
/// Recursive directory removal does many excessive "stat" syscalls under the hood.
LOG_ERROR(log, "Cannot quickly remove directory {} by removing files; fallback to recursive removal. Reason: {}", fullPath(disk, dir), getCurrentExceptionMessage(false));
2022-07-12 08:31:09 +00:00
disk->removeSharedRecursive(fs::path(dir) / "", !can_remove_shared_data, names_not_to_remove);
2022-04-19 19:34:41 +00:00
}
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageOnDisk::getRelativePathForPrefix(Poco::Logger * log, const String & prefix, bool detached) const
2022-04-22 16:58:09 +00:00
{
2022-06-20 18:18:17 +00:00
String res;
2022-04-22 16:58:09 +00:00
2022-06-20 18:18:17 +00:00
auto full_relative_path = fs::path(root_path);
if (detached)
full_relative_path /= "detached";
for (int try_no = 0; try_no < 10; ++try_no)
{
res = (prefix.empty() ? "" : prefix + "_") + part_dir + (try_no ? "_try" + DB::toString(try_no) : "");
if (!volume->getDisk()->exists(full_relative_path / res))
return res;
LOG_WARNING(log, "Directory {} (to detach to) already exists. Will detach to directory with '_tryN' suffix.", res);
}
return res;
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageBuilderOnDisk::setRelativePath(const std::string & path)
2022-04-21 19:19:13 +00:00
{
2022-06-20 18:18:17 +00:00
part_dir = path;
2022-04-21 19:19:13 +00:00
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageOnDisk::getDiskName() const
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
return volume->getDisk()->getName();
2022-04-05 19:12:48 +00:00
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageOnDisk::getDiskType() const
2022-04-05 19:12:48 +00:00
{
2022-08-19 14:58:30 +00:00
return toString(volume->getDisk()->getDataSourceDescription().type);
2022-04-05 19:12:48 +00:00
}
2022-04-08 18:56:08 +00:00
bool DataPartStorageOnDisk::isStoredOnRemoteDisk() const
{
return volume->getDisk()->isRemote();
}
bool DataPartStorageOnDisk::supportZeroCopyReplication() const
{
return volume->getDisk()->supportZeroCopyReplication();
}
2022-04-22 16:58:09 +00:00
bool DataPartStorageOnDisk::supportParallelWrite() const
{
return volume->getDisk()->supportParallelWrite();
}
2022-04-19 19:34:41 +00:00
bool DataPartStorageOnDisk::isBroken() const
{
return volume->getDisk()->isBroken();
}
2022-06-20 18:18:17 +00:00
void DataPartStorageOnDisk::syncRevision(UInt64 revision)
{
volume->getDisk()->syncRevision(revision);
}
UInt64 DataPartStorageOnDisk::getRevision() const
{
return volume->getDisk()->getRevision();
}
std::unordered_map<String, String> DataPartStorageOnDisk::getSerializedMetadata(const std::vector<String> & paths) const
{
return volume->getDisk()->getSerializedMetadata(paths);
}
std::string DataPartStorageOnDisk::getDiskPath() const
2022-04-19 19:34:41 +00:00
{
return volume->getDisk()->getPath();
}
2022-06-20 18:18:17 +00:00
DataPartStorageOnDisk::DisksSet::const_iterator DataPartStorageOnDisk::isStoredOnDisk(const DisksSet & disks) const
{
return disks.find(volume->getDisk());
}
ReservationPtr DataPartStorageOnDisk::reserve(UInt64 bytes) const
{
auto res = volume->reserve(bytes);
if (!res)
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve {}, not enough space", ReadableSize(bytes));
return res;
}
ReservationPtr DataPartStorageOnDisk::tryReserve(UInt64 bytes) const
{
return volume->reserve(bytes);
}
size_t DataPartStorageOnDisk::getVolumeIndex(const IStoragePolicy & storage_policy) const
{
return storage_policy.getVolumeIndexByDisk(volume->getDisk());
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk::writeChecksums(const MergeTreeDataPartChecksums & checksums, const WriteSettings & settings) const
2022-04-05 19:12:48 +00:00
{
2022-04-12 18:59:49 +00:00
std::string path = fs::path(root_path) / part_dir / "checksums.txt";
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
try
2022-04-05 19:12:48 +00:00
{
2022-04-21 19:19:13 +00:00
{
2022-05-03 15:48:05 +00:00
auto out = volume->getDisk()->writeFile(path + ".tmp", 4096, WriteMode::Rewrite, settings);
2022-04-21 19:19:13 +00:00
checksums.write(*out);
}
volume->getDisk()->moveFile(path + ".tmp", path);
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
catch (...)
{
try
{
if (volume->getDisk()->exists(path + ".tmp"))
volume->getDisk()->removeFile(path + ".tmp");
}
catch (...)
{
tryLogCurrentException("DataPartStorageOnDisk");
}
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
throw;
}
2022-04-05 19:12:48 +00:00
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk::writeColumns(const NamesAndTypesList & columns, const WriteSettings & settings) const
2022-04-05 19:12:48 +00:00
{
2022-04-12 18:59:49 +00:00
std::string path = fs::path(root_path) / part_dir / "columns.txt";
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
try
2022-04-05 19:12:48 +00:00
{
2022-06-20 18:18:17 +00:00
auto buf = volume->getDisk()->writeFile(path + ".tmp", 4096, WriteMode::Rewrite, settings);
columns.writeText(*buf);
buf->finalize();
2022-04-21 19:19:13 +00:00
volume->getDisk()->moveFile(path + ".tmp", path);
2022-04-05 19:12:48 +00:00
}
2022-04-21 19:19:13 +00:00
catch (...)
{
try
{
if (volume->getDisk()->exists(path + ".tmp"))
volume->getDisk()->removeFile(path + ".tmp");
}
catch (...)
{
tryLogCurrentException("DataPartStorageOnDisk");
}
2022-04-05 19:12:48 +00:00
2022-04-21 19:19:13 +00:00
throw;
}
2022-04-05 19:12:48 +00:00
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk::writeVersionMetadata(const VersionMetadata & version, bool fsync_part_dir) const
{
std::string path = fs::path(root_path) / part_dir / "txn_version.txt";
try
{
{
/// TODO IDisk interface does not allow to open file with O_EXCL flag (for DiskLocal),
/// so we create empty file at first (expecting that createFile throws if file already exists)
/// and then overwrite it.
2022-06-03 18:49:12 +00:00
volume->getDisk()->createFile(path + ".tmp");
auto buf = volume->getDisk()->writeFile(path + ".tmp", 256);
2022-05-03 15:48:05 +00:00
version.write(*buf);
buf->finalize();
buf->sync();
}
SyncGuardPtr sync_guard;
if (fsync_part_dir)
2022-06-20 18:18:17 +00:00
sync_guard = volume->getDisk()->getDirectorySyncGuard(getRelativePath());
2022-05-03 15:48:05 +00:00
volume->getDisk()->replaceFile(path + ".tmp", path);
}
catch (...)
{
try
{
if (volume->getDisk()->exists(path + ".tmp"))
volume->getDisk()->removeFile(path + ".tmp");
}
catch (...)
{
tryLogCurrentException("DataPartStorageOnDisk");
}
throw;
}
}
void DataPartStorageOnDisk::appendCSNToVersionMetadata(const VersionMetadata & version, VersionMetadata::WhichCSN which_csn) const
{
/// Small enough appends to file are usually atomic,
/// so we append new metadata instead of rewriting file to reduce number of fsyncs.
/// We don't need to do fsync when writing CSN, because in case of hard restart
/// we will be able to restore CSN from transaction log in Keeper.
std::string version_file_name = fs::path(root_path) / part_dir / "txn_version.txt";
DiskPtr disk = volume->getDisk();
auto out = disk->writeFile(version_file_name, 256, WriteMode::Append);
version.writeCSN(*out, which_csn);
out->finalize();
}
void DataPartStorageOnDisk::appendRemovalTIDToVersionMetadata(const VersionMetadata & version, bool clear) const
{
String version_file_name = fs::path(root_path) / part_dir / "txn_version.txt";
DiskPtr disk = volume->getDisk();
auto out = disk->writeFile(version_file_name, 256, WriteMode::Append);
version.writeRemovalTID(*out, clear);
out->finalize();
/// fsync is not required when we clearing removal TID, because after hard restart we will fix metadata
if (!clear)
out->sync();
}
2022-04-08 18:56:08 +00:00
void DataPartStorageOnDisk::writeDeleteOnDestroyMarker(Poco::Logger * log) const
{
2022-04-12 18:59:49 +00:00
String marker_path = fs::path(root_path) / part_dir / "delete-on-destroy.txt";
2022-04-08 18:56:08 +00:00
auto disk = volume->getDisk();
try
{
volume->getDisk()->createFile(marker_path);
}
catch (Poco::Exception & e)
{
LOG_ERROR(log, "{} (while creating DeleteOnDestroy marker: {})", e.what(), backQuote(fullPath(disk, marker_path)));
}
}
2022-05-03 15:48:05 +00:00
void DataPartStorageOnDisk::removeDeleteOnDestroyMarker() const
{
std::string delete_on_destroy_file_name = fs::path(root_path) / part_dir / "delete-on-destroy.txt";
volume->getDisk()->removeFileIfExists(delete_on_destroy_file_name);
}
void DataPartStorageOnDisk::removeVersionMetadata() const
{
std::string version_file_name = fs::path(root_path) / part_dir / "txn_version.txt";
volume->getDisk()->removeFileIfExists(version_file_name);
}
2022-06-20 18:18:17 +00:00
String DataPartStorageOnDisk::getUniqueId() const
2022-05-05 09:19:12 +00:00
{
2022-06-21 07:26:43 +00:00
auto disk = volume->getDisk();
if (!disk->supportZeroCopyReplication())
throw Exception(fmt::format("Disk {} doesn't support zero-copy replication", disk->getName()), ErrorCodes::LOGICAL_ERROR);
return disk->getUniqueId(fs::path(getRelativePath()) / "checksums.txt");
2022-05-05 09:19:12 +00:00
}
2022-04-07 11:58:38 +00:00
bool DataPartStorageOnDisk::shallParticipateInMerges(const IStoragePolicy & storage_policy) const
{
/// `IMergeTreeDataPart::volume` describes space where current part belongs, and holds
/// `SingleDiskVolume` object which does not contain up-to-date settings of corresponding volume.
/// Therefore we shall obtain volume from storage policy.
auto volume_ptr = storage_policy.getVolume(storage_policy.getVolumeIndexByDisk(volume->getDisk()));
return !volume_ptr->areMergesAvoided();
}
2022-04-19 19:34:41 +00:00
void DataPartStorageOnDisk::backup(
TemporaryFilesOnDisks & temp_dirs,
const MergeTreeDataPartChecksums & checksums,
const NameSet & files_without_checksums,
2022-06-22 20:30:50 +00:00
const String & path_in_backup,
2022-04-19 19:34:41 +00:00
BackupEntries & backup_entries) const
{
2022-06-22 20:30:50 +00:00
fs::path part_path_on_disk = fs::path{root_path} / part_dir;
fs::path part_path_in_backup = fs::path{path_in_backup} / part_dir;
2022-04-19 19:34:41 +00:00
2022-06-22 20:30:50 +00:00
auto disk = volume->getDisk();
2022-04-19 19:34:41 +00:00
auto temp_dir_it = temp_dirs.find(disk);
if (temp_dir_it == temp_dirs.end())
2022-07-03 14:32:11 +00:00
temp_dir_it = temp_dirs.emplace(disk, std::make_shared<TemporaryFileOnDisk>(disk, "tmp/")).first;
2022-04-19 19:34:41 +00:00
auto temp_dir_owner = temp_dir_it->second;
fs::path temp_dir = temp_dir_owner->getPath();
2022-06-22 20:30:50 +00:00
fs::path temp_part_dir = temp_dir / part_path_in_backup.relative_path();
2022-04-19 19:34:41 +00:00
disk->createDirectories(temp_part_dir);
2022-06-22 20:30:50 +00:00
/// For example,
/// part_path_in_backup = /data/test/table/0_1_1_0
/// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0
2022-07-03 14:32:11 +00:00
/// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0
2022-06-22 20:30:50 +00:00
/// Or, for projections:
/// part_path_in_backup = /data/test/table/0_1_1_0/prjmax.proj
/// part_path_on_disk = store/f57/f5728353-44bb-4575-85e8-28deb893657a/0_1_1_0/prjmax.proj
2022-07-03 14:32:11 +00:00
/// tmp_part_dir = tmp/1aaaaaa/data/test/table/0_1_1_0/prjmax.proj
2022-06-22 20:30:50 +00:00
2022-04-19 19:34:41 +00:00
for (const auto & [filepath, checksum] : checksums.files)
{
2022-06-22 20:30:50 +00:00
if (filepath.ends_with(".proj"))
continue; /// Skip *.proj files - they're actually directories and will be handled.
String filepath_on_disk = part_path_on_disk / filepath;
String filepath_in_backup = part_path_in_backup / filepath;
2022-04-19 19:34:41 +00:00
String hardlink_filepath = temp_part_dir / filepath;
2022-06-30 08:10:12 +00:00
2022-06-22 20:30:50 +00:00
disk->createHardLink(filepath_on_disk, hardlink_filepath);
2022-04-19 19:34:41 +00:00
UInt128 file_hash{checksum.file_hash.first, checksum.file_hash.second};
backup_entries.emplace_back(
2022-06-22 20:30:50 +00:00
filepath_in_backup,
2022-04-19 19:34:41 +00:00
std::make_unique<BackupEntryFromImmutableFile>(disk, hardlink_filepath, checksum.file_size, file_hash, temp_dir_owner));
}
for (const auto & filepath : files_without_checksums)
{
2022-06-22 20:30:50 +00:00
String filepath_on_disk = part_path_on_disk / filepath;
String filepath_in_backup = part_path_in_backup / filepath;
backup_entries.emplace_back(filepath_in_backup, std::make_unique<BackupEntryFromSmallFile>(disk, filepath_on_disk));
2022-04-19 19:34:41 +00:00
}
}
DataPartStoragePtr DataPartStorageOnDisk::freeze(
2022-05-05 09:19:12 +00:00
const std::string & to,
2022-04-19 19:34:41 +00:00
const std::string & dir_path,
2022-05-03 15:48:05 +00:00
bool make_source_readonly,
std::function<void(const DiskPtr &)> save_metadata_callback,
bool copy_instead_of_hardlink) const
2022-04-19 19:34:41 +00:00
{
auto disk = volume->getDisk();
disk->createDirectories(to);
2022-06-20 18:18:17 +00:00
localBackup(disk, getRelativePath(), fs::path(to) / dir_path, make_source_readonly, {}, copy_instead_of_hardlink);
2022-04-19 19:34:41 +00:00
if (save_metadata_callback)
save_metadata_callback(disk);
disk->removeFileIfExists(fs::path(to) / dir_path / "delete-on-destroy.txt");
2022-05-03 15:48:05 +00:00
disk->removeFileIfExists(fs::path(to) / dir_path / "txn_version.txt");
2022-04-19 19:34:41 +00:00
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
return std::make_shared<DataPartStorageOnDisk>(single_disk_volume, to, dir_path);
}
2022-06-02 15:00:07 +00:00
DataPartStoragePtr DataPartStorageOnDisk::clone(
2022-04-19 19:34:41 +00:00
const std::string & to,
const std::string & dir_path,
2022-06-15 11:41:08 +00:00
const DiskPtr & disk,
2022-04-19 19:34:41 +00:00
Poco::Logger * log) const
{
String path_to_clone = fs::path(to) / dir_path / "";
if (disk->exists(path_to_clone))
{
LOG_WARNING(log, "Path {} already exists. Will remove it and clone again.", fullPath(disk, path_to_clone));
disk->removeRecursive(path_to_clone);
}
disk->createDirectories(to);
2022-06-20 18:18:17 +00:00
volume->getDisk()->copy(getRelativePath(), disk, to);
2022-04-19 19:34:41 +00:00
volume->getDisk()->removeFileIfExists(fs::path(path_to_clone) / "delete-on-destroy.txt");
auto single_disk_volume = std::make_shared<SingleDiskVolume>(disk->getName(), disk, 0);
return std::make_shared<DataPartStorageOnDisk>(single_disk_volume, to, dir_path);
}
2022-06-27 19:41:29 +00:00
void DataPartStorageOnDisk::onRename(const std::string & new_root_path, const std::string & new_part_dir)
{
part_dir = new_part_dir;
root_path = new_root_path;
}
2022-06-26 16:43:28 +00:00
void DataPartStorageBuilderOnDisk::rename(
const std::string & new_root_path,
const std::string & new_part_dir,
Poco::Logger * log,
bool remove_new_dir_if_exists,
bool fsync_part_dir)
2022-06-20 18:18:17 +00:00
{
String to = fs::path(new_root_path) / new_part_dir / "";
if (volume->getDisk()->exists(to))
{
if (remove_new_dir_if_exists)
{
Names files;
volume->getDisk()->listFiles(to, files);
if (log)
LOG_WARNING(log,
"Part directory {} already exists and contains {} files. Removing it.",
fullPath(volume->getDisk(), to), files.size());
2022-06-27 19:41:29 +00:00
transaction->removeRecursive(to);
2022-06-20 18:18:17 +00:00
}
else
{
throw Exception(
ErrorCodes::DIRECTORY_ALREADY_EXISTS,
"Part directory {} already exists",
fullPath(volume->getDisk(), to));
}
}
String from = getRelativePath();
/// Why?
2022-06-27 19:41:29 +00:00
transaction->setLastModified(from, Poco::Timestamp::fromEpochTime(time(nullptr)));
transaction->moveDirectory(from, to);
2022-06-20 18:18:17 +00:00
part_dir = new_part_dir;
root_path = new_root_path;
SyncGuardPtr sync_guard;
if (fsync_part_dir)
sync_guard = volume->getDisk()->getDirectorySyncGuard(getRelativePath());
}
void DataPartStorageOnDisk::changeRootPath(const std::string & from_root, const std::string & to_root)
{
/// This is a very dumb implementation, here for root path like
/// "some/current/path/to/part" and change like
/// "some/current" -> "other/different", we just replace prefix to make new root like
/// "other/different/path/to/part".
/// Here we expect that actual move was done by somebody else.
size_t prefix_size = from_root.size();
if (prefix_size > 0 && from_root.back() == '/')
--prefix_size;
if (prefix_size > root_path.size()
|| std::string_view(from_root).substr(0, prefix_size) != std::string_view(root_path).substr(0, prefix_size))
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot change part root to {} because it is not a prefix of current root {}",
from_root, root_path);
size_t dst_size = to_root.size();
if (dst_size > 0 && to_root.back() == '/')
--dst_size;
root_path = to_root.substr(0, dst_size) + root_path.substr(prefix_size);
}
2022-04-19 19:34:41 +00:00
2022-06-23 16:21:46 +00:00
DataPartStorageBuilderOnDisk::DataPartStorageBuilderOnDisk(
VolumePtr volume_,
std::string root_path_,
std::string part_dir_)
: volume(std::move(volume_))
, root_path(std::move(root_path_))
, part_dir(std::move(part_dir_))
, transaction(volume->getDisk()->createTransaction())
2022-05-05 09:19:12 +00:00
{
2022-04-12 18:59:49 +00:00
}
std::unique_ptr<WriteBufferFromFileBase> DataPartStorageBuilderOnDisk::writeFile(
2022-06-20 18:18:17 +00:00
const String & name,
2022-05-03 15:48:05 +00:00
size_t buf_size,
const WriteSettings & settings)
2022-04-12 18:59:49 +00:00
{
2022-06-23 16:21:46 +00:00
return transaction->writeFile(fs::path(root_path) / part_dir / name, buf_size, WriteMode::Rewrite, settings, /* autocommit = */ false);
2022-04-12 18:59:49 +00:00
}
2022-06-20 18:18:17 +00:00
void DataPartStorageBuilderOnDisk::removeFile(const String & name)
2022-04-12 18:59:49 +00:00
{
2022-06-23 16:21:46 +00:00
transaction->removeFile(fs::path(root_path) / part_dir / name);
2022-04-12 18:59:49 +00:00
}
2022-06-23 12:01:26 +00:00
void DataPartStorageBuilderOnDisk::removeFileIfExists(const String & name)
{
2022-06-23 16:21:46 +00:00
transaction->removeFileIfExists(fs::path(root_path) / part_dir / name);
2022-06-23 12:01:26 +00:00
}
2022-04-12 18:59:49 +00:00
void DataPartStorageBuilderOnDisk::removeRecursive()
{
2022-06-23 16:21:46 +00:00
transaction->removeRecursive(fs::path(root_path) / part_dir);
2022-04-12 18:59:49 +00:00
}
2022-04-21 19:19:13 +00:00
void DataPartStorageBuilderOnDisk::removeSharedRecursive(bool keep_in_remote_fs)
{
2022-06-23 16:21:46 +00:00
transaction->removeSharedRecursive(fs::path(root_path) / part_dir, keep_in_remote_fs, {});
2022-04-21 19:19:13 +00:00
}
SyncGuardPtr DataPartStorageBuilderOnDisk::getDirectorySyncGuard() const
{
return volume->getDisk()->getDirectorySyncGuard(fs::path(root_path) / part_dir);
}
void DataPartStorageBuilderOnDisk::createHardLinkFrom(const IDataPartStorage & source, const std::string & from, const std::string & to) const
{
const auto * source_on_disk = typeid_cast<const DataPartStorageOnDisk *>(&source);
if (!source_on_disk)
throw Exception(
2022-05-05 09:19:12 +00:00
ErrorCodes::LOGICAL_ERROR,
2022-04-21 19:19:13 +00:00
"Cannot create hardlink from different storage. Expected DataPartStorageOnDisk, got {}",
typeid(source).name());
2022-06-23 16:21:46 +00:00
transaction->createHardLink(
2022-06-20 18:18:17 +00:00
fs::path(source_on_disk->getRelativePath()) / from,
2022-04-21 19:19:13 +00:00
fs::path(root_path) / part_dir / to);
}
2022-04-12 18:59:49 +00:00
bool DataPartStorageBuilderOnDisk::exists() const
{
return volume->getDisk()->exists(fs::path(root_path) / part_dir);
}
std::string DataPartStorageBuilderOnDisk::getFullPath() const
{
return fs::path(volume->getDisk()->getPath()) / root_path / part_dir;
}
2022-06-20 18:18:17 +00:00
std::string DataPartStorageBuilderOnDisk::getRelativePath() const
2022-04-21 19:19:13 +00:00
{
return fs::path(root_path) / part_dir;
}
2022-04-12 18:59:49 +00:00
void DataPartStorageBuilderOnDisk::createDirectories()
{
2022-06-23 16:21:46 +00:00
transaction->createDirectories(fs::path(root_path) / part_dir);
2022-04-12 18:59:49 +00:00
}
2022-04-21 19:19:13 +00:00
void DataPartStorageBuilderOnDisk::createProjection(const std::string & name)
{
2022-06-23 16:21:46 +00:00
transaction->createDirectory(fs::path(root_path) / part_dir / name);
2022-04-21 19:19:13 +00:00
}
2022-04-12 18:59:49 +00:00
ReservationPtr DataPartStorageBuilderOnDisk::reserve(UInt64 bytes)
{
auto res = volume->reserve(bytes);
if (!res)
throw Exception(ErrorCodes::NOT_ENOUGH_SPACE, "Cannot reserve {}, not enough space", ReadableSize(bytes));
return res;
}
DataPartStorageBuilderPtr DataPartStorageBuilderOnDisk::getProjection(const std::string & name) const
{
return std::make_shared<DataPartStorageBuilderOnDisk>(volume, std::string(fs::path(root_path) / part_dir), name);
}
2022-04-19 19:34:41 +00:00
DataPartStoragePtr DataPartStorageBuilderOnDisk::getStorage() const
{
return std::make_shared<DataPartStorageOnDisk>(volume, root_path, part_dir);
}
2022-06-23 16:21:46 +00:00
void DataPartStorageBuilderOnDisk::commit()
{
transaction->commit();
}
2022-04-05 19:12:48 +00:00
}