Merge pull request #71395 from ClickHouse/backport/24.8/70823

Backport #70823 to 24.8: Fix excessive LIST API calls to the object storage in the plain_rewritable disk
This commit is contained in:
robot-ch-test-poll 2024-11-02 17:54:57 +01:00 committed by GitHub
commit 640f8bd1c4
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 334 additions and 109 deletions

View File

@ -180,8 +180,14 @@
M(ObjectStorageAzureThreadsScheduled, "Number of queued or active jobs in the AzureObjectStorage thread pool.") \
\
M(DiskPlainRewritableAzureDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \
M(DiskPlainRewritableAzureFileCount, "Number of file entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \
M(DiskPlainRewritableAzureUniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for AzureObjectStorage.") \
M(DiskPlainRewritableLocalDirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \
M(DiskPlainRewritableLocalFileCount, "Number of file entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \
M(DiskPlainRewritableLocalUniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for LocalObjectStorage.") \
M(DiskPlainRewritableS3DirectoryMapSize, "Number of local-to-remote path entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \
M(DiskPlainRewritableS3FileCount, "Number of file entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \
M(DiskPlainRewritableS3UniqueFileNamesCount, "Number of unique file name entries in the 'plain_rewritable' in-memory map for S3ObjectStorage.") \
\
M(MergeTreePartsLoaderThreads, "Number of threads in the MergeTree parts loader thread pool.") \
M(MergeTreePartsLoaderThreadsActive, "Number of threads in the MergeTree parts loader thread pool running a task.") \

View File

@ -2,7 +2,9 @@
#include <filesystem>
#include <map>
#include <memory>
#include <optional>
#include <set>
#include <shared_mutex>
#include <base/defines.h>
#include <Common/SharedLockGuard.h>
@ -25,10 +27,19 @@ struct InMemoryDirectoryPathMap
return path1 < path2;
}
};
using FileNames = std::set<std::string>;
using FileNamesIterator = FileNames::iterator;
struct FileNameIteratorComparator
{
bool operator()(const FileNames::iterator & lhs, const FileNames::iterator & rhs) const { return *lhs < *rhs; }
};
struct RemotePathInfo
{
std::string path;
time_t last_modified = 0;
std::set<FileNamesIterator, FileNameIteratorComparator> filename_iterators;
};
using Map = std::map<std::filesystem::path, RemotePathInfo, PathComparator>;
@ -49,9 +60,11 @@ struct InMemoryDirectoryPathMap
mutable SharedMutex mutex;
#ifdef OS_LINUX
FileNames TSA_GUARDED_BY(mutex) unique_filenames;
Map TSA_GUARDED_BY(mutex) map;
/// std::shared_mutex may not be annotated with the 'capability' attribute in libcxx.
#else
FileNames unique_filenames;
Map map;
#endif
};

View File

@ -203,6 +203,21 @@ void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std
}
}
void MetadataStorageFromPlainObjectStorageTransaction::createEmptyMetadataFile(const std::string & path)
{
if (metadata_storage.object_storage->isWriteOnce())
return;
addOperation(
std::make_unique<MetadataStorageFromPlainObjectStorageWriteFileOperation>(path, *metadata_storage.getPathMap(), object_storage));
}
void MetadataStorageFromPlainObjectStorageTransaction::createMetadataFile(
const std::string & path, ObjectStorageKey /*object_key*/, uint64_t /* size_in_bytes */)
{
createEmptyMetadataFile(path);
}
void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std::string & path)
{
if (metadata_storage.object_storage->isWriteOnce())
@ -235,12 +250,6 @@ void MetadataStorageFromPlainObjectStorageTransaction::moveDirectory(const std::
metadata_storage.getMetadataKeyPrefix()));
}
void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata(
const std::string &, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */)
{
/// Noop, local metadata files is only one file, it is the metadata file itself.
}
UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string & path)
{
/// The record has become stale, remove it from cache.
@ -252,8 +261,11 @@ UnlinkMetadataFileOperationOutcomePtr MetadataStorageFromPlainObjectStorageTrans
metadata_storage.object_metadata_cache->remove(hash.get128());
}
/// No hardlinks, so will always remove file.
return std::make_shared<UnlinkMetadataFileOperationOutcome>(UnlinkMetadataFileOperationOutcome{0});
auto result = std::make_shared<UnlinkMetadataFileOperationOutcome>(UnlinkMetadataFileOperationOutcome{0});
if (!metadata_storage.object_storage->isWriteOnce())
addOperation(std::make_unique<MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation>(
path, *metadata_storage.getPathMap(), object_storage));
return result;
}
void MetadataStorageFromPlainObjectStorageTransaction::commit()

