2020-04-28 18:36:43 +00:00
|
|
|
#include <Databases/DatabaseFactory.h>
|
|
|
|
|
2019-10-23 13:46:38 +00:00
|
|
|
#include <Databases/DatabaseAtomic.h>
|
2019-09-26 13:43:08 +00:00
|
|
|
#include <Databases/DatabaseDictionary.h>
|
|
|
|
#include <Databases/DatabaseLazy.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/DatabaseMemory.h>
|
2019-09-26 13:43:08 +00:00
|
|
|
#include <Databases/DatabaseOrdinary.h>
|
2021-05-17 11:02:35 +00:00
|
|
|
#include <Databases/DatabaseReplicated.h>
|
|
|
|
#include <Interpreters/Context.h>
|
2019-06-14 16:18:48 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2019-08-27 20:43:08 +00:00
|
|
|
#include <Parsers/ASTFunction.h>
|
2020-05-06 06:40:05 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/formatAST.h>
|
2021-02-15 20:00:59 +00:00
|
|
|
#include <Common/Macros.h>
|
2021-09-11 16:29:23 +00:00
|
|
|
#include <Storages/ExternalDataSourceConfiguration.h>
|
2021-04-30 17:20:53 +00:00
|
|
|
#include <filesystem>
|
2019-12-12 12:17:27 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
|
|
|
# include "config_core.h"
|
|
|
|
#endif
|
2019-06-13 08:19:26 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if USE_MYSQL
|
2020-08-13 12:41:36 +00:00
|
|
|
# include <Core/MySQL/MySQLClient.h>
|
2020-09-28 10:02:30 +00:00
|
|
|
# include <Databases/MySQL/ConnectionMySQLSettings.h>
|
2021-06-17 19:26:34 +00:00
|
|
|
# include <Databases/MySQL/DatabaseMySQL.h>
|
2021-07-26 18:17:28 +00:00
|
|
|
# include <Databases/MySQL/MaterializedMySQLSettings.h>
|
|
|
|
# include <Databases/MySQL/DatabaseMaterializedMySQL.h>
|
2020-05-06 06:40:05 +00:00
|
|
|
# include <mysqlxx/Pool.h>
|
2019-06-13 08:19:26 +00:00
|
|
|
#endif
|
2019-06-10 09:40:33 +00:00
|
|
|
|
2020-12-28 19:16:10 +00:00
|
|
|
#if USE_MYSQL || USE_LIBPQXX
|
2021-03-31 14:02:51 +00:00
|
|
|
#include <Common/parseRemoteDescription.h>
|
2020-12-28 19:16:10 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Common/parseAddress.h>
|
|
|
|
#endif
|
|
|
|
|
2020-12-21 19:20:56 +00:00
|
|
|
#if USE_LIBPQXX
|
2021-01-07 03:45:12 +00:00
|
|
|
#include <Databases/PostgreSQL/DatabasePostgreSQL.h> // Y_IGNORE
|
2021-06-27 19:09:17 +00:00
|
|
|
#include <Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h>
|
|
|
|
#include <Storages/PostgreSQL/MaterializedPostgreSQLSettings.h>
|
2021-09-01 23:17:15 +00:00
|
|
|
#include <Storages/StoragePostgreSQL.h>
|
2020-12-21 19:20:56 +00:00
|
|
|
#endif
|
|
|
|
|
2021-07-08 13:27:30 +00:00
|
|
|
#if USE_SQLITE
|
|
|
|
#include <Databases/SQLite/DatabaseSQLite.h>
|
|
|
|
#endif
|
|
|
|
|
2021-04-30 17:20:53 +00:00
|
|
|
namespace fs = std::filesystem;
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int UNKNOWN_ELEMENT_IN_AST;
|
2019-12-12 12:17:27 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
|
|
|
extern const int UNKNOWN_DATABASE_ENGINE;
|
|
|
|
extern const int CANNOT_CREATE_DATABASE;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
2019-12-12 12:17:27 +00:00
|
|
|
{
|
2020-04-22 19:37:30 +00:00
|
|
|
bool created = false;
|
2020-04-22 16:03:58 +00:00
|
|
|
|
2019-12-12 12:17:27 +00:00
|
|
|
try
|
|
|
|
{
|
2020-07-17 13:11:44 +00:00
|
|
|
/// Creates store/xxx/ for Atomic
|
2021-05-09 11:59:49 +00:00
|
|
|
fs::create_directories(fs::path(metadata_path).parent_path());
|
|
|
|
|
2020-07-17 13:11:44 +00:00
|
|
|
/// Before 20.7 it's possible that .sql metadata file does not exist for some old database.
|
|
|
|
/// In this case Ordinary database is created on server startup if the corresponding metadata directory exists.
|
|
|
|
/// So we should remove metadata directory if database creation failed.
|
2021-04-30 17:20:53 +00:00
|
|
|
created = fs::create_directory(metadata_path);
|
2021-01-23 10:18:40 +00:00
|
|
|
|
|
|
|
DatabasePtr impl = getImpl(create, metadata_path, context);
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
if (impl && context->hasQueryContext() && context->getSettingsRef().log_queries)
|
|
|
|
context->getQueryContext()->addQueryFactoriesInfo(Context::QueryLogFactories::Database, impl->getEngineName());
|
2021-01-23 10:18:40 +00:00
|
|
|
|
|
|
|
return impl;
|
|
|
|
|
2019-12-12 12:17:27 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2021-04-30 17:20:53 +00:00
|
|
|
if (created && fs::exists(metadata_path))
|
|
|
|
fs::remove_all(metadata_path);
|
2019-12-12 12:17:27 +00:00
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-07 05:14:49 +00:00
|
|
|
template <typename ValueType>
|
2020-04-08 05:41:11 +00:00
|
|
|
static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &engine_name)
|
2020-04-07 05:14:49 +00:00
|
|
|
{
|
|
|
|
if (!ast || !ast->as<ASTLiteral>())
|
|
|
|
throw Exception("Database engine " + engine_name + " requested literal argument.", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
|
|
|
|
return ast->as<ASTLiteral>()->value.safeGet<ValueType>();
|
|
|
|
}
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2020-08-04 04:22:12 +00:00
|
|
|
auto * engine_define = create.storage;
|
2020-07-17 13:11:44 +00:00
|
|
|
const String & database_name = create.database;
|
|
|
|
const String & engine_name = engine_define->engine->name;
|
|
|
|
const UUID & uuid = create.uuid;
|
2019-06-10 09:40:33 +00:00
|
|
|
|
2021-08-05 09:04:00 +00:00
|
|
|
static const std::unordered_set<std::string_view> engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL",
|
2021-08-04 20:42:29 +00:00
|
|
|
"Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"};
|
|
|
|
|
|
|
|
bool engine_may_have_arguments = engines_with_arguments.contains(engine_name);
|
|
|
|
|
2020-10-20 16:14:54 +00:00
|
|
|
if (engine_define->engine->arguments && !engine_may_have_arguments)
|
2019-06-14 16:18:48 +00:00
|
|
|
throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
|
2021-02-19 23:41:58 +00:00
|
|
|
bool has_unexpected_element = engine_define->engine->parameters || engine_define->partition_by ||
|
|
|
|
engine_define->primary_key || engine_define->order_by ||
|
|
|
|
engine_define->sample_by;
|
2021-06-27 19:09:17 +00:00
|
|
|
bool may_have_settings = endsWith(engine_name, "MySQL") || engine_name == "Replicated" || engine_name == "MaterializedPostgreSQL";
|
2021-08-04 20:42:29 +00:00
|
|
|
|
2021-02-19 23:41:58 +00:00
|
|
|
if (has_unexpected_element || (!may_have_settings && engine_define->settings))
|
2019-06-14 16:18:48 +00:00
|
|
|
throw Exception("Database engine " + engine_name + " cannot have parameters, primary_key, order_by, sample_by, settings",
|
|
|
|
ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
|
2019-06-10 09:40:33 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (engine_name == "Ordinary")
|
2017-11-03 19:53:10 +00:00
|
|
|
return std::make_shared<DatabaseOrdinary>(database_name, metadata_path, context);
|
2019-10-23 13:46:38 +00:00
|
|
|
else if (engine_name == "Atomic")
|
2020-07-02 20:39:31 +00:00
|
|
|
return std::make_shared<DatabaseAtomic>(database_name, metadata_path, uuid, context);
|
2017-04-01 07:20:54 +00:00
|
|
|
else if (engine_name == "Memory")
|
2020-05-28 20:10:45 +00:00
|
|
|
return std::make_shared<DatabaseMemory>(database_name, context);
|
2017-06-22 15:44:19 +00:00
|
|
|
else if (engine_name == "Dictionary")
|
2020-04-01 22:41:29 +00:00
|
|
|
return std::make_shared<DatabaseDictionary>(database_name, context);
|
2019-06-13 08:19:26 +00:00
|
|
|
|
|
|
|
#if USE_MYSQL
|
|
|
|
|
2021-07-26 18:24:32 +00:00
|
|
|
else if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL")
|
2019-06-10 09:40:33 +00:00
|
|
|
{
|
2019-06-14 16:18:48 +00:00
|
|
|
const ASTFunction * engine = engine_define->engine;
|
2021-09-15 18:11:49 +00:00
|
|
|
if (!engine->arguments)
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
|
|
|
|
ASTs & arguments = engine->arguments->children;
|
|
|
|
auto [common_configuration, storage_specific_args, with_named_collection] = getExternalDataSourceConfiguration(arguments, context, true);
|
2021-09-02 13:01:26 +00:00
|
|
|
StorageMySQLConfiguration configuration(common_configuration);
|
2020-04-07 05:14:49 +00:00
|
|
|
|
2021-09-02 13:01:26 +00:00
|
|
|
if (with_named_collection)
|
|
|
|
{
|
|
|
|
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
|
|
|
|
if (!storage_specific_args.empty())
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.");
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-09-15 18:11:49 +00:00
|
|
|
if (arguments.size() != 4)
|
2021-09-02 13:01:26 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"MySQL database require mysql_hostname, mysql_database_name, mysql_username, mysql_password arguments.");
|
2020-04-08 05:41:11 +00:00
|
|
|
|
2021-09-02 13:01:26 +00:00
|
|
|
arguments[1] = evaluateConstantExpressionOrIdentifierAsLiteral(arguments[1], context);
|
|
|
|
|
|
|
|
const auto & host_port = safeGetLiteralValue<String>(arguments[0], engine_name);
|
|
|
|
|
|
|
|
if (engine_name == "MySQL")
|
|
|
|
{
|
|
|
|
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
|
|
|
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
const auto & [remote_host, remote_port] = parseAddress(host_port, 3306);
|
|
|
|
configuration.host = remote_host;
|
|
|
|
configuration.port = remote_port;
|
|
|
|
}
|
|
|
|
|
|
|
|
configuration.database = safeGetLiteralValue<String>(arguments[1], engine_name);
|
|
|
|
configuration.username = safeGetLiteralValue<String>(arguments[2], engine_name);
|
|
|
|
configuration.password = safeGetLiteralValue<String>(arguments[3], engine_name);
|
|
|
|
}
|
2019-06-10 09:40:33 +00:00
|
|
|
|
2019-12-12 12:17:27 +00:00
|
|
|
try
|
|
|
|
{
|
2020-09-15 13:30:30 +00:00
|
|
|
if (engine_name == "MySQL")
|
2020-05-29 02:39:22 +00:00
|
|
|
{
|
2020-10-02 17:09:37 +00:00
|
|
|
auto mysql_database_settings = std::make_unique<ConnectionMySQLSettings>();
|
2021-09-02 13:01:26 +00:00
|
|
|
auto mysql_pool = mysqlxx::PoolWithFailover(configuration.database, configuration.addresses, configuration.username, configuration.password);
|
2020-05-29 02:39:22 +00:00
|
|
|
|
2020-10-02 17:09:37 +00:00
|
|
|
mysql_database_settings->loadFromQueryContext(context);
|
|
|
|
mysql_database_settings->loadFromQuery(*engine_define); /// higher priority
|
2019-12-12 12:17:27 +00:00
|
|
|
|
2021-06-17 19:26:34 +00:00
|
|
|
return std::make_shared<DatabaseMySQL>(
|
2021-09-02 13:01:26 +00:00
|
|
|
context, database_name, metadata_path, engine_define, configuration.database, std::move(mysql_database_settings), std::move(mysql_pool));
|
2020-05-29 02:39:22 +00:00
|
|
|
}
|
2020-05-06 06:40:05 +00:00
|
|
|
|
2021-09-02 13:01:26 +00:00
|
|
|
MySQLClient client(configuration.host, configuration.port, configuration.username, configuration.password);
|
|
|
|
auto mysql_pool = mysqlxx::Pool(configuration.database, configuration.host, configuration.username, configuration.password, configuration.port);
|
2021-08-27 18:07:10 +00:00
|
|
|
|
2021-07-26 18:17:28 +00:00
|
|
|
auto materialize_mode_settings = std::make_unique<MaterializedMySQLSettings>();
|
2020-09-28 10:02:30 +00:00
|
|
|
|
2020-09-15 13:30:30 +00:00
|
|
|
if (engine_define->settings)
|
|
|
|
materialize_mode_settings->loadFromQuery(*engine_define);
|
2020-09-28 10:02:30 +00:00
|
|
|
|
2020-09-15 13:30:30 +00:00
|
|
|
if (create.uuid == UUIDHelpers::Nil)
|
2021-07-26 18:17:28 +00:00
|
|
|
return std::make_shared<DatabaseMaterializedMySQL<DatabaseOrdinary>>(
|
2021-09-02 13:01:26 +00:00
|
|
|
context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool),
|
|
|
|
std::move(client), std::move(materialize_mode_settings));
|
2020-09-15 13:30:30 +00:00
|
|
|
else
|
2021-07-26 18:17:28 +00:00
|
|
|
return std::make_shared<DatabaseMaterializedMySQL<DatabaseAtomic>>(
|
2021-09-02 13:01:26 +00:00
|
|
|
context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool),
|
|
|
|
std::move(client), std::move(materialize_mode_settings));
|
2019-12-12 12:17:27 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
const auto & exception_message = getCurrentExceptionMessage(true);
|
|
|
|
throw Exception("Cannot create MySQL database, because " + exception_message, ErrorCodes::CANNOT_CREATE_DATABASE);
|
|
|
|
}
|
2019-06-10 09:40:33 +00:00
|
|
|
}
|
2019-06-13 08:19:26 +00:00
|
|
|
#endif
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2019-09-26 13:43:08 +00:00
|
|
|
else if (engine_name == "Lazy")
|
|
|
|
{
|
|
|
|
const ASTFunction * engine = engine_define->engine;
|
|
|
|
|
2019-10-22 10:47:43 +00:00
|
|
|
if (!engine->arguments || engine->arguments->children.size() != 1)
|
|
|
|
throw Exception("Lazy database require cache_expiration_time_seconds argument", ErrorCodes::BAD_ARGUMENTS);
|
2019-09-26 13:43:08 +00:00
|
|
|
|
2019-10-22 10:47:43 +00:00
|
|
|
const auto & arguments = engine->arguments->children;
|
2019-09-26 13:43:08 +00:00
|
|
|
|
2020-04-08 05:41:11 +00:00
|
|
|
const auto cache_expiration_time_seconds = safeGetLiteralValue<UInt64>(arguments[0], "Lazy");
|
2019-09-26 13:43:08 +00:00
|
|
|
return std::make_shared<DatabaseLazy>(database_name, metadata_path, cache_expiration_time_seconds, context);
|
|
|
|
}
|
|
|
|
|
2020-04-29 11:19:16 +00:00
|
|
|
else if (engine_name == "Replicated")
|
|
|
|
{
|
|
|
|
const ASTFunction * engine = engine_define->engine;
|
|
|
|
|
2020-10-27 09:19:45 +00:00
|
|
|
if (!engine->arguments || engine->arguments->children.size() != 3)
|
|
|
|
throw Exception("Replicated database requires 3 arguments: zookeeper path, shard name and replica name", ErrorCodes::BAD_ARGUMENTS);
|
2020-04-29 11:19:16 +00:00
|
|
|
|
|
|
|
const auto & arguments = engine->arguments->children;
|
|
|
|
|
2021-02-15 20:00:59 +00:00
|
|
|
String zookeeper_path = safeGetLiteralValue<String>(arguments[0], "Replicated");
|
|
|
|
String shard_name = safeGetLiteralValue<String>(arguments[1], "Replicated");
|
|
|
|
String replica_name = safeGetLiteralValue<String>(arguments[2], "Replicated");
|
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
zookeeper_path = context->getMacros()->expand(zookeeper_path);
|
|
|
|
shard_name = context->getMacros()->expand(shard_name);
|
|
|
|
replica_name = context->getMacros()->expand(replica_name);
|
2020-06-27 13:39:41 +00:00
|
|
|
|
2021-02-19 23:41:58 +00:00
|
|
|
DatabaseReplicatedSettings database_replicated_settings{};
|
|
|
|
if (engine_define->settings)
|
|
|
|
database_replicated_settings.loadFromQuery(*engine_define);
|
|
|
|
|
|
|
|
return std::make_shared<DatabaseReplicated>(database_name, metadata_path, uuid,
|
|
|
|
zookeeper_path, shard_name, replica_name,
|
|
|
|
std::move(database_replicated_settings), context);
|
2020-04-29 11:19:16 +00:00
|
|
|
}
|
|
|
|
|
2020-12-21 19:20:56 +00:00
|
|
|
#if USE_LIBPQXX
|
|
|
|
|
|
|
|
else if (engine_name == "PostgreSQL")
|
|
|
|
{
|
|
|
|
const ASTFunction * engine = engine_define->engine;
|
2021-09-15 18:11:49 +00:00
|
|
|
if (!engine->arguments)
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
|
2020-12-21 19:20:56 +00:00
|
|
|
|
|
|
|
ASTs & engine_args = engine->arguments->children;
|
2021-09-03 11:16:32 +00:00
|
|
|
auto [common_configuration, storage_specific_args, with_named_collection] = getExternalDataSourceConfiguration(engine_args, context, true);
|
2021-09-01 23:17:15 +00:00
|
|
|
StoragePostgreSQLConfiguration configuration(common_configuration);
|
2021-09-15 18:11:49 +00:00
|
|
|
auto use_table_cache = false;
|
2020-12-21 19:20:56 +00:00
|
|
|
|
2021-09-01 23:17:15 +00:00
|
|
|
if (with_named_collection)
|
|
|
|
{
|
|
|
|
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
|
|
|
|
for (const auto & [arg_name, arg_value] : storage_specific_args)
|
|
|
|
{
|
2021-09-15 18:11:49 +00:00
|
|
|
if (arg_name == "use_table_cache")
|
|
|
|
use_table_cache = true;
|
2021-09-01 23:17:15 +00:00
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
2021-09-15 18:11:49 +00:00
|
|
|
"Unexpected key-value argument."
|
2021-09-01 23:17:15 +00:00
|
|
|
"Got: {}, but expected one of:"
|
2021-09-15 18:11:49 +00:00
|
|
|
"host, port, username, password, database, schema, use_table_cache.", arg_name);
|
2021-09-02 13:01:26 +00:00
|
|
|
}
|
2021-09-01 23:17:15 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2021-09-15 18:11:49 +00:00
|
|
|
if (engine_args.size() < 4 || engine_args.size() > 6)
|
2021-09-01 23:17:15 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"PostgreSQL Database require `host:port`, `database_name`, `username`, `password`"
|
2021-09-15 18:11:49 +00:00
|
|
|
"[, `schema` = "", `use_table_cache` = 0");
|
2021-09-01 23:17:15 +00:00
|
|
|
|
|
|
|
for (auto & engine_arg : engine_args)
|
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
2020-12-21 19:20:56 +00:00
|
|
|
|
2021-09-01 23:17:15 +00:00
|
|
|
const auto & host_port = safeGetLiteralValue<String>(engine_args[0], engine_name);
|
|
|
|
size_t max_addresses = context->getSettingsRef().glob_expansion_max_elements;
|
2020-12-21 19:20:56 +00:00
|
|
|
|
2021-09-01 23:17:15 +00:00
|
|
|
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 5432);
|
|
|
|
configuration.database = safeGetLiteralValue<String>(engine_args[1], engine_name);
|
|
|
|
configuration.username = safeGetLiteralValue<String>(engine_args[2], engine_name);
|
|
|
|
configuration.password = safeGetLiteralValue<String>(engine_args[3], engine_name);
|
|
|
|
|
|
|
|
if (engine_args.size() >= 5)
|
|
|
|
configuration.schema = safeGetLiteralValue<String>(engine_args[4], engine_name);
|
|
|
|
}
|
2021-08-04 20:42:29 +00:00
|
|
|
|
2021-09-01 23:17:15 +00:00
|
|
|
if (engine_args.size() >= 6)
|
2021-08-04 20:42:29 +00:00
|
|
|
use_table_cache = safeGetLiteralValue<UInt8>(engine_args[5], engine_name);
|
2020-12-26 19:38:10 +00:00
|
|
|
|
2021-09-01 23:17:15 +00:00
|
|
|
auto pool = std::make_shared<postgres::PoolWithFailover>(configuration,
|
2021-04-10 23:33:54 +00:00
|
|
|
context->getSettingsRef().postgresql_connection_pool_size,
|
|
|
|
context->getSettingsRef().postgresql_connection_pool_wait_timeout);
|
2020-12-21 19:20:56 +00:00
|
|
|
|
|
|
|
return std::make_shared<DatabasePostgreSQL>(
|
2021-09-01 23:17:15 +00:00
|
|
|
context, metadata_path, engine_define, database_name, configuration, pool, use_table_cache);
|
2020-12-21 19:20:56 +00:00
|
|
|
}
|
2021-06-27 19:09:17 +00:00
|
|
|
else if (engine_name == "MaterializedPostgreSQL")
|
2021-02-08 23:23:51 +00:00
|
|
|
{
|
|
|
|
const ASTFunction * engine = engine_define->engine;
|
2021-09-15 18:11:49 +00:00
|
|
|
if (!engine->arguments)
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Engine `{}` must have arguments", engine_name);
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
ASTs & engine_args = engine->arguments->children;
|
|
|
|
auto [common_configuration, storage_specific_args, with_named_collection] = getExternalDataSourceConfiguration(engine_args, context, true);
|
|
|
|
StoragePostgreSQLConfiguration configuration(common_configuration);
|
|
|
|
|
|
|
|
if (with_named_collection)
|
2021-02-08 23:23:51 +00:00
|
|
|
{
|
2021-09-03 11:16:32 +00:00
|
|
|
if (!storage_specific_args.empty())
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"MaterializedPostgreSQL Database requires only `host`, `port`, `database_name`, `username`, `password`.");
|
2021-02-08 23:23:51 +00:00
|
|
|
}
|
2021-09-03 11:16:32 +00:00
|
|
|
else
|
|
|
|
{
|
2021-09-15 18:11:49 +00:00
|
|
|
if (engine_args.size() != 4)
|
2021-09-03 11:16:32 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"MaterializedPostgreSQL Database require `host:port`, `database_name`, `username`, `password`.");
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
for (auto & engine_arg : engine_args)
|
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context);
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
auto parsed_host_port = parseAddress(safeGetLiteralValue<String>(engine_args[0], engine_name), 5432);
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
configuration.host = parsed_host_port.first;
|
|
|
|
configuration.port = parsed_host_port.second;
|
|
|
|
configuration.database = safeGetLiteralValue<String>(engine_args[1], engine_name);
|
|
|
|
configuration.username = safeGetLiteralValue<String>(engine_args[2], engine_name);
|
|
|
|
configuration.password = safeGetLiteralValue<String>(engine_args[3], engine_name);
|
|
|
|
}
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-09-03 11:16:32 +00:00
|
|
|
auto connection_info = postgres::formatConnectionString(
|
|
|
|
configuration.database, configuration.host, configuration.port, configuration.username, configuration.password);
|
2021-02-08 23:23:51 +00:00
|
|
|
|
2021-06-27 19:09:17 +00:00
|
|
|
auto postgresql_replica_settings = std::make_unique<MaterializedPostgreSQLSettings>();
|
2021-02-08 23:23:51 +00:00
|
|
|
if (engine_define->settings)
|
|
|
|
postgresql_replica_settings->loadFromQuery(*engine_define);
|
|
|
|
|
2021-06-27 19:09:17 +00:00
|
|
|
return std::make_shared<DatabaseMaterializedPostgreSQL>(
|
2021-07-04 14:56:31 +00:00
|
|
|
context, metadata_path, uuid, engine_define, create.attach,
|
2021-09-03 11:16:32 +00:00
|
|
|
database_name, configuration.database, connection_info,
|
2021-04-08 22:38:17 +00:00
|
|
|
std::move(postgresql_replica_settings));
|
2021-02-08 23:23:51 +00:00
|
|
|
}
|
|
|
|
|
2020-12-21 19:20:56 +00:00
|
|
|
|
2021-07-08 13:27:30 +00:00
|
|
|
#endif
|
|
|
|
|
|
|
|
#if USE_SQLITE
|
|
|
|
else if (engine_name == "SQLite")
|
|
|
|
{
|
|
|
|
const ASTFunction * engine = engine_define->engine;
|
|
|
|
|
|
|
|
if (!engine->arguments || engine->arguments->children.size() != 1)
|
|
|
|
throw Exception("SQLite database requires 1 argument: database path", ErrorCodes::BAD_ARGUMENTS);
|
|
|
|
|
|
|
|
const auto & arguments = engine->arguments->children;
|
|
|
|
|
|
|
|
String database_path = safeGetLiteralValue<String>(arguments[0], "SQLite");
|
|
|
|
|
2021-07-26 07:05:28 +00:00
|
|
|
return std::make_shared<DatabaseSQLite>(context, engine_define, create.attach, database_path);
|
2021-07-08 13:27:30 +00:00
|
|
|
}
|
2020-12-21 19:20:56 +00:00
|
|
|
#endif
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Unknown database engine: " + engine_name, ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|