mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Add prefix_path parameter to object key generator
This commit is contained in:
parent
d6da86dad2
commit
98ad45ba96
@ -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);
|
||||
}
|
||||
|
@ -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>;
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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
|
||||
|
@ -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); }
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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),
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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()
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user