2020-05-13 23:20:45 +00:00
|
|
|
#include "StorageMongoDB.h"
|
2021-03-23 15:01:13 +00:00
|
|
|
#include "StorageMongoDBSocketFactory.h"
|
2020-05-13 23:20:45 +00:00
|
|
|
|
|
|
|
#include <Poco/MongoDB/Connection.h>
|
|
|
|
#include <Poco/MongoDB/Cursor.h>
|
|
|
|
#include <Poco/MongoDB/Database.h>
|
|
|
|
#include <Poco/Version.h>
|
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Core/Settings.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <DataStreams/IBlockOutputStream.h>
|
|
|
|
#include <Common/parseAddress.h>
|
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Processors/Sources/SourceFromInputStream.h>
|
|
|
|
#include <Processors/Pipe.h>
|
2021-08-11 16:44:34 +00:00
|
|
|
#include <DataStreams/MongoDBSource.h>
|
2020-05-13 23:20:45 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int MONGODB_CANNOT_AUTHENTICATE;
|
2021-09-04 18:46:09 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2020-05-13 23:20:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
StorageMongoDB::StorageMongoDB(
|
|
|
|
const StorageID & table_id_,
|
|
|
|
const std::string & host_,
|
2020-05-26 22:20:25 +00:00
|
|
|
uint16_t port_,
|
2020-05-13 23:20:45 +00:00
|
|
|
const std::string & database_name_,
|
|
|
|
const std::string & collection_name_,
|
|
|
|
const std::string & username_,
|
|
|
|
const std::string & password_,
|
2021-03-23 15:01:13 +00:00
|
|
|
const std::string & options_,
|
2020-05-13 23:20:45 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const ConstraintsDescription & constraints_,
|
|
|
|
const String & comment)
|
2020-05-13 23:20:45 +00:00
|
|
|
: IStorage(table_id_)
|
|
|
|
, host(host_)
|
|
|
|
, port(port_)
|
|
|
|
, database_name(database_name_)
|
|
|
|
, collection_name(collection_name_)
|
|
|
|
, username(username_)
|
|
|
|
, password(password_)
|
2021-03-23 15:01:13 +00:00
|
|
|
, options(options_)
|
2021-07-28 15:28:30 +00:00
|
|
|
, uri("mongodb://" + host_ + ":" + std::to_string(port_) + "/" + database_name_ + "?" + options_)
|
2020-05-13 23:20:45 +00:00
|
|
|
{
|
2020-06-26 14:28:00 +00:00
|
|
|
StorageInMemoryMetadata storage_metadata;
|
|
|
|
storage_metadata.setColumns(columns_);
|
|
|
|
storage_metadata.setConstraints(constraints_);
|
2021-04-23 12:18:23 +00:00
|
|
|
storage_metadata.setComment(comment);
|
2020-06-26 14:28:00 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2020-05-13 23:20:45 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-02-05 08:47:02 +00:00
|
|
|
void StorageMongoDB::connectIfNotConnected()
|
|
|
|
{
|
|
|
|
std::lock_guard lock{connection_mutex};
|
|
|
|
if (!connection)
|
2021-03-23 15:01:13 +00:00
|
|
|
{
|
|
|
|
StorageMongoDBSocketFactory factory;
|
|
|
|
connection = std::make_shared<Poco::MongoDB::Connection>(uri, factory);
|
|
|
|
}
|
2021-02-05 08:47:02 +00:00
|
|
|
|
2021-06-28 17:02:22 +00:00
|
|
|
if (!authenticated)
|
2021-02-05 08:47:02 +00:00
|
|
|
{
|
|
|
|
# if POCO_VERSION >= 0x01070800
|
|
|
|
Poco::MongoDB::Database poco_db(database_name);
|
|
|
|
if (!poco_db.authenticate(*connection, username, password, Poco::MongoDB::Database::AUTH_SCRAM_SHA1))
|
|
|
|
throw Exception("Cannot authenticate in MongoDB, incorrect user or password", ErrorCodes::MONGODB_CANNOT_AUTHENTICATE);
|
|
|
|
# else
|
|
|
|
authenticate(*connection, database_name, username, password);
|
|
|
|
# endif
|
2021-06-28 17:02:22 +00:00
|
|
|
authenticated = true;
|
2021-02-05 08:47:02 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageMongoDB::read(
|
2020-05-13 23:20:45 +00:00
|
|
|
const Names & column_names,
|
2020-06-26 14:28:00 +00:00
|
|
|
const StorageMetadataPtr & metadata_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context*/,
|
2020-05-13 23:20:45 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
|
|
|
size_t max_block_size,
|
|
|
|
unsigned)
|
|
|
|
{
|
2021-02-05 08:47:02 +00:00
|
|
|
connectIfNotConnected();
|
2020-05-13 23:20:45 +00:00
|
|
|
|
2021-02-05 08:47:02 +00:00
|
|
|
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
|
2020-05-13 23:20:45 +00:00
|
|
|
|
|
|
|
Block sample_block;
|
|
|
|
for (const String & column_name : column_names)
|
|
|
|
{
|
2020-06-26 14:28:00 +00:00
|
|
|
auto column_data = metadata_snapshot->getColumns().getPhysical(column_name);
|
2020-05-13 23:20:45 +00:00
|
|
|
sample_block.insert({ column_data.type, column_data.name });
|
|
|
|
}
|
|
|
|
|
2021-08-06 08:41:45 +00:00
|
|
|
return Pipe(std::make_shared<MongoDBSource>(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true));
|
2020-05-13 23:20:45 +00:00
|
|
|
}
|
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
|
|
|
|
StorageMongoDBConfiguration StorageMongoDB::getConfiguration(ASTs engine_args, ContextPtr context)
|
2020-05-13 23:20:45 +00:00
|
|
|
{
|
2021-09-03 11:16:32 +00:00
|
|
|
auto [common_configuration, storage_specific_args, with_named_collection] = getExternalDataSourceConfiguration(engine_args, context);
|
|
|
|
StorageMongoDBConfiguration configuration(common_configuration);
|
2020-05-13 23:20:45 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
if (with_named_collection)
|
|
|
|
{
|
|
|
|
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
|
|
|
{
|
|
|
|
if (arg_name == "collection")
|
|
|
|
configuration.collection = arg_value.safeGet<String>();
|
|
|
|
else if (arg_name == "options")
|
|
|
|
configuration.options = arg_value.safeGet<String>();
|
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
2021-09-04 18:46:09 +00:00
|
|
|
"Unexpected argument name for key-value defined argument."
|
2021-09-03 11:16:32 +00:00
|
|
|
"Got: {}, but expected one of:"
|
|
|
|
"host, port, username, password, database, table, options.", arg_name);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-03-23 15:01:13 +00:00
|
|
|
if (engine_args.size() < 5 || engine_args.size() > 6)
|
2020-05-13 23:20:45 +00:00
|
|
|
throw Exception(
|
2021-03-23 15:01:13 +00:00
|
|
|
"Storage MongoDB requires from 5 to 6 parameters: MongoDB('host:port', database, collection, 'user', 'password' [, 'options']).",
|
2020-05-13 23:20:45 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
|
|
|
for (auto & engine_arg : engine_args)
|
2021-09-03 11:16:32 +00:00
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
2020-05-13 23:20:45 +00:00
|
|
|
|
|
|
|
/// 27017 is the default MongoDB port.
|
|
|
|
auto parsed_host_port = parseAddress(engine_args[0]->as<ASTLiteral &>().value.safeGet<String>(), 27017);
|
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
configuration.database = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
configuration.collection = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
configuration.username = engine_args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
configuration.password = engine_args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
2020-05-13 23:20:45 +00:00
|
|
|
|
2021-07-29 08:38:39 +00:00
|
|
|
String options;
|
2021-03-23 15:01:13 +00:00
|
|
|
|
|
|
|
if (engine_args.size() >= 6)
|
|
|
|
options = engine_args[5]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return configuration;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void registerStorageMongoDB(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("MongoDB", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
auto configuration = StorageMongoDB::getConfiguration(args.engine_args, args.getContext());
|
|
|
|
|
2020-05-13 23:20:45 +00:00
|
|
|
return StorageMongoDB::create(
|
|
|
|
args.table_id,
|
2021-09-03 11:16:32 +00:00
|
|
|
configuration.host,
|
|
|
|
configuration.port,
|
|
|
|
configuration.database,
|
|
|
|
configuration.collection,
|
|
|
|
configuration.username,
|
|
|
|
configuration.password,
|
|
|
|
configuration.options,
|
2020-05-13 23:20:45 +00:00
|
|
|
args.columns,
|
2021-04-23 12:18:23 +00:00
|
|
|
args.constraints,
|
|
|
|
args.comment);
|
2020-05-13 23:20:45 +00:00
|
|
|
},
|
|
|
|
{
|
|
|
|
.source_access_type = AccessType::MONGO,
|
|
|
|
});
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|