mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Allow use encryption for distributed tables (#2065)
* Allow use encryption for distributed tables * wip * fix * rename encryption to secure * Fix DirectoryMonitor with +secure * fix * fix * Update DirectoryMonitor.cpp * Update DirectoryMonitor.cpp
This commit is contained in:
parent
54986af313
commit
a6f9df105c
@ -51,14 +51,15 @@ void Connection::connect()
|
||||
if (connected)
|
||||
disconnect();
|
||||
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user);
|
||||
LOG_TRACE(log_wrapper.get(), "Connecting. Database: " << (default_database.empty() ? "(not specified)" : default_database) << ". User: " << user
|
||||
<< (static_cast<bool>(secure) ? ". Secure" : "") << (static_cast<bool>(compression) ? "" : ". Uncompressed") );
|
||||
|
||||
if (static_cast<bool>(encryption))
|
||||
if (static_cast<bool>(secure))
|
||||
{
|
||||
#if Poco_NetSSL_FOUND
|
||||
socket = std::make_unique<Poco::Net::SecureStreamSocket>();
|
||||
#else
|
||||
throw Exception{"tcp_ssl protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
}
|
||||
else
|
||||
|
@ -58,14 +58,14 @@ public:
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable,
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable,
|
||||
Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0))
|
||||
:
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(host, port),
|
||||
client_name(client_name_),
|
||||
compression(compression_),
|
||||
encryption(encryption_),
|
||||
secure(secure_),
|
||||
timeouts(timeouts_),
|
||||
sync_request_timeout(sync_request_timeout_),
|
||||
log_wrapper(*this)
|
||||
@ -84,7 +84,7 @@ public:
|
||||
const ConnectionTimeouts & timeouts_,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable,
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable,
|
||||
Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0))
|
||||
:
|
||||
host(host_), port(port_),
|
||||
@ -93,7 +93,7 @@ public:
|
||||
resolved_address(resolved_address_),
|
||||
client_name(client_name_),
|
||||
compression(compression_),
|
||||
encryption(encryption_),
|
||||
secure(secure_),
|
||||
timeouts(timeouts_),
|
||||
sync_request_timeout(sync_request_timeout_),
|
||||
log_wrapper(*this)
|
||||
@ -222,7 +222,7 @@ private:
|
||||
|
||||
String query_id;
|
||||
Protocol::Compression compression; /// Enable data compression for communication.
|
||||
Protocol::Encryption encryption; /// Enable data encryption for communication.
|
||||
Protocol::Secure secure; /// Enable data encryption for communication.
|
||||
|
||||
/// What compression settings to use while sending data for INSERT queries and external tables.
|
||||
CompressionSettings compression_settings;
|
||||
|
@ -51,11 +51,12 @@ public:
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable)
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable)
|
||||
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(host_, port_),
|
||||
client_name(client_name_), compression(compression_), encryption(encryption_),
|
||||
client_name(client_name_), compression(compression_),
|
||||
secure{secure_},
|
||||
timeouts(timeouts)
|
||||
{
|
||||
}
|
||||
@ -67,11 +68,12 @@ public:
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const String & client_name_ = "client",
|
||||
Protocol::Compression compression_ = Protocol::Compression::Enable,
|
||||
Protocol::Encryption encryption_ = Protocol::Encryption::Disable)
|
||||
Protocol::Secure secure_ = Protocol::Secure::Disable)
|
||||
: Base(max_connections_, &Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),
|
||||
host(host_), port(port_), default_database(default_database_),
|
||||
user(user_), password(password_), resolved_address(resolved_address_),
|
||||
client_name(client_name_), compression(compression_), encryption(encryption_),
|
||||
client_name(client_name_), compression(compression_),
|
||||
secure{secure_},
|
||||
timeouts(timeouts)
|
||||
{
|
||||
}
|
||||
@ -102,7 +104,7 @@ protected:
|
||||
return std::make_shared<Connection>(
|
||||
host, port, resolved_address,
|
||||
default_database, user, password, timeouts,
|
||||
client_name, compression, encryption);
|
||||
client_name, compression, secure);
|
||||
}
|
||||
|
||||
private:
|
||||
@ -119,7 +121,7 @@ private:
|
||||
|
||||
String client_name;
|
||||
Protocol::Compression compression; /// Whether to compress data when interacting with the server.
|
||||
Protocol::Encryption encryption; /// Whether to encrypt data when interacting with the server.
|
||||
Protocol::Secure secure; /// Whether to encrypt data when interacting with the server.
|
||||
|
||||
ConnectionTimeouts timeouts;
|
||||
};
|
||||
|
@ -116,7 +116,7 @@ namespace Protocol
|
||||
};
|
||||
|
||||
/// Whether the ssl must be used.
|
||||
enum class Encryption
|
||||
enum class Secure
|
||||
{
|
||||
Disable = 0,
|
||||
Enable = 1,
|
||||
|
@ -21,13 +21,15 @@ namespace ErrorCodes
|
||||
static const size_t MAX_CONNECTIONS = 16;
|
||||
|
||||
static ConnectionPoolWithFailoverPtr createPool(
|
||||
const std::string & host, UInt16 port, const std::string & db,
|
||||
const std::string & host, UInt16 port, bool secure, const std::string & db,
|
||||
const std::string & user, const std::string & password, const Context & context)
|
||||
{
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(context.getSettingsRef());
|
||||
ConnectionPoolPtrs pools;
|
||||
pools.emplace_back(std::make_shared<ConnectionPool>(
|
||||
MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource"));
|
||||
MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource",
|
||||
Protocol::Compression::Enable,
|
||||
secure ? Protocol::Secure::Enable : Protocol::Secure::Disable));
|
||||
return std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
|
||||
}
|
||||
|
||||
@ -41,6 +43,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
dict_struct{dict_struct_},
|
||||
host{config.getString(config_prefix + ".host")},
|
||||
port(config.getInt(config_prefix + ".port")),
|
||||
secure(config.getBool(config_prefix + ".secure", false)),
|
||||
user{config.getString(config_prefix + ".user", "")},
|
||||
password{config.getString(config_prefix + ".password", "")},
|
||||
db{config.getString(config_prefix + ".db", "")},
|
||||
@ -50,7 +53,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||
sample_block{sample_block}, context(context),
|
||||
is_local{isLocalAddress({ host, port }, config.getInt("tcp_port", 0))},
|
||||
pool{is_local ? nullptr : createPool(host, port, db, user, password, context)},
|
||||
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},
|
||||
load_all_query{query_builder.composeLoadAllQuery()}
|
||||
{}
|
||||
|
||||
@ -58,14 +61,16 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
|
||||
ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other)
|
||||
: update_time{other.update_time},
|
||||
dict_struct{other.dict_struct},
|
||||
host{other.host}, port{other.port}, user{other.user}, password{other.password},
|
||||
host{other.host}, port{other.port},
|
||||
secure{other.secure},
|
||||
user{other.user}, password{other.password},
|
||||
db{other.db}, table{other.table},
|
||||
where{other.where},
|
||||
update_field{other.update_field},
|
||||
query_builder{dict_struct, db, table, where, ExternalQueryBuilder::Backticks},
|
||||
sample_block{other.sample_block}, context(other.context),
|
||||
is_local{other.is_local},
|
||||
pool{is_local ? nullptr : createPool(host, port, db, user, password, context)},
|
||||
pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)},
|
||||
load_all_query{other.load_all_query}
|
||||
{}
|
||||
|
||||
|
@ -53,6 +53,7 @@ private:
|
||||
const DictionaryStructure dict_struct;
|
||||
const std::string host;
|
||||
const UInt16 port;
|
||||
const bool secure;
|
||||
const std::string user;
|
||||
const std::string password;
|
||||
const std::string db;
|
||||
|
@ -65,6 +65,8 @@ Cluster::Address::Address(Poco::Util::AbstractConfiguration & config, const Stri
|
||||
password = config.getString(config_prefix + ".password", "");
|
||||
default_database = config.getString(config_prefix + ".default_database", "");
|
||||
is_local = isLocal(*this, clickhouse_port);
|
||||
secure = config.getBool(config_prefix + ".secure", false) ? Protocol::Secure::Enable : Protocol::Secure::Disable;
|
||||
compression = config.getBool(config_prefix + ".compression", true) ? Protocol::Compression::Enable : Protocol::Compression::Disable;
|
||||
}
|
||||
|
||||
|
||||
@ -113,7 +115,8 @@ String Cluster::Address::toStringFull() const
|
||||
(password.empty() ? "" : (':' + escapeForFileName(password))) + '@' +
|
||||
escapeForFileName(resolved_address.host().toString()) + ':' +
|
||||
std::to_string(resolved_address.port()) +
|
||||
(default_database.empty() ? "" : ('#' + escapeForFileName(default_database)));
|
||||
(default_database.empty() ? "" : ('#' + escapeForFileName(default_database)))
|
||||
+ ((secure == Protocol::Secure::Enable) ? "+secure" : "");
|
||||
}
|
||||
|
||||
|
||||
@ -220,7 +223,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
address.host_name, address.port, address.resolved_address,
|
||||
address.default_database, address.user, address.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithoutFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
|
||||
"server", address.compression, address.secure);
|
||||
|
||||
info.pool = std::make_shared<ConnectionPoolWithFailover>(
|
||||
ConnectionPoolPtrs{pool}, settings.load_balancing, settings.connections_with_failover_max_tries);
|
||||
@ -303,7 +306,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
replica.host_name, replica.port, replica.resolved_address,
|
||||
replica.default_database, replica.user, replica.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
|
||||
"server", replica.compression, replica.secure);
|
||||
|
||||
remote_replicas_pools.emplace_back(replica_pool);
|
||||
all_replicas_pools.emplace_back(replica_pool);
|
||||
@ -367,7 +370,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
|
||||
replica.host_name, replica.port, replica.resolved_address,
|
||||
replica.default_database, replica.user, replica.password,
|
||||
ConnectionTimeouts::getTCPTimeoutsWithFailover(settings).getSaturated(settings.max_execution_time),
|
||||
"server", Protocol::Compression::Enable, Protocol::Encryption::Disable);
|
||||
"server", replica.compression, replica.secure);
|
||||
all_replicas.emplace_back(replica_pool);
|
||||
remote_replicas.emplace_back(replica_pool);
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ public:
|
||||
* <replica>
|
||||
* <host>example01-01-1</host>
|
||||
* <port>9000</port>
|
||||
* <!-- <user>, <password>, <default_database> if needed -->
|
||||
* <!-- <user>, <password>, <default_database>. <secure> if needed -->
|
||||
* </replica>
|
||||
* </shard>
|
||||
*/
|
||||
@ -60,6 +60,8 @@ public:
|
||||
String default_database; /// this database is selected when no database is specified for Distributed table
|
||||
UInt32 replica_num;
|
||||
bool is_local;
|
||||
Protocol::Compression compression = Protocol::Compression::Enable;
|
||||
Protocol::Secure secure = Protocol::Secure::Disable;
|
||||
|
||||
Address() = default;
|
||||
Address(Poco::Util::AbstractConfiguration & config, const String & config_prefix);
|
||||
|
@ -177,7 +177,7 @@ private:
|
||||
String default_database;
|
||||
String user;
|
||||
String password;
|
||||
Protocol::Encryption security;
|
||||
Protocol::Secure security;
|
||||
Protocol::Compression compression;
|
||||
ConnectionTimeouts timeouts;
|
||||
|
||||
@ -187,8 +187,8 @@ private:
|
||||
{
|
||||
bool is_secure = config.getBool("secure", false);
|
||||
security = is_secure
|
||||
? Protocol::Encryption::Enable
|
||||
: Protocol::Encryption::Disable;
|
||||
? Protocol::Secure::Enable
|
||||
: Protocol::Secure::Disable;
|
||||
|
||||
host = config.getString("host", "localhost");
|
||||
port = config.getInt("port",
|
||||
@ -1371,7 +1371,7 @@ public:
|
||||
("config-file,c", boost::program_options::value<std::string>(), "config-file path")
|
||||
("host,h", boost::program_options::value<std::string>()->default_value("localhost"), "server host")
|
||||
("port", boost::program_options::value<int>()->default_value(9000), "server port")
|
||||
("ssl,s", "ssl")
|
||||
("secure,s", "secure")
|
||||
("user,u", boost::program_options::value<std::string>(), "user")
|
||||
("password", boost::program_options::value<std::string>(), "password")
|
||||
("query_id", boost::program_options::value<std::string>(), "query_id")
|
||||
@ -1474,8 +1474,8 @@ public:
|
||||
|
||||
if (options.count("port") && !options["port"].defaulted())
|
||||
config().setInt("port", options["port"].as<int>());
|
||||
if (options.count("ssl"))
|
||||
config().setBool("ssl", true);
|
||||
if (options.count("secure"))
|
||||
config().setBool("secure", true);
|
||||
if (options.count("user"))
|
||||
config().setString("user", options["user"].as<std::string>());
|
||||
if (options.count("password"))
|
||||
|
@ -425,10 +425,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
|
||||
/// TCP with SSL
|
||||
if (config().has("tcp_ssl_port"))
|
||||
if (config().has("tcp_port_secure"))
|
||||
{
|
||||
#if Poco_NetSSL_FOUND
|
||||
Poco::Net::SocketAddress tcp_address = make_socket_address(listen_host, config().getInt("tcp_ssl_port"));
|
||||
Poco::Net::SocketAddress tcp_address = make_socket_address(listen_host, config().getInt("tcp_port_secure"));
|
||||
Poco::Net::SecureServerSocket tcp_socket(tcp_address);
|
||||
tcp_socket.setReceiveTimeout(settings.receive_timeout);
|
||||
tcp_socket.setSendTimeout(settings.send_timeout);
|
||||
@ -437,7 +437,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
server_pool,
|
||||
tcp_socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening tcp_ssl: " + tcp_address.toString());
|
||||
LOG_INFO(log, "Listening tcp_secure: " + tcp_address.toString());
|
||||
#else
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
|
@ -16,10 +16,10 @@
|
||||
<!-- For HTTPS and SSL over native protocol. -->
|
||||
<!--
|
||||
<https_port>8443</https_port>
|
||||
<tcp_ssl_port>9440</tcp_ssl_port>
|
||||
<tcp_port_secure>9440</tcp_port_secure>
|
||||
-->
|
||||
|
||||
<!-- Used with https_port and tcp_ssl_port. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
|
||||
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
|
||||
@ -150,6 +150,15 @@
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>9440</port>
|
||||
<secure>1</secure>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost_secure>
|
||||
</remote_servers>
|
||||
|
||||
|
||||
|
@ -52,6 +52,14 @@ namespace
|
||||
const char * address_begin = static_cast<const char*>(address.data());
|
||||
const char * address_end = address_begin + address.size();
|
||||
|
||||
Protocol::Secure secure = Protocol::Secure::Disable;
|
||||
const char * secure_tag = "+secure";
|
||||
if (endsWith(address, secure_tag))
|
||||
{
|
||||
address_end -= strlen(secure_tag);
|
||||
secure = Protocol::Secure::Enable;
|
||||
}
|
||||
|
||||
const char * user_pw_end = strchr(address.data(), '@');
|
||||
const char * colon = strchr(address.data(), ':');
|
||||
if (!user_pw_end || !colon)
|
||||
@ -76,7 +84,7 @@ namespace
|
||||
const auto database = has_db ? unescapeForFileName(std::string(has_db + 1, address_end))
|
||||
: std::string();
|
||||
|
||||
pools.emplace_back(factory(host, port, user, password, database));
|
||||
pools.emplace_back(factory(host, port, secure, user, password, database));
|
||||
}
|
||||
|
||||
return pools;
|
||||
@ -152,13 +160,16 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
|
||||
{
|
||||
auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.context.getSettingsRef());
|
||||
const auto pool_factory = [&storage, &name, &timeouts] (const std::string & host, const UInt16 port,
|
||||
const Protocol::Secure secure,
|
||||
const std::string & user, const std::string & password,
|
||||
const std::string & default_database)
|
||||
{
|
||||
return std::make_shared<ConnectionPool>(
|
||||
1, host, port, default_database,
|
||||
user, password, timeouts,
|
||||
storage.getName() + '_' + name);
|
||||
storage.getName() + '_' + name,
|
||||
Protocol::Compression::Enable,
|
||||
secure);
|
||||
};
|
||||
|
||||
auto pools = createPoolsForAddresses(name, pool_factory);
|
||||
|
@ -1,7 +1,7 @@
|
||||
<!-- Config for connecting to test server -->
|
||||
<yandex>
|
||||
<tcp_port>59000</tcp_port>
|
||||
<tcp_ssl_port>59440</tcp_ssl_port>
|
||||
<tcp_port_secure>59440</tcp_port_secure>
|
||||
<openSSL>
|
||||
<client>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
|
22
dbms/tests/queries/0_stateless/00505_distributed_secure.data
Normal file
22
dbms/tests/queries/0_stateless/00505_distributed_secure.data
Normal file
@ -0,0 +1,22 @@
|
||||
DROP TABLE IF EXISTS test.secure1;
|
||||
DROP TABLE IF EXISTS test.secure2;
|
||||
DROP TABLE IF EXISTS test.secure3;
|
||||
|
||||
CREATE TABLE test.secure1 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = MergeTree(date, (a, date), 8192);
|
||||
CREATE TABLE test.secure2 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost_secure, 'test', 'secure1');
|
||||
CREATE TABLE test.secure3 ( date Date, a Int32, b Int32, c Int32, d Int32) ENGINE = Distributed(test_shard_localhost_secure, 'test', 'secure2');
|
||||
|
||||
INSERT INTO test.secure1 VALUES (1, 2, 3, 4, 5);
|
||||
INSERT INTO test.secure1 VALUES (11,12,13,14,15);
|
||||
INSERT INTO test.secure2 VALUES (21,22,23,24,25);
|
||||
INSERT INTO test.secure3 VALUES (31,32,33,34,35);
|
||||
|
||||
SELECT sleep(1);
|
||||
|
||||
SELECT * FROM test.secure1 ORDER BY a;
|
||||
SELECT * FROM test.secure2 ORDER BY a;
|
||||
SELECT * FROM test.secure3 ORDER BY a;
|
||||
|
||||
DROP TABLE test.secure1;
|
||||
DROP TABLE test.secure2;
|
||||
DROP TABLE test.secure3;
|
17
dbms/tests/queries/0_stateless/00505_secure.reference
Normal file
17
dbms/tests/queries/0_stateless/00505_secure.reference
Normal file
@ -0,0 +1,17 @@
|
||||
1
|
||||
2
|
||||
3
|
||||
1
|
||||
0
|
||||
1970-01-02 2 3 4 5
|
||||
1970-01-12 12 13 14 15
|
||||
1970-01-22 22 23 24 25
|
||||
1970-02-01 32 33 34 35
|
||||
1970-01-02 2 3 4 5
|
||||
1970-01-12 12 13 14 15
|
||||
1970-01-22 22 23 24 25
|
||||
1970-02-01 32 33 34 35
|
||||
1970-01-02 2 3 4 5
|
||||
1970-01-12 12 13 14 15
|
||||
1970-01-22 22 23 24 25
|
||||
1970-02-01 32 33 34 35
|
22
dbms/tests/queries/0_stateless/00505_secure.sh
Executable file
22
dbms/tests/queries/0_stateless/00505_secure.sh
Executable file
@ -0,0 +1,22 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
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_secure.reference
|
||||
else
|
||||
# Auto port detect
|
||||
${CLICKHOUSE_CLIENT} --secure -q "SELECT 1";
|
||||
${CLICKHOUSE_CLIENT} --secure --port=${CLICKHOUSE_PORT_TCP_SECURE} -q "SELECT 2";
|
||||
|
||||
${CLICKHOUSE_CURL} -sS --insecure ${CLICKHOUSE_URL_HTTPS}?query=SELECT%203
|
||||
|
||||
${CLICKHOUSE_CLIENT} --secure -q "SELECT 1";
|
||||
|
||||
cat $CURDIR/00505_distributed_secure.data | $CLICKHOUSE_CLIENT --secure -n -m
|
||||
fi
|
@ -1,3 +0,0 @@
|
||||
1
|
||||
2
|
||||
3
|
@ -1,18 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. $CURDIR/../shell_config.sh
|
||||
|
||||
# Not default server config needed
|
||||
|
||||
tcp_ssl_port=`${CLICKHOUSE_EXTRACT_CONFIG} -k tcp_ssl_port 2>/dev/null`
|
||||
if [ -z ${tcp_ssl_port} ]; then
|
||||
# Secure port disabled. Fake result
|
||||
cat $CURDIR/00505_tcp_ssl.reference
|
||||
else
|
||||
# Auto port detect
|
||||
${CLICKHOUSE_CLIENT} --ssl -q "SELECT 1";
|
||||
${CLICKHOUSE_CLIENT} --ssl --port=${CLICKHOUSE_PORT_TCP_SSL} -q "SELECT 2";
|
||||
|
||||
${CLICKHOUSE_CURL} -sS --insecure ${CLICKHOUSE_URL_HTTPS}?query=SELECT%203
|
||||
fi
|
@ -10,8 +10,8 @@ export CLICKHOUSE_CONFIG_GREP=${CLICKHOUSE_CONFIG_GREP:="/etc/clickhouse-server/
|
||||
export CLICKHOUSE_HOST=${CLICKHOUSE_HOST:="localhost"}
|
||||
export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:=`${CLICKHOUSE_EXTRACT_CONFIG} --try --key=tcp_port 2>/dev/null`} 2>/dev/null
|
||||
export CLICKHOUSE_PORT_TCP=${CLICKHOUSE_PORT_TCP:="9000"}
|
||||
export CLICKHOUSE_PORT_TCP_SSL=${CLICKHOUSE_PORT_TCP_SSL:=`${CLICKHOUSE_EXTRACT_CONFIG} --try --key=tcp_ssl_port 2>/dev/null`} 2>/dev/null
|
||||
export CLICKHOUSE_PORT_TCP_SSL=${CLICKHOUSE_PORT_TCP_SSL:="9440"}
|
||||
export CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:=`${CLICKHOUSE_EXTRACT_CONFIG} --try --key=tcp_port_secure 2>/dev/null`} 2>/dev/null
|
||||
export CLICKHOUSE_PORT_TCP_SECURE=${CLICKHOUSE_PORT_TCP_SECURE:="9440"}
|
||||
export CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:=`${CLICKHOUSE_EXTRACT_CONFIG} --key=http_port 2>/dev/null`}
|
||||
export CLICKHOUSE_PORT_HTTP=${CLICKHOUSE_PORT_HTTP:="8123"}
|
||||
export CLICKHOUSE_PORT_HTTPS=${CLICKHOUSE_PORT_HTTPS:=`${CLICKHOUSE_EXTRACT_CONFIG} --try --key=https_port 2>/dev/null`} 2>/dev/null
|
||||
|
@ -7,6 +7,7 @@
|
||||
<errorlog>/tmp/clickhouse/log/clickhouse-server.err.log</errorlog>
|
||||
<size>10M</size>
|
||||
<count>1</count>
|
||||
<compress>0</compress>
|
||||
</logger>
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
<listen_host>::</listen_host>
|
||||
@ -14,7 +15,7 @@
|
||||
<http_port>58123</http_port>
|
||||
<tcp_port>59000</tcp_port>
|
||||
<https_port>58443</https_port>
|
||||
<tcp_ssl_port>59440</tcp_ssl_port>
|
||||
<tcp_port_secure>59440</tcp_port_secure>
|
||||
<interserver_http_port>59009</interserver_http_port>
|
||||
<openSSL>
|
||||
<server> <!-- Used for https server AND secure tcp port -->
|
||||
@ -60,6 +61,15 @@
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost>
|
||||
<test_shard_localhost_secure>
|
||||
<shard>
|
||||
<replica>
|
||||
<host>localhost</host>
|
||||
<port>59440</port>
|
||||
<secure>1</secure>
|
||||
</replica>
|
||||
</shard>
|
||||
</test_shard_localhost_secure>
|
||||
</remote_servers>
|
||||
<zookeeper incl="zookeeper-servers" optional="true" />
|
||||
<macros incl="macros" optional="true" />
|
||||
|
4
debian/pbuilder-hooks/B90test-server
vendored
4
debian/pbuilder-hooks/B90test-server
vendored
@ -23,12 +23,12 @@ if [ "${TEST_CONNECT}" ]; then
|
||||
|
||||
TEST_SSL=${TEST_SSL=1}
|
||||
if [ "${TEST_SSL}" ]; then
|
||||
echo "<yandex><https_port>8443</https_port><tcp_ssl_port>9440</tcp_ssl_port></yandex>" > /etc/clickhouse-server/config.d/ssl.xml
|
||||
echo "<yandex><https_port>8443</https_port><tcp_port_secure>9440</tcp_port_secure></yandex>" > /etc/clickhouse-server/config.d/ssl.xml
|
||||
echo "<yandex><openSSL><client><verificationMode>none</verificationMode><invalidCertificateHandler><name>AcceptCertificateHandler</name></invalidCertificateHandler></client></openSSL></yandex>" > /etc/clickhouse-client/config.d/ssl.xml
|
||||
|
||||
openssl dhparam -out /etc/clickhouse-server/dhparam.pem 256
|
||||
openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt
|
||||
CLIENT_ADD="--ssl"
|
||||
CLIENT_ADD="--secure"
|
||||
fi
|
||||
|
||||
function finish {
|
||||
|
Loading…
Reference in New Issue
Block a user