Replace old named collections code for mongo

This commit is contained in:
kssenii 2022-12-17 00:49:07 +01:00
parent 0c63ce9731
commit 2ce5af421e
5 changed files with 42 additions and 25 deletions

View File

@ -46,12 +46,6 @@ struct StorageMySQLConfiguration : ExternalDataSourceConfiguration
String on_duplicate_clause;
};
struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration
{
String options;
};
using StorageSpecificArgs = std::vector<std::pair<String, ASTPtr>>;
struct ExternalDataSourceInfo

View File

@ -2,6 +2,7 @@
#include <Storages/StorageMongoDBSocketFactory.h>
#include <Storages/StorageFactory.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <Poco/MongoDB/Connection.h>
#include <Poco/MongoDB/Cursor.h>
@ -11,6 +12,7 @@
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Common/parseAddress.h>
#include <Common/NamedCollections/NamedCollections.h>
#include <IO/Operators.h>
#include <Parsers/ASTLiteral.h>
#include <QueryPipeline/Pipe.h>
@ -27,6 +29,19 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
static const std::unordered_set<std::string_view> required_configuration_keys = {
"host",
"port",
"username",
"passowrd",
"database",
"collection",
"table",
};
static const std::unordered_set<std::string_view> optional_configuration_keys = {
"options",
};
StorageMongoDB::StorageMongoDB(
const StorageID & table_id_,
const std::string & host_,
@ -172,24 +187,21 @@ SinkToStoragePtr StorageMongoDB::write(const ASTPtr & /* query */, const Storage
return std::make_shared<StorageMongoDBSink>(collection_name, database_name, metadata_snapshot, connection);
}
StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context)
StorageMongoDB::Configuration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context)
{
StorageMongoDBConfiguration configuration;
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context))
{
auto [common_configuration, storage_specific_args, _] = named_collection.value();
configuration.set(common_configuration);
Configuration configuration;
for (const auto & [arg_name, arg_value] : storage_specific_args)
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args))
{
if (arg_name == "options")
configuration.options = checkAndGetLiteralArgument<String>(arg_value, "options");
else
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unexpected key-value argument."
"Got: {}, but expected one of:"
"host, port, username, password, database, table, options.", arg_name);
}
validateNamedCollection(*named_collection, required_configuration_keys, optional_configuration_keys);
configuration.host = named_collection->get<String>("host");
configuration.port = static_cast<UInt16>(named_collection->get<UInt64>("port"));
configuration.username = named_collection->get<String>("username");
configuration.password = named_collection->get<String>("password");
configuration.database = named_collection->get<String>("database");
configuration.table = named_collection->getOrDefault<String>("collection", named_collection->get<String>("table"));
configuration.options = named_collection->getOrDefault<String>("options", "");
}
else
{

View File

@ -44,7 +44,18 @@ public:
const StorageMetadataPtr & /*metadata_snapshot*/,
ContextPtr context) override;
static StorageMongoDBConfiguration getConfiguration(ASTs engine_args, ContextPtr context);
struct Configuration
{
std::string host;
UInt16 port;
std::string username;
std::string password;
std::string database;
std::string table;
std::string options;
};
static Configuration getConfiguration(ASTs engine_args, ContextPtr context);
private:
void connectIfNotConnected();

View File

@ -80,7 +80,7 @@ static const String PARTITION_ID_WILDCARD = "{_partition_id}";
static const std::unordered_set<std::string_view> required_configuration_keys = {
"url",
};
static std::unordered_set<std::string_view> optional_configuration_keys = {
static const std::unordered_set<std::string_view> optional_configuration_keys = {
"format",
"compression",
"compression_method",

View File

@ -24,7 +24,7 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
std::optional<StorageMongoDBConfiguration> configuration;
std::optional<StorageMongoDB::Configuration> configuration;
String structure;
};