2020-04-16 12:31:57 +00:00
|
|
|
#if !defined(ARCADIA_BUILD)
|
2021-03-31 14:02:51 +00:00
|
|
|
#include "config_core.h"
|
2020-04-16 12:31:57 +00:00
|
|
|
#endif
|
2017-12-28 04:29:53 +00:00
|
|
|
|
2020-04-16 12:31:57 +00:00
|
|
|
#if USE_MYSQL
|
2021-03-31 14:02:51 +00:00
|
|
|
#include <Core/Defines.h>
|
|
|
|
#include <Databases/MySQL/FetchTablesColumnsList.h>
|
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <DataTypes/convertMySQLDataType.h>
|
|
|
|
#include <Formats/MySQLBlockInputStream.h>
|
|
|
|
#include <IO/Operators.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Storages/StorageMySQL.h>
|
|
|
|
#include <TableFunctions/ITableFunction.h>
|
|
|
|
#include <TableFunctions/TableFunctionFactory.h>
|
|
|
|
#include <TableFunctions/TableFunctionMySQL.h>
|
|
|
|
#include <Common/Exception.h>
|
|
|
|
#include <Common/parseAddress.h>
|
|
|
|
#include <Common/quoteString.h>
|
|
|
|
#include "registerTableFunctions.h"
|
|
|
|
|
|
|
|
#include <Databases/MySQL/DatabaseConnectionMySQL.h> // for fetchTablesColumnsList
|
|
|
|
#include <Common/parseRemoteDescription.h>
|
2020-09-09 12:18:02 +00:00
|
|
|
|
2017-12-28 04:28:05 +00:00
|
|
|
|
2017-12-05 13:32:02 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2017-12-28 04:28:05 +00:00
|
|
|
|
2017-12-05 13:32:02 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2020-02-25 18:02:41 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2017-12-05 13:32:02 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
2018-05-14 23:59:58 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2019-06-10 06:13:05 +00:00
|
|
|
extern const int UNKNOWN_TABLE;
|
2017-12-05 13:32:02 +00:00
|
|
|
}
|
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, const Context & context)
|
2017-12-05 13:32:02 +00:00
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
const auto & args_func = ast_function->as<ASTFunction &>();
|
2017-12-05 13:32:02 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
if (!args_func.arguments)
|
2017-12-27 21:45:05 +00:00
|
|
|
throw Exception("Table function 'mysql' must have arguments.", ErrorCodes::LOGICAL_ERROR);
|
2017-12-05 13:32:02 +00:00
|
|
|
|
2019-03-15 16:14:13 +00:00
|
|
|
ASTs & args = args_func.arguments->children;
|
2017-12-05 13:32:02 +00:00
|
|
|
|
2018-05-13 02:34:49 +00:00
|
|
|
if (args.size() < 5 || args.size() > 7)
|
2018-05-14 11:00:22 +00:00
|
|
|
throw Exception("Table function 'mysql' requires 5-7 parameters: MySQL('host:port', database, table, 'user', 'password'[, replace_query, 'on_duplicate_clause']).",
|
2017-12-05 13:32:02 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
|
2020-03-09 00:08:02 +00:00
|
|
|
for (auto & arg : args)
|
|
|
|
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
|
2017-12-05 13:32:02 +00:00
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
String host_port = args[0]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
remote_database_name = args[1]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
remote_table_name = args[2]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
user_name = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
|
|
|
password = args[4]->as<ASTLiteral &>().value.safeGet<String>();
|
2017-12-27 21:45:05 +00:00
|
|
|
|
2021-04-01 10:27:24 +00:00
|
|
|
/// Split into replicas if needed. 3306 is the default MySQL port number
|
2021-03-31 14:02:51 +00:00
|
|
|
size_t max_addresses = context.getSettingsRef().storage_external_distributed_max_addresses;
|
2021-04-01 10:27:24 +00:00
|
|
|
auto addresses = parseRemoteDescriptionForExternalDatabase(host_port, max_addresses);
|
|
|
|
pool.emplace(remote_database_name, addresses, user_name, password);
|
2021-03-31 14:02:51 +00:00
|
|
|
|
2018-05-14 11:00:22 +00:00
|
|
|
if (args.size() >= 6)
|
2019-03-15 16:14:13 +00:00
|
|
|
replace_query = args[5]->as<ASTLiteral &>().value.safeGet<UInt64>() > 0;
|
2018-05-14 11:00:22 +00:00
|
|
|
if (args.size() == 7)
|
2019-03-15 16:14:13 +00:00
|
|
|
on_duplicate_clause = args[6]->as<ASTLiteral &>().value.safeGet<String>();
|
2018-05-13 02:34:49 +00:00
|
|
|
|
2018-05-14 11:00:22 +00:00
|
|
|
if (replace_query && !on_duplicate_clause.empty())
|
|
|
|
throw Exception(
|
|
|
|
"Only one of 'replace_query' and 'on_duplicate_clause' can be specified, or none of them",
|
|
|
|
ErrorCodes::BAD_ARGUMENTS);
|
2020-10-14 12:19:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
ColumnsDescription TableFunctionMySQL::getActualTableStructure(const Context & context) const
|
|
|
|
{
|
2020-09-09 12:18:02 +00:00
|
|
|
const auto & settings = context.getSettingsRef();
|
2020-10-14 12:19:29 +00:00
|
|
|
const auto tables_and_columns = fetchTablesColumnsList(*pool, remote_database_name, {remote_table_name}, settings.external_table_functions_use_nulls, settings.mysql_datatypes_support_level);
|
2017-12-26 21:34:06 +00:00
|
|
|
|
2020-09-09 12:18:02 +00:00
|
|
|
const auto columns = tables_and_columns.find(remote_table_name);
|
|
|
|
if (columns == tables_and_columns.end())
|
2019-07-19 13:28:28 +00:00
|
|
|
throw Exception("MySQL table " + backQuoteIfNeed(remote_database_name) + "." + backQuoteIfNeed(remote_table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
return ColumnsDescription{columns->second};
|
|
|
|
}
|
|
|
|
|
|
|
|
StoragePtr TableFunctionMySQL::executeImpl(const ASTPtr & /*ast_function*/, const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
|
|
|
{
|
|
|
|
auto columns = getActualTableStructure(context);
|
|
|
|
|
2020-08-31 17:45:21 +00:00
|
|
|
auto res = StorageMySQL::create(
|
2019-12-04 16:06:55 +00:00
|
|
|
StorageID(getDatabaseName(), table_name),
|
2020-10-14 12:19:29 +00:00
|
|
|
std::move(*pool),
|
2019-07-19 13:28:28 +00:00
|
|
|
remote_database_name,
|
|
|
|
remote_table_name,
|
2018-05-13 02:34:49 +00:00
|
|
|
replace_query,
|
|
|
|
on_duplicate_clause,
|
2020-10-14 12:19:29 +00:00
|
|
|
columns,
|
2019-08-24 21:20:20 +00:00
|
|
|
ConstraintsDescription{},
|
2018-05-11 04:15:22 +00:00
|
|
|
context);
|
2017-12-26 21:34:06 +00:00
|
|
|
|
2020-10-14 12:19:29 +00:00
|
|
|
pool.reset();
|
|
|
|
|
2017-12-05 13:32:02 +00:00
|
|
|
res->startup();
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
void registerTableFunctionMySQL(TableFunctionFactory & factory)
|
|
|
|
{
|
2017-12-28 04:28:05 +00:00
|
|
|
factory.registerFunction<TableFunctionMySQL>();
|
2017-12-05 13:32:02 +00:00
|
|
|
}
|
|
|
|
}
|
2017-12-28 04:29:53 +00:00
|
|
|
|
|
|
|
#endif
|