Fix usage plain metadata type with new configuration option

This commit is contained in:
kssenii 2024-02-26 14:03:53 +08:00
parent 07c828ece9
commit 277e8d9655
11 changed files with 178 additions and 97 deletions

View File

@ -1,7 +1,27 @@
#include "DiskType.h"
#include <Poco/String.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
}
MetadataStorageType metadataTypeFromString(const String & type)
{
auto check_type = Poco::toLower(type);
if (check_type == "local")
return MetadataStorageType::Local;
if (check_type == "plain")
return MetadataStorageType::Plain;
if (check_type == "web")
return MetadataStorageType::StaticWeb;
throw Exception(ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG,
"MetadataStorageFactory: unknown metadata storage type: {}", type);
}
bool DataSourceDescription::operator==(const DataSourceDescription & other) const
{
@ -14,4 +34,32 @@ bool DataSourceDescription::sameKind(const DataSourceDescription & other) const
== std::tie(other.type, other.object_storage_type, other.description);
}
std::string DataSourceDescription::toString() const
{
switch (type)
{
case DataSourceType::Local:
return "local";
case DataSourceType::RAM:
return "memory";
case DataSourceType::ObjectStorage:
{
switch (object_storage_type)
{
case ObjectStorageType::S3:
return "s3";
case ObjectStorageType::HDFS:
return "hdfs";
case ObjectStorageType::Azure:
return "azure_blob_storage";
case ObjectStorageType::Local:
return "local_blob_storage";
case ObjectStorageType::Web:
return "web";
case ObjectStorageType::None:
return "none";
}
}
}
}
}

View File

@ -17,7 +17,6 @@ enum class ObjectStorageType
{
None,
S3,
S3_Plain,
Azure,
HDFS,
Web,
@ -30,9 +29,9 @@ enum class MetadataStorageType
Local,
Plain,
StaticWeb,
Memory,
};
MetadataStorageType metadataTypeFromString(const String & type);
String toString(DataSourceType data_source_type);
struct DataSourceDescription
@ -49,36 +48,7 @@ struct DataSourceDescription
bool operator==(const DataSourceDescription & other) const;
bool sameKind(const DataSourceDescription & other) const;
std::string toString() const
{
switch (type)
{
case DataSourceType::Local:
return "local";
case DataSourceType::RAM:
return "memory";
case DataSourceType::ObjectStorage:
{
switch (object_storage_type)
{
case ObjectStorageType::S3:
return "s3";
case ObjectStorageType::S3_Plain:
return "s3_plain";
case ObjectStorageType::HDFS:
return "hdfs";
case ObjectStorageType::Azure:
return "azure_blob_storage";
case ObjectStorageType::Local:
return "local_blob_storage";
case ObjectStorageType::Web:
return "web";
case ObjectStorageType::None:
return "none";
}
}
}
}
std::string toString() const;
};
}

View File

@ -218,6 +218,7 @@ public:
virtual bool isReadOnly() const { return false; }
virtual bool isWriteOnce() const { return false; }
virtual bool isPlain() const { return false; }
virtual bool supportParallelWrite() const { return false; }

View File

