CLICKHOUSE-4257 remoteSecure

This commit is contained in:
proller 2019-01-17 20:55:44 +03:00
parent 178302cc5d
commit f2bbac6523
8 changed files with 52 additions and 11 deletions

View File

@ -67,12 +67,13 @@ Cluster::Address::Address(const Poco::Util::AbstractConfiguration & config, cons
}
Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port)
Cluster::Address::Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port, bool secure_)
: user(user_), password(password_)
{
auto parsed_host_port = parseAddress(host_port_, clickhouse_port);
host_name = parsed_host_port.first;
port = parsed_host_port.second;
secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable;
initially_resolved_address = DNSResolver::instance().resolveAddress(parsed_host_port.first, parsed_host_port.second);
is_local = isLocal(*this, initially_resolved_address, clickhouse_port);
@ -319,7 +320,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String>> & names,
const String & username, const String & password, UInt16 clickhouse_port, bool treat_local_as_remote)
const String & username, const String & password, UInt16 clickhouse_port, bool treat_local_as_remote, bool secure)
{
UInt32 current_shard_num = 1;
@ -327,7 +328,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
{
Addresses current;
for (auto & replica : shard)
current.emplace_back(replica, username, password, clickhouse_port);
current.emplace_back(replica, username, password, clickhouse_port, secure);
addresses_with_failover.emplace_back(current);

View File

@ -24,7 +24,7 @@ public:
/// This parameter is needed only to check that some address is local (points to ourself).
Cluster(const Settings & settings, const std::vector<std::vector<String>> & names,
const String & username, const String & password,
UInt16 clickhouse_port, bool treat_local_as_remote);
UInt16 clickhouse_port, bool treat_local_as_remote, bool secure = false);
Cluster(const Cluster &) = delete;
Cluster & operator=(const Cluster &) = delete;
@ -69,7 +69,7 @@ public:
Address() = default;
Address(const Poco::Util::AbstractConfiguration & config, const String & config_prefix);
Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port);
Address(const String & host_port_, const String & user_, const String & password_, UInt16 clickhouse_port, bool secure_ = false);
/// Returns 'escaped_host_name:port'
String toString() const;

View File

@ -152,7 +152,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
if (names.empty())
throw Exception("Shard list is empty after parsing first argument", ErrorCodes::BAD_ARGUMENTS);
cluster = std::make_shared<Cluster>(context.getSettings(), names, username, password, context.getTCPPort(), false);
auto maybe_secure_port = context.getTCPPortSecure();
cluster = std::make_shared<Cluster>(context.getSettings(), names, username, password, (secure && maybe_secure_port) ? *maybe_secure_port : context.getTCPPort(), false, secure);
}
auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_database, remote_table, context, remote_table_function_ptr);
@ -177,8 +178,8 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
}
TableFunctionRemote::TableFunctionRemote(const std::string & name_)
: name(name_)
TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure)
: name{name_}, secure{secure}
{
is_cluster_function = name == "cluster";
@ -193,6 +194,7 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_)
void registerTableFunctionRemote(TableFunctionFactory & factory)
{
factory.registerFunction("remote", [] () -> TableFunctionPtr { return std::make_shared<TableFunctionRemote>("remote"); });
factory.registerFunction("remoteSecure", [] () -> TableFunctionPtr { return std::make_shared<TableFunctionRemote>("remote", /* secure = */ true); });
factory.registerFunction("cluster", [] () -> TableFunctionPtr { return std::make_shared<TableFunctionRemote>("cluster"); });
}

View File

@ -16,7 +16,7 @@ namespace DB
class TableFunctionRemote : public ITableFunction
{
public:
explicit TableFunctionRemote(const std::string & name_ = "remote");
explicit TableFunctionRemote(const std::string & name_ = "remote", bool secure = false);
std::string getName() const override { return name; }
@ -26,6 +26,7 @@ private:
std::string name;
bool is_cluster_function;
std::string help_message;
bool secure;
};
}

View File

@ -0,0 +1,10 @@
0
0
0
0
0
0
0
0
0
0

View File

@ -0,0 +1,24 @@
#!/usr/bin/env bash
# set -x
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
# Not default server config needed
tcp_port_secure=`$CLICKHOUSE_EXTRACT_CONFIG -k tcp_port_secure 2>/dev/null`
if [ -z $tcp_port_secure ]; then
# Secure port disabled. Fake result
cat $CURDIR/00505_shard_secure.reference
else
$CLICKHOUSE_CLIENT -q "SELECT * FROM remoteSecure('127.0.0.{1,2}', system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remoteSecure('127.0.0.{1,2}:$CLICKHOUSE_PORT_TCP_SECURE', system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remoteSecure('127.0.0.{1,2}', system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remoteSecure(test_shard_localhost_secure, system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remote(test_shard_localhost_secure, system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remoteSecure(test_shard_localhost, system.one);"
$CLICKHOUSE_CLIENT -q "SELECT * FROM remote(test_shard_localhost, system.one);"
fi

View File

@ -1,5 +1,5 @@
# remote
# remote, remoteSecure
Allows you to access remote servers without creating a `Distributed` table.
@ -72,5 +72,6 @@ The `remote` table function can be useful in the following cases:
If the user is not specified, `default` is used.
If the password is not specified, an empty password is used.
`remoteSecure` - same as `remote` but with secured connection. Default port - `tcp_port_secure` from config or 9440.
[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/remote/) <!--hide-->

View File

@ -1,5 +1,5 @@
# remote
# remote, remoteSecure
Позволяет обратиться к удалённым серверам без создания таблицы типа `Distributed`.
@ -72,4 +72,6 @@ example01-{01..02}-{1|2}
Если пользователь не задан,то используется `default`.
Если пароль не задан, то используется пустой пароль.
`remoteSecure` - аналогично функции `remote` но с соединением по шифрованому каналу. Порт по умолчанию - `tcp_port_secure` из конфига или 9440.
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/remote/) <!--hide-->