Check refcount in RemoveManyObjectStorageOperation::finalize instead of execute

This commit is contained in:
vdimir 2023-07-07 15:08:21 +00:00
parent eed1ecb6ba
commit 227e415d6d
No known key found for this signature in database
GPG Key ID: 6EE4CE2BEDC51862
8 changed files with 63 additions and 20 deletions

View File

@ -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);
}
};

View File

@ -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;

View File

@ -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;
}
}

View File

@ -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;
};

View File

@ -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)

View File

@ -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_,

View File

@ -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;
}
}

View File

@ -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
{