@ -32,6 +32,35 @@ void MetadataStorageFactory::registerMetadataStorageType(const std::string & met
}
}
std::string MetadataStorageFactory::getCompatibilityMetadataTypeHint(const ObjectStorageType & type)
{
switch (type)
{
case ObjectStorageType::S3:
case ObjectStorageType::HDFS:
case ObjectStorageType::Local:
case ObjectStorageType::Azure:
return "local";
case ObjectStorageType::Web:
return "web";
default:
return "";
}
}
std::string MetadataStorageFactory::getMetadataType(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const std::string & compatibility_type_hint)
{
if (compatibility_type_hint.empty() && !config.has(config_prefix + ".metadata_type"))
{
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Expected `metadata_type` in config");
}
return config.getString(config_prefix + ".metadata_type", compatibility_type_hint);
}
MetadataStoragePtr MetadataStorageFactory::create(
const std::string & name,
const Poco::Util::AbstractConfiguration & config,
@ -39,12 +68,7 @@ MetadataStoragePtr MetadataStorageFactory::create(
ObjectStoragePtr object_storage,
const std::string & compatibility_type_hint) const
{
if (compatibility_type_hint.empty() && !config.has(config_prefix + ".metadata_type"))
{
throw Exception(ErrorCodes::NO_ELEMENTS_IN_CONFIG, "Expected `metadata_type` in config");
}
const auto type = config.getString(config_prefix + ".metadata_type", compatibility_type_hint);
const auto type = getMetadataType(config, config_prefix, compatibility_type_hint);
const auto it = registry.find(type);
if (it == registry.end())

View File

@ -25,6 +25,13 @@ public:
ObjectStoragePtr object_storage,
const std::string & compatibility_type_hint) const;
static std::string getMetadataType(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
const std::string & compatibility_type_hint = "");
static std::string getCompatibilityMetadataTypeHint(const ObjectStorageType & type);
private:
using Registry = std::unordered_map<String, Creator>;
Registry registry;

View File

@ -16,8 +16,10 @@
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
#include <Disks/ObjectStorages/Web/WebObjectStorage.h>
#include <Disks/ObjectStorages/Local/LocalObjectStorage.h>
#include <Disks/ObjectStorages/PlainObjectStorage.h>
#include <Disks/loadLocalDiskConfig.h>
#endif
#include <Disks/ObjectStorages/MetadataStorageFactory.h>
#include <Interpreters/Context.h>
#include <Common/Macros.h>
@ -32,6 +34,28 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
namespace
{
template <typename BaseObjectStorage, class ...Args>
ObjectStoragePtr createObjectStorage(
const Poco::Util::AbstractConfiguration & config,
const std::string & config_prefix,
Args && ...args)
{
auto compatibility_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(ObjectStorageType::S3);
auto metadata_type = MetadataStorageFactory::getMetadataType(config, config_prefix, compatibility_hint);
if (metadataTypeFromString(metadata_type) == MetadataStorageType::Plain)
{
return std::make_shared<PlainObjectStorage<BaseObjectStorage>>(std::forward<Args>(args)...);
}
else
{
return std::make_shared<BaseObjectStorage>(std::forward<Args>(args)...);
}
}
}
ObjectStorageFactory & ObjectStorageFactory::instance()
{
static ObjectStorageFactory factory;
@ -129,12 +153,12 @@ void registerS3ObjectStorage(ObjectStorageFactory & factory)
auto client = getClient(config, config_prefix, context, *settings);
auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix);
auto object_storage = std::make_shared<S3ObjectStorage>(
std::move(client), std::move(settings), uri, s3_capabilities, key_generator, name);
auto object_storage = createObjectStorage<S3ObjectStorage>(
config, config_prefix, 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(*object_storage, s3_capabilities, name, uri.key);
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name, uri.key);
return object_storage;
});
@ -165,12 +189,12 @@ void registerS3PlainObjectStorage(ObjectStorageFactory & factory)
auto client = getClient(config, config_prefix, context, *settings);
auto key_generator = getKeyGenerator(disk_type, uri, config, config_prefix);
auto object_storage = std::make_shared<S3PlainObjectStorage>(
auto object_storage = std::make_shared<PlainObjectStorage<S3ObjectStorage>>(
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(*object_storage, s3_capabilities, name, uri.key);
checkS3Capabilities(*dynamic_cast<S3ObjectStorage *>(object_storage.get()), s3_capabilities, name, uri.key);
return object_storage;
});
@ -198,7 +222,7 @@ void registerHDFSObjectStorage(ObjectStorageFactory & factory)
context->getSettingsRef().hdfs_replication
);
return std::make_unique<HDFSObjectStorage>(uri, std::move(settings), config);
return createObjectStorage<HDFSObjectStorage>(config, config_prefix, uri, std::move(settings), config);
});
}
#endif
@ -214,7 +238,8 @@ void registerAzureObjectStorage(ObjectStorageFactory & factory)
bool /* skip_access_check */) -> ObjectStoragePtr
{
String container_name = config.getString(config_prefix + ".container_name", "default-container");
return std::make_unique<AzureObjectStorage>(
return createObjectStorage<AzureObjectStorage>(
config, config_prefix,
name,
getAzureBlobContainerClient(config, config_prefix),
getAzureBlobStorageSettings(config, config_prefix, context),
@ -248,7 +273,7 @@ void registerWebObjectStorage(ObjectStorageFactory & factory)
ErrorCodes::BAD_ARGUMENTS, "Bad URI: `{}`. Error: {}", uri, e.what());
}
return std::make_shared<WebObjectStorage>(uri, context);
return createObjectStorage<WebObjectStorage>(config, config_prefix, uri, context);
});
}
@ -266,7 +291,7 @@ void registerLocalObjectStorage(ObjectStorageFactory & factory)
loadDiskLocalConfig(name, config, config_prefix, context, object_key_prefix, keep_free_space_bytes);
/// keys are mapped to the fs, object_key_prefix is a directory also
fs::create_directories(object_key_prefix);
return std::make_shared<LocalObjectStorage>(object_key_prefix);
return createObjectStorage<LocalObjectStorage>(config, config_prefix, object_key_prefix);
});
}
#endif

View File

@ -0,0 +1,29 @@
#pragma once
#include <Disks/ObjectStorages/IObjectStorage.h>
namespace DB
{
/// Do not encode keys, store as-is, and do not require separate disk for metadata.
/// But because of this does not support renames/hardlinks/attrs/...
///
/// NOTE: This disk has excessive API calls.
template <typename BaseObjectStorage>
class PlainObjectStorage : public BaseObjectStorage
{
public:
template <class ...Args>
explicit PlainObjectStorage(Args && ...args)
: BaseObjectStorage(std::forward<Args>(args)...) {}
std::string getName() const override { return "" + BaseObjectStorage::getName(); }
/// Notes:
/// - supports BACKUP to this disk
/// - does not support INSERT into MergeTree table on this disk
bool isWriteOnce() const override { return true; }
bool isPlain() const override { return true; }
};
}

