mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 09:02:00 +00:00
improvements, cleanups, comments
This commit is contained in:
parent
8c99b0d5eb
commit
01ee500b06
@ -20,12 +20,16 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo
|
||||
const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path);
|
||||
|
||||
auto key = std::filesystem::path(object_key_prefix.empty() ? storage_key_prefix : object_key_prefix);
|
||||
|
||||
/// The longest prefix is the same as path, meaning that the path is already mapped.
|
||||
if (suffix_parts.empty())
|
||||
return ObjectStorageKey::createAsRelative(std::move(key));
|
||||
|
||||
/// File and top-level directory paths are mapped as is.
|
||||
if (!is_directory || object_key_prefix.empty())
|
||||
for (const auto & part : suffix_parts)
|
||||
key /= part;
|
||||
/// Replace the last part of the directory path with a pseudorandom suffix.
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i + 1 < suffix_parts.size(); ++i)
|
||||
|
@ -8,6 +8,15 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Object storage key generator used specifically with the
|
||||
/// MetadataStorageFromPlainObjectStorage if multiple writes are allowed.
|
||||
|
||||
/// It searches for the local (metadata) path in a pre-loaded path map.
|
||||
/// If no such path exists, it searches for the parent path, until it is found
|
||||
/// or no parent path exists.
|
||||
///
|
||||
/// The key generator ensures that the original directory hierarchy is
|
||||
/// preserved, which is required for the MergeTree family.
|
||||
class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator
|
||||
{
|
||||
public:
|
||||
@ -18,6 +27,7 @@ public:
|
||||
ObjectStorageKey generate(const String & path, bool is_directory) const override;
|
||||
|
||||
private:
|
||||
/// Longest key prefix and unresolved parts of the source path.
|
||||
std::tuple<std::string, std::vector<String>> getLongestObjectKeyPrefix(const String & path) const;
|
||||
|
||||
const String storage_key_prefix;
|
||||
|
@ -1,10 +1,13 @@
|
||||
#pragma once
|
||||
|
||||
#include <Disks/ObjectStorages/IMetadataOperation.h>
|
||||
|
||||
/// TODO: rename to MetadataStorageTransactionState.
|
||||
#include <Disks/ObjectStorages/MetadataFromDiskTransactionState.h>
|
||||
|
||||
/**
|
||||
* Implementations for transactional operations with metadata used by MetadataStorageFromDisk.
|
||||
* Implementations for transactional operations with metadata used by MetadataStorageFromDisk
|
||||
* and MetadataStorageFromPlainObjectStorage.
|
||||
*/
|
||||
|
||||
namespace DB
|
||||
|
@ -46,11 +46,23 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri
|
||||
|
||||
StoredObject object{file.relative_path};
|
||||
|
||||
auto read_buf = object_storage->readObject(object, {});
|
||||
String content;
|
||||
readString(content, *read_buf);
|
||||
auto read_buf = object_storage->readObject(object);
|
||||
String local_path;
|
||||
readStringUntilEOF(local_path, *read_buf);
|
||||
|
||||
result.emplace(content, remote_path.parent_path().string());
|
||||
chassert(remote_path.has_parent_path());
|
||||
auto res = result.emplace(local_path, remote_path.parent_path());
|
||||
|
||||
/// This can happen if table replication is enabled, then the same local path is written
|
||||
/// in `prefix.path` of each replica.
|
||||
/// TODO: should replicated tables (e.g., RMT) be explicitly disallowed?
|
||||
if (!res.second)
|
||||
LOG_WARNING(
|
||||
getLogger("MetadataStorageFromPlainObjectStorage"),
|
||||
"The local path '{}' is already mapped to a remote path '{}', ignoring: '{}'",
|
||||
local_path,
|
||||
res.first->second,
|
||||
remote_path.parent_path().string());
|
||||
}
|
||||
return result;
|
||||
}
|
||||
@ -62,9 +74,6 @@ MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage(Obj
|
||||
, storage_path_prefix(std::move(storage_path_prefix_))
|
||||
, path_map(std::make_shared<PathMap>(loadPathPrefixMap(object_storage->getCommonKeyPrefix(), object_storage)))
|
||||
{
|
||||
LOG_TRACE(
|
||||
getLogger("MetadataStorageFromPlainObjectStorage"), "MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage");
|
||||
|
||||
if (!object_storage->isWriteOnce())
|
||||
{
|
||||
auto keys_gen = std::make_shared<CommonPathPrefixKeyGenerator>(object_storage->getCommonKeyPrefix(), metadata_mutex, path_map);
|
||||
|
@ -46,7 +46,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
|
||||
object,
|
||||
WriteMode::Rewrite,
|
||||
/* object_attributes */ std::nullopt,
|
||||
/* buf_size */ 4096,
|
||||
/* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
|
||||
/* settings */ {});
|
||||
|
||||
write_created = true;
|
||||
@ -97,7 +97,7 @@ std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMo
|
||||
if (validate_content)
|
||||
{
|
||||
std::string data;
|
||||
auto readBuf = object_storage->readObject(object, {});
|
||||
auto readBuf = object_storage->readObject(object);
|
||||
readStringUntilEOF(data, *readBuf);
|
||||
if (data != path_from)
|
||||
throw Exception(
|
||||
@ -111,8 +111,8 @@ std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMo
|
||||
auto write_buf = object_storage->writeObject(
|
||||
object,
|
||||
WriteMode::Rewrite,
|
||||
std::nullopt,
|
||||
/*buf_size*/ 4096,
|
||||
/* object_attributes */ std::nullopt,
|
||||
/*buf_size*/ DBMS_DEFAULT_BUFFER_SIZE,
|
||||
/*settings*/ {});
|
||||
|
||||
return write_buf;
|
||||
@ -177,7 +177,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo()
|
||||
object,
|
||||
WriteMode::Rewrite,
|
||||
/* object_attributes */ std::nullopt,
|
||||
/* buf_size */ 4096,
|
||||
/* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
|
||||
/* settings */ {});
|
||||
writeString(path.string(), *buf);
|
||||
buf->finalize();
|
||||
|
@ -222,11 +222,10 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const ContextPtr & context,
|
||||
bool /* skip_access_check */) -> ObjectStoragePtr
|
||||
bool skip_access_check) -> ObjectStoragePtr
|
||||
{
|
||||
/// send_metadata changes the filenames (includes revision), while
|
||||
/// s3_plain do not care about this, and expect that the file name
|
||||
/// will not be changed.
|
||||
/// s3_plain_rewritable does not support file renaming.
|
||||
if (config.getBool(config_prefix + ".send_metadata", false))
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain_rewritable does not supports send_metadata");
|
||||
|
||||
@ -239,6 +238,10 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
|
||||
auto object_storage = std::make_shared<S3PlainRewritableObjectStorage>(
|
||||
std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
|
||||
|
||||
/// NOTE: should we still perform this check for clickhouse-disks?
|
||||
if (!skip_access_check)
|
||||
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name);
|
||||
|
||||
return object_storage;
|
||||
});
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user