style and doc

This commit is contained in:
Julia Kartseva 2024-07-12 02:27:45 +00:00
parent c0e6780dfe
commit 3f066018fb
13 changed files with 35 additions and 34 deletions

View File

@ -3,7 +3,6 @@
#include <Common/getRandomASCIIString.h>
#include <Common/MatchGenerator.h>
#include <optional>
#include <fmt/format.h>

View File

@ -1,6 +1,7 @@
#pragma once
#include <memory>
#include <optional>
#include "ObjectStorageKey.h"
namespace DB
@ -12,6 +13,9 @@ public:
virtual ~IObjectStorageKeysGenerator() = default;
/// Generates an object storage key based on a path in the virtual filesystem.
/// @param path - Path in the virtual filesystem.
/// @param is_directory - If the path in the virtual filesystem corresponds to a directory.
/// @param key_prefix - Optional key prefix for the generated object storage key. If provided, this prefix will be added to the beginning of the generated key.
virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const = 0;
};

View File

@ -9,6 +9,7 @@
namespace DB
{
/// Deprecated. Used for backward compatibility with plain rewritable disks without a separate metadata layout.
/// Object storage key generator used specifically with the
/// MetadataStorageFromPlainObjectStorage if multiple writes are allowed.

View File

@ -1,4 +1,4 @@
#include "FlatStructureKeyGenerator.h"
#include "FlatDirectoryStructureKeyGenerator.h"
#include <Disks/ObjectStorages/InMemoryPathMap.h>
#include "Common/ObjectStorageKey.h"
#include "Common/SharedMutex.h"
@ -11,12 +11,12 @@
namespace DB
{
FlatStructureKeyGenerator::FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr<InMemoryPathMap> path_map_)
FlatDirectoryStructureKeyGenerator::FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr<InMemoryPathMap> path_map_)
: storage_key_prefix(storage_key_prefix_), path_map(std::move(path_map_))
{
}
ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const
ObjectStorageKey FlatDirectoryStructureKeyGenerator::generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const
{
if (is_directory)
chassert(path.ends_with('/'));
@ -24,8 +24,6 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i
const auto p = std::filesystem::path(path);
auto directory = p.parent_path();
constexpr size_t part_size = 32;
std::optional<std::filesystem::path> remote_path;
{
auto ptr = path_map.lock();
@ -38,6 +36,7 @@ ObjectStorageKey FlatStructureKeyGenerator::generate(const String & path, bool i
if (it != ptr->map.end())
remote_path = it->second;
}
constexpr size_t part_size = 32;
std::filesystem::path key = remote_path.has_value() ? *remote_path
: is_directory ? std::filesystem::path(getRandomASCIIString(part_size))
: directory;

View File

@ -7,10 +7,10 @@ namespace DB
{
struct InMemoryPathMap;
class FlatStructureKeyGenerator : public IObjectStorageKeysGenerator
class FlatDirectoryStructureKeyGenerator : public IObjectStorageKeysGenerator
{
public:
explicit FlatStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr<InMemoryPathMap> path_map_);
explicit FlatDirectoryStructureKeyGenerator(String storage_key_prefix_, std::weak_ptr<InMemoryPathMap> path_map_);
ObjectStorageKey generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const override;

View File

@ -4,7 +4,6 @@
#include <Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h>
#include <Storages/ObjectStorage/HDFS/HDFSCommon.h>
#include <optional>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h>
#include <Common/getRandomASCIIString.h>

View File

@ -2,7 +2,7 @@
#include <filesystem>
#include <map>
#include "Common/SharedMutex.h"
#include <Common/SharedMutex.h>
namespace DB
{

View File

@ -1,7 +1,6 @@
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#include <filesystem>
#include <optional>
#include <Disks/IO/AsynchronousBoundedReadBuffer.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/createReadBufferFromFileBase.h>

View File

@ -146,10 +146,8 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std
return;
auto normalized_path = normalizeDirectoryPath(path);
auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, "" /* key_prefix */).serialize();
auto op = std::make_unique<MetadataStorageFromPlainObjectStorageCreateDirectoryOperation>(
std::move(normalized_path),
std::move(key_prefix),
*metadata_storage.getPathMap(),
object_storage,
metadata_storage.getMetadataKeyPrefix());

View File

@ -78,11 +78,10 @@ public:
bool supportsStat() const override { return false; }
protected:
/// Get the object storage prefix for storing metadata files. If stored behind a separate endpoint,
/// the metadata keys reflect the layout of the regular files.
/// Get the object storage prefix for storing metadata files.
virtual std::string getMetadataKeyPrefix() const { return object_storage->getCommonKeyPrefix(); }
/// Returns a map of local paths to paths in object storage.
/// Returns a map of virtual filesystem paths to paths in the object storage.
virtual std::shared_ptr<InMemoryPathMap> getPathMap() const { throwNotImplemented(); }
};

View File

@ -29,25 +29,21 @@ ObjectStorageKey createMetadataObjectKey(const std::string & key_prefix, const s
}
MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::MetadataStorageFromPlainObjectStorageCreateDirectoryOperation(
std::filesystem::path && path_,
std::string && key_prefix_,
InMemoryPathMap & path_map_,
ObjectStoragePtr object_storage_,
const std::string & metadata_key_prefix_)
std::filesystem::path && path_, InMemoryPathMap & path_map_, ObjectStoragePtr object_storage_, const std::string & metadata_key_prefix_)
: path(std::move(path_))
, key_prefix(key_prefix_)
, path_map(path_map_)
, object_storage(object_storage_)
, metadata_key_prefix(metadata_key_prefix_)
, key_prefix(object_storage->generateObjectKeyPrefixForDirectoryPath(path, "" /* key_prefix */).serialize())
{
}
void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std::unique_lock<SharedMutex> &)
{
auto & map = path_map.map;
auto & mutex = path_map.mutex;
{
std::shared_lock lock(mutex);
auto & map = path_map.map;
if (map.contains(path.parent_path()))
return;
}
@ -72,6 +68,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
{
std::unique_lock lock(mutex);
auto & map = path_map.map;
[[maybe_unused]] auto result = map.emplace(path.parent_path(), std::move(key_prefix));
chassert(result.second);
}
@ -89,7 +86,6 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::unique_lock<SharedMutex> &)
{
auto & map = path_map.map;
auto & mutex = path_map.mutex;
auto metadata_object_key = createMetadataObjectKey(key_prefix, metadata_key_prefix);
@ -98,6 +94,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::undo(std::un
{
{
std::unique_lock lock(mutex);
auto & map = path_map.map;
map.erase(path.parent_path());
}
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
@ -126,12 +123,12 @@ MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::MetadataStorageFrom
std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::createWriteBuf(
const std::filesystem::path & expected_path, const std::filesystem::path & new_path, bool validate_content)
{
auto & map = path_map.map;
auto & mutex = path_map.mutex;
std::filesystem::path remote_path;
{
std::shared_lock lock(mutex);
auto & map = path_map.map;
auto expected_it = map.find(expected_path.parent_path());
if (expected_it == map.end())
throw Exception(
@ -182,10 +179,10 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::execute(std::u
writeString(path_to.string(), *write_buf);
write_buf->finalize();
auto & map = path_map.map;
auto & mutex = path_map.mutex;
{
std::unique_lock lock(mutex);
auto & map = path_map.map;
[[maybe_unused]] auto result = map.emplace(path_to.parent_path(), map.extract(path_from.parent_path()).mapped());
chassert(result.second);
}
@ -197,9 +194,9 @@ void MetadataStorageFromPlainObjectStorageMoveDirectoryOperation::undo(std::uniq
{
if (write_finalized)
{
auto & map = path_map.map;
auto & mutex = path_map.mutex;
std::unique_lock lock(mutex);
auto & map = path_map.map;
map.emplace(path_from.parent_path(), map.extract(path_to.parent_path()).mapped());
}
@ -219,10 +216,10 @@ MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::MetadataStorageFr
void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std::unique_lock<SharedMutex> & /* metadata_lock */)
{
auto & map = path_map.map;
auto & mutex = path_map.mutex;
{
std::shared_lock lock(mutex);
auto & map = path_map.map;
auto path_it = map.find(path.parent_path());
if (path_it == map.end())
return;
@ -237,6 +234,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::execute(std:
{
std::unique_lock lock(mutex);
auto & map = path_map.map;
map.erase(path.parent_path());
}
@ -265,10 +263,10 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un
writeString(path.string(), *buf);
buf->finalize();
auto & map = path_map.map;
auto & mutex = path_map.mutex;
{
std::unique_lock lock(mutex);
auto & map = path_map.map;
map.emplace(path.parent_path(), std::move(key_prefix));
}
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;

View File

@ -14,10 +14,10 @@ class MetadataStorageFromPlainObjectStorageCreateDirectoryOperation final : publ
{
private:
std::filesystem::path path;
std::string key_prefix;
InMemoryPathMap & path_map;
ObjectStoragePtr object_storage;
const std::string metadata_key_prefix;
const std::string key_prefix;
bool write_created = false;
bool write_finalized = false;
@ -26,7 +26,6 @@ public:
// Assuming that paths are normalized.
MetadataStorageFromPlainObjectStorageCreateDirectoryOperation(
std::filesystem::path && path_,
std::string && key_prefix_,
InMemoryPathMap & path_map_,
ObjectStoragePtr object_storage_,
const std::string & metadata_key_prefix_);

View File

@ -1,4 +1,4 @@
#include <Disks/ObjectStorages/FlatStructureKeyGenerator.h>
#include <Disks/ObjectStorages/FlatDirectoryStructureKeyGenerator.h>
#include <Disks/ObjectStorages/InMemoryPathMap.h>
#include <Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h>
#include <Disks/ObjectStorages/ObjectStorageIterator.h>
@ -132,16 +132,20 @@ void getDirectChildrenOnDiskImpl(
SharedMutex & shared_mutex,
std::unordered_set<std::string> & result)
{
/// Directories are retrieved from the in-memory path map.
{
std::shared_lock lock(shared_mutex);
auto end_it = local_path_prefixes.end();
for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it)
{
const auto & [k, v] = std::make_tuple(it->first.string(), it->second);
const auto & [k, _] = std::make_tuple(it->first.string(), it->second);
if (!k.starts_with(local_path))
break;
auto slash_num = count(k.begin() + local_path.size(), k.end(), '/');
/// The local_path_prefixes comparator ensures that the paths with the smallest number of
/// hops from the local_path are iterated first. The paths do not end with '/', hence
/// break the loop if the number of slashes is greater than 0.
if (slash_num != 0)
break;
@ -149,6 +153,7 @@ void getDirectChildrenOnDiskImpl(
}
}
/// Files.
auto skip_list = std::set<std::string>{PREFIX_PATH_FILE_NAME};
for (const auto & elem : remote_paths)
{
@ -189,7 +194,8 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita
}
else
{
auto keys_gen = std::make_shared<FlatStructureKeyGenerator>(object_storage->getCommonKeyPrefix(), path_map);
/// Use flat directory structure if the metadata is stored separately from the table data.
auto keys_gen = std::make_shared<FlatDirectoryStructureKeyGenerator>(object_storage->getCommonKeyPrefix(), path_map);
object_storage->setKeysGenerator(keys_gen);
}
}