View File

@ -114,22 +114,19 @@ public:
const IMetadataStorage & getStorageForNonTransactionalReads() const override;
void addBlobToMetadata(const std::string & path, ObjectStorageKey object_key, uint64_t size_in_bytes) override;
void addBlobToMetadata(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override
{
// Noop
}
void setLastModified(const String &, const Poco::Timestamp &) override
{
/// Noop
}
void createEmptyMetadataFile(const std::string & /* path */) override
{
/// No metadata, no need to create anything.
}
void createEmptyMetadataFile(const std::string & /* path */) override;
void createMetadataFile(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override
{
/// Noop
}
void createMetadataFile(const std::string & /* path */, ObjectStorageKey /* object_key */, uint64_t /* size_in_bytes */) override;
void createDirectory(const std::string & path) override;

View File

@ -1,6 +1,8 @@
#include "MetadataStorageFromPlainObjectStorageOperations.h"
#include <Disks/ObjectStorages/InMemoryDirectoryPathMap.h>
#include <filesystem>
#include <mutex>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Poco/Timestamp.h>
@ -76,7 +78,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
std::lock_guard lock(path_map.mutex);
auto & map = path_map.map;
[[maybe_unused]] auto result
= map.emplace(base_path, InMemoryDirectoryPathMap::RemotePathInfo{object_key_prefix, Poco::Timestamp{}.epochTime()});
= map.emplace(base_path, InMemoryDirectoryPathMap::RemotePathInfo{object_key_prefix, Poco::Timestamp{}.epochTime(), {}});
chassert(result.second);
}
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
@ -287,4 +289,122 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo(std::un
CurrentMetrics::add(metric, 1);
}
MetadataStorageFromPlainObjectStorageWriteFileOperation::MetadataStorageFromPlainObjectStorageWriteFileOperation(
const std::string & path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_)
: path(path_), path_map(path_map_), object_storage(object_storage_)
{
}
void MetadataStorageFromPlainObjectStorageWriteFileOperation::execute(std::unique_lock<SharedMutex> &)
{
LOG_TEST(getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"), "Creating metadata for a file '{}'", path);
std::lock_guard lock(path_map.mutex);
auto it = path_map.map.find(path.parent_path());
/// Some paths (e.g., clickhouse_access_check) may not have parent directories.
if (it == path_map.map.end())
LOG_TRACE(
getLogger("MetadataStorageFromPlainObjectStorageWriteFileOperation"),
"Parent dirrectory does not exist, skipping path {}",
path);
else
{
auto [filename_it, inserted] = path_map.unique_filenames.emplace(path.filename());
if (inserted)
{
auto metric = object_storage->getMetadataStorageMetrics().unique_filenames_count;
CurrentMetrics::add(metric, 1);
}
written = it->second.filename_iterators.emplace(filename_it).second;
if (written)
{
auto metric = object_storage->getMetadataStorageMetrics().file_count;
CurrentMetrics::add(metric, 1);
}
}
}
void MetadataStorageFromPlainObjectStorageWriteFileOperation::undo(std::unique_lock<SharedMutex> &)
{
if (written)
{
std::lock_guard lock(path_map.mutex);
auto it = path_map.map.find(path.parent_path());
chassert(it != path_map.map.end());
if (it != path_map.map.end())
{
auto filename_it = path_map.unique_filenames.find(path.filename());
if (filename_it != path_map.unique_filenames.end())
{
if (it->second.filename_iterators.erase(filename_it) > 0)
{
auto metric = object_storage->getMetadataStorageMetrics().file_count;
CurrentMetrics::sub(metric, 1);
}
}
}
}
}
MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation(
std::filesystem::path && path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_)
: path(path_)
, remote_path(std::filesystem::path(object_storage_->generateObjectKeyForPath(path_, std::nullopt).serialize()))
, path_map(path_map_)
, object_storage(object_storage_)
{
}
void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::execute(std::unique_lock<SharedMutex> &)
{
LOG_TEST(
getLogger("MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation"),
"Unlinking metadata for a write '{}' with remote path '{}'",
path,
remote_path);
std::lock_guard lock(path_map.mutex);
auto it = path_map.map.find(path.parent_path());
if (it == path_map.map.end())
LOG_TRACE(
getLogger("MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation"),
"Parent directory does not exist, skipping path {}",
path);
else
{
auto & filename_iterators = it->second.filename_iterators;
auto filename_it = path_map.unique_filenames.find(path.filename());
if (filename_it != path_map.unique_filenames.end())
unlinked = (filename_iterators.erase(filename_it) > 0);
if (unlinked)
{
auto metric = object_storage->getMetadataStorageMetrics().file_count;
CurrentMetrics::sub(metric, 1);
}
}
}
void MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation::undo(std::unique_lock<SharedMutex> &)
{
if (unlinked)
{
std::lock_guard lock(path_map.mutex);
auto it = path_map.map.find(path.parent_path());
chassert(it != path_map.map.end());
if (it != path_map.map.end())
{
auto filename_it = path_map.unique_filenames.find(path.filename());
if (filename_it != path_map.unique_filenames.end())
{
if (it->second.filename_iterators.emplace(filename_it).second)
{
auto metric = object_storage->getMetadataStorageMetrics().file_count;
CurrentMetrics::add(metric, 1);
}
}
}
}
}
}

View File

@ -87,4 +87,38 @@ public:
void undo(std::unique_lock<SharedMutex> & metadata_lock) override;
};
class MetadataStorageFromPlainObjectStorageWriteFileOperation final : public IMetadataOperation
{
private:
std::filesystem::path path;
InMemoryDirectoryPathMap & path_map;
ObjectStoragePtr object_storage;
bool written = false;
public:
MetadataStorageFromPlainObjectStorageWriteFileOperation(
const std::string & path, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_);
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
void undo(std::unique_lock<SharedMutex> & metadata_lock) override;
};
class MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation final : public IMetadataOperation
{
private:
std::filesystem::path path;
std::filesystem::path remote_path;
InMemoryDirectoryPathMap & path_map;
ObjectStoragePtr object_storage;
bool unlinked = false;
public:
MetadataStorageFromPlainObjectStorageUnlinkMetadataFileOperation(
std::filesystem::path && path_, InMemoryDirectoryPathMap & path_map_, ObjectStoragePtr object_storage_);
void execute(std::unique_lock<SharedMutex> & metadata_lock) override;
void undo(std::unique_lock<SharedMutex> & metadata_lock) override;
};
}

