Add prefix_path parameter to object key generator

This commit is contained in:
Julia Kartseva 2024-06-28 00:58:20 +00:00
parent d6da86dad2
commit 98ad45ba96
22 changed files with 74 additions and 51 deletions

View File

@ -3,6 +3,7 @@
#include <Common/getRandomASCIIString.h>
#include <Common/MatchGenerator.h>
#include <optional>
#include <fmt/format.h>
@ -14,7 +15,10 @@ public:
, re_gen(key_template)
{
}
DB::ObjectStorageKey generate(const String &, bool) const override { return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate()); }
DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional<String> & /* key_prefix */) const override
{
return DB::ObjectStorageKey::createAsAbsolute(re_gen.generate());
}
private:
String key_template;
@ -29,7 +33,7 @@ public:
: key_prefix(std::move(key_prefix_))
{}
DB::ObjectStorageKey generate(const String &, bool) const override
DB::ObjectStorageKey generate(const String &, bool /* is_directory */, const std::optional<String> & /* key_prefix */) const override
{
/// Path to store the new S3 object.
@ -60,7 +64,8 @@ public:
: key_prefix(std::move(key_prefix_))
{}
DB::ObjectStorageKey generate(const String & path, bool) const override
DB::ObjectStorageKey
generate(const String & path, bool /* is_directory */, const std::optional<String> & /* key_prefix */) const override
{
return DB::ObjectStorageKey::createAsRelative(key_prefix, path);
}

View File

@ -11,7 +11,8 @@ class IObjectStorageKeysGenerator
public:
virtual ~IObjectStorageKeysGenerator() = default;
virtual ObjectStorageKey generate(const String & path, bool is_directory) const = 0;
/// Generates an object storage key based on a path in the virtual filesystem.
virtual ObjectStorageKey generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const = 0;
};
using ObjectStorageKeysGeneratorPtr = std::shared_ptr<IObjectStorageKeysGenerator>;

View File

@ -1,3 +1,4 @@
#include <optional>
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
#include "Common/Exception.h"
@ -117,7 +118,8 @@ AzureObjectStorage::AzureObjectStorage(
{
}
ObjectStorageKey AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const
ObjectStorageKey
AzureObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional<std::string> & /* key_prefix */) const
{
return ObjectStorageKey::createAsRelative(getRandomASCIIString(32));
}

View File

@ -101,7 +101,7 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
bool isRemote() const override { return true; }

View File

@ -34,14 +34,16 @@ FileCache::Key CachedObjectStorage::getCacheKey(const std::string & path) const
return cache->createKeyForPath(path);
}
ObjectStorageKey CachedObjectStorage::generateObjectKeyForPath(const std::string & path) const
ObjectStorageKey
CachedObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const
{
return object_storage->generateObjectKeyForPath(path);
return object_storage->generateObjectKeyForPath(path, key_prefix);
}
ObjectStorageKey CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const
ObjectStorageKey
CachedObjectStorage::generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional<std::string> & key_prefix) const
{
return object_storage->generateObjectKeyPrefixForDirectoryPath(path);
return object_storage->generateObjectKeyPrefixForDirectoryPath(path, key_prefix);
}
ReadSettings CachedObjectStorage::patchSettings(const ReadSettings & read_settings) const

View File

@ -98,9 +98,10 @@ public:
const std::string & getCacheName() const override { return cache_config_name; }
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override;
ObjectStorageKey
generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { object_storage->setKeysGenerator(gen); }

View File

@ -15,7 +15,8 @@ CommonPathPrefixKeyGenerator::CommonPathPrefixKeyGenerator(
{
}
ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, bool is_directory) const
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);

View File

