mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
in-memory map path comparator
This commit is contained in:
parent
ecca720f9e
commit
97519ae800
@ -18,7 +18,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator(
|
||||
ObjectStorageKey
|
||||
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);
|
||||
|
||||
@ -54,7 +55,7 @@ std::tuple<std::string, std::vector<std::string>> CommonPathPrefixKeyGenerator::
|
||||
|
||||
while (p != p.root_path())
|
||||
{
|
||||
auto it = ptr->find(p / "");
|
||||
auto it = ptr->find(p);
|
||||
if (it != ptr->end())
|
||||
{
|
||||
std::vector<std::string> vec(std::make_move_iterator(dq.begin()), std::make_move_iterator(dq.end()));
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Common/ObjectStorageKeyGenerator.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
|
||||
#include <Disks/ObjectStorages/PathComparator.h>
|
||||
|
||||
#include <filesystem>
|
||||
#include <map>
|
||||
#include <optional>
|
||||
@ -23,7 +25,7 @@ class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator
|
||||
{
|
||||
public:
|
||||
/// 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_);
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Disks/ObjectStorages/IMetadataStorage.h>
|
||||
#include <Disks/ObjectStorages/MetadataOperationsHolder.h>
|
||||
#include <Disks/ObjectStorages/MetadataStorageTransactionState.h>
|
||||
#include <Disks/ObjectStorages/PathComparator.h>
|
||||
|
||||
#include <map>
|
||||
#include <string>
|
||||
@ -29,7 +30,7 @@ class MetadataStorageFromPlainObjectStorage : public IMetadataStorage
|
||||
{
|
||||
public:
|
||||
/// 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:
|
||||
friend class MetadataStorageFromPlainObjectStorageTransaction;
|
||||
|
@ -43,7 +43,7 @@ MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFr
|
||||
|
||||
void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
if (path_map.contains(path))
|
||||
if (path_map.contains(path.parent_path()))
|
||||
return;
|
||||
|
||||
auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix);
|
||||
@ -64,7 +64,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
||||
|
||||
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);
|
||||
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
||||
CurrentMetrics::add(metric, 1);
|
||||
@ -84,7 +84,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
|
||||
|
||||
if (write_finalized)
|
||||
{
|
||||
path_map.erase(path);
|
||||
path_map.erase(path.parent_path());
|
||||
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
|
||||
CurrentMetrics::sub(metric, 1);
|
||||
|
||||
@ -111,11 +111,11 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom
|
||||
std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf(
|
||||
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())
|
||||
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);
|
||||
|
||||
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);
|
||||
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);
|
||||
|
||||
write_finalized = true;
|
||||
@ -165,7 +165,7 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u
|
||||
void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
|
||||
{
|
||||
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)
|
||||
{
|
||||
@ -186,7 +186,7 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr
|
||||
|
||||
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())
|
||||
return;
|
||||
|
||||
@ -223,7 +223,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un
|
||||
writeString(path.string(), *buf);
|
||||
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;
|
||||
CurrentMetrics::add(metric, 1);
|
||||
}
|
||||
|
@ -97,7 +97,7 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri
|
||||
std::pair<MetadataStorageFromPlainObjectStorage::PathMap::iterator, bool> res;
|
||||
{
|
||||
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
|
||||
@ -145,11 +145,10 @@ void getDirectChildrenOnDiskImpl(
|
||||
break;
|
||||
|
||||
auto slash_num = count(k.begin() + local_path.size(), k.end(), '/');
|
||||
if (slash_num != 1)
|
||||
if (slash_num != 0)
|
||||
continue;
|
||||
|
||||
chassert(k.back() == '/');
|
||||
remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end() - 1));
|
||||
remote_to_local_subdir.emplace(v, std::string(k.begin() + local_path.size(), k.end()) + "/");
|
||||
}
|
||||
}
|
||||
|
||||
@ -243,7 +242,7 @@ std::vector<std::string> MetadataStorageFromPlainRewritableObjectStorage::listDi
|
||||
object_storage->listObjects(abs_key, files, 0);
|
||||
|
||||
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
|
||||
/// metadata along with regular files.
|
||||
if (object_storage->getCommonKeyPrefix() != getMetadataKeyPrefix())
|
||||
|
20
src/Disks/ObjectStorages/PathComparator.h
Normal file
20
src/Disks/ObjectStorages/PathComparator.h
Normal 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;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user