View File

@ -3,9 +3,14 @@
#include <Disks/ObjectStorages/MetadataStorageFromPlainRewritableObjectStorage.h>
#include <Disks/ObjectStorages/ObjectStorageIterator.h>
#include <algorithm>
#include <any>
#include <cstddef>
#include <exception>
#include <iterator>
#include <mutex>
#include <optional>
#include <unordered_map>
#include <unordered_set>
#include <IO/ReadHelpers.h>
#include <IO/S3Common.h>
@ -15,6 +20,7 @@
#include <Common/SharedLockGuard.h>
#include <Common/SharedMutex.h>
#include <Common/logger_useful.h>
#include <Common/setThreadName.h>
#include "CommonPathPrefixKeyGenerator.h"
@ -45,6 +51,61 @@ std::string getMetadataKeyPrefix(ObjectStoragePtr object_storage)
: metadata_key_prefix;
}
void loadDirectoryTree(
InMemoryDirectoryPathMap::Map & map, InMemoryDirectoryPathMap::FileNames & unique_filenames, ObjectStoragePtr object_storage)
{
using FileNamesIterator = InMemoryDirectoryPathMap::FileNamesIterator;
using FileNameIteratorComparator = InMemoryDirectoryPathMap::FileNameIteratorComparator;
const auto common_key_prefix = object_storage->getCommonKeyPrefix();
ThreadPool & pool = getIOThreadPool().get();
ThreadPoolCallbackRunnerLocal<void> runner(pool, "PlainRWTreeLoad");
std::atomic<size_t> num_files = 0;
LOG_DEBUG(getLogger("MetadataStorageFromPlainObjectStorage"), "Loading directory tree");
std::mutex mutex;
for (auto & item : map)
{
auto & remote_path_info = item.second;
const auto remote_path = std::filesystem::path(common_key_prefix) / remote_path_info.path / "";
runner(
[remote_path, &mutex, &remote_path_info, &unique_filenames, &object_storage, &num_files]
{
setThreadName("PlainRWTreeLoad");
std::set<FileNamesIterator, FileNameIteratorComparator> filename_iterators;
for (auto iterator = object_storage->iterate(remote_path, 0); iterator->isValid(); iterator->next())
{
auto file = iterator->current();
String path = file->getPath();
chassert(path.starts_with(remote_path.string()));
auto filename = std::filesystem::path(path).filename();
/// Check that the file is a direct child.
if (path.substr(remote_path.string().size()) == filename)
{
auto filename_it = unique_filenames.end();
{
std::lock_guard lock(mutex);
filename_it = unique_filenames.emplace(filename).first;
}
auto inserted = filename_iterators.emplace(filename_it).second;
chassert(inserted);
if (inserted)
++num_files;
}
}
auto metric = object_storage->getMetadataStorageMetrics().file_count;
CurrentMetrics::add(metric, filename_iterators.size());
remote_path_info.filename_iterators = std::move(filename_iterators);
});
}
runner.waitForAllToFinishAndRethrowFirstError();
LOG_DEBUG(
getLogger("MetadataStorageFromPlainObjectStorage"),
"Loaded directory tree for {} directories, found {} files",
map.size(),
num_files);
}
std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string & metadata_key_prefix, ObjectStoragePtr object_storage)
{
auto result = std::make_shared<InMemoryDirectoryPathMap>();
@ -62,6 +123,9 @@ std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string &
LOG_DEBUG(log, "Loading metadata");
size_t num_files = 0;
std::mutex mutex;
InMemoryDirectoryPathMap::Map map;
for (auto iterator = object_storage->iterate(metadata_key_prefix, 0); iterator->isValid(); iterator->next())
{
++num_files;
@ -72,7 +136,7 @@ std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string &
continue;
runner(
[remote_metadata_path, path, &object_storage, &result, &log, &settings, &metadata_key_prefix]
[remote_metadata_path, path, &object_storage, &mutex, &map, &log, &settings, &metadata_key_prefix]
{
setThreadName("PlainRWMetaLoad");
@ -109,13 +173,13 @@ std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string &
chassert(remote_metadata_path.has_parent_path());
chassert(remote_metadata_path.string().starts_with(metadata_key_prefix));
auto suffix = remote_metadata_path.string().substr(metadata_key_prefix.size());
auto remote_path = std::filesystem::path(std::move(suffix));
auto rel_path = std::filesystem::path(std::move(suffix));
std::pair<Map::iterator, bool> res;
{
std::lock_guard lock(result->mutex);
res = result->map.emplace(
std::lock_guard lock(mutex);
res = map.emplace(
std::filesystem::path(local_path).parent_path(),
InMemoryDirectoryPathMap::RemotePathInfo{remote_path.parent_path(), last_modified.epochTime()});
InMemoryDirectoryPathMap::RemotePathInfo{rel_path.parent_path(), last_modified.epochTime(), {}});
}
/// This can happen if table replication is enabled, then the same local path is written
@ -126,14 +190,19 @@ std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string &
"The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'",
local_path,
res.first->second.path,
remote_path.parent_path().string());
rel_path.parent_path().string());
});
}
runner.waitForAllToFinishAndRethrowFirstError();
InMemoryDirectoryPathMap::FileNames unique_filenames;
LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, map.size());
loadDirectoryTree(map, unique_filenames, object_storage);
{
SharedLockGuard lock(result->mutex);
LOG_DEBUG(log, "Loaded metadata for {} files, found {} directories", num_files, result->map.size());
std::lock_guard lock(result->mutex);
result->map = std::move(map);
result->unique_filenames = std::move(unique_filenames);
auto metric = object_storage->getMetadataStorageMetrics().directory_map_size;
CurrentMetrics::add(metric, result->map.size());
@ -141,55 +210,6 @@ std::shared_ptr<InMemoryDirectoryPathMap> loadPathPrefixMap(const std::string &
return result;
}
void getDirectChildrenOnDiskImpl(
const std::string & storage_key,
const RelativePathsWithMetadata & remote_paths,
const std::string & local_path,
const InMemoryDirectoryPathMap & path_map,
std::unordered_set<std::string> & result)
{
/// Directories are retrieved from the in-memory path map.
{
SharedLockGuard lock(path_map.mutex);
const auto & local_path_prefixes = path_map.map;
const auto end_it = local_path_prefixes.end();
for (auto it = local_path_prefixes.lower_bound(local_path); it != end_it; ++it)
{
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;
result.emplace(std::string(k.begin() + local_path.size(), k.end()) + "/");
}
}
/// Files.
auto skip_list = std::set<std::string>{PREFIX_PATH_FILE_NAME};
for (const auto & elem : remote_paths)
{
const auto & path = elem->relative_path;
chassert(path.find(storage_key) == 0);
const auto child_pos = storage_key.size();
auto slash_pos = path.find('/', child_pos);
if (slash_pos == std::string::npos)
{
/// File names.
auto filename = path.substr(child_pos);
if (!skip_list.contains(filename))
result.emplace(std::move(filename));
}
}
}
}
MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewritableObjectStorage(
@ -215,6 +235,9 @@ MetadataStorageFromPlainRewritableObjectStorage::MetadataStorageFromPlainRewrita
auto keys_gen = std::make_shared<CommonPathPrefixKeyGenerator>(object_storage->getCommonKeyPrefix(), path_map);
object_storage->setKeysGenerator(keys_gen);
}
auto metric = object_storage->getMetadataStorageMetrics().unique_filenames_count;
CurrentMetrics::add(metric, path_map->unique_filenames.size());
}
MetadataStorageFromPlainRewritableObjectStorage::~MetadataStorageFromPlainRewritableObjectStorage()
@ -246,26 +269,8 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str
std::vector<std::string> MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const
{
auto key_prefix = object_storage->generateObjectKeyForPath(path, "" /* key_prefix */).serialize();
RelativePathsWithMetadata files;
auto abs_key = std::filesystem::path(object_storage->getCommonKeyPrefix()) / key_prefix / "";
object_storage->listObjects(abs_key, files, 0);
std::unordered_set<std::string> directories;
getDirectChildrenOnDisk(abs_key, 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 (useSeparateLayoutForMetadata())
{
auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix / "";
RelativePathsWithMetadata metadata_files;
object_storage->listObjects(metadata_key, metadata_files, 0);
getDirectChildrenOnDisk(metadata_key, metadata_files, std::filesystem::path(path) / "", directories);
}
return std::vector<std::string>(std::make_move_iterator(directories.begin()), std::make_move_iterator(directories.end()));
std::unordered_set<std::string> result = getDirectChildrenOnDisk(std::filesystem::path(path) / "");
return std::vector<std::string>(std::make_move_iterator(result.begin()), std::make_move_iterator(result.end()));
}
std::optional<Poco::Timestamp> MetadataStorageFromPlainRewritableObjectStorage::getLastModifiedIfExists(const String & path) const
@ -280,13 +285,41 @@ std::optional<Poco::Timestamp> MetadataStorageFromPlainRewritableObjectStorage::
return std::nullopt;
}
void MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(
const std::string & storage_key,
const RelativePathsWithMetadata & remote_paths,
const std::string & local_path,
std::unordered_set<std::string> & result) const
std::unordered_set<std::string>
MetadataStorageFromPlainRewritableObjectStorage::getDirectChildrenOnDisk(const std::filesystem::path & local_path) const
{
getDirectChildrenOnDiskImpl(storage_key, remote_paths, local_path, *getPathMap(), result);
std::unordered_set<std::string> result;
SharedLockGuard lock(path_map->mutex);
const auto end_it = path_map->map.end();
/// Directories.
for (auto it = path_map->map.lower_bound(local_path); it != end_it; ++it)
{
const auto & subdirectory = it->first.string();
if (!subdirectory.starts_with(local_path.string()))
break;
auto slash_num = count(subdirectory.begin() + local_path.string().size(), subdirectory.end(), '/');
/// The directory map 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 to the right from the offset is greater than 0.
if (slash_num != 0)
break;
result.emplace(std::string(subdirectory.begin() + local_path.string().size(), subdirectory.end()) + "/");
}
/// Files.
auto it = path_map->map.find(local_path.parent_path());
if (it != path_map->map.end())
{
for (const auto & filename_it : it->second.filename_iterators)
{
chassert(filename_it != path_map->unique_filenames.end());
result.insert(*filename_it);
}
}
return result;
}
bool MetadataStorageFromPlainRewritableObjectStorage::useSeparateLayoutForMetadata() const

