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;
|
String on_duplicate_clause;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct StorageMongoDBConfiguration : ExternalDataSourceConfiguration
|
|
||||||
{
|
|
||||||
String options;
|
|
||||||
};
|
|
||||||
|
|
||||||
|
|
||||||
using StorageSpecificArgs = std::vector<std::pair<String, ASTPtr>>;
|
using StorageSpecificArgs = std::vector<std::pair<String, ASTPtr>>;
|
||||||
|
|
||||||
struct ExternalDataSourceInfo
|
struct ExternalDataSourceInfo
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <Storages/StorageMongoDBSocketFactory.h>
|
#include <Storages/StorageMongoDBSocketFactory.h>
|
||||||
#include <Storages/StorageFactory.h>
|
#include <Storages/StorageFactory.h>
|
||||||
#include <Storages/checkAndGetLiteralArgument.h>
|
#include <Storages/checkAndGetLiteralArgument.h>
|
||||||
|
#include <Storages/NamedCollectionsHelpers.h>
|
||||||
|
|
||||||
#include <Poco/MongoDB/Connection.h>
|
#include <Poco/MongoDB/Connection.h>
|
||||||
#include <Poco/MongoDB/Cursor.h>
|
#include <Poco/MongoDB/Cursor.h>
|
||||||
@ -11,6 +12,7 @@
|
|||||||
#include <Core/Settings.h>
|
#include <Core/Settings.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Common/parseAddress.h>
|
#include <Common/parseAddress.h>
|
||||||
|
#include <Common/NamedCollections/NamedCollections.h>
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <QueryPipeline/Pipe.h>
|
#include <QueryPipeline/Pipe.h>
|
||||||
@ -27,6 +29,19 @@ namespace ErrorCodes
|
|||||||
extern const int BAD_ARGUMENTS;
|
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(
|
StorageMongoDB::StorageMongoDB(
|
||||||
const StorageID & table_id_,
|
const StorageID & table_id_,
|
||||||
const std::string & host_,
|
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);
|
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;
|
Configuration configuration;
|
||||||
if (auto named_collection = getExternalDataSourceConfiguration(engine_args, context))
|
|
||||||
{
|
|
||||||
auto [common_configuration, storage_specific_args, _] = named_collection.value();
|
|
||||||
configuration.set(common_configuration);
|
|
||||||
|
|
||||||
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args))
|
||||||
{
|
{
|
||||||
if (arg_name == "options")
|
validateNamedCollection(*named_collection, required_configuration_keys, optional_configuration_keys);
|
||||||
configuration.options = checkAndGetLiteralArgument<String>(arg_value, "options");
|
|
||||||
else
|
configuration.host = named_collection->get<String>("host");
|
||||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
configuration.port = static_cast<UInt16>(named_collection->get<UInt64>("port"));
|
||||||
"Unexpected key-value argument."
|
configuration.username = named_collection->get<String>("username");
|
||||||
"Got: {}, but expected one of:"
|
configuration.password = named_collection->get<String>("password");
|
||||||
"host, port, username, password, database, table, options.", arg_name);
|
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
|
else
|
||||||
{
|
{
|
||||||
|
@ -44,7 +44,18 @@ public:
|
|||||||
const StorageMetadataPtr & /*metadata_snapshot*/,
|
const StorageMetadataPtr & /*metadata_snapshot*/,
|
||||||
ContextPtr context) override;
|
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:
|
private:
|
||||||
void connectIfNotConnected();
|
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 = {
|
static const std::unordered_set<std::string_view> required_configuration_keys = {
|
||||||
"url",
|
"url",
|
||||||
};
|
};
|
||||||
static std::unordered_set<std::string_view> optional_configuration_keys = {
|
static const std::unordered_set<std::string_view> optional_configuration_keys = {
|
||||||
"format",
|
"format",
|
||||||
"compression",
|
"compression",
|
||||||
"compression_method",
|
"compression_method",
|
||||||
|
@ -24,7 +24,7 @@ private:
|
|||||||
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
|
||||||
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
|
||||||
|
|
||||||
std::optional<StorageMongoDBConfiguration> configuration;
|
std::optional<StorageMongoDB::Configuration> configuration;
|
||||||
String structure;
|
String structure;
|
||||||
};
|
};
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user