@ -5,6 +5,7 @@
#include <filesystem>
#include <map>
#include <optional>
namespace DB
{
@ -26,7 +27,7 @@ public:
explicit CommonPathPrefixKeyGenerator(String key_prefix_, SharedMutex & shared_mutex_, std::weak_ptr<PathMap> path_map_);
ObjectStorageKey generate(const String & path, bool is_directory) const override;
ObjectStorageKey generate(const String & path, bool is_directory, const std::optional<String> & key_prefix) const override;
private:
/// Longest key prefix and unresolved parts of the source path.

View File

@ -537,7 +537,7 @@ struct CopyFileObjectStorageOperation final : public IDiskObjectStorageOperation
for (const auto & object_from : source_blobs)
{
auto object_key = destination_object_storage.generateObjectKeyForPath(to_path);
auto object_key = destination_object_storage.generateObjectKeyForPath(to_path, std::nullopt /* key_prefix */);
auto object_to = StoredObject(object_key.serialize());
object_storage.copyObjectToAnotherObjectStorage(object_from, object_to,read_settings,write_settings, destination_object_storage);
@ -738,7 +738,7 @@ std::unique_ptr<WriteBufferFromFileBase> DiskObjectStorageTransaction::writeFile
const WriteSettings & settings,
bool autocommit)
{
auto object_key = object_storage.generateObjectKeyForPath(path);
auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
std::optional<ObjectAttributes> object_attributes;
if (metadata_helper)
@ -835,7 +835,7 @@ void DiskObjectStorageTransaction::writeFileUsingBlobWritingFunction(
const String & path, WriteMode mode, WriteBlobFunction && write_blob_function)
{
/// This function is a simplified and adapted version of DiskObjectStorageTransaction::writeFile().
auto object_key = object_storage.generateObjectKeyForPath(path);
auto object_key = object_storage.generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
std::optional<ObjectAttributes> object_attributes;
if (metadata_helper)

View File

@ -4,8 +4,9 @@
#include <Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.h>
#include <Storages/ObjectStorage/HDFS/HDFSCommon.h>
#include <Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h>
#include <optional>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.h>
#include <Common/getRandomASCIIString.h>
#include <Common/logger_useful.h>
@ -53,7 +54,8 @@ std::string HDFSObjectStorage::extractObjectKeyFromURL(const StoredObject & obje
return path;
}
ObjectStorageKey HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const
ObjectStorageKey
HDFSObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional<std::string> & /* key_prefix */) const
{
initializeHDFSFS();
/// what ever data_source_description.description value is, consider that key as relative key

View File

@ -111,7 +111,7 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
bool isRemote() const override { return true; }

View File

@ -232,10 +232,11 @@ public:
/// Generate blob name for passed absolute local path.
/// Path can be generated either independently or based on `path`.
virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path) const = 0;
virtual ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const = 0;
/// Object key prefix for local paths in the directory 'path'.
virtual ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */) const
virtual ObjectStorageKey
generateObjectKeyPrefixForDirectoryPath(const std::string & /* path */, const std::optional<std::string> & /* key_prefix */) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'generateObjectKeyPrefixForDirectoryPath' is not implemented");
}

View File

