in-memory map path comparator

This commit is contained in:
Julia Kartseva 2024-07-04 07:02:13 +00:00
parent ecca720f9e
commit 97519ae800
6 changed files with 41 additions and 18 deletions

View File

@ -18,7 +18,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator(
ObjectStorageKey ObjectStorageKey
CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const
{ {
const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); const auto & [object_key_prefix, suffix_parts]
= getLongestObjectKeyPrefix(is_directory ? std::filesystem::path(path).parent_path().string() : path);
auto key = std::filesystem::path(object_key_prefix); auto key = std::filesystem::path(object_key_prefix);
@ -54,7 +55,7 @@ std::tuple<std::string, std::vector<std::string>> CommonPathPrefixKeyGenerator::
while (p != p.root_path()) while (p != p.root_path())
{ {
auto it = ptr->find(p / ""); auto it = ptr->find(p);
if (it != ptr->end()) if (it != ptr->end())
{ {
std::vector<std::string> vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end())); std::vector<std::string> vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end()));

View File

@ -3,6 +3,8 @@
#include <Common/ObjectStorageKeyGenerator.h> #include <Common/ObjectStorageKeyGenerator.h>
#include <Common/SharedMutex.h> #include <Common/SharedMutex.h>
#include <Disks/ObjectStorages/PathComparator.h>
#include <filesystem> #include <filesystem>
#include <map> #include <map>
#include <optional> #include <optional>
@ -23,7 +25,7 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator
{ {
public: public:
/// Local to remote path map. Leverages filesystem::path comparator for paths. /// Local to remote path map. Leverages filesystem::path comparator for paths.
using PathMap = std::map<std::filesystem::path, std::string>; using PathMap = std::map<std::filesystem::path, std::string, PathComparator>;
explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr<PathMap> path_map_); explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr<PathMap> path_map_);

View File

@ -4,6 +4,7 @@
#include <Disks/ObjectStorages/IMetadataStorage.h> #include <Disks/ObjectStorages/IMetadataStorage.h>
#include <Disks/ObjectStorages/MetadataOperationsHolder.h> #include <Disks/ObjectStorages/MetadataOperationsHolder.h>
#include <Disks/ObjectStorages/MetadataStorageTransactionState.h> #include <Disks/ObjectStorages/MetadataStorageTransactionState.h>
#include <Disks/ObjectStorages/PathComparator.h>
#include <map> #include <map>
#include <string> #include <string>
@ -29,7 +30,7 @@ class MetadataStorageFromPlainObjectStorage : public IMetadataStorage
{ {
public: public:
/// Local path prefixes mapped to storage key prefixes. /// Local path prefixes mapped to storage key prefixes.
using PathMap = std::map<std::filesystem::path, std::string>; using PathMap = std::map<std::filesystem::path, std::string, PathComparator>;
private: private:
friend class MetadataStorageFromPlainObjectStorageTransaction; friend class MetadataStorageFromPlainObjectStorageTransaction;

View File

@ -43,7 +43,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr
void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock<SharedMutex> &) void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
{ {
if (path_map.contains(path)) if (path_map.contains(path.parent_path()))
return; return;
auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix); auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix);
@ -64,7 +64,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
write_created = true; write_created = true;
[[maybe_unused]] auto result = path_map.emplace(path, std::move(key_prefix)); [[maybe_unused]] auto result = path_map.emplace(path.parent_path(), std::move(key_prefix));
chassert(result.second); chassert(result.second);
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, 1); CurrentMetrics::add(metric, 1);
@ -84,7 +84,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
if (write_finalized) if (write_finalized)
{ {
path_map.erase(path); path_map.erase(path.parent_path());
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::sub(metric, 1); CurrentMetrics::sub(metric, 1);
@ -111,11 +111,11 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom
std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf( std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf(
const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content) const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content)
{ {
auto expected_it = path_map.find(expected_path); auto expected_it = path_map.find(expected_path.parent_path());
if (expected_it == path_map.end()) if (expected_it == path_map.end())
throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path); throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Metadata object for the expected (source) path '{}' does not exist", expected_path);
if (path_map.contains(new_path)) if (path_map.contains(new_path.parent_path()))
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path); throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Metadata object for the new (destination) path '{}' already exists", new_path);
auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix); auto metadata_object_key = createMetadataObjectKey(expected_it->second, metadata_key_prefix);
@ -156,7 +156,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u
writeString(path_to.string(), *write_buf); writeString(path_to.string(), *write_buf);
write_buf->finalize(); write_buf->finalize();
[[maybe_unused]] auto result = path_map.emplace(path_to, path_map.extract(path_from).mapped()); [[maybe_unused]] auto result = path_map.emplace(path_to.parent_path(), path_map.extract(path_from.parent_path()).mapped());
chassert(result.second); chassert(result.second);
write_finalized = true; write_finalized = true;
@ -165,7 +165,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u
void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &) void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
{ {
if (write_finalized) if (write_finalized)
path_map.emplace(path_from, path_map.extract(path_to).mapped()); path_map.emplace(path_from.parent_path(), path_map.extract(path_to.parent_path()).mapped());
if (write_created) if (write_created)
{ {
@ -186,7 +186,7 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr
void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock<SharedMutex> & /* metadata_lock */) void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock<SharedMutex> & /* metadata_lock */)
{ {
auto path_it = path_map.find(path); auto path_it = path_map.find(path.parent_path());
if (path_it == path_map.end()) if (path_it == path_map.end())
return; return;
@ -223,7 +223,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un
writeString(path.string(), *buf); writeString(path.string(), *buf);
buf->finalize(); buf->finalize();
path_map.emplace(path, std::move(key_prefix)); path_map.emplace(path.parent_path(), std::move(key_prefix));
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size; auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, 1); CurrentMetrics::add(metric, 1);
} }

View File

@ -97,7 +97,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri
std::pair<MetadataStorageFromPlainObjectStorage::PathMap::iterator, bool> res; std::pair<MetadataStorageFromPlainObjectStorage::PathMap::iterator, bool> res;
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
res = result.emplace(local_path, remote_path.parent_path()); res = result.emplace(std::filesystem::path(local_path).parent_path(), remote_path.parent_path());
} }
/// This can happen if table replication is enabled, then the same local path is written /// This can happen if table replication is enabled, then the same local path is written
@ -145,11 +145,10 @@ void getDirectChildrenOnDiskImpl(
break; break;
auto slash_num = count(k.begin() + local_path.size(), k.end(), '/'); auto slash_num = count(k.begin() + local_path.size(), k.end(), '/');
if (slash_num != 1) if (slash_num != 0)
continue; continue;
chassert(k.back() == '/'); remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/");
remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end() - 1));
} }
} }
@ -243,7 +242,7 @@ std::vector<std::string> MetadataStorageFromPlainRewritableObjectStorage::listDi
object_storage->listObjects(abs_key, files, 0); object_storage->listObjects(abs_key, files, 0);
std::unordered_set<std::string> directories; std::unordered_set<std::string> directories;
getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, path, directories); getDirectChildrenOnDisk(abs_key, object_storage->getCommonKeyPrefix(), files, std::filesystem::path(path) / "", directories);
/// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove /// List empty directories that are identified by the `prefix.path` metadata files. This is required to, e.g., remove
/// metadata along with regular files. /// metadata along with regular files.
if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix()) if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix())

View File

@ -0,0 +1,20 @@
#pragma once
#include <filesystem>
namespace DB
{
// TODO: rename
struct PathComparator
{
bool operator()(const std::filesystem::path & path1, const std::filesystem::path & path2) const
{
auto d1 = std::distance(path1.begin(), path1.end());
auto d2 = std::distance(path2.begin(), path2.end());
if (d1 != d2)
return d1 < d2;
return path1 < path2;
}
};
}