2019-06-24 11:17:15 +00:00
|
|
|
#include "StorageMySQL.h"
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2017-12-29 15:48:20 +00:00
|
|
|
#if USE_MYSQL
|
2018-05-10 09:23:38 +00:00
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Storages/StorageFactory.h>
|
2017-12-26 21:34:06 +00:00
|
|
|
#include <Storages/transformQueryForExternalDatabase.h>
|
2021-12-13 22:06:46 +00:00
|
|
|
#include <Storages/MySQL/MySQLHelpers.h>
|
2022-06-23 20:04:06 +00:00
|
|
|
#include <Storages/checkAndGetLiteralArgument.h>
|
2021-10-15 20:18:20 +00:00
|
|
|
#include <Processors/Sources/MySQLSource.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
2020-06-27 01:10:07 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <Formats/FormatFactory.h>
|
2021-10-11 16:11:50 +00:00
|
|
|
#include <Processors/Formats/IOutputFormat.h>
|
2018-05-10 09:23:38 +00:00
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2017-12-30 00:36:06 +00:00
|
|
|
#include <Parsers/ASTLiteral.h>
|
2021-05-15 04:40:43 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2018-05-10 09:23:38 +00:00
|
|
|
#include <mysqlxx/Transaction.h>
|
2021-07-23 19:33:59 +00:00
|
|
|
#include <Processors/Sinks/SinkToStorage.h>
|
2021-10-16 14:03:50 +00:00
|
|
|
#include <QueryPipeline/Pipe.h>
|
2021-03-31 14:02:51 +00:00
|
|
|
#include <Common/parseRemoteDescription.h>
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2023-02-20 20:37:38 +00:00
|
|
|
#include <Storages/NamedCollectionsHelpers.h>
|
2017-12-05 13:32:02 +00:00
|
|
|
|
2017-12-26 18:32:17 +00:00
|
|
|
|
2017-12-05 13:32:02 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2017-12-26 18:32:17 +00:00
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2018-05-14 11:00:22 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2017-12-30 00:36:06 +00:00
|
|
|
}
|
|
|
|
|
2019-12-15 06:34:43 +00:00
|
|
|
static String backQuoteMySQL(const String & x)
|
2019-06-21 05:22:04 +00:00
|
|
|
{
|
|
|
|
String res(x.size(), '\0');
|
|
|
|
{
|
|
|
|
WriteBufferFromString wb(res);
|
2019-06-24 14:51:37 +00:00
|
|
|
writeBackQuotedStringMySQL(x, wb);
|
2019-06-21 05:22:04 +00:00
|
|
|
}
|
|
|
|
return res;
|
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2019-07-09 15:40:21 +00:00
|
|
|
StorageMySQL::StorageMySQL(
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2021-03-24 18:15:31 +00:00
|
|
|
mysqlxx::PoolWithFailover && pool_,
|
2019-08-03 11:02:40 +00:00
|
|
|
const std::string & remote_database_name_,
|
|
|
|
const std::string & remote_table_name_,
|
|
|
|
const bool replace_query_,
|
|
|
|
const std::string & on_duplicate_clause_,
|
2018-05-11 04:15:22 +00:00
|
|
|
const ColumnsDescription & columns_,
|
2019-08-24 21:20:20 +00:00
|
|
|
const ConstraintsDescription & constraints_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const String & comment,
|
2021-05-15 04:40:43 +00:00
|
|
|
ContextPtr context_,
|
|
|
|
const MySQLSettings & mysql_settings_)
|
2019-12-04 16:06:55 +00:00
|
|
|
: IStorage(table_id_)
|
2021-04-10 23:33:54 +00:00
|
|
|
, WithContext(context_->getGlobalContext())
|
2019-08-03 11:02:40 +00:00
|
|
|
, remote_database_name(remote_database_name_)
|
|
|
|
, remote_table_name(remote_table_name_)
|
|
|
|
, replace_query{replace_query_}
|
|
|
|
, on_duplicate_clause{on_duplicate_clause_}
|
2021-05-15 04:40:43 +00:00
|
|
|
, mysql_settings(mysql_settings_)
|
2021-03-27 14:35:44 +00:00
|
|
|
, pool(std::make_shared<mysqlxx::PoolWithFailover>(pool_))
|
2021-09-28 22:17:26 +00:00
|
|
|
, log(&Poco::Logger::get("StorageMySQL (" + table_id_.table_name + ")"))
|
2017-12-05 13:32:02 +00:00
|
|
|
{
|
2020-06-19 15:39:41 +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-19 15:39:41 +00:00
|
|
|
setInMemoryMetadata(storage_metadata);
|
2017-12-05 13:32:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageMySQL::read(
|
2019-08-03 11:02:40 +00:00
|
|
|
const Names & column_names_,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info_,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2018-09-08 11:29:23 +00:00
|
|
|
QueryProcessingStage::Enum /*processed_stage*/,
|
2021-04-16 01:42:55 +00:00
|
|
|
size_t /*max_block_size*/,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t /*num_streams*/)
|
2017-12-05 13:32:02 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names_);
|
2018-07-16 01:02:46 +00:00
|
|
|
String query = transformQueryForExternalDatabase(
|
2020-06-16 15:51:29 +00:00
|
|
|
query_info_,
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->metadata->getColumns().getOrdinary(),
|
2020-06-16 15:51:29 +00:00
|
|
|
IdentifierQuotingStyle::BackticksMySQL,
|
|
|
|
remote_database_name,
|
|
|
|
remote_table_name,
|
|
|
|
context_);
|
2021-09-28 22:17:26 +00:00
|
|
|
LOG_TRACE(log, "Query: {}", query);
|
2017-12-28 05:26:45 +00:00
|
|
|
|
|
|
|
Block sample_block;
|
2019-08-03 11:02:40 +00:00
|
|
|
for (const String & column_name : column_names_)
|
2017-12-28 05:26:45 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
auto column_data = storage_snapshot->metadata->getColumns().getPhysical(column_name);
|
2020-06-27 01:10:07 +00:00
|
|
|
|
|
|
|
WhichDataType which(column_data.type);
|
|
|
|
/// Convert enum to string.
|
|
|
|
if (which.isEnum())
|
|
|
|
column_data.type = std::make_shared<DataTypeString>();
|
2017-12-28 05:26:45 +00:00
|
|
|
sample_block.insert({ column_data.type, column_data.name });
|
|
|
|
}
|
|
|
|
|
2021-04-14 05:41:15 +00:00
|
|
|
|
2021-05-15 04:40:43 +00:00
|
|
|
StreamSettings mysql_input_stream_settings(context_->getSettingsRef(),
|
|
|
|
mysql_settings.connection_auto_close);
|
2021-08-06 08:41:45 +00:00
|
|
|
return Pipe(std::make_shared<MySQLWithFailoverSource>(pool, query, sample_block, mysql_input_stream_settings));
|
2017-12-05 13:32:02 +00:00
|
|
|
}
|
2017-12-26 21:34:06 +00:00
|
|
|
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
class StorageMySQLSink : public SinkToStorage
|
2018-05-10 09:23:38 +00:00
|
|
|
{
|
|
|
|
public:
|
2021-07-23 19:33:59 +00:00
|
|
|
explicit StorageMySQLSink(
|
2020-06-16 15:51:29 +00:00
|
|
|
const StorageMySQL & storage_,
|
|
|
|
const StorageMetadataPtr & metadata_snapshot_,
|
2019-08-03 11:02:40 +00:00
|
|
|
const std::string & remote_database_name_,
|
|
|
|
const std::string & remote_table_name_,
|
|
|
|
const mysqlxx::PoolWithFailover::Entry & entry_,
|
2018-05-14 11:00:22 +00:00
|
|
|
const size_t & mysql_max_rows_to_insert)
|
2021-07-26 10:08:40 +00:00
|
|
|
: SinkToStorage(metadata_snapshot_->getSampleBlock())
|
2021-07-23 19:33:59 +00:00
|
|
|
, storage{storage_}
|
2020-06-16 15:51:29 +00:00
|
|
|
, metadata_snapshot{metadata_snapshot_}
|
2019-08-03 11:02:40 +00:00
|
|
|
, remote_database_name{remote_database_name_}
|
|
|
|
, remote_table_name{remote_table_name_}
|
|
|
|
, entry{entry_}
|
2018-05-10 09:23:38 +00:00
|
|
|
, max_batch_rows{mysql_max_rows_to_insert}
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
String getName() const override { return "StorageMySQLSink"; }
|
2018-05-14 11:00:22 +00:00
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
void consume(Chunk chunk) override
|
2018-05-14 11:00:22 +00:00
|
|
|
{
|
2021-09-03 17:29:36 +00:00
|
|
|
auto block = getHeader().cloneWithColumns(chunk.detachColumns());
|
2018-05-14 11:00:22 +00:00
|
|
|
auto blocks = splitBlocks(block, max_batch_rows);
|
|
|
|
mysqlxx::Transaction trans(entry);
|
|
|
|
try
|
|
|
|
{
|
|
|
|
for (const Block & batch_data : blocks)
|
|
|
|
{
|
|
|
|
writeBlockData(batch_data);
|
|
|
|
}
|
|
|
|
trans.commit();
|
|
|
|
}
|
2018-11-23 18:52:00 +00:00
|
|
|
catch (...)
|
2018-05-14 11:00:22 +00:00
|
|
|
{
|
|
|
|
trans.rollback();
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void writeBlockData(const Block & block)
|
|
|
|
{
|
|
|
|
WriteBufferFromOwnString sqlbuf;
|
|
|
|
sqlbuf << (storage.replace_query ? "REPLACE" : "INSERT") << " INTO ";
|
2021-04-19 20:36:13 +00:00
|
|
|
if (!remote_database_name.empty())
|
|
|
|
sqlbuf << backQuoteMySQL(remote_database_name) << ".";
|
|
|
|
sqlbuf << backQuoteMySQL(remote_table_name);
|
2018-11-24 01:48:06 +00:00
|
|
|
sqlbuf << " (" << dumpNamesWithBackQuote(block) << ") VALUES ";
|
2018-05-14 11:00:22 +00:00
|
|
|
|
2021-10-11 16:11:50 +00:00
|
|
|
auto writer = FormatFactory::instance().getOutputFormat("Values", sqlbuf, metadata_snapshot->getSampleBlock(), storage.getContext());
|
2018-05-14 11:00:22 +00:00
|
|
|
writer->write(block);
|
|
|
|
|
|
|
|
if (!storage.on_duplicate_clause.empty())
|
|
|
|
sqlbuf << " ON DUPLICATE KEY " << storage.on_duplicate_clause;
|
|
|
|
|
|
|
|
sqlbuf << ";";
|
|
|
|
|
|
|
|
auto query = this->entry->query(sqlbuf.str());
|
|
|
|
query.execute();
|
|
|
|
}
|
|
|
|
|
|
|
|
Blocks splitBlocks(const Block & block, const size_t & max_rows) const
|
|
|
|
{
|
|
|
|
/// Avoid Excessive copy when block is small enough
|
|
|
|
if (block.rows() <= max_rows)
|
2022-03-02 17:22:12 +00:00
|
|
|
return {block};
|
2018-05-14 11:00:22 +00:00
|
|
|
|
2022-09-11 01:21:34 +00:00
|
|
|
const size_t split_block_size = static_cast<size_t>(ceil(block.rows() * 1.0 / max_rows));
|
2021-06-28 17:02:22 +00:00
|
|
|
Blocks split_blocks(split_block_size);
|
2018-05-14 11:00:22 +00:00
|
|
|
|
2021-06-28 17:02:22 +00:00
|
|
|
for (size_t idx = 0; idx < split_block_size; ++idx)
|
|
|
|
split_blocks[idx] = block.cloneEmpty();
|
2018-05-14 11:00:22 +00:00
|
|
|
|
|
|
|
const size_t columns = block.columns();
|
|
|
|
const size_t rows = block.rows();
|
|
|
|
size_t offsets = 0;
|
2019-02-10 15:17:45 +00:00
|
|
|
UInt64 limits = max_batch_rows;
|
2021-06-28 17:02:22 +00:00
|
|
|
for (size_t idx = 0; idx < split_block_size; ++idx)
|
2018-05-14 11:00:22 +00:00
|
|
|
{
|
|
|
|
/// For last batch, limits should be the remain size
|
2021-06-28 17:02:22 +00:00
|
|
|
if (idx == split_block_size - 1) limits = rows - offsets;
|
2018-05-14 11:00:22 +00:00
|
|
|
for (size_t col_idx = 0; col_idx < columns; ++col_idx)
|
|
|
|
{
|
2021-06-28 17:02:22 +00:00
|
|
|
split_blocks[idx].getByPosition(col_idx).column = block.getByPosition(col_idx).column->cut(offsets, limits);
|
2018-05-14 11:00:22 +00:00
|
|
|
}
|
|
|
|
offsets += max_batch_rows;
|
|
|
|
}
|
|
|
|
|
2021-06-28 17:02:22 +00:00
|
|
|
return split_blocks;
|
2018-05-14 11:00:22 +00:00
|
|
|
}
|
|
|
|
|
2020-03-18 00:57:00 +00:00
|
|
|
static std::string dumpNamesWithBackQuote(const Block & block)
|
2018-05-14 11:00:22 +00:00
|
|
|
{
|
|
|
|
WriteBufferFromOwnString out;
|
|
|
|
for (auto it = block.begin(); it != block.end(); ++it)
|
|
|
|
{
|
|
|
|
if (it != block.begin())
|
|
|
|
out << ", ";
|
2019-06-21 05:22:04 +00:00
|
|
|
out << backQuoteMySQL(it->name);
|
2018-05-14 11:00:22 +00:00
|
|
|
}
|
|
|
|
return out.str();
|
|
|
|
}
|
2018-05-10 09:23:38 +00:00
|
|
|
|
|
|
|
private:
|
2018-05-14 11:00:22 +00:00
|
|
|
const StorageMySQL & storage;
|
2020-06-16 15:51:29 +00:00
|
|
|
StorageMetadataPtr metadata_snapshot;
|
2018-05-14 11:00:22 +00:00
|
|
|
std::string remote_database_name;
|
|
|
|
std::string remote_table_name;
|
|
|
|
mysqlxx::PoolWithFailover::Entry entry;
|
|
|
|
size_t max_batch_rows;
|
2018-05-10 09:23:38 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2021-07-23 19:33:59 +00:00
|
|
|
SinkToStoragePtr StorageMySQL::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
|
2018-05-10 09:23:38 +00:00
|
|
|
{
|
2021-07-23 19:33:59 +00:00
|
|
|
return std::make_shared<StorageMySQLSink>(
|
2021-04-10 23:33:54 +00:00
|
|
|
*this,
|
|
|
|
metadata_snapshot,
|
|
|
|
remote_database_name,
|
|
|
|
remote_table_name,
|
|
|
|
pool->get(),
|
|
|
|
local_context->getSettingsRef().mysql_max_rows_to_insert);
|
2018-05-10 09:23:38 +00:00
|
|
|
}
|
|
|
|
|
2023-02-20 20:37:38 +00:00
|
|
|
StorageMySQL::Configuration StorageMySQL::processNamedCollectionResult(
|
|
|
|
const NamedCollection & named_collection, MySQLSettings & storage_settings, bool require_table)
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
2023-02-20 20:37:38 +00:00
|
|
|
StorageMySQL::Configuration configuration;
|
|
|
|
|
2023-02-24 19:38:40 +00:00
|
|
|
ValidateKeysMultiset<ExternalDatabaseEqualKeysSet> optional_arguments = {"replace_query", "on_duplicate_clause", "addresses_expr", "host", "hostname", "port"};
|
2023-02-20 20:37:38 +00:00
|
|
|
auto mysql_settings = storage_settings.all();
|
|
|
|
for (const auto & setting : mysql_settings)
|
|
|
|
optional_arguments.insert(setting.getName());
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2023-02-24 19:38:40 +00:00
|
|
|
ValidateKeysMultiset<ExternalDatabaseEqualKeysSet> required_arguments = {"user", "username", "password", "database", "db", "table"};
|
2023-02-20 20:37:38 +00:00
|
|
|
if (require_table)
|
|
|
|
required_arguments.insert("table");
|
2023-02-24 19:38:40 +00:00
|
|
|
validateNamedCollection<ValidateKeysMultiset<ExternalDatabaseEqualKeysSet>>(named_collection, required_arguments, optional_arguments);
|
2023-02-20 20:37:38 +00:00
|
|
|
|
|
|
|
configuration.addresses_expr = named_collection.getOrDefault<String>("addresses_expr", "");
|
|
|
|
if (configuration.addresses_expr.empty())
|
2017-12-30 00:36:06 +00:00
|
|
|
{
|
2023-02-24 19:38:40 +00:00
|
|
|
configuration.host = named_collection.getOrDefault<String>("host", named_collection.getOrDefault<String>("hostname", ""));
|
2023-02-20 20:37:38 +00:00
|
|
|
configuration.port = static_cast<UInt16>(named_collection.get<UInt64>("port"));
|
2021-09-02 13:01:26 +00:00
|
|
|
configuration.addresses = {std::make_pair(configuration.host, configuration.port)};
|
2023-02-20 20:37:38 +00:00
|
|
|
}
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2023-02-24 19:38:40 +00:00
|
|
|
configuration.username = named_collection.getOrDefault<String>("username", named_collection.getOrDefault<String>("user", ""));
|
2023-02-20 20:37:38 +00:00
|
|
|
configuration.password = named_collection.get<String>("password");
|
2023-02-24 19:38:40 +00:00
|
|
|
configuration.database = named_collection.getOrDefault<String>("db", named_collection.getOrDefault<String>("database", ""));
|
2023-02-20 20:37:38 +00:00
|
|
|
if (require_table)
|
|
|
|
configuration.table = named_collection.get<String>("table");
|
|
|
|
configuration.replace_query = named_collection.getOrDefault<UInt64>("replace_query", false);
|
|
|
|
configuration.on_duplicate_clause = named_collection.getOrDefault<String>("on_duplicate_clause", "");
|
|
|
|
|
|
|
|
for (const auto & setting : mysql_settings)
|
|
|
|
{
|
|
|
|
const auto & setting_name = setting.getName();
|
|
|
|
if (named_collection.has(setting_name))
|
|
|
|
storage_settings.set(setting_name, named_collection.get<String>(setting_name));
|
|
|
|
}
|
|
|
|
|
|
|
|
return configuration;
|
|
|
|
}
|
|
|
|
|
|
|
|
StorageMySQL::Configuration StorageMySQL::getConfiguration(ASTs engine_args, ContextPtr context_, MySQLSettings & storage_settings)
|
|
|
|
{
|
|
|
|
StorageMySQL::Configuration configuration;
|
|
|
|
if (auto named_collection = tryGetNamedCollectionWithOverrides(engine_args))
|
|
|
|
{
|
|
|
|
configuration = StorageMySQL::processNamedCollectionResult(*named_collection, storage_settings);
|
2021-09-02 13:01:26 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-05-13 02:34:49 +00:00
|
|
|
if (engine_args.size() < 5 || engine_args.size() > 7)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage MySQL requires 5-7 parameters: "
|
|
|
|
"MySQL('host:port' (or 'addresses_pattern'), database, table, "
|
|
|
|
"'user', 'password'[, replace_query, 'on_duplicate_clause']).");
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2020-03-09 01:22:33 +00:00
|
|
|
for (auto & engine_arg : engine_args)
|
2021-09-02 13:01:26 +00:00
|
|
|
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context_);
|
|
|
|
|
2022-06-23 20:04:06 +00:00
|
|
|
const auto & host_port = checkAndGetLiteralArgument<String>(engine_args[0], "host:port");
|
2021-09-02 13:01:26 +00:00
|
|
|
size_t max_addresses = context_->getSettingsRef().glob_expansion_max_elements;
|
|
|
|
|
|
|
|
configuration.addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses, 3306);
|
2022-06-23 20:04:06 +00:00
|
|
|
configuration.database = checkAndGetLiteralArgument<String>(engine_args[1], "database");
|
|
|
|
configuration.table = checkAndGetLiteralArgument<String>(engine_args[2], "table");
|
|
|
|
configuration.username = checkAndGetLiteralArgument<String>(engine_args[3], "username");
|
|
|
|
configuration.password = checkAndGetLiteralArgument<String>(engine_args[4], "password");
|
2021-09-02 13:01:26 +00:00
|
|
|
if (engine_args.size() >= 6)
|
2022-06-23 20:04:06 +00:00
|
|
|
configuration.replace_query = checkAndGetLiteralArgument<UInt64>(engine_args[5], "replace_query");
|
2021-09-02 13:01:26 +00:00
|
|
|
if (engine_args.size() == 7)
|
2022-06-23 20:04:06 +00:00
|
|
|
configuration.on_duplicate_clause = checkAndGetLiteralArgument<String>(engine_args[6], "on_duplicate_clause");
|
2021-09-02 13:01:26 +00:00
|
|
|
}
|
2022-03-14 14:37:59 +00:00
|
|
|
for (const auto & address : configuration.addresses)
|
2022-03-11 15:33:23 +00:00
|
|
|
context_->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second));
|
2021-09-02 13:01:26 +00:00
|
|
|
if (configuration.replace_query && !configuration.on_duplicate_clause.empty())
|
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
|
|
|
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them");
|
|
|
|
|
|
|
|
return configuration;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void registerStorageMySQL(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerStorage("MySQL", [](const StorageFactory::Arguments & args)
|
|
|
|
{
|
|
|
|
MySQLSettings mysql_settings; /// TODO: move some arguments from the arguments to the SETTINGS.
|
2021-12-27 14:41:37 +00:00
|
|
|
auto configuration = StorageMySQL::getConfiguration(args.engine_args, args.getLocalContext(), mysql_settings);
|
|
|
|
|
2021-05-15 04:40:43 +00:00
|
|
|
if (args.storage_def->settings)
|
|
|
|
mysql_settings.loadFromQuery(*args.storage_def);
|
|
|
|
|
|
|
|
if (!mysql_settings.connection_pool_size)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::BAD_ARGUMENTS, "connection_pool_size cannot be zero.");
|
2021-05-15 04:40:43 +00:00
|
|
|
|
2021-12-13 22:06:46 +00:00
|
|
|
mysqlxx::PoolWithFailover pool = createMySQLPoolWithFailover(configuration, mysql_settings);
|
2017-12-30 00:36:06 +00:00
|
|
|
|
2022-05-04 10:01:29 +00:00
|
|
|
return std::make_shared<StorageMySQL>(
|
2019-12-04 16:06:55 +00:00
|
|
|
args.table_id,
|
2017-12-30 00:36:06 +00:00
|
|
|
std::move(pool),
|
2021-09-02 13:01:26 +00:00
|
|
|
configuration.database,
|
|
|
|
configuration.table,
|
|
|
|
configuration.replace_query,
|
|
|
|
configuration.on_duplicate_clause,
|
2018-05-11 04:15:22 +00:00
|
|
|
args.columns,
|
2019-08-24 21:20:20 +00:00
|
|
|
args.constraints,
|
2021-04-23 12:18:23 +00:00
|
|
|
args.comment,
|
2021-05-15 04:40:43 +00:00
|
|
|
args.getContext(),
|
|
|
|
mysql_settings);
|
2020-04-06 05:19:40 +00:00
|
|
|
},
|
|
|
|
{
|
2021-05-15 04:40:43 +00:00
|
|
|
.supports_settings = true,
|
2020-04-06 05:19:40 +00:00
|
|
|
.source_access_type = AccessType::MYSQL,
|
2017-12-30 00:36:06 +00:00
|
|
|
});
|
|
|
|
}
|
|
|
|
|
2017-12-26 18:32:17 +00:00
|
|
|
}
|
2017-12-29 15:48:20 +00:00
|
|
|
|
|
|
|
#endif
|