mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
CLICKHOUSE-4257 remoteSecure
This commit is contained in:
parent
178302cc5d
commit
f2bbac6523
@ -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);
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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"); });
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
10
dbms/tests/queries/0_stateless/00505_shard_secure.reference
Normal file
10
dbms/tests/queries/0_stateless/00505_shard_secure.reference
Normal file
@ -0,0 +1,10 @@
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
24
dbms/tests/queries/0_stateless/00505_shard_secure.sh
Executable file
24
dbms/tests/queries/0_stateless/00505_shard_secure.sh
Executable 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
|
@ -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-->
|
||||
|
@ -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-->
|
||||
|
Loading…
Reference in New Issue
Block a user