mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 18:12:02 +00:00
Check refcount in RemoveManyObjectStorageOperation::finalize
instead of execute
This commit is contained in:
parent
eed1ecb6ba
commit
227e415d6d
@ -6,6 +6,8 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
#include <Disks/ObjectStorages/MetadataStorageFromDisk.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -150,7 +152,15 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati
|
||||
RemoveBatchRequest remove_paths;
|
||||
bool keep_all_batch_data;
|
||||
NameSet file_names_remove_metadata_only;
|
||||
StoredObjects objects_to_remove;
|
||||
|
||||
struct ObjectsToRemove
|
||||
{
|
||||
StoredObjects objects;
|
||||
UnlinkMetadataFileOperationOutcomePtr unlink_outcome;
|
||||
};
|
||||
|
||||
std::vector<ObjectsToRemove> objects_to_remove;
|
||||
|
||||
bool remove_from_cache = false;
|
||||
|
||||
RemoveManyObjectStorageOperation(
|
||||
@ -174,7 +184,6 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati
|
||||
{
|
||||
for (const auto & [path, if_exists] : remove_paths)
|
||||
{
|
||||
|
||||
if (!metadata_storage.exists(path))
|
||||
{
|
||||
if (if_exists)
|
||||
@ -188,14 +197,12 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati
|
||||
|
||||
try
|
||||
{
|
||||
uint32_t hardlink_count = metadata_storage.getHardlinkCount(path);
|
||||
auto objects = metadata_storage.getStorageObjects(path);
|
||||
|
||||
tx->unlinkMetadata(path);
|
||||
|
||||
/// File is really redundant
|
||||
if (hardlink_count == 0 && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename()))
|
||||
std::move(objects.begin(), objects.end(), std::back_inserter(objects_to_remove));
|
||||
auto unlink_outcome = tx->unlinkMetadata(path);
|
||||
if (unlink_outcome && !keep_all_batch_data && !file_names_remove_metadata_only.contains(fs::path(path).filename()))
|
||||
{
|
||||
auto objects = metadata_storage.getStorageObjects(path);
|
||||
objects_to_remove.emplace_back(ObjectsToRemove{std::move(objects), std::move(unlink_outcome)});
|
||||
}
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
@ -215,15 +222,21 @@ struct RemoveManyObjectStorageOperation final : public IDiskObjectStorageOperati
|
||||
|
||||
void undo() override
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
void finalize() override
|
||||
{
|
||||
StoredObjects remove_from_remote;
|
||||
for (auto && [objects, unlink_outcome] : objects_to_remove)
|
||||
{
|
||||
if (unlink_outcome->num_hardlinks == 0)
|
||||
std::move(objects.begin(), objects.end(), std::back_inserter(remove_from_remote));
|
||||
}
|
||||
|
||||
/// Read comment inside RemoveObjectStorageOperation class
|
||||
/// TL;DR Don't pay any attention to 404 status code
|
||||
if (!objects_to_remove.empty())
|
||||
object_storage.removeObjectsIfExist(objects_to_remove);
|
||||
if (!remove_from_remote.empty())
|
||||
object_storage.removeObjectsIfExist(remove_from_remote);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -22,6 +22,8 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
class IMetadataStorage;
|
||||
struct UnlinkMetadataFileOperationOutcome;
|
||||
using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr<UnlinkMetadataFileOperationOutcome>;
|
||||
|
||||
/// Tries to provide some "transactions" interface, which allow
|
||||
/// to execute (commit) operations simultaneously. We don't provide
|
||||
@ -127,9 +129,10 @@ public:
|
||||
|
||||
/// Unlink metadata file and do something special if required
|
||||
/// By default just remove file (unlink file).
|
||||
virtual void unlinkMetadata(const std::string & path)
|
||||
virtual UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path)
|
||||
{
|
||||
unlinkFile(path);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
virtual ~IMetadataTransaction() = default;
|
||||
|
@ -340,9 +340,12 @@ void MetadataStorageFromDiskTransaction::addBlobToMetadata(const std::string & p
|
||||
addOperation(std::make_unique<AddBlobOperation>(path, blob_name, metadata_storage.object_storage_root_path, size_in_bytes, *metadata_storage.disk, metadata_storage));
|
||||
}
|
||||
|
||||
void MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path)
|
||||
UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromDiskTransaction::unlinkMetadata(const std::string & path)
|
||||
{
|
||||
addOperation(std::make_unique<UnlinkMetadataFileOperation>(path, *metadata_storage.disk, metadata_storage));
|
||||
auto operation = std::make_unique<UnlinkMetadataFileOperation>(path, *metadata_storage.getDisk(), metadata_storage);
|
||||
auto result = operation->outcome;
|
||||
addOperation(std::move(operation));
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -11,6 +11,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct UnlinkMetadataFileOperationOutcome;
|
||||
using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr<UnlinkMetadataFileOperationOutcome>;
|
||||
|
||||
/// Store metadata on a separate disk
|
||||
/// (used for object storages, like S3 and related).
|
||||
class MetadataStorageFromDisk final : public IMetadataStorage
|
||||
@ -131,7 +134,7 @@ public:
|
||||
|
||||
void replaceFile(const std::string & path_from, const std::string & path_to) override;
|
||||
|
||||
void unlinkMetadata(const std::string & path) override;
|
||||
UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override;
|
||||
|
||||
|
||||
};
|
||||
|
@ -319,6 +319,8 @@ void UnlinkMetadataFileOperation::execute(std::unique_lock<SharedMutex> & metada
|
||||
write_operation = std::make_unique<WriteFileOperation>(path, disk, metadata->serializeToString());
|
||||
write_operation->execute(metadata_lock);
|
||||
}
|
||||
outcome->num_hardlinks = ref_count;
|
||||
|
||||
unlink_operation = std::make_unique<UnlinkFileOperation>(path, disk);
|
||||
unlink_operation->execute(metadata_lock);
|
||||
}
|
||||
@ -334,6 +336,9 @@ void UnlinkMetadataFileOperation::undo()
|
||||
|
||||
if (write_operation)
|
||||
write_operation->undo();
|
||||
|
||||
/// Update outcome to reflect the fact that we have restored the file.
|
||||
outcome->num_hardlinks++;
|
||||
}
|
||||
|
||||
void SetReadonlyFileOperation::execute(std::unique_lock<SharedMutex> & metadata_lock)
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
|
||||
#include <numeric>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class MetadataStorageFromDisk;
|
||||
@ -242,9 +244,19 @@ private:
|
||||
std::unique_ptr<WriteFileOperation> write_operation;
|
||||
};
|
||||
|
||||
/// Return the result of operation to the caller.
|
||||
/// It is used in `IDiskObjectStorageOperation::finalize` after metadata transaction executed to make decision on blob removal.
|
||||
struct UnlinkMetadataFileOperationOutcome
|
||||
{
|
||||
UInt32 num_hardlinks = std::numeric_limits<UInt32>::max();
|
||||
};
|
||||
|
||||
using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr<UnlinkMetadataFileOperationOutcome>;
|
||||
|
||||
struct UnlinkMetadataFileOperation final : public IMetadataOperation
|
||||
{
|
||||
const UnlinkMetadataFileOperationOutcomePtr outcome = std::make_shared<UnlinkMetadataFileOperationOutcome>();
|
||||
|
||||
UnlinkMetadataFileOperation(
|
||||
const std::string & path_,
|
||||
IDisk & disk_,
|
||||
|
@ -135,9 +135,10 @@ void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata(
|
||||
{
|
||||
/// Noop, local metadata files is only one file, it is the metadata file itself.
|
||||
}
|
||||
void MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &)
|
||||
|
||||
UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &)
|
||||
{
|
||||
/// Noop, no separate metadata.
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -9,6 +9,9 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct UnlinkMetadataFileOperationOutcome;
|
||||
using UnlinkMetadataFileOperationOutcomePtr = std::shared_ptr<UnlinkMetadataFileOperationOutcome>;
|
||||
|
||||
/// Object storage is used as a filesystem, in a limited form:
|
||||
/// - no directory concept, files only
|
||||
/// - no stat/chmod/...
|
||||
@ -104,7 +107,7 @@ public:
|
||||
|
||||
void unlinkFile(const std::string & path) override;
|
||||
|
||||
void unlinkMetadata(const std::string & path) override;
|
||||
UnlinkMetadataFileOperationOutcomePtr unlinkMetadata(const std::string & path) override;
|
||||
|
||||
void commit() override
|
||||
{
|
||||
|
Loading…
Reference in New Issue
Block a user