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:
proller 2018-03-28 18:41:06 -07:00 committed by alexey-milovidov
parent 54986af313
commit a6f9df105c
21 changed files with 149 additions and 65 deletions

View File

@ -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

View File

@ -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;

View File

@ -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;
};

View File

@ -116,7 +116,7 @@ namespace Protocol
};
/// Whether the ssl must be used.
enum class Encryption
enum class Secure
{
Disable = 0,
Enable = 1,

View File

@ -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}
{}

View File

@ -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;

View File

@ -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);
}

View File

@ -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);

View File

@ -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"))

View File

@ -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};

View File

@ -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>

View File

@ -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);

View File

@ -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>

View 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;

View 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

View 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

View File

@ -1,3 +0,0 @@
1
2
3

View File

@ -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

View File

@ -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

View File

@ -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" />

View File

@ -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 {