View File

@ -10,25 +10,6 @@ namespace DB
void registerObjectStorages();
void registerMetadataStorages();
static std::string getCompatibilityMetadataTypeHint(const ObjectStorageType & type)
{
switch (type)
{
case ObjectStorageType::S3:
case ObjectStorageType::HDFS:
case ObjectStorageType::Local:
case ObjectStorageType::Azure:
return "local";
case ObjectStorageType::S3_Plain:
return "plain";
case ObjectStorageType::Web:
return "web";
case ObjectStorageType::None:
return "";
}
UNREACHABLE();
}
void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_check)
{
registerObjectStorages();
@ -47,7 +28,10 @@ void registerDiskObjectStorage(DiskFactory & factory, bool global_skip_access_ch
std::string compatibility_metadata_type_hint;
if (!config.has(config_prefix + ".metadata_type"))
{
compatibility_metadata_type_hint = getCompatibilityMetadataTypeHint(object_storage->getType());
if (object_storage->isPlain())
compatibility_metadata_type_hint = "plain";
else
compatibility_metadata_type_hint = MetadataStorageFactory::getCompatibilityMetadataTypeHint(object_storage->getType());
}
auto metadata_storage = MetadataStorageFactory::instance().create(

View File

@ -182,27 +182,6 @@ private:
LoggerPtr log;
};
/// Do not encode keys, store as-is, and do not require separate disk for metadata.
/// But because of this does not support renames/hardlinks/attrs/...
///
/// NOTE: This disk has excessive API calls.
class S3PlainObjectStorage : public S3ObjectStorage
{
public:
std::string getName() const override { return "S3PlainObjectStorage"; }
template <class ...Args>
explicit S3PlainObjectStorage(Args && ...args)
: S3ObjectStorage("S3PlainObjectStorage", std::forward<Args>(args)...) {}
ObjectStorageType getType() const override { return ObjectStorageType::S3_Plain; }
/// Notes:
/// - supports BACKUP to this disk
/// - does not support INSERT into MergeTree table on this disk
bool isWriteOnce() const override { return true; }
};
}
#endif

View File

@ -8,9 +8,16 @@
<access_key_id>minio</access_key_id>
<secret_access_key>minio123</secret_access_key>
</backup_disk_s3_plain>
<backup_disk_local_plain>
<type>object_storage</type>
<object_storage_type>local</object_storage_type>
<metadata_type>plain</metadata_type>
<path>local_plain/</path>
</backup_disk_local_plain>
</disks>
</storage_configuration>
<backups>
<allowed_disk>backup_disk_s3_plain</allowed_disk>
<allowed_disk>backup_disk_local_plain</allowed_disk>
</backups>
</clickhouse>

View File

@ -20,17 +20,27 @@ def start_cluster():
finally:
cluster.shutdown()
s3_disk_def = """disk(type=s3_plain,
endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/',
access_key_id='minio',
secret_access_key='minio123');"""
local_disk_def = "disk(type=object_storage, object_storage_type = 'local', metadata_type = 'plain'"
@pytest.mark.parametrize(
"table_name,backup_name,storage_policy,min_bytes_for_wide_part",
"table_name,backup_name,storage_policy,disk_def,min_bytes_for_wide_part",
[
pytest.param(
"compact", "backup_compact", "s3_backup_compact", int(1e9), id="compact"
"compact", "backup_compact_s3", "backup_disk_s3_plain", s3_disk_def, int(1e9), id="compact"
),
pytest.param("wide", "backup_wide", "s3_backup_wide", int(0), id="wide"),
pytest.param("wide", "backup_wide_s3", "backup_disk_s3_plain", s3_disk_def, int(0), id="wide"),
pytest.param(
"compact", "backup_compact_local", "backup_disk_local_plain", local_disk_def, int(1e9), id="compact"
),
pytest.param("wide", "backup_wide_local", "backup_disk_local_plain", local_disk_def, int(0), id="wide"),
],
)
def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide_part):
def test_attach_part(table_name, backup_name, storage_policy, disk_def, min_bytes_for_wide_part):
node.query(
f"""
-- Catch any errors (NOTE: warnings are ok)
@ -45,7 +55,7 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide
settings min_bytes_for_wide_part={min_bytes_for_wide_part}
as select number%5 part, number key from numbers(100);
backup table ordinary_db.{table_name} TO Disk('backup_disk_s3_plain', '{backup_name}') settings deduplicate_files=0;
backup table ordinary_db.{table_name} TO Disk('{storage_policy}', '{backup_name}') settings deduplicate_files=0;
drop table ordinary_db.{table_name};
attach table ordinary_db.{table_name} (part UInt8, key UInt64)
@ -53,10 +63,7 @@ def test_attach_part(table_name, backup_name, storage_policy, min_bytes_for_wide
order by key partition by part
settings
max_suspicious_broken_parts=0,
disk=disk(type=s3_plain,
endpoint='http://minio1:9001/root/data/disks/disk_s3_plain/{backup_name}/',
access_key_id='minio',
secret_access_key='minio123');
disk={disk_def}
"""
)