mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-28 02:21:59 +00:00
Replace old named collections code for mongo
This commit is contained in:
parent
0c63ce9731
commit
2ce5af421e
@ -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
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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();
|
||||
|
@ -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",
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user