View File

@ -35,11 +35,7 @@ public:
protected:
std::string getMetadataKeyPrefix() const override { return metadata_key_prefix; }
std::shared_ptr<InMemoryDirectoryPathMap> getPathMap() const override { return path_map; }
void getDirectChildrenOnDisk(
const std::string & storage_key,
const RelativePathsWithMetadata & remote_paths,
const std::string & local_path,
std::unordered_set<std::string> & result) const;
std::unordered_set<std::string> getDirectChildrenOnDisk(const std::filesystem::path & local_path) const;
private:
bool useSeparateLayoutForMetadata() const;

View File

@ -13,6 +13,8 @@ struct MetadataStorageMetrics
const ProfileEvents::Event directory_removed = ProfileEvents::end();
CurrentMetrics::Metric directory_map_size = CurrentMetrics::end();
CurrentMetrics::Metric unique_filenames_count = CurrentMetrics::end();
CurrentMetrics::Metric file_count = CurrentMetrics::end();
template <typename ObjectStorage, MetadataStorageType metadata_type>
static MetadataStorageMetrics create()

View File

@ -24,8 +24,14 @@ extern const Event DiskPlainRewritableS3DirectoryRemoved;
namespace CurrentMetrics
{
extern const Metric DiskPlainRewritableAzureDirectoryMapSize;
extern const Metric DiskPlainRewritableAzureUniqueFileNamesCount;
extern const Metric DiskPlainRewritableAzureFileCount;
extern const Metric DiskPlainRewritableLocalDirectoryMapSize;
extern const Metric DiskPlainRewritableLocalUniqueFileNamesCount;
extern const Metric DiskPlainRewritableLocalFileCount;
extern const Metric DiskPlainRewritableS3DirectoryMapSize;
extern const Metric DiskPlainRewritableS3UniqueFileNamesCount;
extern const Metric DiskPlainRewritableS3FileCount;
}
namespace DB
@ -38,7 +44,9 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<S3ObjectStorage, Me
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableS3DirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableS3DirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableS3DirectoryMapSize};
.directory_map_size = CurrentMetrics::DiskPlainRewritableS3DirectoryMapSize,
.unique_filenames_count = CurrentMetrics::DiskPlainRewritableS3UniqueFileNamesCount,
.file_count = CurrentMetrics::DiskPlainRewritableS3FileCount};
}
#endif
@ -49,7 +57,9 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<AzureObjectStorage,
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableAzureDirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableAzureDirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableAzureDirectoryMapSize};
.directory_map_size = CurrentMetrics::DiskPlainRewritableAzureDirectoryMapSize,
.unique_filenames_count = CurrentMetrics::DiskPlainRewritableAzureUniqueFileNamesCount,
.file_count = CurrentMetrics::DiskPlainRewritableAzureFileCount};
}
#endif
@ -59,7 +69,9 @@ inline MetadataStorageMetrics MetadataStorageMetrics::create<LocalObjectStorage,
return MetadataStorageMetrics{
.directory_created = ProfileEvents::DiskPlainRewritableLocalDirectoryCreated,
.directory_removed = ProfileEvents::DiskPlainRewritableLocalDirectoryRemoved,
.directory_map_size = CurrentMetrics::DiskPlainRewritableLocalDirectoryMapSize};
.directory_map_size = CurrentMetrics::DiskPlainRewritableLocalDirectoryMapSize,
.unique_filenames_count = CurrentMetrics::DiskPlainRewritableLocalUniqueFileNamesCount,
.file_count = CurrentMetrics::DiskPlainRewritableLocalFileCount};
}
}