@ -1,15 +1,16 @@
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#include <Interpreters/Context.h>
#include <Common/filesystemHelpers.h>
#include <Common/logger_useful.h>
#include <filesystem>
#include <optional>
#include <Disks/IO/AsynchronousBoundedReadBuffer.h>
#include <Disks/IO/ReadBufferFromRemoteFSGather.h>
#include <Disks/IO/createReadBufferFromFileBase.h>
#include <Disks/IO/AsynchronousBoundedReadBuffer.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/copyData.h>
#include <Interpreters/Context.h>
#include <Common/filesystemHelpers.h>
#include <Common/getRandomASCIIString.h>
#include <filesystem>
#include <Common/logger_useful.h>
namespace fs = std::filesystem;
@ -222,7 +223,8 @@ std::unique_ptr<IObjectStorage> LocalObjectStorage::cloneObjectStorage(
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "cloneObjectStorage() is not implemented for LocalObjectStorage");
}
ObjectStorageKey LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */) const
ObjectStorageKey
LocalObjectStorage::generateObjectKeyForPath(const std::string & /* path */, const std::optional<std::string> & /* key_prefix */) const
{
constexpr size_t key_name_total_size = 32;
return ObjectStorageKey::createAsRelative(key_prefix, getRandomASCIIString(key_name_total_size));

View File

@ -81,7 +81,7 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
bool isRemote() const override { return false; }

View File

@ -42,7 +42,7 @@ bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) con
{
/// NOTE: exists() cannot be used here since it works only for existing
/// key, and does not work for some intermediate path.
auto object_key = object_storage->generateObjectKeyForPath(path);
auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
return object_storage->existsOrHasAnyChild(object_key.serialize());
}
@ -54,7 +54,7 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con
bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const
{
auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize();
auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize();
auto directory = std::filesystem::path(std::move(key_prefix)) / "";
return object_storage->existsOrHasAnyChild(directory);
@ -62,7 +62,7 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path
uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const
{
auto object_key = object_storage->generateObjectKeyForPath(path);
auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
auto metadata = object_storage->tryGetObjectMetadata(object_key.serialize());
if (metadata)
return metadata->size_bytes;
@ -71,7 +71,7 @@ uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path)
std::vector<std::string> MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const
{
auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize();
auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize();
RelativePathsWithMetadata files;
std::string abs_key = key_prefix;
@ -98,7 +98,7 @@ DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(con
StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std::string & path) const
{
size_t object_size = getFileSize(path);
auto object_key = object_storage->generateObjectKeyForPath(path);
auto object_key = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
return {StoredObject(object_key.serialize(), path, object_size)};
}
@ -130,7 +130,7 @@ const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getSt
void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path)
{
auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path);
auto object_key = metadata_storage.object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */);
auto object = StoredObject(object_key.serialize());
metadata_storage.object_storage->removeObject(object);
}
@ -155,7 +155,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std
return;
auto normalized_path = normalizeDirectoryPath(path);
auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path).serialize();
auto key_prefix = object_storage->generateObjectKeyPrefixForDirectoryPath(normalized_path, std::nullopt /* key_prefix */).serialize();
chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix()));
auto op = std::make_unique<MetadataStorageFromPlainObjectStorageCreateDirectoryOperation>(
std::move(normalized_path),

View File

@ -215,7 +215,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::exists(const std::string &
if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix())
{
auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize();
auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize();
chassert(key_prefix.starts_with(object_storage->getCommonKeyPrefix()));
auto metadata_key = std::filesystem::path(getMetadataKeyPrefix()) / key_prefix.substr(object_storage->getCommonKeyPrefix().size());
return object_storage->existsOrHasAnyChild(metadata_key);
@ -228,7 +228,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str
{
if (getMetadataKeyPrefix() != object_storage->getCommonKeyPrefix())
{
auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path).serialize()) / "";
auto directory = std::filesystem::path(object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize()) / "";
chassert(directory.string().starts_with(object_storage->getCommonKeyPrefix()));
auto metadata_key
= std::filesystem::path(getMetadataKeyPrefix()) / directory.string().substr(object_storage->getCommonKeyPrefix().size());
@ -240,7 +240,7 @@ bool MetadataStorageFromPlainRewritableObjectStorage::isDirectory(const std::str
std::vector<std::string> MetadataStorageFromPlainRewritableObjectStorage::listDirectory(const std::string & path) const
{
auto key_prefix = object_storage->generateObjectKeyForPath(path).serialize();
auto key_prefix = object_storage->generateObjectKeyForPath(path, std::nullopt /* key_prefix */).serialize();
RelativePathsWithMetadata files;
std::string abs_key = key_prefix;

View File

@ -26,7 +26,7 @@ public:
bool isPlain() const override { return true; }
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & /* key_prefix */) const override
{
return ObjectStorageKey::createAsRelative(BaseObjectStorage::getCommonKeyPrefix(), path);
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <optional>
#include <string>
#include <Disks/ObjectStorages/IObjectStorage.h>
#include <Common/ObjectStorageKeyGenerator.h>
#include "CommonPathPrefixKeyGenerator.h"
@ -33,9 +35,10 @@ public:
bool isPlain() const override { return true; }
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
ObjectStorageKey generateObjectKeyPrefixForDirectoryPath(const std::string & path) const override;
ObjectStorageKey
generateObjectKeyPrefixForDirectoryPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
void setKeysGenerator(ObjectStorageKeysGeneratorPtr gen) override { key_generator = gen; }
@ -46,20 +49,22 @@ private:
template <typename BaseObjectStorage>
ObjectStorageKey PlainRewritableObjectStorage<BaseObjectStorage>::generateObjectKeyForPath(const std::string & path) const
ObjectStorageKey PlainRewritableObjectStorage<BaseObjectStorage>::generateObjectKeyForPath(
const std::string & path, const std::optional<std::string> & key_prefix) const
{
if (!key_generator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set");
return key_generator->generate(path, /* is_directory */ false);
return key_generator->generate(path, /* is_directory */ false, key_prefix);
}
template <typename BaseObjectStorage>
ObjectStorageKey PlainRewritableObjectStorage<BaseObjectStorage>::generateObjectKeyPrefixForDirectoryPath(const std::string & path) const
ObjectStorageKey PlainRewritableObjectStorage<BaseObjectStorage>::generateObjectKeyPrefixForDirectoryPath(
const std::string & path, const std::optional<std::string> & key_prefix) const
{
if (!key_generator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set");
return key_generator->generate(path, /* is_directory */ true);
return key_generator->generate(path, /* is_directory */ true, key_prefix);
}
}

View File

@ -79,7 +79,7 @@ bool checkBatchRemove(S3ObjectStorage & storage)
/// We are using generateObjectKeyForPath() which returns random object key.
/// That generated key is placed in a right directory where we should have write access.
const String path = fmt::format("clickhouse_remove_objects_capability_{}", getServerUUID());
const auto key = storage.generateObjectKeyForPath(path);
const auto key = storage.generateObjectKeyForPath(path, {} /* key_prefix */);
StoredObject object(key.serialize(), path);
try
{

View File

@ -624,12 +624,12 @@ std::unique_ptr<IObjectStorage> S3ObjectStorage::cloneObjectStorage(
std::move(new_client), std::move(new_s3_settings), new_uri, s3_capabilities, key_generator, disk_name);
}
ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path) const
ObjectStorageKey S3ObjectStorage::generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const
{
if (!key_generator)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Key generator is not set");
return key_generator->generate(path, /* is_directory */ false);
return key_generator->generate(path, /* is_directory */ false, key_prefix);
}
std::shared_ptr<const S3::Client> S3ObjectStorage::getS3StorageClient()

View File

@ -164,7 +164,7 @@ public:
bool supportParallelWrite() const override { return true; }
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & key_prefix) const override;
bool isReadOnly() const override { return s3_settings.get()->read_only; }

View File

@ -82,7 +82,7 @@ public:
const std::string & config_prefix,
ContextPtr context) override;
ObjectStorageKey generateObjectKeyForPath(const std::string & path) const override
ObjectStorageKey generateObjectKeyForPath(const std::string & path, const std::optional<std::string> & /* key_prefix */) const override
{
return ObjectStorageKey::createAsRelative(path);
}