improvements, cleanups, comments

This commit is contained in:
Julia Kartseva 2024-04-08 23:11:18 +00:00
parent 8c99b0d5eb
commit 01ee500b06
6 changed files with 45 additions and 16 deletions

View File

@ -20,12 +20,16 @@ ObjectStorageKey CommonPathPrefixKeyGenerator::generate(const String & path, boo
const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path); const auto & [object_key_prefix, suffix_parts] = getLongestObjectKeyPrefix(path);
auto key = std::filesystem::path(object_key_prefix.empty() ? storage_key_prefix : object_key_prefix); 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()) if (suffix_parts.empty())
return ObjectStorageKey::createAsRelative(std::move(key)); return ObjectStorageKey::createAsRelative(std::move(key));
/// File and top-level directory paths are mapped as is.
if (!is_directory || object_key_prefix.empty()) if (!is_directory || object_key_prefix.empty())
for (const auto & part : suffix_parts) for (const auto & part : suffix_parts)
key /= part; key /= part;
/// Replace the last part of the directory path with a pseudorandom suffix.
else else
{ {
for (size_t i = 0; i + 1 < suffix_parts.size(); ++i) for (size_t i = 0; i + 1 < suffix_parts.size(); ++i)

View File

@ -8,6 +8,15 @@
namespace DB 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 class CommonPathPrefixKeyGenerator : public IObjectStorageKeysGenerator
{ {
public: public:
@ -18,6 +27,7 @@ public:
ObjectStorageKey generate(const String & path, bool is_directory) const override; ObjectStorageKey generate(const String & path, bool is_directory) const override;
private: private:
/// Longest key prefix and unresolved parts of the source path.
std::tuple<std::string, std::vector<String>> getLongestObjectKeyPrefix(const String & path) const; std::tuple<std::string, std::vector<String>> getLongestObjectKeyPrefix(const String & path) const;
const String storage_key_prefix; const String storage_key_prefix;

View File

@ -1,10 +1,13 @@
#pragma once #pragma once
#include <Disks/ObjectStorages/IMetadataOperation.h> #include <Disks/ObjectStorages/IMetadataOperation.h>
/// TODO: rename to MetadataStorageTransactionState.
#include <Disks/ObjectStorages/MetadataFromDiskTransactionState.h> #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 namespace DB

View File

@ -46,11 +46,23 @@ MetadataStorageFromPlainObjectStorage::PathMap loadPathPrefixMap(const std::stri
StoredObject object{file.relative_path}; StoredObject object{file.relative_path};
auto read_buf = object_storage->readObject(object, {}); auto read_buf = object_storage->readObject(object);
String content; String local_path;
readString(content, *read_buf); 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; return result;
} }
@ -62,9 +74,6 @@ MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage(Obj
, storage_path_prefix(std::move(storage_path_prefix_)) , storage_path_prefix(std::move(storage_path_prefix_))
, path_map(std::make_shared<PathMap>(loadPathPrefixMap(object_storage->getCommonKeyPrefix(), object_storage))) , path_map(std::make_shared<PathMap>(loadPathPrefixMap(object_storage->getCommonKeyPrefix(), object_storage)))
{ {
LOG_TRACE(
getLogger("MetadataStorageFromPlainObjectStorage"), "MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage");
if (!object_storage->isWriteOnce()) if (!object_storage->isWriteOnce())
{ {
auto keys_gen = std::make_shared<CommonPathPrefixKeyGenerator>(object_storage->getCommonKeyPrefix(), metadata_mutex, path_map); auto keys_gen = std::make_shared<CommonPathPrefixKeyGenerator>(object_storage->getCommonKeyPrefix(), metadata_mutex, path_map);

View File

@ -46,7 +46,7 @@ void MetadataStorageFromPlainObjectStorageCreateDirectoryOperation::execute(std:
object, object,
WriteMode::Rewrite, WriteMode::Rewrite,
/* object_attributes */ std::nullopt, /* object_attributes */ std::nullopt,
/* buf_size */ 4096, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
/* settings */ {}); /* settings */ {});
write_created = true; write_created = true;
@ -97,7 +97,7 @@ std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMo
if (validate_content) if (validate_content)
{ {
std::string data; std::string data;
auto readBuf = object_storage->readObject(object, {}); auto readBuf = object_storage->readObject(object);
readStringUntilEOF(data, *readBuf); readStringUntilEOF(data, *readBuf);
if (data != path_from) if (data != path_from)
throw Exception( throw Exception(
@ -111,8 +111,8 @@ std::unique_ptr<WriteBufferFromFileBase> MetadataStorageFromPlainObjectStorageMo
auto write_buf = object_storage->writeObject( auto write_buf = object_storage->writeObject(
object, object,
WriteMode::Rewrite, WriteMode::Rewrite,
std::nullopt, /* object_attributes */ std::nullopt,
/*buf_size*/ 4096, /*buf_size*/ DBMS_DEFAULT_BUFFER_SIZE,
/*settings*/ {}); /*settings*/ {});
return write_buf; return write_buf;
@ -177,7 +177,7 @@ void MetadataStorageFromPlainObjectStorageRemoveDirectoryOperation::undo()
object, object,
WriteMode::Rewrite, WriteMode::Rewrite,
/* object_attributes */ std::nullopt, /* object_attributes */ std::nullopt,
/* buf_size */ 4096, /* buf_size */ DBMS_DEFAULT_BUFFER_SIZE,
/* settings */ {}); /* settings */ {});
writeString(path.string(), *buf); writeString(path.string(), *buf);
buf->finalize(); buf->finalize();

View File

@ -222,11 +222,10 @@ void registerS3PlainRewritableObjectStorage(ObjectStorageFactory & factory)
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix, const std::string & config_prefix,
const ContextPtr & context, const ContextPtr & context,
bool /* skip_access_check */) -> ObjectStoragePtr bool skip_access_check) -> ObjectStoragePtr
{ {
/// send_metadata changes the filenames (includes revision), while /// send_metadata changes the filenames (includes revision), while
/// s3_plain do not care about this, and expect that the file name /// s3_plain_rewritable does not support file renaming.
/// will not be changed.
if (config.getBool(config_prefix + ".send_metadata", false)) if (config.getBool(config_prefix + ".send_metadata", false))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "s3_plain_rewritable does not supports send_metadata"); 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>( auto object_storage = std::make_shared<S3PlainRewritableObjectStorage>(
std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name); 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; return object_storage;
}); });
} }