mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-15 19:02:04 +00:00
Merge branch 'rocksdb_metacache' of https://github.com/bigo-sg/ClickHouse into rocksdb_metacache
This commit is contained in:
commit
e070cdc247
@ -0,0 +1,15 @@
|
||||
---
|
||||
title: How do I contribute code to ClickHouse?
|
||||
toc_hidden: true
|
||||
toc_priority: 120
|
||||
---
|
||||
|
||||
# How do I contribute code to ClickHouse? {#how-do-i-contribute-code-to-clickhouse}
|
||||
|
||||
ClickHouse is an open-source project [developed on GitHub](https://github.com/ClickHouse/ClickHouse).
|
||||
|
||||
As customary, contribution instructions are published in [CONTRIBUTING.md](https://github.com/ClickHouse/ClickHouse/blob/master/CONTRIBUTING.md) file in the root of the source code repository.
|
||||
|
||||
If you want to suggest a substantial change to ClickHouse, consider [opening a GitHub issue](https://github.com/ClickHouse/ClickHouse/issues/new/choose) explaining what you want to do, to discuss it with maintainers and community first. [Examples of such RFC issues](https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aissue+is%3Aopen+rfc).
|
||||
|
||||
If your contributions are security related, please check out [our security policy](https://github.com/ClickHouse/ClickHouse/security/policy/) too.
|
@ -17,6 +17,7 @@ Questions:
|
||||
- [What is OLAP?](../../faq/general/olap.md)
|
||||
- [What is a columnar database?](../../faq/general/columnar-database.md)
|
||||
- [Why not use something like MapReduce?](../../faq/general/mapreduce.md)
|
||||
- [How do I contribute code to ClickHouse?](../../faq/general/how-do-i-contribute-code-to-clickhouse.md)
|
||||
|
||||
!!! info "Don’t see what you were looking for?"
|
||||
Check out [other F.A.Q. categories](../../faq/index.md) or browse around main documentation articles found in the left sidebar.
|
||||
|
@ -6,7 +6,7 @@ toc_title: Client Libraries
|
||||
# Client Libraries from Third-party Developers {#client-libraries-from-third-party-developers}
|
||||
|
||||
!!! warning "Disclaimer"
|
||||
Yandex does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality.
|
||||
ClickHouse Inc does **not** maintain the libraries listed below and hasn’t done any extensive testing to ensure their quality.
|
||||
|
||||
- Python
|
||||
- [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm)
|
||||
|
@ -817,9 +817,19 @@ If the number of rows to be read from a file of a [MergeTree](../../engines/tabl
|
||||
|
||||
Possible values:
|
||||
|
||||
- Any positive integer.
|
||||
- Positive integer.
|
||||
|
||||
Default value: 163840.
|
||||
Default value: `163840`.
|
||||
|
||||
## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem}
|
||||
|
||||
The minimum number of lines to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem.
|
||||
|
||||
Possible values:
|
||||
|
||||
- Positive integer.
|
||||
|
||||
Default value: `163840`.
|
||||
|
||||
## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read}
|
||||
|
||||
@ -827,9 +837,19 @@ If the number of bytes to read from one file of a [MergeTree](../../engines/tabl
|
||||
|
||||
Possible value:
|
||||
|
||||
- Any positive integer.
|
||||
- Positive integer.
|
||||
|
||||
Default value: 251658240.
|
||||
Default value: `251658240`.
|
||||
|
||||
## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem}
|
||||
|
||||
The minimum number of bytes to read from one file before [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) engine can parallelize reading, when reading from remote filesystem.
|
||||
|
||||
Possible values:
|
||||
|
||||
- Positive integer.
|
||||
|
||||
Default value: `251658240`.
|
||||
|
||||
## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek}
|
||||
|
||||
|
@ -24,9 +24,7 @@ Function:
|
||||
|
||||
- Calculates a hash for all parameters in the aggregate, then uses it in calculations.
|
||||
|
||||
- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values up to 65536.
|
||||
|
||||
This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions.
|
||||
- Uses an adaptive sampling algorithm. For the calculation state, the function uses a sample of element hash values up to 65536. This algorithm is very accurate and very efficient on the CPU. When the query contains several of these functions, using `uniq` is almost as fast as using other aggregate functions.
|
||||
|
||||
- Provides the result deterministically (it does not depend on the query processing order).
|
||||
|
||||
|
@ -11,9 +11,7 @@ Aggregate functions can have an implementation-defined intermediate state that c
|
||||
|
||||
**Parameters**
|
||||
|
||||
- Name of the aggregate function.
|
||||
|
||||
If the function is parametric, specify its parameters too.
|
||||
- Name of the aggregate function. If the function is parametric, specify its parameters too.
|
||||
|
||||
- Types of the aggregate function arguments.
|
||||
|
||||
|
1
docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
1
docs/ja/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md
|
1
docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
1
docs/ru/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md
|
@ -761,9 +761,20 @@ ClickHouse может парсить только базовый формат `Y
|
||||
|
||||
Возможные значения:
|
||||
|
||||
- Любое положительное целое число.
|
||||
- Положительное целое число.
|
||||
|
||||
Значение по умолчанию: 163840.
|
||||
Значение по умолчанию: `163840`.
|
||||
|
||||
|
||||
## merge_tree_min_rows_for_concurrent_read_for_remote_filesystem {#merge-tree-min-rows-for-concurrent-read-for-remote-filesystem}
|
||||
|
||||
Минимальное количество строк для чтения из одного файла, прежде чем движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы.
|
||||
|
||||
Возможные значения:
|
||||
|
||||
- Положительное целое число.
|
||||
|
||||
Значение по умолчанию: `163840`.
|
||||
|
||||
## merge_tree_min_bytes_for_concurrent_read {#setting-merge-tree-min-bytes-for-concurrent-read}
|
||||
|
||||
@ -773,7 +784,17 @@ ClickHouse может парсить только базовый формат `Y
|
||||
|
||||
- Положительное целое число.
|
||||
|
||||
Значение по умолчанию: 251658240.
|
||||
Значение по умолчанию: `251658240`.
|
||||
|
||||
## merge_tree_min_bytes_for_concurrent_read_for_remote_filesystem {#merge-tree-min-bytes-for-concurrent-read-for-remote-filesystem}
|
||||
|
||||
Минимальное количество байтов для чтения из одного файла, прежде чем движок [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) может выполнять параллельное чтение из удаленной файловой системы.
|
||||
|
||||
Возможное значение:
|
||||
|
||||
- Положительное целое число.
|
||||
|
||||
Значение по умолчанию: `251658240`.
|
||||
|
||||
## merge_tree_min_rows_for_seek {#setting-merge-tree-min-rows-for-seek}
|
||||
|
||||
|
1
docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
1
docs/zh/faq/general/how-do-i-contribute-code-to-clickhouse.md
Symbolic link
@ -0,0 +1 @@
|
||||
../../../en/faq/general/how-do-i-contribute-code-to-clickhouse.md
|
@ -6,6 +6,7 @@
|
||||
#include <Interpreters/DNSCacheUpdater.h>
|
||||
#include <Coordination/Defines.h>
|
||||
#include <Common/Config/ConfigReloader.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <filesystem>
|
||||
#include <IO/UseSSL.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
@ -379,11 +380,11 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(
|
||||
listen_host,
|
||||
port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString());
|
||||
"Keeper (tcp): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, false), server_pool, socket));
|
||||
});
|
||||
|
||||
const char * secure_port_name = "keeper_server.tcp_port_secure";
|
||||
@ -395,10 +396,11 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(
|
||||
listen_host,
|
||||
secure_port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString());
|
||||
"Keeper with secure protocol (tcp_secure): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, true), server_pool, socket));
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
@ -408,7 +410,10 @@ int Keeper::main(const std::vector<std::string> & /*args*/)
|
||||
}
|
||||
|
||||
for (auto & server : *servers)
|
||||
{
|
||||
server.start();
|
||||
LOG_INFO(log, "Listening for {}", server.getDescription());
|
||||
}
|
||||
|
||||
zkutil::EventPtr unused_event = std::make_shared<Poco::Event>();
|
||||
zkutil::ZooKeeperNodeCache unused_cache([] { return nullptr; });
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/getExecutablePath.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ThreadProfileEvents.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/getMappedArea.h>
|
||||
@ -70,6 +71,7 @@
|
||||
#include "MetricsTransmitter.h"
|
||||
#include <Common/StatusFile.h>
|
||||
#include <Server/TCPHandlerFactory.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <Common/SensitiveDataMasker.h>
|
||||
#include <Common/ThreadFuzzer.h>
|
||||
#include <Common/getHashOfLoadedBinary.h>
|
||||
@ -127,6 +129,11 @@ namespace CurrentMetrics
|
||||
extern const Metric MaxPushedDDLEntryID;
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MainConfigLoads;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
#if USE_JEMALLOC
|
||||
@ -344,16 +351,53 @@ Poco::Net::SocketAddress Server::socketBindListen(Poco::Net::ServerSocket & sock
|
||||
return address;
|
||||
}
|
||||
|
||||
void Server::createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const
|
||||
std::vector<std::string> getListenHosts(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
auto listen_hosts = DB::getMultipleValuesFromConfig(config, "", "listen_host");
|
||||
if (listen_hosts.empty())
|
||||
{
|
||||
listen_hosts.emplace_back("::1");
|
||||
listen_hosts.emplace_back("127.0.0.1");
|
||||
}
|
||||
return listen_hosts;
|
||||
}
|
||||
|
||||
bool getListenTry(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
bool listen_try = config.getBool("listen_try", false);
|
||||
if (!listen_try)
|
||||
listen_try = DB::getMultipleValuesFromConfig(config, "", "listen_host").empty();
|
||||
return listen_try;
|
||||
}
|
||||
|
||||
|
||||
void Server::createServer(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & listen_host,
|
||||
const char * port_name,
|
||||
bool listen_try,
|
||||
bool start_server,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
CreateServerFunc && func) const
|
||||
{
|
||||
/// For testing purposes, user may omit tcp_port or http_port or https_port in configuration file.
|
||||
if (!config().has(port_name))
|
||||
if (config.getString(port_name, "").empty())
|
||||
return;
|
||||
|
||||
auto port = config().getInt(port_name);
|
||||
/// If we already have an active server for this listen_host/port_name, don't create it again
|
||||
for (const auto & server : servers)
|
||||
if (!server.isStopping() && server.getListenHost() == listen_host && server.getPortName() == port_name)
|
||||
return;
|
||||
|
||||
auto port = config.getInt(port_name);
|
||||
try
|
||||
{
|
||||
func(port);
|
||||
servers.push_back(func(port));
|
||||
if (start_server)
|
||||
{
|
||||
servers.back().start();
|
||||
LOG_INFO(&logger(), "Listening for {}", servers.back().getDescription());
|
||||
}
|
||||
global_context->registerServerPort(port_name, port);
|
||||
}
|
||||
catch (const Poco::Exception &)
|
||||
@ -515,6 +559,27 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
config().getUInt("thread_pool_queue_size", 10000)
|
||||
);
|
||||
|
||||
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
|
||||
std::mutex servers_lock;
|
||||
std::vector<ProtocolServerAdapter> servers;
|
||||
std::vector<ProtocolServerAdapter> servers_to_start_before_tables;
|
||||
/// This object will periodically calculate some metrics.
|
||||
AsynchronousMetrics async_metrics(
|
||||
global_context, config().getUInt("asynchronous_metrics_update_period_s", 1),
|
||||
[&]() -> std::vector<ProtocolServerMetrics>
|
||||
{
|
||||
std::vector<ProtocolServerMetrics> metrics;
|
||||
metrics.reserve(servers_to_start_before_tables.size());
|
||||
for (const auto & server : servers_to_start_before_tables)
|
||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
|
||||
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (const auto & server : servers)
|
||||
metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()});
|
||||
return metrics;
|
||||
}
|
||||
);
|
||||
|
||||
ConnectionCollector::init(global_context, config().getUInt("max_threads_for_connection_collector", 10));
|
||||
|
||||
bool has_zookeeper = config().has("zookeeper");
|
||||
@ -883,12 +948,17 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
global_context->reloadZooKeeperIfChanged(config);
|
||||
|
||||
global_context->reloadAuxiliaryZooKeepersConfigIfChanged(config);
|
||||
|
||||
std::lock_guard lock(servers_lock);
|
||||
updateServers(*config, server_pool, async_metrics, servers);
|
||||
}
|
||||
|
||||
global_context->updateStorageConfiguration(*config);
|
||||
global_context->updateInterserverCredentials(*config);
|
||||
|
||||
CompressionCodecEncrypted::Configuration::instance().tryLoad(*config, "encryption_codecs");
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::MainConfigLoads);
|
||||
},
|
||||
/* already_loaded = */ false); /// Reload it right now (initial loading)
|
||||
|
||||
@ -1000,24 +1070,8 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
/// try set up encryption. There are some errors in config, error will be printed and server wouldn't start.
|
||||
CompressionCodecEncrypted::Configuration::instance().load(config(), "encryption_codecs");
|
||||
|
||||
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);
|
||||
|
||||
Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024));
|
||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||
http_params->setTimeout(settings.http_receive_timeout);
|
||||
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
||||
|
||||
auto servers_to_start_before_tables = std::make_shared<std::vector<ProtocolServerAdapter>>();
|
||||
|
||||
std::vector<std::string> listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host");
|
||||
|
||||
bool listen_try = config().getBool("listen_try", false);
|
||||
if (listen_hosts.empty())
|
||||
{
|
||||
listen_hosts.emplace_back("::1");
|
||||
listen_hosts.emplace_back("127.0.0.1");
|
||||
listen_try = true;
|
||||
}
|
||||
const auto listen_hosts = getListenHosts(config());
|
||||
const auto listen_try = getListenTry(config());
|
||||
|
||||
if (config().has("keeper_server"))
|
||||
{
|
||||
@ -1040,39 +1094,46 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
{
|
||||
/// TCP Keeper
|
||||
const char * port_name = "keeper_server.tcp_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers_to_start_before_tables->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, false), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString());
|
||||
});
|
||||
createServer(
|
||||
config(), listen_host, port_name, listen_try, /* start_server: */ false,
|
||||
servers_to_start_before_tables,
|
||||
[&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"Keeper (tcp): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, false), server_pool, socket));
|
||||
});
|
||||
|
||||
const char * secure_port_name = "keeper_server.tcp_port_secure";
|
||||
createServer(listen_host, secure_port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
createServer(
|
||||
config(), listen_host, secure_port_name, listen_try, /* start_server: */ false,
|
||||
servers_to_start_before_tables,
|
||||
[&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers_to_start_before_tables->emplace_back(
|
||||
secure_port_name,
|
||||
std::make_unique<Poco::Net::TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, true), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening for connections to Keeper with secure protocol (tcp_secure): {}", address.toString());
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
secure_port_name,
|
||||
"Keeper with secure protocol (tcp_secure): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new KeeperTCPHandlerFactory(*this, true), server_pool, socket));
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
});
|
||||
}
|
||||
#else
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "ClickHouse server built without NuRaft library. Cannot use internal coordination.");
|
||||
@ -1080,14 +1141,19 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
|
||||
}
|
||||
|
||||
for (auto & server : *servers_to_start_before_tables)
|
||||
for (auto & server : servers_to_start_before_tables)
|
||||
{
|
||||
server.start();
|
||||
LOG_INFO(log, "Listening for {}", server.getDescription());
|
||||
}
|
||||
|
||||
SCOPE_EXIT({
|
||||
/// Stop reloading of the main config. This must be done before `global_context->shutdown()` because
|
||||
/// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart.
|
||||
main_config_reloader.reset();
|
||||
|
||||
async_metrics.stop();
|
||||
|
||||
/** Ask to cancel background jobs all table engines,
|
||||
* and also query_log.
|
||||
* It is important to do early, not in destructor of Context, because
|
||||
@ -1099,11 +1165,11 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
|
||||
LOG_DEBUG(log, "Shut down storages.");
|
||||
|
||||
if (!servers_to_start_before_tables->empty())
|
||||
if (!servers_to_start_before_tables.empty())
|
||||
{
|
||||
LOG_DEBUG(log, "Waiting for current connections to servers for tables to finish.");
|
||||
int current_connections = 0;
|
||||
for (auto & server : *servers_to_start_before_tables)
|
||||
for (auto & server : servers_to_start_before_tables)
|
||||
{
|
||||
server.stop();
|
||||
current_connections += server.currentConnections();
|
||||
@ -1115,7 +1181,7 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
LOG_INFO(log, "Closed all listening sockets.");
|
||||
|
||||
if (current_connections > 0)
|
||||
current_connections = waitServersToFinish(*servers_to_start_before_tables, config().getInt("shutdown_wait_unfinished", 5));
|
||||
current_connections = waitServersToFinish(servers_to_start_before_tables, config().getInt("shutdown_wait_unfinished", 5));
|
||||
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed connections to servers for tables. But {} remain. Probably some tables of other users cannot finish their connections after context shutdown.", current_connections);
|
||||
@ -1269,223 +1335,18 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
|
||||
#endif
|
||||
|
||||
auto servers = std::make_shared<std::vector<ProtocolServerAdapter>>();
|
||||
{
|
||||
/// This object will periodically calculate some metrics.
|
||||
AsynchronousMetrics async_metrics(
|
||||
global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), servers_to_start_before_tables, servers);
|
||||
attachSystemTablesAsync(global_context, *DatabaseCatalog::instance().getSystemDatabase(), async_metrics);
|
||||
|
||||
for (const auto & listen_host : listen_hosts)
|
||||
{
|
||||
/// HTTP
|
||||
const char * port_name = "http_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
|
||||
servers->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for http://{}", address.toString());
|
||||
});
|
||||
|
||||
/// HTTPS
|
||||
port_name = "https_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
servers->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for https://{}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
/// TCP
|
||||
port_name = "tcp_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections with native protocol (tcp): {}", address.toString());
|
||||
});
|
||||
|
||||
/// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt
|
||||
port_name = "tcp_with_proxy_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for connections with native protocol (tcp) with PROXY: {}", address.toString());
|
||||
});
|
||||
|
||||
/// TCP with SSL
|
||||
port_name = "tcp_port_secure";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
LOG_INFO(log, "Listening for connections with secure native protocol (tcp_secure): {}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
/// Interserver IO HTTP
|
||||
port_name = "interserver_http_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
servers->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for replica communication (interserver): http://{}", address.toString());
|
||||
});
|
||||
|
||||
port_name = "interserver_https_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
servers->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for secure replica communication (interserver): https://{}", address.toString());
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
port_name = "mysql_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(Poco::Timespan());
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
|
||||
new MySQLHandlerFactory(*this),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for MySQL compatibility protocol: {}", address.toString());
|
||||
});
|
||||
|
||||
port_name = "postgresql_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(Poco::Timespan());
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
servers->emplace_back(port_name, std::make_unique<Poco::Net::TCPServer>(
|
||||
new PostgreSQLHandlerFactory(*this),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
|
||||
LOG_INFO(log, "Listening for PostgreSQL compatibility protocol: " + address.toString());
|
||||
});
|
||||
|
||||
#if USE_GRPC
|
||||
port_name = "grpc_port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::SocketAddress server_address(listen_host, port);
|
||||
servers->emplace_back(port_name, std::make_unique<GRPCServer>(*this, makeSocketAddress(listen_host, port, log)));
|
||||
LOG_INFO(log, "Listening for gRPC protocol: " + server_address.toString());
|
||||
});
|
||||
#endif
|
||||
|
||||
/// Prometheus (if defined and not setup yet with http_port)
|
||||
port_name = "prometheus.port";
|
||||
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
servers->emplace_back(
|
||||
port_name,
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
|
||||
LOG_INFO(log, "Listening for Prometheus: http://{}", address.toString());
|
||||
});
|
||||
std::lock_guard lock(servers_lock);
|
||||
createServers(config(), listen_hosts, listen_try, server_pool, async_metrics, servers);
|
||||
if (servers.empty())
|
||||
throw Exception(
|
||||
"No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
|
||||
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
}
|
||||
|
||||
if (servers->empty())
|
||||
throw Exception("No servers started (add valid listen_host and 'tcp_port' or 'http_port' to configuration file.)",
|
||||
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
|
||||
|
||||
/// Must be done after initialization of `servers`, because async_metrics will access `servers` variable from its thread.
|
||||
async_metrics.start();
|
||||
|
||||
{
|
||||
@ -1564,9 +1425,15 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
&CurrentMetrics::MaxDDLEntryID, &CurrentMetrics::MaxPushedDDLEntryID));
|
||||
}
|
||||
|
||||
for (auto & server : *servers)
|
||||
server.start();
|
||||
LOG_INFO(log, "Ready for connections.");
|
||||
{
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (auto & server : servers)
|
||||
{
|
||||
server.start();
|
||||
LOG_INFO(log, "Listening for {}", server.getDescription());
|
||||
}
|
||||
LOG_INFO(log, "Ready for connections.");
|
||||
}
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
LOG_DEBUG(log, "Received termination signal.");
|
||||
@ -1575,10 +1442,13 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
is_cancelled = true;
|
||||
|
||||
int current_connections = 0;
|
||||
for (auto & server : *servers)
|
||||
{
|
||||
server.stop();
|
||||
current_connections += server.currentConnections();
|
||||
std::lock_guard lock(servers_lock);
|
||||
for (auto & server : servers)
|
||||
{
|
||||
server.stop();
|
||||
current_connections += server.currentConnections();
|
||||
}
|
||||
}
|
||||
|
||||
if (current_connections)
|
||||
@ -1591,7 +1461,7 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
global_context->getProcessList().killAllQueries();
|
||||
|
||||
if (current_connections)
|
||||
current_connections = waitServersToFinish(*servers, config().getInt("shutdown_wait_unfinished", 5));
|
||||
current_connections = waitServersToFinish(servers, config().getInt("shutdown_wait_unfinished", 5));
|
||||
|
||||
if (current_connections)
|
||||
LOG_INFO(log, "Closed connections. But {} remain."
|
||||
@ -1627,4 +1497,273 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
|
||||
return Application::EXIT_OK;
|
||||
}
|
||||
|
||||
void Server::createServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const std::vector<std::string> & listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
bool start_servers)
|
||||
{
|
||||
const Settings & settings = global_context->getSettingsRef();
|
||||
|
||||
Poco::Timespan keep_alive_timeout(config.getUInt("keep_alive_timeout", 10), 0);
|
||||
Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams;
|
||||
http_params->setTimeout(settings.http_receive_timeout);
|
||||
http_params->setKeepAliveTimeout(keep_alive_timeout);
|
||||
|
||||
for (const auto & listen_host : listen_hosts)
|
||||
{
|
||||
/// HTTP
|
||||
const char * port_name = "http_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPHandler-factory"), server_pool, socket, http_params));
|
||||
});
|
||||
|
||||
/// HTTPS
|
||||
port_name = "https_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"https://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "HTTPSHandler-factory"), server_pool, socket, http_params));
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"HTTPS protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
/// TCP
|
||||
port_name = "tcp_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"native protocol (tcp): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ false),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
});
|
||||
|
||||
/// TCP with PROXY protocol, see https://github.com/wolfeidau/proxyv2/blob/master/docs/proxy-protocol.txt
|
||||
port_name = "tcp_with_proxy_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"native protocol (tcp) with PROXY: " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ false, /* proxy protocol */ true),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
});
|
||||
|
||||
/// TCP with SSL
|
||||
port_name = "tcp_port_secure";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.receive_timeout);
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"secure native protocol (tcp_secure): " + address.toString(),
|
||||
std::make_unique<TCPServer>(
|
||||
new TCPHandlerFactory(*this, /* secure */ true, /* proxy protocol */ false),
|
||||
server_pool,
|
||||
socket,
|
||||
new Poco::Net::TCPServerParams));
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
/// Interserver IO HTTP
|
||||
port_name = "interserver_http_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"replica communication (interserver): http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
});
|
||||
|
||||
port_name = "interserver_https_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
#if USE_SSL
|
||||
Poco::Net::SecureServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"secure replica communication (interserver): https://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(),
|
||||
createHandlerFactory(*this, async_metrics, "InterserverIOHTTPSHandler-factory"),
|
||||
server_pool,
|
||||
socket,
|
||||
http_params));
|
||||
#else
|
||||
UNUSED(port);
|
||||
throw Exception{"SSL support for TCP protocol is disabled because Poco library was built without NetSSL support.",
|
||||
ErrorCodes::SUPPORT_IS_DISABLED};
|
||||
#endif
|
||||
});
|
||||
|
||||
port_name = "mysql_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(Poco::Timespan());
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"MySQL compatibility protocol: " + address.toString(),
|
||||
std::make_unique<TCPServer>(new MySQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
});
|
||||
|
||||
port_name = "postgresql_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port, /* secure = */ true);
|
||||
socket.setReceiveTimeout(Poco::Timespan());
|
||||
socket.setSendTimeout(settings.send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"PostgreSQL compatibility protocol: " + address.toString(),
|
||||
std::make_unique<TCPServer>(new PostgreSQLHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
|
||||
});
|
||||
|
||||
#if USE_GRPC
|
||||
port_name = "grpc_port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::SocketAddress server_address(listen_host, port);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"gRPC protocol: " + server_address.toString(),
|
||||
std::make_unique<GRPCServer>(*this, makeSocketAddress(listen_host, port, &logger())));
|
||||
});
|
||||
#endif
|
||||
|
||||
/// Prometheus (if defined and not setup yet with http_port)
|
||||
port_name = "prometheus.port";
|
||||
createServer(config, listen_host, port_name, listen_try, start_servers, servers, [&](UInt16 port) -> ProtocolServerAdapter
|
||||
{
|
||||
Poco::Net::ServerSocket socket;
|
||||
auto address = socketBindListen(socket, listen_host, port);
|
||||
socket.setReceiveTimeout(settings.http_receive_timeout);
|
||||
socket.setSendTimeout(settings.http_send_timeout);
|
||||
return ProtocolServerAdapter(
|
||||
listen_host,
|
||||
port_name,
|
||||
"Prometheus: http://" + address.toString(),
|
||||
std::make_unique<HTTPServer>(
|
||||
context(), createHandlerFactory(*this, async_metrics, "PrometheusHandler-factory"), server_pool, socket, http_params));
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
void Server::updateServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers)
|
||||
{
|
||||
Poco::Logger * log = &logger();
|
||||
/// Gracefully shutdown servers when their port is removed from config
|
||||
const auto listen_hosts = getListenHosts(config);
|
||||
const auto listen_try = getListenTry(config);
|
||||
|
||||
for (auto & server : servers)
|
||||
if (!server.isStopping())
|
||||
{
|
||||
bool has_host = std::find(listen_hosts.begin(), listen_hosts.end(), server.getListenHost()) != listen_hosts.end();
|
||||
bool has_port = !config.getString(server.getPortName(), "").empty();
|
||||
if (!has_host || !has_port || config.getInt(server.getPortName()) != server.portNumber())
|
||||
{
|
||||
server.stop();
|
||||
LOG_INFO(log, "Stopped listening for {}", server.getDescription());
|
||||
}
|
||||
}
|
||||
|
||||
createServers(config, listen_hosts, listen_try, server_pool, async_metrics, servers, /* start_servers: */ true);
|
||||
|
||||
/// Remove servers once all their connections are closed
|
||||
while (std::any_of(servers.begin(), servers.end(), [](const auto & server) { return server.isStopping(); }))
|
||||
{
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
std::erase_if(servers, [&log](auto & server)
|
||||
{
|
||||
if (!server.isStopping())
|
||||
return false;
|
||||
auto is_finished = server.currentConnections() == 0;
|
||||
if (is_finished)
|
||||
LOG_DEBUG(log, "Server finished: {}", server.getDescription());
|
||||
else
|
||||
LOG_TRACE(log, "Waiting server to finish: {}", server.getDescription());
|
||||
return is_finished;
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,6 +24,8 @@ namespace Poco
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class AsynchronousMetrics;
|
||||
class ProtocolServerAdapter;
|
||||
|
||||
class Server : public BaseDaemon, public IServer
|
||||
{
|
||||
@ -67,8 +69,30 @@ private:
|
||||
ContextMutablePtr global_context;
|
||||
Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure = false) const;
|
||||
|
||||
using CreateServerFunc = std::function<void(UInt16)>;
|
||||
void createServer(const std::string & listen_host, const char * port_name, bool listen_try, CreateServerFunc && func) const;
|
||||
using CreateServerFunc = std::function<ProtocolServerAdapter(UInt16)>;
|
||||
void createServer(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & listen_host,
|
||||
const char * port_name,
|
||||
bool listen_try,
|
||||
bool start_server,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
CreateServerFunc && func) const;
|
||||
|
||||
void createServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
const std::vector<std::string> & listen_hosts,
|
||||
bool listen_try,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers,
|
||||
bool start_servers = false);
|
||||
|
||||
void updateServers(
|
||||
Poco::Util::AbstractConfiguration & config,
|
||||
Poco::ThreadPool & server_pool,
|
||||
AsynchronousMetrics & async_metrics,
|
||||
std::vector<ProtocolServerAdapter> & servers);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -284,7 +284,8 @@
|
||||
M(MergeTreeMetaCacheHit, "Number of times the read of meta file was done from MergeTree meta cache") \
|
||||
M(MergeTreeMetaCacheMiss, "Number of times the read of meta file was not done from MergeTree meta cache") \
|
||||
\
|
||||
|
||||
M(MainConfigLoads, "Number of times the main configuration was reloaded.") \
|
||||
\
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
@ -69,12 +69,10 @@ static std::unique_ptr<ReadBufferFromFilePRead> openFileIfExists(const std::stri
|
||||
AsynchronousMetrics::AsynchronousMetrics(
|
||||
ContextPtr global_context_,
|
||||
int update_period_seconds,
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_to_start_before_tables_,
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_)
|
||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_)
|
||||
: WithContext(global_context_)
|
||||
, update_period(update_period_seconds)
|
||||
, servers_to_start_before_tables(servers_to_start_before_tables_)
|
||||
, servers(servers_)
|
||||
, protocol_server_metrics_func(protocol_server_metrics_func_)
|
||||
, log(&Poco::Logger::get("AsynchronousMetrics"))
|
||||
{
|
||||
#if defined(OS_LINUX)
|
||||
@ -238,7 +236,7 @@ void AsynchronousMetrics::start()
|
||||
thread = std::make_unique<ThreadFromGlobalPool>([this] { run(); });
|
||||
}
|
||||
|
||||
AsynchronousMetrics::~AsynchronousMetrics()
|
||||
void AsynchronousMetrics::stop()
|
||||
{
|
||||
try
|
||||
{
|
||||
@ -249,7 +247,10 @@ AsynchronousMetrics::~AsynchronousMetrics()
|
||||
|
||||
wait_cond.notify_one();
|
||||
if (thread)
|
||||
{
|
||||
thread->join();
|
||||
thread.reset();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -257,6 +258,11 @@ AsynchronousMetrics::~AsynchronousMetrics()
|
||||
}
|
||||
}
|
||||
|
||||
AsynchronousMetrics::~AsynchronousMetrics()
|
||||
{
|
||||
stop();
|
||||
}
|
||||
|
||||
|
||||
AsynchronousMetricValues AsynchronousMetrics::getValues() const
|
||||
{
|
||||
@ -1381,22 +1387,11 @@ void AsynchronousMetrics::update(std::chrono::system_clock::time_point update_ti
|
||||
return it->second;
|
||||
};
|
||||
|
||||
if (servers_to_start_before_tables)
|
||||
const auto server_metrics = protocol_server_metrics_func();
|
||||
for (const auto & server_metric : server_metrics)
|
||||
{
|
||||
for (const auto & server : *servers_to_start_before_tables)
|
||||
{
|
||||
if (const auto * name = get_metric_name(server.getPortName()))
|
||||
new_values[name] = server.currentThreads();
|
||||
}
|
||||
}
|
||||
|
||||
if (servers)
|
||||
{
|
||||
for (const auto & server : *servers)
|
||||
{
|
||||
if (const auto * name = get_metric_name(server.getPortName()))
|
||||
new_values[name] = server.currentThreads();
|
||||
}
|
||||
if (const auto * name = get_metric_name(server_metric.port_name))
|
||||
new_values[name] = server_metric.current_threads;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -30,6 +30,11 @@ class ReadBuffer;
|
||||
using AsynchronousMetricValue = double;
|
||||
using AsynchronousMetricValues = std::unordered_map<std::string, AsynchronousMetricValue>;
|
||||
|
||||
struct ProtocolServerMetrics
|
||||
{
|
||||
String port_name;
|
||||
size_t current_threads;
|
||||
};
|
||||
|
||||
/** Periodically (by default, each minute, starting at 30 seconds offset)
|
||||
* calculates and updates some metrics,
|
||||
@ -41,24 +46,25 @@ using AsynchronousMetricValues = std::unordered_map<std::string, AsynchronousMet
|
||||
class AsynchronousMetrics : WithContext
|
||||
{
|
||||
public:
|
||||
using ProtocolServerMetricsFunc = std::function<std::vector<ProtocolServerMetrics>()>;
|
||||
AsynchronousMetrics(
|
||||
ContextPtr global_context_,
|
||||
int update_period_seconds,
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_to_start_before_tables_,
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_);
|
||||
const ProtocolServerMetricsFunc & protocol_server_metrics_func_);
|
||||
|
||||
~AsynchronousMetrics();
|
||||
|
||||
/// Separate method allows to initialize the `servers` variable beforehand.
|
||||
void start();
|
||||
|
||||
void stop();
|
||||
|
||||
/// Returns copy of all values.
|
||||
AsynchronousMetricValues getValues() const;
|
||||
|
||||
private:
|
||||
const std::chrono::seconds update_period;
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_to_start_before_tables{nullptr};
|
||||
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers{nullptr};
|
||||
ProtocolServerMetricsFunc protocol_server_metrics_func;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
std::condition_variable wait_cond;
|
||||
|
@ -4,6 +4,7 @@
|
||||
|
||||
#if USE_GRPC
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
#include <base/types.h>
|
||||
#include "clickhouse_grpc.grpc.pb.h"
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -30,6 +31,9 @@ public:
|
||||
/// Stops the server. No new connections will be accepted.
|
||||
void stop();
|
||||
|
||||
/// Returns the port this server is listening to.
|
||||
UInt16 portNumber() const { return address_to_listen.port(); }
|
||||
|
||||
/// Returns the number of currently handled connections.
|
||||
size_t currentConnections() const;
|
||||
|
||||
|
@ -5,31 +5,13 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
HTTPServer::HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory_,
|
||||
UInt16 port_number,
|
||||
Poco::Net::HTTPServerParams::Ptr params)
|
||||
: TCPServer(new HTTPServerConnectionFactory(context, params, factory_), port_number, params), factory(factory_)
|
||||
{
|
||||
}
|
||||
|
||||
HTTPServer::HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory_,
|
||||
const Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params)
|
||||
: TCPServer(new HTTPServerConnectionFactory(context, params, factory_), socket, params), factory(factory_)
|
||||
{
|
||||
}
|
||||
|
||||
HTTPServer::HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory_,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
const Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::ServerSocket & socket_,
|
||||
Poco::Net::HTTPServerParams::Ptr params)
|
||||
: TCPServer(new HTTPServerConnectionFactory(context, params, factory_), thread_pool, socket, params), factory(factory_)
|
||||
: TCPServer(new HTTPServerConnectionFactory(context, params, factory_), thread_pool, socket_, params), factory(factory_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/TCPServer.h>
|
||||
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
#include <Poco/Net/TCPServer.h>
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
@ -13,26 +13,14 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
class HTTPServer : public Poco::Net::TCPServer
|
||||
class HTTPServer : public TCPServer
|
||||
{
|
||||
public:
|
||||
explicit HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory,
|
||||
UInt16 port_number = 80,
|
||||
Poco::Net::HTTPServerParams::Ptr params = new Poco::Net::HTTPServerParams);
|
||||
|
||||
HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory,
|
||||
const Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params);
|
||||
|
||||
HTTPServer(
|
||||
ContextPtr context,
|
||||
HTTPRequestHandlerFactoryPtr factory,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
const Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params);
|
||||
|
||||
~HTTPServer() override;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Server/HTTP/HTTPServerConnection.h>
|
||||
#include <Server/TCPServer.h>
|
||||
|
||||
#include <Poco/Net/NetException.h>
|
||||
|
||||
@ -7,10 +8,11 @@ namespace DB
|
||||
|
||||
HTTPServerConnection::HTTPServerConnection(
|
||||
ContextPtr context_,
|
||||
TCPServer & tcp_server_,
|
||||
const Poco::Net::StreamSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params_,
|
||||
HTTPRequestHandlerFactoryPtr factory_)
|
||||
: TCPServerConnection(socket), context(Context::createCopy(context_)), params(params_), factory(factory_), stopped(false)
|
||||
: TCPServerConnection(socket), context(Context::createCopy(context_)), tcp_server(tcp_server_), params(params_), factory(factory_), stopped(false)
|
||||
{
|
||||
poco_check_ptr(factory);
|
||||
}
|
||||
@ -20,12 +22,12 @@ void HTTPServerConnection::run()
|
||||
std::string server = params->getSoftwareVersion();
|
||||
Poco::Net::HTTPServerSession session(socket(), params);
|
||||
|
||||
while (!stopped && session.hasMoreRequests())
|
||||
while (!stopped && tcp_server.isOpen() && session.hasMoreRequests())
|
||||
{
|
||||
try
|
||||
{
|
||||
std::unique_lock<std::mutex> lock(mutex);
|
||||
if (!stopped)
|
||||
if (!stopped && tcp_server.isOpen())
|
||||
{
|
||||
HTTPServerResponse response(session);
|
||||
HTTPServerRequest request(context, response, session);
|
||||
@ -48,6 +50,11 @@ void HTTPServerConnection::run()
|
||||
response.set("Server", server);
|
||||
try
|
||||
{
|
||||
if (!tcp_server.isOpen())
|
||||
{
|
||||
sendErrorResponse(session, Poco::Net::HTTPResponse::HTTP_SERVICE_UNAVAILABLE);
|
||||
break;
|
||||
}
|
||||
std::unique_ptr<HTTPRequestHandler> handler(factory->createRequestHandler(request));
|
||||
|
||||
if (handler)
|
||||
|
@ -9,12 +9,14 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class TCPServer;
|
||||
|
||||
class HTTPServerConnection : public Poco::Net::TCPServerConnection
|
||||
{
|
||||
public:
|
||||
HTTPServerConnection(
|
||||
ContextPtr context,
|
||||
TCPServer & tcp_server,
|
||||
const Poco::Net::StreamSocket & socket,
|
||||
Poco::Net::HTTPServerParams::Ptr params,
|
||||
HTTPRequestHandlerFactoryPtr factory);
|
||||
@ -26,6 +28,7 @@ protected:
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
TCPServer & tcp_server;
|
||||
Poco::Net::HTTPServerParams::Ptr params;
|
||||
HTTPRequestHandlerFactoryPtr factory;
|
||||
bool stopped;
|
||||
|
@ -11,9 +11,9 @@ HTTPServerConnectionFactory::HTTPServerConnectionFactory(
|
||||
poco_check_ptr(factory);
|
||||
}
|
||||
|
||||
Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket)
|
||||
Poco::Net::TCPServerConnection * HTTPServerConnectionFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server)
|
||||
{
|
||||
return new HTTPServerConnection(context, socket, params, factory);
|
||||
return new HTTPServerConnection(context, tcp_server, socket, params, factory);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -2,19 +2,19 @@
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Server/HTTP/HTTPRequestHandlerFactory.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
|
||||
#include <Poco/Net/HTTPServerParams.h>
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class HTTPServerConnectionFactory : public Poco::Net::TCPServerConnectionFactory
|
||||
class HTTPServerConnectionFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
public:
|
||||
HTTPServerConnectionFactory(ContextPtr context, Poco::Net::HTTPServerParams::Ptr params, HTTPRequestHandlerFactoryPtr factory);
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override;
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override;
|
||||
|
||||
private:
|
||||
ContextPtr context;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Server/KeeperTCPHandler.h>
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Server/IServer.h>
|
||||
@ -10,7 +10,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
|
||||
class KeeperTCPHandlerFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
private:
|
||||
IServer & server;
|
||||
@ -29,7 +29,7 @@ public:
|
||||
{
|
||||
}
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer &) override
|
||||
{
|
||||
try
|
||||
{
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <IO/WriteBufferFromPocoSocket.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <regex>
|
||||
#include <Common/setThreadName.h>
|
||||
@ -62,10 +63,11 @@ static String showTableStatusReplacementQuery(const String & query);
|
||||
static String killConnectionIdReplacementQuery(const String & query);
|
||||
static String selectLimitReplacementQuery(const String & query);
|
||||
|
||||
MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_,
|
||||
MySQLHandler::MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_,
|
||||
bool ssl_enabled, size_t connection_id_)
|
||||
: Poco::Net::TCPServerConnection(socket_)
|
||||
, server(server_)
|
||||
, tcp_server(tcp_server_)
|
||||
, log(&Poco::Logger::get("MySQLHandler"))
|
||||
, connection_id(connection_id_)
|
||||
, auth_plugin(new MySQLProtocol::Authentication::Native41())
|
||||
@ -138,11 +140,14 @@ void MySQLHandler::run()
|
||||
OKPacket ok_packet(0, handshake_response.capability_flags, 0, 0, 0);
|
||||
packet_endpoint->sendPacket(ok_packet, true);
|
||||
|
||||
while (true)
|
||||
while (tcp_server.isOpen())
|
||||
{
|
||||
packet_endpoint->resetSequenceId();
|
||||
MySQLPacketPayloadReadBuffer payload = packet_endpoint->getPayload();
|
||||
|
||||
while (!in->poll(1000000))
|
||||
if (!tcp_server.isOpen())
|
||||
return;
|
||||
char command = 0;
|
||||
payload.readStrict(command);
|
||||
|
||||
@ -152,6 +157,8 @@ void MySQLHandler::run()
|
||||
LOG_DEBUG(log, "Received command: {}. Connection id: {}.",
|
||||
static_cast<int>(static_cast<unsigned char>(command)), connection_id);
|
||||
|
||||
if (!tcp_server.isOpen())
|
||||
return;
|
||||
try
|
||||
{
|
||||
switch (command)
|
||||
@ -369,8 +376,8 @@ void MySQLHandler::finishHandshakeSSL(
|
||||
}
|
||||
|
||||
#if USE_SSL
|
||||
MySQLHandlerSSL::MySQLHandlerSSL(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_)
|
||||
: MySQLHandler(server_, socket_, ssl_enabled, connection_id_)
|
||||
MySQLHandlerSSL::MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_)
|
||||
: MySQLHandler(server_, tcp_server_, socket_, ssl_enabled, connection_id_)
|
||||
, public_key(public_key_)
|
||||
, private_key(private_key_)
|
||||
{}
|
||||
|
@ -24,11 +24,14 @@ namespace CurrentMetrics
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class ReadBufferFromPocoSocket;
|
||||
class TCPServer;
|
||||
|
||||
/// Handler for MySQL wire protocol connections. Allows to connect to ClickHouse using MySQL client.
|
||||
class MySQLHandler : public Poco::Net::TCPServerConnection
|
||||
{
|
||||
public:
|
||||
MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_);
|
||||
MySQLHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_);
|
||||
|
||||
void run() final;
|
||||
|
||||
@ -52,6 +55,7 @@ protected:
|
||||
virtual void finishHandshakeSSL(size_t packet_size, char * buf, size_t pos, std::function<void(size_t)> read_bytes, MySQLProtocol::ConnectionPhase::HandshakeResponse & packet);
|
||||
|
||||
IServer & server;
|
||||
TCPServer & tcp_server;
|
||||
Poco::Logger * log;
|
||||
UInt64 connection_id = 0;
|
||||
|
||||
@ -68,7 +72,7 @@ protected:
|
||||
Replacements replacements;
|
||||
|
||||
std::unique_ptr<MySQLProtocol::Authentication::IPlugin> auth_plugin;
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
std::shared_ptr<ReadBufferFromPocoSocket> in;
|
||||
std::shared_ptr<WriteBuffer> out;
|
||||
bool secure_connection = false;
|
||||
};
|
||||
@ -77,7 +81,7 @@ protected:
|
||||
class MySQLHandlerSSL : public MySQLHandler
|
||||
{
|
||||
public:
|
||||
MySQLHandlerSSL(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_);
|
||||
MySQLHandlerSSL(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_, RSA & public_key_, RSA & private_key_);
|
||||
|
||||
private:
|
||||
void authPluginSSL() override;
|
||||
|
@ -118,14 +118,14 @@ void MySQLHandlerFactory::generateRSAKeys()
|
||||
}
|
||||
#endif
|
||||
|
||||
Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket)
|
||||
Poco::Net::TCPServerConnection * MySQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server)
|
||||
{
|
||||
size_t connection_id = last_connection_id++;
|
||||
LOG_TRACE(log, "MySQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString());
|
||||
#if USE_SSL
|
||||
return new MySQLHandlerSSL(server, socket, ssl_enabled, connection_id, *public_key, *private_key);
|
||||
return new MySQLHandlerSSL(server, tcp_server, socket, ssl_enabled, connection_id, *public_key, *private_key);
|
||||
#else
|
||||
return new MySQLHandler(server, socket, ssl_enabled, connection_id);
|
||||
return new MySQLHandler(server, tcp_server, socket, ssl_enabled, connection_id);
|
||||
#endif
|
||||
|
||||
}
|
||||
|
@ -1,9 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <Server/IServer.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
@ -13,8 +13,9 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class TCPServer;
|
||||
|
||||
class MySQLHandlerFactory : public Poco::Net::TCPServerConnectionFactory
|
||||
class MySQLHandlerFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
private:
|
||||
IServer & server;
|
||||
@ -43,7 +44,7 @@ public:
|
||||
|
||||
void generateRSAKeys();
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override;
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Interpreters/executeQuery.h>
|
||||
#include "PostgreSQLHandler.h"
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <random>
|
||||
@ -28,11 +29,13 @@ namespace ErrorCodes
|
||||
PostgreSQLHandler::PostgreSQLHandler(
|
||||
const Poco::Net::StreamSocket & socket_,
|
||||
IServer & server_,
|
||||
TCPServer & tcp_server_,
|
||||
bool ssl_enabled_,
|
||||
Int32 connection_id_,
|
||||
std::vector<std::shared_ptr<PostgreSQLProtocol::PGAuthentication::AuthenticationMethod>> & auth_methods_)
|
||||
: Poco::Net::TCPServerConnection(socket_)
|
||||
, server(server_)
|
||||
, tcp_server(tcp_server_)
|
||||
, ssl_enabled(ssl_enabled_)
|
||||
, connection_id(connection_id_)
|
||||
, authentication_manager(auth_methods_)
|
||||
@ -60,11 +63,18 @@ void PostgreSQLHandler::run()
|
||||
if (!startup())
|
||||
return;
|
||||
|
||||
while (true)
|
||||
while (tcp_server.isOpen())
|
||||
{
|
||||
message_transport->send(PostgreSQLProtocol::Messaging::ReadyForQuery(), true);
|
||||
|
||||
constexpr size_t connection_check_timeout = 1; // 1 second
|
||||
while (!in->poll(1000000 * connection_check_timeout))
|
||||
if (!tcp_server.isOpen())
|
||||
return;
|
||||
PostgreSQLProtocol::Messaging::FrontMessageType message_type = message_transport->receiveMessageType();
|
||||
|
||||
if (!tcp_server.isOpen())
|
||||
return;
|
||||
switch (message_type)
|
||||
{
|
||||
case PostgreSQLProtocol::Messaging::FrontMessageType::QUERY:
|
||||
|
@ -18,8 +18,9 @@ namespace CurrentMetrics
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ReadBufferFromPocoSocket;
|
||||
class Session;
|
||||
class TCPServer;
|
||||
|
||||
/** PostgreSQL wire protocol implementation.
|
||||
* For more info see https://www.postgresql.org/docs/current/protocol.html
|
||||
@ -30,6 +31,7 @@ public:
|
||||
PostgreSQLHandler(
|
||||
const Poco::Net::StreamSocket & socket_,
|
||||
IServer & server_,
|
||||
TCPServer & tcp_server_,
|
||||
bool ssl_enabled_,
|
||||
Int32 connection_id_,
|
||||
std::vector<std::shared_ptr<PostgreSQLProtocol::PGAuthentication::AuthenticationMethod>> & auth_methods_);
|
||||
@ -40,12 +42,13 @@ private:
|
||||
Poco::Logger * log = &Poco::Logger::get("PostgreSQLHandler");
|
||||
|
||||
IServer & server;
|
||||
TCPServer & tcp_server;
|
||||
std::unique_ptr<Session> session;
|
||||
bool ssl_enabled = false;
|
||||
Int32 connection_id = 0;
|
||||
Int32 secret_key = 0;
|
||||
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
std::shared_ptr<ReadBufferFromPocoSocket> in;
|
||||
std::shared_ptr<WriteBuffer> out;
|
||||
std::shared_ptr<PostgreSQLProtocol::Messaging::MessageTransport> message_transport;
|
||||
|
||||
|
@ -1,5 +1,4 @@
|
||||
#include "PostgreSQLHandlerFactory.h"
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <memory>
|
||||
#include <Server/PostgreSQLHandler.h>
|
||||
|
||||
@ -17,11 +16,11 @@ PostgreSQLHandlerFactory::PostgreSQLHandlerFactory(IServer & server_)
|
||||
};
|
||||
}
|
||||
|
||||
Poco::Net::TCPServerConnection * PostgreSQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket)
|
||||
Poco::Net::TCPServerConnection * PostgreSQLHandlerFactory::createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server)
|
||||
{
|
||||
Int32 connection_id = last_connection_id++;
|
||||
LOG_TRACE(log, "PostgreSQL connection. Id: {}. Address: {}", connection_id, socket.peerAddress().toString());
|
||||
return new PostgreSQLHandler(socket, server, ssl_enabled, connection_id, auth_methods);
|
||||
return new PostgreSQLHandler(socket, server, tcp_server, ssl_enabled, connection_id, auth_methods);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -1,16 +1,16 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
#include <Server/IServer.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
#include <Core/PostgreSQLProtocol.h>
|
||||
#include <Common/config.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class PostgreSQLHandlerFactory : public Poco::Net::TCPServerConnectionFactory
|
||||
class PostgreSQLHandlerFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
private:
|
||||
IServer & server;
|
||||
@ -28,6 +28,6 @@ private:
|
||||
public:
|
||||
explicit PostgreSQLHandlerFactory(IServer & server_);
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override;
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & server) override;
|
||||
};
|
||||
}
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Server/ProtocolServerAdapter.h>
|
||||
#include <Poco/Net/TCPServer.h>
|
||||
#include <Server/TCPServer.h>
|
||||
|
||||
#if USE_GRPC
|
||||
#include <Server/GRPCServer.h>
|
||||
@ -11,20 +11,29 @@ namespace DB
|
||||
class ProtocolServerAdapter::TCPServerAdapterImpl : public Impl
|
||||
{
|
||||
public:
|
||||
explicit TCPServerAdapterImpl(std::unique_ptr<Poco::Net::TCPServer> tcp_server_) : tcp_server(std::move(tcp_server_)) {}
|
||||
explicit TCPServerAdapterImpl(std::unique_ptr<TCPServer> tcp_server_) : tcp_server(std::move(tcp_server_)) {}
|
||||
~TCPServerAdapterImpl() override = default;
|
||||
|
||||
void start() override { tcp_server->start(); }
|
||||
void stop() override { tcp_server->stop(); }
|
||||
bool isStopping() const override { return !tcp_server->isOpen(); }
|
||||
UInt16 portNumber() const override { return tcp_server->portNumber(); }
|
||||
size_t currentConnections() const override { return tcp_server->currentConnections(); }
|
||||
size_t currentThreads() const override { return tcp_server->currentThreads(); }
|
||||
|
||||
private:
|
||||
std::unique_ptr<Poco::Net::TCPServer> tcp_server;
|
||||
std::unique_ptr<TCPServer> tcp_server;
|
||||
};
|
||||
|
||||
ProtocolServerAdapter::ProtocolServerAdapter(const char * port_name_, std::unique_ptr<Poco::Net::TCPServer> tcp_server_)
|
||||
: port_name(port_name_), impl(std::make_unique<TCPServerAdapterImpl>(std::move(tcp_server_)))
|
||||
ProtocolServerAdapter::ProtocolServerAdapter(
|
||||
const std::string & listen_host_,
|
||||
const char * port_name_,
|
||||
const std::string & description_,
|
||||
std::unique_ptr<TCPServer> tcp_server_)
|
||||
: listen_host(listen_host_)
|
||||
, port_name(port_name_)
|
||||
, description(description_)
|
||||
, impl(std::make_unique<TCPServerAdapterImpl>(std::move(tcp_server_)))
|
||||
{
|
||||
}
|
||||
|
||||
@ -36,16 +45,30 @@ public:
|
||||
~GRPCServerAdapterImpl() override = default;
|
||||
|
||||
void start() override { grpc_server->start(); }
|
||||
void stop() override { grpc_server->stop(); }
|
||||
void stop() override
|
||||
{
|
||||
is_stopping = true;
|
||||
grpc_server->stop();
|
||||
}
|
||||
bool isStopping() const override { return is_stopping; }
|
||||
UInt16 portNumber() const override { return grpc_server->portNumber(); }
|
||||
size_t currentConnections() const override { return grpc_server->currentConnections(); }
|
||||
size_t currentThreads() const override { return grpc_server->currentThreads(); }
|
||||
|
||||
private:
|
||||
std::unique_ptr<GRPCServer> grpc_server;
|
||||
bool is_stopping = false;
|
||||
};
|
||||
|
||||
ProtocolServerAdapter::ProtocolServerAdapter(const char * port_name_, std::unique_ptr<GRPCServer> grpc_server_)
|
||||
: port_name(port_name_), impl(std::make_unique<GRPCServerAdapterImpl>(std::move(grpc_server_)))
|
||||
ProtocolServerAdapter::ProtocolServerAdapter(
|
||||
const std::string & listen_host_,
|
||||
const char * port_name_,
|
||||
const std::string & description_,
|
||||
std::unique_ptr<GRPCServer> grpc_server_)
|
||||
: listen_host(listen_host_)
|
||||
, port_name(port_name_)
|
||||
, description(description_)
|
||||
, impl(std::make_unique<GRPCServerAdapterImpl>(std::move(grpc_server_)))
|
||||
{
|
||||
}
|
||||
#endif
|
||||
|
@ -2,14 +2,14 @@
|
||||
|
||||
#include <Common/config.h>
|
||||
|
||||
#include <base/types.h>
|
||||
#include <memory>
|
||||
#include <string>
|
||||
|
||||
namespace Poco::Net { class TCPServer; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class GRPCServer;
|
||||
class TCPServer;
|
||||
|
||||
/// Provides an unified interface to access a protocol implementing server
|
||||
/// no matter what type it has (HTTPServer, TCPServer, MySQLServer, GRPCServer, ...).
|
||||
@ -19,10 +19,10 @@ class ProtocolServerAdapter
|
||||
public:
|
||||
ProtocolServerAdapter(ProtocolServerAdapter && src) = default;
|
||||
ProtocolServerAdapter & operator =(ProtocolServerAdapter && src) = default;
|
||||
ProtocolServerAdapter(const char * port_name_, std::unique_ptr<Poco::Net::TCPServer> tcp_server_);
|
||||
ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr<TCPServer> tcp_server_);
|
||||
|
||||
#if USE_GRPC
|
||||
ProtocolServerAdapter(const char * port_name_, std::unique_ptr<GRPCServer> grpc_server_);
|
||||
ProtocolServerAdapter(const std::string & listen_host_, const char * port_name_, const std::string & description_, std::unique_ptr<GRPCServer> grpc_server_);
|
||||
#endif
|
||||
|
||||
/// Starts the server. A new thread will be created that waits for and accepts incoming connections.
|
||||
@ -31,14 +31,23 @@ public:
|
||||
/// Stops the server. No new connections will be accepted.
|
||||
void stop() { impl->stop(); }
|
||||
|
||||
bool isStopping() const { return impl->isStopping(); }
|
||||
|
||||
/// Returns the number of currently handled connections.
|
||||
size_t currentConnections() const { return impl->currentConnections(); }
|
||||
|
||||
/// Returns the number of current threads.
|
||||
size_t currentThreads() const { return impl->currentThreads(); }
|
||||
|
||||
/// Returns the port this server is listening to.
|
||||
UInt16 portNumber() const { return impl->portNumber(); }
|
||||
|
||||
const std::string & getListenHost() const { return listen_host; }
|
||||
|
||||
const std::string & getPortName() const { return port_name; }
|
||||
|
||||
const std::string & getDescription() const { return description; }
|
||||
|
||||
private:
|
||||
class Impl
|
||||
{
|
||||
@ -46,13 +55,17 @@ private:
|
||||
virtual ~Impl() {}
|
||||
virtual void start() = 0;
|
||||
virtual void stop() = 0;
|
||||
virtual bool isStopping() const = 0;
|
||||
virtual UInt16 portNumber() const = 0;
|
||||
virtual size_t currentConnections() const = 0;
|
||||
virtual size_t currentThreads() const = 0;
|
||||
};
|
||||
class TCPServerAdapterImpl;
|
||||
class GRPCServerAdapterImpl;
|
||||
|
||||
std::string listen_host;
|
||||
std::string port_name;
|
||||
std::string description;
|
||||
std::unique_ptr<Impl> impl;
|
||||
};
|
||||
|
||||
|
@ -32,6 +32,7 @@
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/Session.h>
|
||||
#include <Interpreters/ProfileEventsExt.h>
|
||||
#include <Server/TCPServer.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Storages/StorageS3Cluster.h>
|
||||
@ -81,9 +82,10 @@ namespace ErrorCodes
|
||||
extern const int UNKNOWN_PROTOCOL;
|
||||
}
|
||||
|
||||
TCPHandler::TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_)
|
||||
TCPHandler::TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_)
|
||||
: Poco::Net::TCPServerConnection(socket_)
|
||||
, server(server_)
|
||||
, tcp_server(tcp_server_)
|
||||
, parse_proxy_protocol(parse_proxy_protocol_)
|
||||
, log(&Poco::Logger::get("TCPHandler"))
|
||||
, server_display_name(std::move(server_display_name_))
|
||||
@ -172,13 +174,13 @@ void TCPHandler::runImpl()
|
||||
throw;
|
||||
}
|
||||
|
||||
while (true)
|
||||
while (tcp_server.isOpen())
|
||||
{
|
||||
/// We are waiting for a packet from the client. Thus, every `poll_interval` seconds check whether we need to shut down.
|
||||
{
|
||||
Stopwatch idle_time;
|
||||
UInt64 timeout_ms = std::min(poll_interval, idle_connection_timeout) * 1000000;
|
||||
while (!server.isCancelled() && !static_cast<ReadBufferFromPocoSocket &>(*in).poll(timeout_ms))
|
||||
while (tcp_server.isOpen() && !server.isCancelled() && !static_cast<ReadBufferFromPocoSocket &>(*in).poll(timeout_ms))
|
||||
{
|
||||
if (idle_time.elapsedSeconds() > idle_connection_timeout)
|
||||
{
|
||||
@ -189,7 +191,7 @@ void TCPHandler::runImpl()
|
||||
}
|
||||
|
||||
/// If we need to shut down, or client disconnects.
|
||||
if (server.isCancelled() || in->eof())
|
||||
if (!tcp_server.isOpen() || server.isCancelled() || in->eof())
|
||||
break;
|
||||
|
||||
Stopwatch watch;
|
||||
|
@ -35,6 +35,7 @@ class Session;
|
||||
struct Settings;
|
||||
class ColumnsDescription;
|
||||
struct ProfileInfo;
|
||||
class TCPServer;
|
||||
|
||||
/// State of query processing.
|
||||
struct QueryState
|
||||
@ -127,7 +128,7 @@ public:
|
||||
* because it allows to check the IP ranges of the trusted proxy.
|
||||
* Proxy-forwarded (original client) IP address is used for quota accounting if quota is keyed by forwarded IP.
|
||||
*/
|
||||
TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_);
|
||||
TCPHandler(IServer & server_, TCPServer & tcp_server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_);
|
||||
~TCPHandler() override;
|
||||
|
||||
void run() override;
|
||||
@ -137,6 +138,7 @@ public:
|
||||
|
||||
private:
|
||||
IServer & server;
|
||||
TCPServer & tcp_server;
|
||||
bool parse_proxy_protocol = false;
|
||||
Poco::Logger * log;
|
||||
|
||||
|
@ -1,17 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <base/logger_useful.h>
|
||||
#include <Server/IServer.h>
|
||||
#include <Server/TCPHandler.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
|
||||
class TCPHandlerFactory : public TCPServerConnectionFactory
|
||||
{
|
||||
private:
|
||||
IServer & server;
|
||||
@ -38,13 +38,13 @@ public:
|
||||
server_display_name = server.config().getString("display_name", getFQDNOrHostName());
|
||||
}
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) override
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_TRACE(log, "TCP Request. Address: {}", socket.peerAddress().toString());
|
||||
|
||||
return new TCPHandler(server, socket, parse_proxy_protocol, server_display_name);
|
||||
return new TCPHandler(server, tcp_server, socket, parse_proxy_protocol, server_display_name);
|
||||
}
|
||||
catch (const Poco::Net::NetException &)
|
||||
{
|
||||
|
36
src/Server/TCPServer.cpp
Normal file
36
src/Server/TCPServer.cpp
Normal file
@ -0,0 +1,36 @@
|
||||
#include <Poco/Net/TCPServerConnectionFactory.h>
|
||||
#include <Server/TCPServer.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TCPServerConnectionFactoryImpl : public Poco::Net::TCPServerConnectionFactory
|
||||
{
|
||||
public:
|
||||
TCPServerConnectionFactoryImpl(TCPServer & tcp_server_, DB::TCPServerConnectionFactory::Ptr factory_)
|
||||
: tcp_server(tcp_server_)
|
||||
, factory(factory_)
|
||||
{}
|
||||
|
||||
Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket) override
|
||||
{
|
||||
return factory->createConnection(socket, tcp_server);
|
||||
}
|
||||
private:
|
||||
TCPServer & tcp_server;
|
||||
DB::TCPServerConnectionFactory::Ptr factory;
|
||||
};
|
||||
|
||||
TCPServer::TCPServer(
|
||||
TCPServerConnectionFactory::Ptr factory_,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
Poco::Net::ServerSocket & socket_,
|
||||
Poco::Net::TCPServerParams::Ptr params)
|
||||
: Poco::Net::TCPServer(new TCPServerConnectionFactoryImpl(*this, factory_), thread_pool, socket_, params)
|
||||
, factory(factory_)
|
||||
, socket(socket_)
|
||||
, is_open(true)
|
||||
, port_number(socket.address().port())
|
||||
{}
|
||||
|
||||
}
|
47
src/Server/TCPServer.h
Normal file
47
src/Server/TCPServer.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Net/TCPServer.h>
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Server/TCPServerConnectionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class Context;
|
||||
|
||||
class TCPServer : public Poco::Net::TCPServer
|
||||
{
|
||||
public:
|
||||
explicit TCPServer(
|
||||
TCPServerConnectionFactory::Ptr factory,
|
||||
Poco::ThreadPool & thread_pool,
|
||||
Poco::Net::ServerSocket & socket,
|
||||
Poco::Net::TCPServerParams::Ptr params = new Poco::Net::TCPServerParams);
|
||||
|
||||
/// Close the socket and ask existing connections to stop serving queries
|
||||
void stop()
|
||||
{
|
||||
Poco::Net::TCPServer::stop();
|
||||
// This notifies already established connections that they should stop serving
|
||||
// queries and close their socket as soon as they can.
|
||||
is_open = false;
|
||||
// Poco's stop() stops listening on the socket but leaves it open.
|
||||
// To be able to hand over control of the listening port to a new server, and
|
||||
// to get fast connection refusal instead of timeouts, we also need to close
|
||||
// the listening socket.
|
||||
socket.close();
|
||||
}
|
||||
|
||||
bool isOpen() const { return is_open; }
|
||||
|
||||
UInt16 portNumber() const { return port_number; }
|
||||
|
||||
private:
|
||||
TCPServerConnectionFactory::Ptr factory;
|
||||
Poco::Net::ServerSocket socket;
|
||||
std::atomic<bool> is_open;
|
||||
UInt16 port_number;
|
||||
};
|
||||
|
||||
}
|
27
src/Server/TCPServerConnectionFactory.h
Normal file
27
src/Server/TCPServerConnectionFactory.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/SharedPtr.h>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
namespace Net
|
||||
{
|
||||
class StreamSocket;
|
||||
class TCPServerConnection;
|
||||
}
|
||||
}
|
||||
namespace DB
|
||||
{
|
||||
class TCPServer;
|
||||
|
||||
class TCPServerConnectionFactory
|
||||
{
|
||||
public:
|
||||
using Ptr = Poco::SharedPtr<TCPServerConnectionFactory>;
|
||||
|
||||
virtual ~TCPServerConnectionFactory() = default;
|
||||
|
||||
/// Same as Poco::Net::TCPServerConnectionFactory except we can pass the TCPServer
|
||||
virtual Poco::Net::TCPServerConnection * createConnection(const Poco::Net::StreamSocket & socket, TCPServer & tcp_server) = 0;
|
||||
};
|
||||
}
|
@ -278,9 +278,8 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ
|
||||
{
|
||||
postgres::Connection connection(connection_info);
|
||||
pqxx::nontransaction tx(connection.getRef());
|
||||
auto table_structure = std::make_unique<PostgreSQLTableStructure>(fetchPostgreSQLTableStructure(tx, table_name, postgres_schema, true, true, true));
|
||||
if (!table_structure)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get PostgreSQL table structure");
|
||||
auto [postgres_table_schema, postgres_table_name] = getSchemaAndTableName(table_name);
|
||||
auto table_structure = std::make_unique<PostgreSQLTableStructure>(fetchPostgreSQLTableStructure(tx, postgres_table_name, postgres_table_schema, true, true, true));
|
||||
|
||||
auto table_override = tryGetTableOverride(current_database_name, table_name);
|
||||
return storage->getCreateNestedTableQuery(std::move(table_structure), table_override ? table_override->as<ASTTableOverride>() : nullptr);
|
||||
@ -516,17 +515,25 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx)
|
||||
|
||||
void PostgreSQLReplicationHandler::addTableToPublication(pqxx::nontransaction & ntx, const String & table_name)
|
||||
{
|
||||
std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteString(table_name));
|
||||
std::string query_str = fmt::format("ALTER PUBLICATION {} ADD TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name));
|
||||
ntx.exec(query_str);
|
||||
LOG_TRACE(log, "Added table `{}` to publication `{}`", table_name, publication_name);
|
||||
LOG_TRACE(log, "Added table {} to publication `{}`", doubleQuoteWithSchema(table_name), publication_name);
|
||||
}
|
||||
|
||||
|
||||
void PostgreSQLReplicationHandler::removeTableFromPublication(pqxx::nontransaction & ntx, const String & table_name)
|
||||
{
|
||||
std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", publication_name, doubleQuoteString(table_name));
|
||||
ntx.exec(query_str);
|
||||
LOG_TRACE(log, "Removed table `{}` from publication `{}`", table_name, publication_name);
|
||||
try
|
||||
{
|
||||
std::string query_str = fmt::format("ALTER PUBLICATION {} DROP TABLE ONLY {}", publication_name, doubleQuoteWithSchema(table_name));
|
||||
ntx.exec(query_str);
|
||||
LOG_TRACE(log, "Removed table `{}` from publication `{}`", doubleQuoteWithSchema(table_name), publication_name);
|
||||
}
|
||||
catch (const pqxx::undefined_table &)
|
||||
{
|
||||
/// Removing table from replication must succeed even if table does not exist in PostgreSQL.
|
||||
LOG_WARNING(log, "Did not remove table {} from publication, because table does not exist in PostgreSQL", doubleQuoteWithSchema(table_name), publication_name);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -455,10 +455,8 @@ static void appendBlock(const Block & from, Block & to)
|
||||
size_t rows = from.rows();
|
||||
size_t bytes = from.bytes();
|
||||
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows);
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, bytes);
|
||||
|
||||
size_t old_rows = to.rows();
|
||||
size_t old_bytes = to.bytes();
|
||||
|
||||
MutableColumnPtr last_col;
|
||||
try
|
||||
@ -468,6 +466,8 @@ static void appendBlock(const Block & from, Block & to)
|
||||
if (to.rows() == 0)
|
||||
{
|
||||
to = from;
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows);
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, bytes);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -480,6 +480,8 @@ static void appendBlock(const Block & from, Block & to)
|
||||
|
||||
to.getByPosition(column_no).column = std::move(last_col);
|
||||
}
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows);
|
||||
CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, to.bytes() - old_bytes);
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
|
@ -9,6 +9,36 @@ get_property (BUILD_COMPILE_DEFINITIONS DIRECTORY ${ClickHouse_SOURCE_DIR} PROPE
|
||||
|
||||
get_property(TZDATA_VERSION GLOBAL PROPERTY TZDATA_VERSION_PROP)
|
||||
|
||||
|
||||
find_package(Git)
|
||||
if(Git_FOUND)
|
||||
# The commit's git hash, and whether the building workspace was dirty or not
|
||||
execute_process(COMMAND
|
||||
"${GIT_EXECUTABLE}" rev-parse HEAD
|
||||
WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}"
|
||||
OUTPUT_VARIABLE GIT_HASH
|
||||
ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
# Git branch name
|
||||
execute_process(COMMAND
|
||||
"${GIT_EXECUTABLE}" rev-parse --abbrev-ref HEAD
|
||||
WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}"
|
||||
OUTPUT_VARIABLE GIT_BRANCH
|
||||
ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
# The date of the commit
|
||||
SET(ENV{TZ} "UTC")
|
||||
execute_process(COMMAND
|
||||
"${GIT_EXECUTABLE}" log -1 --format=%ad --date=iso-local
|
||||
WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}"
|
||||
OUTPUT_VARIABLE GIT_DATE
|
||||
ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
# The subject of the commit
|
||||
execute_process(COMMAND
|
||||
"${GIT_EXECUTABLE}" log -1 --format=%s
|
||||
WORKING_DIRECTORY "${ClickHouse_SOURCE_DIR}"
|
||||
OUTPUT_VARIABLE GIT_COMMIT_SUBJECT
|
||||
ERROR_QUIET OUTPUT_STRIP_TRAILING_WHITESPACE)
|
||||
endif()
|
||||
|
||||
configure_file (StorageSystemBuildOptions.generated.cpp.in ${CONFIG_BUILD})
|
||||
|
||||
include("${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake")
|
||||
|
@ -50,6 +50,10 @@ const char * auto_config_build[]
|
||||
"USE_KRB5", "@USE_KRB5@",
|
||||
"USE_FILELOG", "@USE_FILELOG@",
|
||||
"USE_BZIP2", "@USE_BZIP2@",
|
||||
"GIT_HASH", "@GIT_HASH@",
|
||||
"GIT_BRANCH", "@GIT_BRANCH@",
|
||||
"GIT_DATE", "@GIT_DATE@",
|
||||
"GIT_COMMIT_SUBJECT", "@GIT_COMMIT_SUBJECT@",
|
||||
|
||||
nullptr, nullptr
|
||||
};
|
||||
|
@ -120,7 +120,7 @@ TEST(TransformQueryForExternalDatabase, InWithSingleElement)
|
||||
|
||||
check(state, 1,
|
||||
"SELECT column FROM test.table WHERE 1 IN (1)",
|
||||
R"(SELECT "column" FROM "test"."table" WHERE 1)");
|
||||
R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)");
|
||||
check(state, 1,
|
||||
"SELECT column FROM test.table WHERE column IN (1, 2)",
|
||||
R"(SELECT "column" FROM "test"."table" WHERE "column" IN (1, 2))");
|
||||
@ -135,7 +135,7 @@ TEST(TransformQueryForExternalDatabase, InWithMultipleColumns)
|
||||
|
||||
check(state, 1,
|
||||
"SELECT column FROM test.table WHERE (1,1) IN ((1,1))",
|
||||
R"(SELECT "column" FROM "test"."table" WHERE 1)");
|
||||
R"(SELECT "column" FROM "test"."table" WHERE 1 = 1)");
|
||||
check(state, 1,
|
||||
"SELECT field, value FROM test.table WHERE (field, value) IN (('foo', 'bar'))",
|
||||
R"(SELECT "field", "value" FROM "test"."table" WHERE ("field", "value") IN (('foo', 'bar')))");
|
||||
|
@ -306,6 +306,18 @@ String transformQueryForExternalDatabase(
|
||||
throw Exception("Query contains non-compatible expressions (and external_table_strict_query=true)", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
|
||||
auto * literal_expr = typeid_cast<ASTLiteral *>(original_where.get());
|
||||
UInt64 value;
|
||||
if (literal_expr && literal_expr->value.tryGet<UInt64>(value) && (value == 0 || value == 1))
|
||||
{
|
||||
/// WHERE 1 -> WHERE 1=1, WHERE 0 -> WHERE 1=0.
|
||||
if (value)
|
||||
original_where = makeASTFunction("equals", std::make_shared<ASTLiteral>(1), std::make_shared<ASTLiteral>(1));
|
||||
else
|
||||
original_where = makeASTFunction("equals", std::make_shared<ASTLiteral>(1), std::make_shared<ASTLiteral>(0));
|
||||
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(original_where));
|
||||
}
|
||||
|
||||
ASTPtr select_ptr = select;
|
||||
dropAliases(select_ptr);
|
||||
|
||||
|
@ -11,7 +11,6 @@ import boto3
|
||||
NEED_RERUN_OR_CANCELL_WORKFLOWS = {
|
||||
13241696, # PR
|
||||
15834118, # Docs
|
||||
15522500, # MasterCI
|
||||
15516108, # ReleaseCI
|
||||
15797242, # BackportPR
|
||||
}
|
||||
@ -86,10 +85,23 @@ WorkflowDescription = namedtuple('WorkflowDescription',
|
||||
def get_workflows_description_for_pull_request(pull_request_event):
|
||||
head_branch = pull_request_event['head']['ref']
|
||||
print("PR", pull_request_event['number'], "has head ref", head_branch)
|
||||
workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}")
|
||||
workflows_data = []
|
||||
workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page=1")
|
||||
workflows_data += workflows['workflow_runs']
|
||||
i = 2
|
||||
while len(workflows['workflow_runs']) > 0:
|
||||
workflows = _exec_get_with_retry(API_URL + f"/actions/runs?branch={head_branch}&event=pull_request&page={i}")
|
||||
workflows_data += workflows['workflow_runs']
|
||||
i += 1
|
||||
if i > 30:
|
||||
print("Too many workflows found")
|
||||
break
|
||||
|
||||
workflow_descriptions = []
|
||||
for workflow in workflows['workflow_runs']:
|
||||
if workflow['workflow_id'] in NEED_RERUN_OR_CANCELL_WORKFLOWS:
|
||||
for workflow in workflows_data:
|
||||
# unfortunately we cannot filter workflows from forks in request to API so doing it manually
|
||||
if (workflow['head_repository']['full_name'] == pull_request_event['head']['repo']['full_name']
|
||||
and workflow['workflow_id'] in NEED_RERUN_OR_CANCELL_WORKFLOWS):
|
||||
workflow_descriptions.append(WorkflowDescription(
|
||||
run_id=workflow['id'],
|
||||
status=workflow['status'],
|
||||
|
@ -120,6 +120,8 @@ if __name__ == "__main__":
|
||||
|
||||
pr_info = PRInfo()
|
||||
|
||||
logging.info("Start at PR number %s, commit sha %s labels %s", pr_info.number, pr_info.sha, pr_info.labels)
|
||||
|
||||
if pr_info.number != 0 and 'jepsen-test' not in pr_info.labels():
|
||||
logging.info("Not jepsen test label in labels list, skipping")
|
||||
sys.exit(0)
|
||||
|
@ -2043,7 +2043,8 @@ class ClickHouseInstance:
|
||||
user=user, password=password, database=database)
|
||||
|
||||
# Connects to the instance via HTTP interface, sends a query and returns the answer
|
||||
def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False):
|
||||
def http_query(self, sql, data=None, params=None, user=None, password=None, expect_fail_and_get_error=False,
|
||||
port=8123, timeout=None, retry_strategy=None):
|
||||
logging.debug(f"Executing query {sql} on {self.name} via HTTP interface")
|
||||
if params is None:
|
||||
params = {}
|
||||
@ -2057,12 +2058,19 @@ class ClickHouseInstance:
|
||||
auth = requests.auth.HTTPBasicAuth(user, password)
|
||||
elif user:
|
||||
auth = requests.auth.HTTPBasicAuth(user, '')
|
||||
url = "http://" + self.ip_address + ":8123/?" + urllib.parse.urlencode(params)
|
||||
url = f"http://{self.ip_address}:{port}/?" + urllib.parse.urlencode(params)
|
||||
|
||||
if data:
|
||||
r = requests.post(url, data, auth=auth)
|
||||
if retry_strategy is None:
|
||||
requester = requests
|
||||
else:
|
||||
r = requests.get(url, auth=auth)
|
||||
adapter = requests.adapters.HTTPAdapter(max_retries=retry_strategy)
|
||||
requester = requests.Session()
|
||||
requester.mount("https://", adapter)
|
||||
requester.mount("http://", adapter)
|
||||
if data:
|
||||
r = requester.post(url, data, auth=auth, timeout=timeout)
|
||||
else:
|
||||
r = requester.get(url, auth=auth, timeout=timeout)
|
||||
|
||||
def http_code_and_message():
|
||||
code = r.status_code
|
||||
|
@ -1,4 +1,23 @@
|
||||
<clickhouse>
|
||||
<!-- make it fail earlier -->
|
||||
<max_server_memory_usage>3000000000</max_server_memory_usage> <!-- 3GB -->
|
||||
<!-- To make it fail earlier, we will limit max_server_memory_usage explicitly.
|
||||
|
||||
Also note, that usually it is enough 3Gi,
|
||||
but TSan uses 2.8+- GiB of RAM w/o just at start,
|
||||
so this limit had been increased to 4GB
|
||||
-->
|
||||
<max_server_memory_usage>4000000000</max_server_memory_usage>
|
||||
|
||||
<query_thread_log remove="remove"/>
|
||||
<query_log remove="remove" />
|
||||
<query_views_log remove="remove" />
|
||||
<metric_log remove="remove"/>
|
||||
<text_log remove="remove"/>
|
||||
<trace_log remove="remove"/>
|
||||
<asynchronous_metric_log remove="remove" />
|
||||
<session_log remove="remove" />
|
||||
<part_log remove="remove" />
|
||||
<crash_log remove="remove" />
|
||||
<opentelemetry_span_log remove="remove" />
|
||||
<!-- just in case it will be enabled by default -->
|
||||
<zookeeper_log remove="remove" />
|
||||
</clickhouse>
|
||||
|
@ -24,16 +24,13 @@ def start_cluster():
|
||||
|
||||
|
||||
# max_memory_usage_for_user cannot be used, since the memory for user accounted
|
||||
# correctly, only total is not
|
||||
# correctly, only total is not (it is set via conf.xml)
|
||||
def test_memory_tracking_total():
|
||||
instance.query('''
|
||||
CREATE TABLE null (row String) ENGINE=Null;
|
||||
''')
|
||||
instance.query('CREATE TABLE null (row String) ENGINE=Null')
|
||||
instance.exec_in_container(['bash', '-c',
|
||||
'clickhouse local -q "SELECT arrayStringConcat(arrayMap(x->toString(cityHash64(x)), range(1000)), \' \') from numbers(10000)" > data.json'])
|
||||
for it in range(0, 20):
|
||||
# the problem can be triggered only via HTTP,
|
||||
# since clickhouse-client parses the data by itself.
|
||||
assert instance.exec_in_container(['curl', '--silent', '--show-error', '--data-binary', '@data.json',
|
||||
'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', 'Failed on {} iteration'.format(
|
||||
it)
|
||||
'http://127.1:8123/?query=INSERT%20INTO%20null%20FORMAT%20TSV']) == '', f'Failed on {it} iteration'
|
||||
|
@ -178,7 +178,7 @@ def assert_number_of_columns(expected, table_name, database_name='test_database'
|
||||
def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database', schema_name=''):
|
||||
assert_nested_table_is_created(table_name, materialized_database, schema_name)
|
||||
|
||||
print("Checking table is synchronized:", table_name)
|
||||
print(f"Checking table is synchronized. Table name: {table_name}, table schema: {schema_name}")
|
||||
expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by))
|
||||
if len(schema_name) == 0:
|
||||
result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by))
|
||||
@ -356,6 +356,11 @@ def test_remove_table_from_replication(started_cluster):
|
||||
for i in range(NUM_TABLES):
|
||||
cursor.execute('drop table if exists postgresql_replica_{};'.format(i))
|
||||
|
||||
# Removing from replication table which does not exist in PostgreSQL must be ok.
|
||||
instance.query('DETACH TABLE test_database.postgresql_replica_0');
|
||||
assert instance.contains_in_log("from publication, because table does not exist in PostgreSQL")
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
def test_predefined_connection_configuration(started_cluster):
|
||||
drop_materialized_db()
|
||||
@ -379,6 +384,7 @@ def test_database_with_single_non_default_schema(started_cluster):
|
||||
|
||||
NUM_TABLES=5
|
||||
schema_name = 'test_schema'
|
||||
materialized_db = 'test_database'
|
||||
clickhouse_postgres_db = 'postgres_database_with_schema'
|
||||
global insert_counter
|
||||
insert_counter = 0
|
||||
@ -430,6 +436,14 @@ def test_database_with_single_non_default_schema(started_cluster):
|
||||
instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)")
|
||||
assert_number_of_columns(3, f'postgresql_replica_{altered_table}')
|
||||
check_tables_are_synchronized(f"postgresql_replica_{altered_table}", postgres_database=clickhouse_postgres_db);
|
||||
|
||||
print('DETACH-ATTACH')
|
||||
detached_table_name = "postgresql_replica_1"
|
||||
instance.query(f"DETACH TABLE {materialized_db}.{detached_table_name}")
|
||||
assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL")
|
||||
instance.query(f"ATTACH TABLE {materialized_db}.{detached_table_name}")
|
||||
check_tables_are_synchronized(detached_table_name, postgres_database=clickhouse_postgres_db);
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
@ -440,6 +454,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster):
|
||||
NUM_TABLES = 5
|
||||
schema_name = 'test_schema'
|
||||
clickhouse_postgres_db = 'postgres_database_with_schema'
|
||||
materialized_db = 'test_database'
|
||||
publication_tables = ''
|
||||
global insert_counter
|
||||
insert_counter = 0
|
||||
@ -494,6 +509,15 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster):
|
||||
instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)")
|
||||
assert_number_of_columns(3, f'{schema_name}.postgresql_replica_{altered_table}')
|
||||
check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db);
|
||||
|
||||
print('DETACH-ATTACH')
|
||||
detached_table_name = "postgresql_replica_1"
|
||||
instance.query(f"DETACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`")
|
||||
assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL")
|
||||
instance.query(f"ATTACH TABLE {materialized_db}.`{schema_name}.{detached_table_name}`")
|
||||
assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n")
|
||||
check_tables_are_synchronized(detached_table_name, schema_name=schema_name, postgres_database=clickhouse_postgres_db);
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
@ -504,6 +528,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster):
|
||||
NUM_TABLES = 2
|
||||
schemas_num = 2
|
||||
schema_list = 'schema0, schema1'
|
||||
materialized_db = 'test_database'
|
||||
global insert_counter
|
||||
insert_counter = 0
|
||||
|
||||
@ -557,11 +582,23 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster):
|
||||
print('ALTER')
|
||||
altered_schema = random.randint(0, schemas_num-1)
|
||||
altered_table = random.randint(0, NUM_TABLES-1)
|
||||
clickhouse_postgres_db = f'clickhouse_postgres_db{altered_schema}'
|
||||
cursor.execute(f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer")
|
||||
|
||||
instance.query(f"INSERT INTO clickhouse_postgres_db{altered_schema}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(1000 * {insert_counter}, 1000)")
|
||||
assert_number_of_columns(3, f'schema{altered_schema}.postgresql_replica_{altered_table}')
|
||||
check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db);
|
||||
check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=f"schema{altered_schema}", postgres_database=clickhouse_postgres_db);
|
||||
|
||||
print('DETACH-ATTACH')
|
||||
detached_table_name = "postgresql_replica_1"
|
||||
detached_table_schema = "schema0"
|
||||
clickhouse_postgres_db = f'clickhouse_postgres_db0'
|
||||
instance.query(f"DETACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`")
|
||||
assert not instance.contains_in_log("from publication, because table does not exist in PostgreSQL")
|
||||
instance.query(f"ATTACH TABLE {materialized_db}.`{detached_table_schema}.{detached_table_name}`")
|
||||
assert_show_tables("schema0.postgresql_replica_0\nschema0.postgresql_replica_1\nschema1.postgresql_replica_0\nschema1.postgresql_replica_1\n")
|
||||
check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=detached_table_schema, postgres_database=clickhouse_postgres_db);
|
||||
|
||||
drop_materialized_db()
|
||||
|
||||
|
||||
|
1
tests/integration/test_server_reload/.gitignore
vendored
Normal file
1
tests/integration/test_server_reload/.gitignore
vendored
Normal file
@ -0,0 +1 @@
|
||||
_gen
|
0
tests/integration/test_server_reload/__init__.py
Normal file
0
tests/integration/test_server_reload/__init__.py
Normal file
@ -0,0 +1,13 @@
|
||||
<?xml version="1.0"?>
|
||||
<yandex>
|
||||
<profiles>
|
||||
<default>
|
||||
</default>
|
||||
</profiles>
|
||||
|
||||
<users>
|
||||
<postgresql>
|
||||
<password>123</password>
|
||||
</postgresql>
|
||||
</users>
|
||||
</yandex>
|
8
tests/integration/test_server_reload/configs/dhparam.pem
Normal file
8
tests/integration/test_server_reload/configs/dhparam.pem
Normal file
@ -0,0 +1,8 @@
|
||||
-----BEGIN DH PARAMETERS-----
|
||||
MIIBCAKCAQEAkPGhfLY5nppeQkFBKYRpiisxzrRQfyyTUu6aabZP2CbAMAuoYzaC
|
||||
Z+iqeWSQZKRYeA21SZXkC9xE1e5FJsc5IWzCRiMNZeLuj4ApUNysMu89DpX8/b91
|
||||
+Ka6wRJnaO43ZqHj/9FpU4JiYtxoIpXDC9HeiSAnwLwJc3L+nkYfnSGgvzWIxhGV
|
||||
gCoVmVBoTe7wrqCyVlM5nrNZSjhlSugvXmu2bSK3MwYF08QLKvlF68eedbs0PMWh
|
||||
WC0bFM/X7gMBEqL4DiINufAShbZPKxD6eL2APiHPUo6xun3ed/Po/5j8QBmiku0c
|
||||
5Jb12ZhOTRTQjaRg2aFF8LPdW2tDE7HmewIBAg==
|
||||
-----END DH PARAMETERS-----
|
@ -0,0 +1,9 @@
|
||||
<yandex>
|
||||
<listen_host remove="remove"/>
|
||||
<include from_zk="/clickhouse/listen_hosts" />
|
||||
<tcp_port from_zk="/clickhouse/ports/tcp" replace="replace" />
|
||||
<http_port from_zk="/clickhouse/ports/http" replace="replace" />
|
||||
<mysql_port from_zk="/clickhouse/ports/mysql" replace="replace" />
|
||||
<postgresql_port from_zk="/clickhouse/ports/postgresql" replace="replace" />
|
||||
<grpc_port from_zk="/clickhouse/ports/grpc" replace="replace" />
|
||||
</yandex>
|
18
tests/integration/test_server_reload/configs/server.crt
Normal file
18
tests/integration/test_server_reload/configs/server.crt
Normal file
@ -0,0 +1,18 @@
|
||||
-----BEGIN CERTIFICATE-----
|
||||
MIIC+zCCAeOgAwIBAgIJAIhI9ozZJ+TWMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV
|
||||
BAMMCWxvY2FsaG9zdDAeFw0xOTA0MjIwNDMyNTJaFw0yMDA0MjEwNDMyNTJaMBQx
|
||||
EjAQBgNVBAMMCWxvY2FsaG9zdDCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
|
||||
ggEBAK+wVUEdqF2uXvN0MJBgnAHyXi6JTi4p/F6igsrCjSNjJWzHH0vQmK8ujfcF
|
||||
CkifW88i+W5eHctuEtQqNHK+t9x9YiZtXrj6m/XkOXs20mYgENSmbbbHbriTPnZB
|
||||
zZrq6UqMlwIHNNAa+I3NMORQxVRaI0ybXnGVO5elr70xHpk03xL0JWKHpEqYp4db
|
||||
2aBQgF6y3Ww4khxjIYqpUYXWXGFnVIRU7FKVEAM1xyKqvQzXjQ5sVM/wyHknveEF
|
||||
3b/X4ggN+KNl5KOc0cWDh1/XaatJAPaUUPqZcq76tynLbP64Xm3dxHcj+gtRkO67
|
||||
ef6MSg6l63m3XQP6Qb+MIkd06OsCAwEAAaNQME4wHQYDVR0OBBYEFDmODTO8QLDN
|
||||
ykR3x0LIOnjNhrKhMB8GA1UdIwQYMBaAFDmODTO8QLDNykR3x0LIOnjNhrKhMAwG
|
||||
A1UdEwQFMAMBAf8wDQYJKoZIhvcNAQELBQADggEBAAwaiJc7uqEpnH3aukbftDwX
|
||||
m8GfEnj1HVdgg+9GGNq+9rvUYBF6gdPmjRCX9dO0cclLFx8jc2org0rTSq9WoOhX
|
||||
E6qL4Eqrmc5SE3Y9jZM0h6GRD4oXK014FmtZ3T6ddZU3dQLj3BS2r1XrvmubTvGN
|
||||
ZuTJNY8nx8Hh6H5XINmsEjUF9E5hog+PwCE03xt2adIdYL+gsbxASeNYyeUFpZv5
|
||||
zcXR3VoakBWnAaOVgCHq2qh96QAnL7ZKzFkGf/MdwV10KU3dmb+ICbQUUdf9Gc17
|
||||
aaDCIRws312F433FdXBkGs2UkB7ZZme9dfn6O1QbeTNvex2VLMqYx/CTkfFbOQA=
|
||||
-----END CERTIFICATE-----
|
28
tests/integration/test_server_reload/configs/server.key
Normal file
28
tests/integration/test_server_reload/configs/server.key
Normal file
@ -0,0 +1,28 @@
|
||||
-----BEGIN PRIVATE KEY-----
|
||||
MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCvsFVBHahdrl7z
|
||||
dDCQYJwB8l4uiU4uKfxeooLKwo0jYyVsxx9L0JivLo33BQpIn1vPIvluXh3LbhLU
|
||||
KjRyvrfcfWImbV64+pv15Dl7NtJmIBDUpm22x264kz52Qc2a6ulKjJcCBzTQGviN
|
||||
zTDkUMVUWiNMm15xlTuXpa+9MR6ZNN8S9CVih6RKmKeHW9mgUIBest1sOJIcYyGK
|
||||
qVGF1lxhZ1SEVOxSlRADNcciqr0M140ObFTP8Mh5J73hBd2/1+IIDfijZeSjnNHF
|
||||
g4df12mrSQD2lFD6mXKu+rcpy2z+uF5t3cR3I/oLUZDuu3n+jEoOpet5t10D+kG/
|
||||
jCJHdOjrAgMBAAECggEARF66zrxb6RkSmmt8+rKeA6PuQu3sHsr4C1vyyjUr97l9
|
||||
tvdGlpp20LWtSZQMjHZ3pARYTTsTHTeY3DgQcRcHNicVKx8k3ZepWeeW9vw+pL+V
|
||||
zSt3RsoVrH6gsCSrfr4sS3aqzX9AbjwQvh48CJ3mLQ1m70kHV+xbZIh1+4pB/hyP
|
||||
1wKyUE18ZkOptXvO/TtoHzLQCecpkXtWzmry1Eh2isvXA+NMrAtLibGsyM1mtm7i
|
||||
5ozevzHabvvCDBEe+KgZdONgVhhhvm2eOd+/s4w3rw4ETud4fI/ZAJyWXhiIKFnA
|
||||
VJbElWruSAoVBW7p2bsF5PbmVzvo8vXL+VylxYD+AQKBgQDhLoRKTVhNkn/QjKxq
|
||||
sdOh+QZra0LzjVpAmkQzu7wZMSHEz9qePQciDQQrYKrmRF1vNcIRCVUTqWYheJ/1
|
||||
lKRrCGa0ab6k96zkWMqLHD5u+UeJV7r1dJIx08ME9kNJ+x/XtB8klRIji16NiQUS
|
||||
qc6p8z0M2AnbJzsRfWZRH8FeYwKBgQDHu8dzdtVGI7MtxfPOE/bfajiopDg8BdTC
|
||||
pdug2T8XofRHRq7Q+0vYjTAZFT/slib91Pk6VvvPdo9VBZiL4omv4dAq6mOOdX/c
|
||||
U14mJe1X5GCrr8ExZ8BfNJ3t/6sV1fcxyJwAw7iBguqxA2JqdM/wFk10K8XqvzVn
|
||||
CD6O9yGt2QKBgFX1BMi8N538809vs41S7l9hCQNOQZNo/O+2M5yv6ECRkbtoQKKw
|
||||
1x03bMUGNJaLuELweXE5Z8GGo5bZTe5X3F+DKHlr+DtO1C+ieUaa9HY2MAmMdLCn
|
||||
2/qrREGLo+oEs4YKmuzC/taUp/ZNPKOAMISNdluFyFVg51pozPrgrVbTAoGBAKkE
|
||||
LBl3O67o0t0vH8sJdeVFG8EJhlS0koBMnfgVHqC++dm+5HwPyvTrNQJkyv1HaqNt
|
||||
r6FArkG3ED9gRuBIyT6+lctbIPgSUip9mbQqcBfqOCvQxGksZMur2ODncz09HLtS
|
||||
CUFUXjOqNzOnq4ZuZu/Bz7U4vXiSaXxQq6+LTUKxAoGAFZU/qrI06XxnrE9A1X0W
|
||||
l7DSkpZaDcu11NrZ473yONih/xOZNh4SSBpX8a7F6Pmh9BdtGqphML8NFPvQKcfP
|
||||
b9H2iid2tc292uyrUEb5uTMmv61zoTwtitqLzO0+tS6PT3fXobX+eyeEWKzPBljL
|
||||
HFtxG5CCXpkdnWRmaJnhTzA=
|
||||
-----END PRIVATE KEY-----
|
18
tests/integration/test_server_reload/configs/ssl_conf.xml
Normal file
18
tests/integration/test_server_reload/configs/ssl_conf.xml
Normal file
@ -0,0 +1,18 @@
|
||||
<?xml version="1.0"?>
|
||||
<clickhouse>
|
||||
<!-- 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 -->
|
||||
<certificateFile>/etc/clickhouse-server/config.d/server.crt</certificateFile>
|
||||
<privateKeyFile>/etc/clickhouse-server/config.d/server.key</privateKeyFile>
|
||||
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
|
||||
<dhParamsFile>/etc/clickhouse-server/config.d/dhparam.pem</dhParamsFile>
|
||||
<verificationMode>none</verificationMode>
|
||||
<loadDefaultCAFile>true</loadDefaultCAFile>
|
||||
<cacheSessions>true</cacheSessions>
|
||||
<disableProtocols>sslv2,sslv3</disableProtocols>
|
||||
<preferServerCiphers>true</preferServerCiphers>
|
||||
</server>
|
||||
</openSSL>
|
||||
</clickhouse>
|
@ -0,0 +1,174 @@
|
||||
/* This file describes gRPC protocol supported in ClickHouse.
|
||||
*
|
||||
* To use this protocol a client should send one or more messages of the QueryInfo type
|
||||
* and then receive one or more messages of the Result type.
|
||||
* According to that the service provides four methods for that:
|
||||
* ExecuteQuery(QueryInfo) returns (Result)
|
||||
* ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result)
|
||||
* ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result)
|
||||
* ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result)
|
||||
* It's up to the client to choose which method to use.
|
||||
* For example, ExecuteQueryWithStreamInput() allows the client to add data multiple times
|
||||
* while executing a query, which is suitable for inserting many rows.
|
||||
*/
|
||||
|
||||
syntax = "proto3";
|
||||
|
||||
package clickhouse.grpc;
|
||||
|
||||
message NameAndType {
|
||||
string name = 1;
|
||||
string type = 2;
|
||||
}
|
||||
|
||||
// Describes an external table - a table which will exists only while a query is executing.
|
||||
message ExternalTable {
|
||||
// Name of the table. If omitted, "_data" is used.
|
||||
string name = 1;
|
||||
|
||||
// Columns of the table. Types are required, names can be omitted. If the names are omitted, "_1", "_2", ... is used.
|
||||
repeated NameAndType columns = 2;
|
||||
|
||||
// Data to insert to the external table.
|
||||
// If a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used,
|
||||
// then data for insertion to the same external table can be split between multiple QueryInfos.
|
||||
bytes data = 3;
|
||||
|
||||
// Format of the data to insert to the external table.
|
||||
string format = 4;
|
||||
|
||||
// Settings for executing that insertion, applied after QueryInfo.settings.
|
||||
map<string, string> settings = 5;
|
||||
}
|
||||
|
||||
enum CompressionAlgorithm {
|
||||
NO_COMPRESSION = 0;
|
||||
DEFLATE = 1;
|
||||
GZIP = 2;
|
||||
STREAM_GZIP = 3;
|
||||
}
|
||||
|
||||
enum CompressionLevel {
|
||||
COMPRESSION_NONE = 0;
|
||||
COMPRESSION_LOW = 1;
|
||||
COMPRESSION_MEDIUM = 2;
|
||||
COMPRESSION_HIGH = 3;
|
||||
}
|
||||
|
||||
message Compression {
|
||||
CompressionAlgorithm algorithm = 1;
|
||||
CompressionLevel level = 2;
|
||||
}
|
||||
|
||||
// Information about a query which a client sends to a ClickHouse server.
|
||||
// The first QueryInfo can set any of the following fields. Extra QueryInfos only add extra data.
|
||||
// In extra QueryInfos only `input_data`, `external_tables`, `next_query_info` and `cancel` fields can be set.
|
||||
message QueryInfo {
|
||||
string query = 1;
|
||||
string query_id = 2;
|
||||
map<string, string> settings = 3;
|
||||
|
||||
// Default database.
|
||||
string database = 4;
|
||||
|
||||
// Input data, used both as data for INSERT query and as data for the input() function.
|
||||
bytes input_data = 5;
|
||||
|
||||
// Delimiter for input_data, inserted between input_data from adjacent QueryInfos.
|
||||
bytes input_data_delimiter = 6;
|
||||
|
||||
// Default output format. If not specified, 'TabSeparated' is used.
|
||||
string output_format = 7;
|
||||
|
||||
repeated ExternalTable external_tables = 8;
|
||||
|
||||
string user_name = 9;
|
||||
string password = 10;
|
||||
string quota = 11;
|
||||
|
||||
// Works exactly like sessions in the HTTP protocol.
|
||||
string session_id = 12;
|
||||
bool session_check = 13;
|
||||
uint32 session_timeout = 14;
|
||||
|
||||
// Set `cancel` to true to stop executing the query.
|
||||
bool cancel = 15;
|
||||
|
||||
// If true there will be at least one more QueryInfo in the input stream.
|
||||
// `next_query_info` is allowed to be set only if a method with streaming input (i.e. ExecuteQueryWithStreamInput() or ExecuteQueryWithStreamIO()) is used.
|
||||
bool next_query_info = 16;
|
||||
|
||||
/// Controls how a ClickHouse server will compress query execution results before sending back to the client.
|
||||
/// If not set the compression settings from the configuration file will be used.
|
||||
Compression result_compression = 17;
|
||||
}
|
||||
|
||||
enum LogsLevel {
|
||||
LOG_NONE = 0;
|
||||
LOG_FATAL = 1;
|
||||
LOG_CRITICAL = 2;
|
||||
LOG_ERROR = 3;
|
||||
LOG_WARNING = 4;
|
||||
LOG_NOTICE = 5;
|
||||
LOG_INFORMATION = 6;
|
||||
LOG_DEBUG = 7;
|
||||
LOG_TRACE = 8;
|
||||
}
|
||||
|
||||
message LogEntry {
|
||||
uint32 time = 1;
|
||||
uint32 time_microseconds = 2;
|
||||
uint64 thread_id = 3;
|
||||
string query_id = 4;
|
||||
LogsLevel level = 5;
|
||||
string source = 6;
|
||||
string text = 7;
|
||||
}
|
||||
|
||||
message Progress {
|
||||
uint64 read_rows = 1;
|
||||
uint64 read_bytes = 2;
|
||||
uint64 total_rows_to_read = 3;
|
||||
uint64 written_rows = 4;
|
||||
uint64 written_bytes = 5;
|
||||
}
|
||||
|
||||
message Stats {
|
||||
uint64 rows = 1;
|
||||
uint64 blocks = 2;
|
||||
uint64 allocated_bytes = 3;
|
||||
bool applied_limit = 4;
|
||||
uint64 rows_before_limit = 5;
|
||||
}
|
||||
|
||||
message Exception {
|
||||
int32 code = 1;
|
||||
string name = 2;
|
||||
string display_text = 3;
|
||||
string stack_trace = 4;
|
||||
}
|
||||
|
||||
// Result of execution of a query which is sent back by the ClickHouse server to the client.
|
||||
message Result {
|
||||
// Output of the query, represented in the `output_format` or in a format specified in `query`.
|
||||
bytes output = 1;
|
||||
bytes totals = 2;
|
||||
bytes extremes = 3;
|
||||
|
||||
repeated LogEntry logs = 4;
|
||||
Progress progress = 5;
|
||||
Stats stats = 6;
|
||||
|
||||
// Set by the ClickHouse server if there was an exception thrown while executing.
|
||||
Exception exception = 7;
|
||||
|
||||
// Set by the ClickHouse server if executing was cancelled by the `cancel` field in QueryInfo.
|
||||
bool cancelled = 8;
|
||||
}
|
||||
|
||||
service ClickHouse {
|
||||
rpc ExecuteQuery(QueryInfo) returns (Result) {}
|
||||
rpc ExecuteQueryWithStreamInput(stream QueryInfo) returns (Result) {}
|
||||
rpc ExecuteQueryWithStreamOutput(QueryInfo) returns (stream Result) {}
|
||||
rpc ExecuteQueryWithStreamIO(stream QueryInfo) returns (stream Result) {}
|
||||
}
|
284
tests/integration/test_server_reload/test.py
Normal file
284
tests/integration/test_server_reload/test.py
Normal file
@ -0,0 +1,284 @@
|
||||
import contextlib
|
||||
import grpc
|
||||
import psycopg2
|
||||
import pymysql.connections
|
||||
import pymysql.err
|
||||
import pytest
|
||||
import sys
|
||||
import time
|
||||
from helpers.cluster import ClickHouseCluster, run_and_check
|
||||
from helpers.client import Client, QueryRuntimeException
|
||||
from kazoo.exceptions import NodeExistsError
|
||||
from pathlib import Path
|
||||
from requests.exceptions import ConnectionError
|
||||
from urllib3.util.retry import Retry
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance(
|
||||
"instance",
|
||||
main_configs=[
|
||||
"configs/ports_from_zk.xml", "configs/ssl_conf.xml", "configs/dhparam.pem", "configs/server.crt", "configs/server.key"
|
||||
],
|
||||
user_configs=["configs/default_passwd.xml"],
|
||||
with_zookeeper=True)
|
||||
|
||||
|
||||
LOADS_QUERY = "SELECT value FROM system.events WHERE event = 'MainConfigLoads'"
|
||||
|
||||
|
||||
# Use grpcio-tools to generate *pb2.py files from *.proto.
|
||||
|
||||
proto_dir = Path(__file__).parent / "protos"
|
||||
gen_dir = Path(__file__).parent / "_gen"
|
||||
gen_dir.mkdir(exist_ok=True)
|
||||
run_and_check(
|
||||
f"python3 -m grpc_tools.protoc -I{proto_dir!s} --python_out={gen_dir!s} --grpc_python_out={gen_dir!s} \
|
||||
{proto_dir!s}/clickhouse_grpc.proto", shell=True)
|
||||
|
||||
sys.path.append(str(gen_dir))
|
||||
import clickhouse_grpc_pb2
|
||||
import clickhouse_grpc_pb2_grpc
|
||||
|
||||
|
||||
@pytest.fixture(name="cluster", scope="module")
|
||||
def fixture_cluster():
|
||||
try:
|
||||
cluster.add_zookeeper_startup_command(configure_ports_from_zk)
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
@pytest.fixture(name="zk", scope="module")
|
||||
def fixture_zk(cluster):
|
||||
return cluster.get_kazoo_client("zoo1")
|
||||
|
||||
|
||||
def get_client(cluster, port):
|
||||
return Client(host=cluster.get_instance_ip("instance"), port=port, command=cluster.client_bin_path)
|
||||
|
||||
|
||||
def get_mysql_client(cluster, port):
|
||||
start_time = time.monotonic()
|
||||
while True:
|
||||
try:
|
||||
return pymysql.connections.Connection(
|
||||
host=cluster.get_instance_ip("instance"), user="default", password="", database="default", port=port)
|
||||
except pymysql.err.OperationalError:
|
||||
if time.monotonic() - start_time > 10:
|
||||
raise
|
||||
time.sleep(0.1)
|
||||
|
||||
|
||||
def get_pgsql_client(cluster, port):
|
||||
start_time = time.monotonic()
|
||||
while True:
|
||||
try:
|
||||
return psycopg2.connect(
|
||||
host=cluster.get_instance_ip("instance"), user="postgresql", password="123", database="default", port=port)
|
||||
except psycopg2.OperationalError:
|
||||
if time.monotonic() - start_time > 10:
|
||||
raise
|
||||
time.sleep(0.1)
|
||||
|
||||
|
||||
def get_grpc_channel(cluster, port):
|
||||
host_port = cluster.get_instance_ip("instance") + f":{port}"
|
||||
channel = grpc.insecure_channel(host_port)
|
||||
grpc.channel_ready_future(channel).result(timeout=10)
|
||||
return channel
|
||||
|
||||
|
||||
def grpc_query(channel, query_text):
|
||||
query_info = clickhouse_grpc_pb2.QueryInfo(query=query_text)
|
||||
stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(channel)
|
||||
result = stub.ExecuteQuery(query_info)
|
||||
if result and result.HasField("exception"):
|
||||
raise Exception(result.exception.display_text)
|
||||
return result.output.decode()
|
||||
|
||||
|
||||
def configure_ports_from_zk(zk, querier=None):
|
||||
default_config = [
|
||||
("/clickhouse/listen_hosts", b"<listen_host>0.0.0.0</listen_host>"),
|
||||
("/clickhouse/ports/tcp", b"9000"),
|
||||
("/clickhouse/ports/http", b"8123"),
|
||||
("/clickhouse/ports/mysql", b"9004"),
|
||||
("/clickhouse/ports/postgresql", b"9005"),
|
||||
("/clickhouse/ports/grpc", b"9100"),
|
||||
]
|
||||
for path, value in default_config:
|
||||
if querier is not None:
|
||||
loads_before = querier(LOADS_QUERY)
|
||||
has_changed = False
|
||||
try:
|
||||
zk.create(path=path, value=value, makepath=True)
|
||||
has_changed = True
|
||||
except NodeExistsError:
|
||||
if zk.get(path) != value:
|
||||
zk.set(path=path, value=value)
|
||||
has_changed = True
|
||||
if has_changed and querier is not None:
|
||||
wait_loaded_config_changed(loads_before, querier)
|
||||
|
||||
|
||||
@contextlib.contextmanager
|
||||
def sync_loaded_config(querier):
|
||||
# Depending on whether we test a change on tcp or http
|
||||
# we monitor canges using the other, untouched, protocol
|
||||
loads_before = querier(LOADS_QUERY)
|
||||
yield
|
||||
wait_loaded_config_changed(loads_before, querier)
|
||||
|
||||
|
||||
def wait_loaded_config_changed(loads_before, querier):
|
||||
loads_after = None
|
||||
start_time = time.monotonic()
|
||||
while time.monotonic() - start_time < 10:
|
||||
try:
|
||||
loads_after = querier(LOADS_QUERY)
|
||||
if loads_after != loads_before:
|
||||
return
|
||||
except (QueryRuntimeException, ConnectionError):
|
||||
pass
|
||||
time.sleep(0.1)
|
||||
assert loads_after is not None and loads_after != loads_before
|
||||
|
||||
|
||||
@contextlib.contextmanager
|
||||
def default_client(cluster, zk, restore_via_http=False):
|
||||
client = get_client(cluster, port=9000)
|
||||
try:
|
||||
yield client
|
||||
finally:
|
||||
querier = instance.http_query if restore_via_http else client.query
|
||||
configure_ports_from_zk(zk, querier)
|
||||
|
||||
|
||||
def test_change_tcp_port(cluster, zk):
|
||||
with default_client(cluster, zk, restore_via_http=True) as client:
|
||||
assert client.query("SELECT 1") == "1\n"
|
||||
with sync_loaded_config(instance.http_query):
|
||||
zk.set("/clickhouse/ports/tcp", b"9090")
|
||||
with pytest.raises(QueryRuntimeException, match="Connection refused"):
|
||||
client.query("SELECT 1")
|
||||
client_on_new_port = get_client(cluster, port=9090)
|
||||
assert client_on_new_port.query("SELECT 1") == "1\n"
|
||||
|
||||
|
||||
def test_change_http_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
retry_strategy = Retry(total=10, backoff_factor=0.1)
|
||||
assert instance.http_query("SELECT 1", retry_strategy=retry_strategy) == "1\n"
|
||||
with sync_loaded_config(client.query):
|
||||
zk.set("/clickhouse/ports/http", b"9090")
|
||||
with pytest.raises(ConnectionError, match="Connection refused"):
|
||||
instance.http_query("SELECT 1")
|
||||
instance.http_query("SELECT 1", port=9090) == "1\n"
|
||||
|
||||
|
||||
def test_change_mysql_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
mysql_client = get_mysql_client(cluster, port=9004)
|
||||
assert mysql_client.query("SELECT 1") == 1
|
||||
with sync_loaded_config(client.query):
|
||||
zk.set("/clickhouse/ports/mysql", b"9090")
|
||||
with pytest.raises(pymysql.err.OperationalError, match="Lost connection"):
|
||||
mysql_client.query("SELECT 1")
|
||||
mysql_client_on_new_port = get_mysql_client(cluster, port=9090)
|
||||
assert mysql_client_on_new_port.query("SELECT 1") == 1
|
||||
|
||||
|
||||
def test_change_postgresql_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
pgsql_client = get_pgsql_client(cluster, port=9005)
|
||||
cursor = pgsql_client.cursor()
|
||||
cursor.execute("SELECT 1")
|
||||
assert cursor.fetchall() == [(1,)]
|
||||
with sync_loaded_config(client.query):
|
||||
zk.set("/clickhouse/ports/postgresql", b"9090")
|
||||
with pytest.raises(psycopg2.OperationalError, match="closed"):
|
||||
cursor.execute("SELECT 1")
|
||||
pgsql_client_on_new_port = get_pgsql_client(cluster, port=9090)
|
||||
cursor = pgsql_client_on_new_port.cursor()
|
||||
cursor.execute("SELECT 1")
|
||||
cursor.fetchall() == [(1,)]
|
||||
|
||||
|
||||
def test_change_grpc_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
grpc_channel = get_grpc_channel(cluster, port=9100)
|
||||
assert grpc_query(grpc_channel, "SELECT 1") == "1\n"
|
||||
with sync_loaded_config(client.query):
|
||||
zk.set("/clickhouse/ports/grpc", b"9090")
|
||||
with pytest.raises(grpc._channel._InactiveRpcError, match="StatusCode.UNAVAILABLE"):
|
||||
grpc_query(grpc_channel, "SELECT 1")
|
||||
grpc_channel_on_new_port = get_grpc_channel(cluster, port=9090)
|
||||
assert grpc_query(grpc_channel_on_new_port, "SELECT 1") == "1\n"
|
||||
|
||||
|
||||
def test_remove_tcp_port(cluster, zk):
|
||||
with default_client(cluster, zk, restore_via_http=True) as client:
|
||||
assert client.query("SELECT 1") == "1\n"
|
||||
with sync_loaded_config(instance.http_query):
|
||||
zk.delete("/clickhouse/ports/tcp")
|
||||
with pytest.raises(QueryRuntimeException, match="Connection refused"):
|
||||
client.query("SELECT 1")
|
||||
|
||||
|
||||
def test_remove_http_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
assert instance.http_query("SELECT 1") == "1\n"
|
||||
with sync_loaded_config(client.query):
|
||||
zk.delete("/clickhouse/ports/http")
|
||||
with pytest.raises(ConnectionError, match="Connection refused"):
|
||||
instance.http_query("SELECT 1")
|
||||
|
||||
|
||||
def test_remove_mysql_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
mysql_client = get_mysql_client(cluster, port=9004)
|
||||
assert mysql_client.query("SELECT 1") == 1
|
||||
with sync_loaded_config(client.query):
|
||||
zk.delete("/clickhouse/ports/mysql")
|
||||
with pytest.raises(pymysql.err.OperationalError, match="Lost connection"):
|
||||
mysql_client.query("SELECT 1")
|
||||
|
||||
|
||||
def test_remove_postgresql_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
pgsql_client = get_pgsql_client(cluster, port=9005)
|
||||
cursor = pgsql_client.cursor()
|
||||
cursor.execute("SELECT 1")
|
||||
assert cursor.fetchall() == [(1,)]
|
||||
with sync_loaded_config(client.query):
|
||||
zk.delete("/clickhouse/ports/postgresql")
|
||||
with pytest.raises(psycopg2.OperationalError, match="closed"):
|
||||
cursor.execute("SELECT 1")
|
||||
|
||||
|
||||
def test_remove_grpc_port(cluster, zk):
|
||||
with default_client(cluster, zk) as client:
|
||||
grpc_channel = get_grpc_channel(cluster, port=9100)
|
||||
assert grpc_query(grpc_channel, "SELECT 1") == "1\n"
|
||||
with sync_loaded_config(client.query):
|
||||
zk.delete("/clickhouse/ports/grpc")
|
||||
with pytest.raises(grpc._channel._InactiveRpcError, match="StatusCode.UNAVAILABLE"):
|
||||
grpc_query(grpc_channel, "SELECT 1")
|
||||
|
||||
|
||||
def test_change_listen_host(cluster, zk):
|
||||
localhost_client = Client(host="127.0.0.1", port=9000, command="/usr/bin/clickhouse")
|
||||
localhost_client.command = ["docker", "exec", "-i", instance.docker_id] + localhost_client.command
|
||||
try:
|
||||
client = get_client(cluster, port=9000)
|
||||
with sync_loaded_config(localhost_client.query):
|
||||
zk.set("/clickhouse/listen_hosts", b"<listen_host>127.0.0.1</listen_host>")
|
||||
with pytest.raises(QueryRuntimeException, match="Connection refused"):
|
||||
client.query("SELECT 1")
|
||||
assert localhost_client.query("SELECT 1") == "1\n"
|
||||
finally:
|
||||
with sync_loaded_config(localhost_client.query):
|
||||
configure_ports_from_zk(zk)
|
||||
|
@ -424,6 +424,21 @@ def test_predefined_connection_configuration(started_cluster):
|
||||
cursor.execute(f'DROP TABLE test_table ')
|
||||
|
||||
|
||||
def test_where_false(started_cluster):
|
||||
cursor = started_cluster.postgres_conn.cursor()
|
||||
cursor.execute("DROP TABLE IF EXISTS test")
|
||||
cursor.execute('CREATE TABLE test (a Integer)')
|
||||
cursor.execute("INSERT INTO test SELECT 1")
|
||||
|
||||
result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 1=0")
|
||||
assert(int(result) == 0)
|
||||
result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 0")
|
||||
assert(int(result) == 0)
|
||||
result = node1.query("SELECT count() FROM postgresql('postgres1:5432', 'postgres', 'test', 'postgres', 'mysecretpassword') WHERE 1=1")
|
||||
assert(int(result) == 1)
|
||||
cursor.execute("DROP TABLE test")
|
||||
|
||||
|
||||
if __name__ == '__main__':
|
||||
cluster.start()
|
||||
input("Cluster created, press any key to destroy...")
|
||||
|
@ -59,3 +59,32 @@ def test_readonly_metrics(start_cluster):
|
||||
node1.query("ATTACH TABLE test.test_table")
|
||||
assert_eq_with_retry(node1, "SELECT value FROM system.metrics WHERE metric = 'ReadonlyReplica'", "0\n", retry_count=300, sleep_time=1)
|
||||
|
||||
#For LowCardinality-columns, the bytes for N rows is not N*size of 1 row.
|
||||
def test_metrics_storage_buffer_size(start_cluster):
|
||||
node1.query('''
|
||||
CREATE TABLE test.test_mem_table
|
||||
(
|
||||
`str` LowCardinality(String)
|
||||
)
|
||||
ENGINE = Memory;
|
||||
|
||||
CREATE TABLE test.buffer_table
|
||||
(
|
||||
`str` LowCardinality(String)
|
||||
)
|
||||
ENGINE = Buffer('test', 'test_mem_table', 1, 600, 600, 1000, 100000, 100000, 10000000);
|
||||
''')
|
||||
|
||||
#before flush
|
||||
node1.query("INSERT INTO test.buffer_table VALUES('hello');")
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "1\n"
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "24\n"
|
||||
|
||||
node1.query("INSERT INTO test.buffer_table VALUES('hello');")
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "2\n"
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "25\n"
|
||||
|
||||
#flush
|
||||
node1.query("OPTIMIZE TABLE test.buffer_table")
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferRows'") == "0\n"
|
||||
assert node1.query("SELECT value FROM system.metrics WHERE metric = 'StorageBufferBytes'") == "0\n"
|
||||
|
@ -20,6 +20,7 @@ expect "SET max_distributed"
|
||||
|
||||
# Wait for suggestions to load, they are loaded in background
|
||||
set is_done 0
|
||||
set timeout 1
|
||||
while {$is_done == 0} {
|
||||
send -- "\t"
|
||||
expect {
|
||||
@ -27,10 +28,15 @@ while {$is_done == 0} {
|
||||
set is_done 1
|
||||
}
|
||||
default {
|
||||
sleep 1
|
||||
# expect "_" will wait for timeout,
|
||||
# if completion was not loaded it will fail,
|
||||
# and we will retry,
|
||||
# but for retry on timeout default should be reseted,
|
||||
# this is what for this block.
|
||||
}
|
||||
}
|
||||
}
|
||||
set timeout 60
|
||||
|
||||
send -- "\3\4"
|
||||
expect eof
|
||||
|
@ -1,5 +0,0 @@
|
||||
## Build with debug mode and without many libraries
|
||||
|
||||
This job is intended as first check that build is not broken on wide variety of platforms.
|
||||
|
||||
Results of this build are not intended for production usage.
|
@ -1,30 +0,0 @@
|
||||
#!/usr/bin/env bash
|
||||
set -e -x
|
||||
|
||||
# How to run:
|
||||
# From "ci" directory:
|
||||
# jobs/quick-build/run.sh
|
||||
# or:
|
||||
# ./run-with-docker.sh ubuntu:bionic jobs/quick-build/run.sh
|
||||
|
||||
cd "$(dirname $0)"/../..
|
||||
|
||||
. default-config
|
||||
|
||||
SOURCES_METHOD=local
|
||||
COMPILER=clang
|
||||
COMPILER_INSTALL_METHOD=packages
|
||||
COMPILER_PACKAGE_VERSION=6.0
|
||||
BUILD_METHOD=normal
|
||||
BUILD_TARGETS=clickhouse
|
||||
BUILD_TYPE=Debug
|
||||
ENABLE_EMBEDDED_COMPILER=0
|
||||
|
||||
CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_JEMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_REDIS=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_SSL=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_CASSANDRA=0 -D ENABLE_LDAP=0"
|
||||
|
||||
[[ $(uname) == "FreeBSD" ]] && COMPILER_PACKAGE_VERSION=devel && export COMPILER_PATH=/usr/local/bin
|
||||
|
||||
. get-sources.sh
|
||||
. prepare-toolchain.sh
|
||||
. install-libraries.sh
|
||||
. build-normal.sh
|
1
utils/ci/vagrant-freebsd/.gitignore
vendored
1
utils/ci/vagrant-freebsd/.gitignore
vendored
@ -1 +0,0 @@
|
||||
.vagrant
|
3
utils/ci/vagrant-freebsd/Vagrantfile
vendored
3
utils/ci/vagrant-freebsd/Vagrantfile
vendored
@ -1,3 +0,0 @@
|
||||
Vagrant.configure("2") do |config|
|
||||
config.vm.box = "generic/freebsd11"
|
||||
end
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
@ -1,41 +0,0 @@
|
||||
How to use Fuzzer
|
||||
===
|
||||
|
||||
The fuzzer consists of auto-generated files:
|
||||
|
||||
ClickHouseUnlexer.py
|
||||
ClickHouseUnparser.py
|
||||
|
||||
They are generated from grammar files (.g4) using Grammarinator:
|
||||
|
||||
pip3 install grammarinator
|
||||
grammarinator-process ClickHouseLexer.g4 ClickHouseParser.g4 -o fuzzer/
|
||||
|
||||
Then you can generate test input for ClickHouse client:
|
||||
|
||||
cd fuzzer
|
||||
grammarinator-generate \
|
||||
-r query_list \ # top-level rule
|
||||
-o /tmp/sql_test_%d.sql \ # template for output test names
|
||||
-n 10 \ # number of tests
|
||||
-c 0.3 \
|
||||
-d 20 \ # depth of recursion
|
||||
-p ClickHouseUnparser.py -l ClickHouseUnlexer.py \ # auto-generated unparser and unlexer
|
||||
--test-transformers SpaceTransformer.single_line_whitespace \ # transform function to insert whitespace
|
||||
|
||||
For more details see `grammarinator-generate --help`. As a test-transformer function also can be used `SpaceTransformer.multi_line_transformer` - both functions reside in `fuzzer/SpaceTransformer.py` file.
|
||||
|
||||
|
||||
Parsing steps
|
||||
===
|
||||
|
||||
1. Replace all operators with corresponding functions.
|
||||
2. Replace all asterisks with columns - if it's inside function call, then expand it as multiple arguments. Warn about undeterministic invocations when functions have positional arguments.
|
||||
|
||||
Old vs. new parser
|
||||
===
|
||||
|
||||
- `a as b [c]` - accessing aliased array expression is not possible.
|
||||
- `a as b . 1` - accessing aliased tuple expression is not possible.
|
||||
- `between a is not null and b` - `between` operator should have lower priority than `is null`.
|
||||
- `*.1` - accessing asterisk tuple expression is not possible.
|
@ -1,38 +0,0 @@
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
from grammarinator.runtime.tree import *
|
||||
|
||||
from itertools import tee, islice, zip_longest
|
||||
import random
|
||||
|
||||
|
||||
def single_line_whitespace(node):
|
||||
return _whitespace(node, ' \t')
|
||||
|
||||
|
||||
def multi_line_whitespace(node):
|
||||
return _whitespace(node, ' \t\r\n')
|
||||
|
||||
|
||||
def _whitespace(node, symbols):
|
||||
for child in node.children:
|
||||
_whitespace(child, symbols)
|
||||
|
||||
# helper function to look ahead one child
|
||||
def with_next(iterable):
|
||||
items, nexts = tee(iterable, 2)
|
||||
nexts = islice(nexts, 1, None)
|
||||
return zip_longest(items, nexts)
|
||||
|
||||
if isinstance(node, UnparserRule):
|
||||
new_children = []
|
||||
for child, next_child in with_next(node.children):
|
||||
if (not next_child or
|
||||
next_child and isinstance(next_child, UnlexerRule) and next_child.name == 'DOT' or
|
||||
isinstance(child, UnlexerRule) and child.name == 'DOT'):
|
||||
new_children.append(child)
|
||||
else:
|
||||
new_children.extend([child, UnlexerRule(src=random.choice(symbols))])
|
||||
node.children = new_children
|
||||
|
||||
return node
|
@ -1 +0,0 @@
|
||||
# -*- coding: utf-8 -*-
|
@ -1,390 +0,0 @@
|
||||
<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0"
|
||||
xmlns:lxslt="http://xml.apache.org/xslt"
|
||||
xmlns:stringutils="xalan://org.apache.tools.ant.util.StringUtils">
|
||||
<xsl:output method="html" indent="yes" encoding="UTF-8"
|
||||
doctype-public="-//W3C//DTD HTML 4.01 Transitional//EN" />
|
||||
<xsl:decimal-format decimal-separator="." grouping-separator="," />
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one
|
||||
or more contributor license agreements. See the NOTICE file
|
||||
distributed with this work for additional information
|
||||
regarding copyright ownership. The ASF licenses this file
|
||||
to you under the Apache License, Version 2.0 (the
|
||||
"License"); you may not use this file except in compliance
|
||||
with the License. You may obtain a copy of the License at
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
Unless required by applicable law or agreed to in writing,
|
||||
software distributed under the License is distributed on an
|
||||
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
KIND, either express or implied. See the License for the
|
||||
specific language governing permissions and limitations
|
||||
under the License.
|
||||
-->
|
||||
<xsl:template match="testsuites">
|
||||
<html>
|
||||
<head>
|
||||
<title>Test Results</title>
|
||||
<style type="text/css">
|
||||
|
||||
body {
|
||||
font:normal 68% verdana,arial,helvetica;
|
||||
color:#000000;
|
||||
}
|
||||
table.details tr th{
|
||||
font-weight: bold;
|
||||
text-align:left;
|
||||
background:#ffcc00;
|
||||
}
|
||||
table.details tr td{
|
||||
background:#eeeeef;
|
||||
}
|
||||
p {
|
||||
line-height:1.5em;
|
||||
margin-top:0.5em; margin-bottom:1.0em;
|
||||
}
|
||||
h1 {
|
||||
margin: 0px 0px 5px; font: 165% verdana,arial,helvetica
|
||||
}
|
||||
h2 {
|
||||
margin-top: 1em; margin-bottom: 0.5em; font: bold 125% verdana,arial,helvetica
|
||||
}
|
||||
h3 {
|
||||
margin-bottom: 0.5em; color:#0077ff; font: bold 115% verdana,arial,helvetica
|
||||
}
|
||||
h4 {
|
||||
margin-bottom: 0.5em; font: bold 100% verdana,arial,helvetica
|
||||
}
|
||||
h5 {
|
||||
margin-bottom: 0.5em; font: bold 100% verdana,arial,helvetica
|
||||
}
|
||||
h6 {
|
||||
margin-bottom: 0.5em; font: bold 100% verdana,arial,helvetica
|
||||
}
|
||||
.Error {
|
||||
font-weight:bold; color:#ff3333;
|
||||
}
|
||||
.Failure {
|
||||
font-weight:bold; color:#0077ff;
|
||||
}
|
||||
.Properties {
|
||||
text-align:right;
|
||||
}
|
||||
</style>
|
||||
<!--
|
||||
<script type="text/javascript" language="JavaScript">
|
||||
var Projects = new Array();
|
||||
var cur;
|
||||
<xsl:for-each select="./testsuite">
|
||||
<xsl:apply-templates select="properties"/>
|
||||
</xsl:for-each>
|
||||
</script>
|
||||
<script type="text/javascript" language="JavaScript"><![CDATA[
|
||||
function displayProperties (name) {
|
||||
var win = window.open('','JUnitSystemProperties','scrollbars=1,resizable=1');
|
||||
var doc = win.document;
|
||||
doc.open();
|
||||
doc.write("<html><head><title>Properties of " + name + "</title>");
|
||||
doc.write("<style>")
|
||||
doc.write("body {font:normal 68% verdana,arial,helvetica; color:#000000; }");
|
||||
doc.write("table tr td, table tr th { font-size: 68%; }");
|
||||
doc.write("table.properties { border-collapse:collapse; border-left:solid 1 #cccccc; border-top:solid 1 #cccccc; padding:5px; }");
|
||||
doc.write("table.properties th { text-align:left; border-right:solid 1 #cccccc; border-bottom:solid 1 #cccccc; background-color:#eeeeee; }");
|
||||
doc.write("table.properties td { font:normal; text-align:left; border-right:solid 1 #cccccc; border-bottom:solid 1 #cccccc; background-color:#fffffff; }");
|
||||
doc.write("h3 { margin-bottom: 0.5em; font: bold 115% verdana,arial,helvetica }");
|
||||
doc.write("</style>");
|
||||
doc.write("</head><body>");
|
||||
doc.write("<h3>Properties of " + name + "</h3>");
|
||||
doc.write("<div align=\"right\"><a href=\"javascript:window.close();\">Close</a></div>");
|
||||
doc.write("<table class='properties'>");
|
||||
doc.write("<tr><th>Name</th><th>Value</th></tr>");
|
||||
for (prop in Projects[name]) {
|
||||
doc.write("<tr><th>" + prop + "</th><td>" + Projects[name][prop] + "</td></tr>");
|
||||
}
|
||||
doc.write("</table>");
|
||||
doc.write("</body></html>");
|
||||
doc.close();
|
||||
win.focus();
|
||||
}
|
||||
]]>
|
||||
</script>
|
||||
-->
|
||||
</head>
|
||||
<body>
|
||||
<a name="top"></a>
|
||||
<xsl:call-template name="pageHeader"/>
|
||||
<!-- Summary part -->
|
||||
<xsl:call-template name="summary"/>
|
||||
<hr size="1" width="95%" align="left"/>
|
||||
<!-- For each class create the part -->
|
||||
<xsl:call-template name="classes"/>
|
||||
</body>
|
||||
</html>
|
||||
</xsl:template>
|
||||
<xsl:template name="classes">
|
||||
<!-- Сначала упавшие -->
|
||||
<xsl:for-each select="testsuite[count(testcase/failure) > 0]">
|
||||
<xsl:call-template name="testsuite"/>
|
||||
</xsl:for-each>
|
||||
<xsl:for-each select="testsuite[count(testcase/failure) = 0]">
|
||||
<xsl:call-template name="testsuite"/>
|
||||
</xsl:for-each>
|
||||
</xsl:template>
|
||||
<xsl:template name="testsuite">
|
||||
<!-- create an anchor to this class name -->
|
||||
<a href="#{@name}">
|
||||
<h3 id="{@name}"><xsl:value-of select="@name"/></h3>
|
||||
</a>
|
||||
<table class="details" border="0" cellpadding="5" cellspacing="2" width="95%">
|
||||
<xsl:call-template name="testcase.test.header"/>
|
||||
<!--
|
||||
test can even not be started at all (failure to load the class)
|
||||
so report the error directly
|
||||
-->
|
||||
<xsl:if test="./error">
|
||||
<tr class="Error">
|
||||
<td colspan="4"><xsl:apply-templates select="./error"/></td>
|
||||
</tr>
|
||||
</xsl:if>
|
||||
<xsl:apply-templates select="./testcase" mode="print.test"/>
|
||||
</table>
|
||||
<p/>
|
||||
<a href="#top">Back to top</a>
|
||||
</xsl:template>
|
||||
<xsl:template name="summary">
|
||||
<h2>Summary</h2>
|
||||
<xsl:variable name="testCount" select="sum(testsuite/@tests)"/>
|
||||
<xsl:variable name="errorCount" select="sum(testsuite/@errors)"/>
|
||||
<xsl:variable name="failureCount" select="sum(testsuite/@failures)"/>
|
||||
<xsl:variable name="timeCount" select="sum(testsuite/@time)"/>
|
||||
<xsl:variable name="successRate" select="($testCount - $failureCount - $errorCount) div $testCount"/>
|
||||
<table class="details" border="0" cellpadding="5" cellspacing="2" width="95%">
|
||||
<tr valign="top">
|
||||
<th>Tests</th>
|
||||
<th>Failures</th>
|
||||
<th>Errors</th>
|
||||
<th>Success rate</th>
|
||||
<th>Time</th>
|
||||
</tr>
|
||||
<tr valign="top">
|
||||
<xsl:attribute name="class">
|
||||
<xsl:choose>
|
||||
<xsl:when test="$failureCount > 0">Failure</xsl:when>
|
||||
<xsl:when test="$errorCount > 0">Error</xsl:when>
|
||||
</xsl:choose>
|
||||
</xsl:attribute>
|
||||
<td><xsl:value-of select="$testCount"/></td>
|
||||
<td><xsl:value-of select="$failureCount"/></td>
|
||||
<td><xsl:value-of select="$errorCount"/></td>
|
||||
<td>
|
||||
<xsl:call-template name="display-percent">
|
||||
<xsl:with-param name="value" select="$successRate"/>
|
||||
</xsl:call-template>
|
||||
</td>
|
||||
<td>
|
||||
<xsl:call-template name="display-time">
|
||||
<xsl:with-param name="value" select="$timeCount"/>
|
||||
</xsl:call-template>
|
||||
</td>
|
||||
</tr>
|
||||
</table>
|
||||
<table border="0" width="95%">
|
||||
<tr>
|
||||
<td style="text-align: justify;">
|
||||
Note: <i>failures</i> are anticipated and checked for with assertions while <i>errors</i> are unanticipated.
|
||||
</td>
|
||||
</tr>
|
||||
</table>
|
||||
</xsl:template>
|
||||
<!--
|
||||
Write properties into a JavaScript data structure.
|
||||
This is based on the original idea by Erik Hatcher (ehatcher@apache.org)
|
||||
-->
|
||||
<!--
|
||||
<xsl:template match="properties">
|
||||
cur = Projects['<xsl:value-of select="../@package"/>.<xsl:value-of select="../@name"/>'] = new Array();
|
||||
<xsl:for-each select="property">
|
||||
<xsl:sort select="@name"/>
|
||||
cur['<xsl:value-of select="@name"/>'] = '<xsl:call-template name="JS-escape"><xsl:with-param name="string" select="@value"/></xsl:call-template>';
|
||||
</xsl:for-each>
|
||||
</xsl:template>
|
||||
-->
|
||||
<!-- Page HEADER -->
|
||||
<xsl:template name="pageHeader">
|
||||
<h1>Test Results</h1>
|
||||
<hr size="1"/>
|
||||
</xsl:template>
|
||||
<xsl:template match="testsuite" mode="header">
|
||||
<tr valign="top">
|
||||
<th width="80%">Name</th>
|
||||
<th>Tests</th>
|
||||
<th>Errors</th>
|
||||
<th>Failures</th>
|
||||
<th nowrap="nowrap">Time(s)</th>
|
||||
</tr>
|
||||
</xsl:template>
|
||||
<!-- class header -->
|
||||
<xsl:template name="testsuite.test.header">
|
||||
<tr valign="top">
|
||||
<th width="80%">Name</th>
|
||||
<th>Tests</th>
|
||||
<th>Errors</th>
|
||||
<th>Failures</th>
|
||||
<th nowrap="nowrap">Time(s)</th>
|
||||
<th nowrap="nowrap">Time Stamp</th>
|
||||
<th>Host</th>
|
||||
</tr>
|
||||
</xsl:template>
|
||||
<!-- method header -->
|
||||
<xsl:template name="testcase.test.header">
|
||||
<tr valign="top">
|
||||
<th>Name</th>
|
||||
<th>Status</th>
|
||||
<th width="80%">Type</th>
|
||||
<th nowrap="nowrap">Time(s)</th>
|
||||
</tr>
|
||||
</xsl:template>
|
||||
<!-- class information -->
|
||||
<xsl:template match="testsuite" mode="print.test">
|
||||
<tr valign="top">
|
||||
<!-- set a nice color depending if there is an error/failure -->
|
||||
<xsl:attribute name="class">
|
||||
<xsl:choose>
|
||||
<xsl:when test="failures/text()[.> 0]">Failure</xsl:when>
|
||||
<xsl:when test="errors/text()[.> 0]">Error</xsl:when>
|
||||
</xsl:choose>
|
||||
</xsl:attribute>
|
||||
<!-- print testsuite information -->
|
||||
<td><a href="#{@name}"><xsl:value-of select="@name"/></a></td>
|
||||
<td><xsl:value-of select="tests/text()"/></td>
|
||||
<td><xsl:value-of select="errors/text()"/></td>
|
||||
<td><xsl:value-of select="failures/text()"/></td>
|
||||
<td>
|
||||
<xsl:call-template name="display-time">
|
||||
<xsl:with-param name="value" select="time/text()"/>
|
||||
</xsl:call-template>
|
||||
</td>
|
||||
<td><xsl:apply-templates select="@timestamp"/></td>
|
||||
<td><xsl:apply-templates select="@hostname"/></td>
|
||||
</tr>
|
||||
</xsl:template>
|
||||
<xsl:template match="testcase" mode="print.test">
|
||||
<tr valign="top">
|
||||
<xsl:attribute name="class">
|
||||
<xsl:choose>
|
||||
<xsl:when test="error">Error</xsl:when>
|
||||
<xsl:when test="failure">Failure</xsl:when>
|
||||
<xsl:otherwise>TableRowColor</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
</xsl:attribute>
|
||||
<td><xsl:value-of select="@name"/></td>
|
||||
<xsl:choose>
|
||||
<xsl:when test="failure">
|
||||
<td>Failure</td>
|
||||
<td><xsl:apply-templates select="failure"/></td>
|
||||
</xsl:when>
|
||||
<xsl:when test="error">
|
||||
<td>Error</td>
|
||||
<td><xsl:apply-templates select="error"/></td>
|
||||
</xsl:when>
|
||||
<xsl:otherwise>
|
||||
<td>Success</td>
|
||||
<td></td>
|
||||
</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
<td>
|
||||
<xsl:call-template name="display-time">
|
||||
<xsl:with-param name="value" select="@time"/>
|
||||
</xsl:call-template>
|
||||
</td>
|
||||
</tr>
|
||||
</xsl:template>
|
||||
<xsl:template match="failure">
|
||||
<xsl:call-template name="display-failures"/>
|
||||
<br/><br/>
|
||||
<code>
|
||||
<xsl:call-template name="br-replace">
|
||||
<xsl:with-param name="word" select="."/>
|
||||
</xsl:call-template>
|
||||
</code>
|
||||
</xsl:template>
|
||||
<xsl:template match="error">
|
||||
<xsl:call-template name="display-failures"/>
|
||||
<!-- display the stacktrace -->
|
||||
<br/><br/>
|
||||
<code>
|
||||
<xsl:call-template name="br-replace">
|
||||
<xsl:with-param name="word" select="."/>
|
||||
</xsl:call-template>
|
||||
</code>
|
||||
</xsl:template>
|
||||
<!-- Style for the error and failure in the testcase template -->
|
||||
<xsl:template name="display-failures">
|
||||
<xsl:choose>
|
||||
<xsl:when test="not(@message)">N/A</xsl:when>
|
||||
<xsl:otherwise>
|
||||
<xsl:value-of select="@message"/>
|
||||
</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
<xsl:choose>
|
||||
<xsl:when test="@linenumber">
|
||||
<br></br>
|
||||
at line <xsl:value-of select="@linenumber"/>
|
||||
<xsl:choose>
|
||||
<xsl:when test="@columnnumber">
|
||||
, column <xsl:value-of select="@columnnumber"/>
|
||||
</xsl:when>
|
||||
</xsl:choose>
|
||||
</xsl:when>
|
||||
</xsl:choose>
|
||||
</xsl:template>
|
||||
<xsl:template name="JS-escape">
|
||||
<xsl:param name="string"/>
|
||||
<xsl:param name="tmp1" select="stringutils:replace(string($string),'\','\\')"/>
|
||||
<xsl:param name="tmp2" select="stringutils:replace(string($tmp1),"'","\'")"/>
|
||||
<xsl:value-of select="$tmp2"/>
|
||||
</xsl:template>
|
||||
<!--
|
||||
template that will convert a carriage return into a br tag
|
||||
@param word the text from which to convert CR to BR tag
|
||||
-->
|
||||
<xsl:template name="br-replace">
|
||||
<xsl:param name="word"/>
|
||||
<xsl:param name="splitlimit">32</xsl:param>
|
||||
<xsl:variable name="secondhalflen" select="(string-length($word)+(string-length($word) mod 2)) div 2"/>
|
||||
<xsl:variable name="secondhalfword" select="substring($word, $secondhalflen)"/>
|
||||
<!-- When word is very big, a recursive replace is very heap/stack expensive, so subdivide on line break after middle of string -->
|
||||
<xsl:choose>
|
||||
<xsl:when test="(string-length($word) > $splitlimit) and (contains($secondhalfword, '
'))">
|
||||
<xsl:variable name="secondhalfend" select="substring-after($secondhalfword, '
')"/>
|
||||
<xsl:variable name="firsthalflen" select="string-length($word) - $secondhalflen"/>
|
||||
<xsl:variable name="firsthalfword" select="substring($word, 1, $firsthalflen)"/>
|
||||
<xsl:variable name="firsthalfend" select="substring-before($secondhalfword, '
')"/>
|
||||
<xsl:call-template name="br-replace">
|
||||
<xsl:with-param name="word" select="concat($firsthalfword,$firsthalfend)"/>
|
||||
</xsl:call-template>
|
||||
<br/>
|
||||
<xsl:call-template name="br-replace">
|
||||
<xsl:with-param name="word" select="$secondhalfend"/>
|
||||
</xsl:call-template>
|
||||
</xsl:when>
|
||||
<xsl:when test="contains($word, '
')">
|
||||
<xsl:value-of select="substring-before($word, '
')"/>
|
||||
<br/>
|
||||
<xsl:call-template name="br-replace">
|
||||
<xsl:with-param name="word" select="substring-after($word, '
')"/>
|
||||
</xsl:call-template>
|
||||
</xsl:when>
|
||||
<xsl:otherwise>
|
||||
<xsl:value-of select="$word"/>
|
||||
</xsl:otherwise>
|
||||
</xsl:choose>
|
||||
</xsl:template>
|
||||
<xsl:template name="display-time">
|
||||
<xsl:param name="value"/>
|
||||
<xsl:value-of select="format-number($value,'0.000')"/>
|
||||
</xsl:template>
|
||||
<xsl:template name="display-percent">
|
||||
<xsl:param name="value"/>
|
||||
<xsl:value-of select="format-number($value,'0.00%')"/>
|
||||
</xsl:template>
|
||||
</xsl:stylesheet>
|
@ -1,86 +0,0 @@
|
||||
#!/usr/bin/env python3
|
||||
# -*- coding: utf-8 -*-
|
||||
import os
|
||||
import lxml.etree as etree
|
||||
import json
|
||||
import argparse
|
||||
|
||||
def export_testcases_json(report, path):
|
||||
with open(os.path.join(path, "cases.jer"), "w") as testcases_file:
|
||||
for testsuite in report.getroot():
|
||||
for testcase in testsuite:
|
||||
row = {}
|
||||
row["hostname"] = testsuite.get("hostname")
|
||||
row["suite"] = testsuite.get("name")
|
||||
row["suite_duration"] = testsuite.get("time")
|
||||
row["timestamp"] = testsuite.get("timestamp")
|
||||
row["testname"] = testcase.get("name")
|
||||
row["classname"] = testcase.get("classname")
|
||||
row["file"] = testcase.get("file")
|
||||
row["line"] = testcase.get("line")
|
||||
row["duration"] = testcase.get("time")
|
||||
for el in testcase:
|
||||
if el.tag == "system-err":
|
||||
row["stderr"] = el.text
|
||||
else:
|
||||
row["stderr"] = ""
|
||||
|
||||
if el.tag == "system-out":
|
||||
row["stdout"] = el.text
|
||||
else:
|
||||
row["stdout"] = ""
|
||||
|
||||
json.dump(row, testcases_file)
|
||||
testcases_file.write("\n")
|
||||
|
||||
def export_testsuites_json(report, path):
|
||||
with open(os.path.join(path, "suites.jer"), "w") as testsuites_file:
|
||||
for testsuite in report.getroot():
|
||||
row = {}
|
||||
row["suite"] = testsuite.get("name")
|
||||
row["errors"] = testsuite.get("errors")
|
||||
row["failures"] = testsuite.get("failures")
|
||||
row["hostname"] = testsuite.get("hostname")
|
||||
row["skipped"] = testsuite.get("skipped")
|
||||
row["duration"] = testsuite.get("time")
|
||||
row["timestamp"] = testsuite.get("timestamp")
|
||||
json.dump(row, testsuites_file)
|
||||
testsuites_file.write("\n")
|
||||
|
||||
|
||||
def _convert_junit_to_html(junit_path, result_path, export_cases, export_suites):
|
||||
with open(os.path.join(os.path.dirname(__file__), "junit-noframes.xsl")) as xslt_file:
|
||||
junit_to_html_xslt = etree.parse(xslt_file)
|
||||
if not os.path.exists(result_path):
|
||||
os.makedirs(result_path)
|
||||
|
||||
with open(junit_path) as junit_file:
|
||||
junit_xml = etree.parse(junit_file)
|
||||
|
||||
if export_suites:
|
||||
export_testsuites_json(junit_xml, result_path)
|
||||
if export_cases:
|
||||
export_testcases_json(junit_xml, result_path)
|
||||
transform = etree.XSLT(junit_to_html_xslt)
|
||||
html = etree.tostring(transform(junit_xml), encoding="utf-8")
|
||||
|
||||
with open(os.path.join(result_path, "result.html"), "w") as html_file:
|
||||
html_file.write(html)
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
||||
parser = argparse.ArgumentParser(description='Convert JUnit XML.')
|
||||
parser.add_argument('junit', help='path to junit.xml report')
|
||||
parser.add_argument('result_dir', nargs='?', help='directory for result files. Default to junit.xml directory')
|
||||
parser.add_argument('--export-cases', help='Export JSONEachRow result for testcases to upload in CI', action='store_true')
|
||||
parser.add_argument('--export-suites', help='Export JSONEachRow result for testsuites to upload in CI', action='store_true')
|
||||
|
||||
args = parser.parse_args()
|
||||
|
||||
junit_path = args.junit
|
||||
if args.result_dir:
|
||||
result_path = args.result_dir
|
||||
else:
|
||||
result_path = os.path.dirname(junit_path)
|
||||
print("junit_path: {}, result_path: {}, export cases:{}, export suites: {}".format(junit_path, result_path, args.export_cases, args.export_suites))
|
||||
_convert_junit_to_html(junit_path, result_path, args.export_cases, args.export_suites)
|
@ -1,42 +0,0 @@
|
||||
#/bin/sh
|
||||
#
|
||||
# This script is used to validate the shared libraries
|
||||
#
|
||||
# Authors: FoundationDB team, https://github.com/apple/foundationdb/blame/master/build/link-validate.sh
|
||||
# License: Apache License 2.0
|
||||
|
||||
verlte() {
|
||||
[ "$1" = "`echo -e "$1\n$2" | sort -V | head -n1`" ]
|
||||
}
|
||||
|
||||
ALLOWED_SHARED_LIBS=("libdl.so.2" "libpthread.so.0" "librt.so.1" "libm.so.6" "libc.so.6" "ld-linux-x86-64.so.2")
|
||||
|
||||
if [ "$#" -lt 1 ]; then
|
||||
echo "USAGE: link-validate.sh BINNAME GLIBC_VERSION"
|
||||
exit 1
|
||||
fi
|
||||
|
||||
# Step 1: glibc version
|
||||
|
||||
for i in $(objdump -T "$1" | awk '{print $5}' | grep GLIBC | sed 's/ *$//g' | sed 's/GLIBC_//' | sort | uniq); do
|
||||
if ! verlte "$i" "${2:-2.10}"; then
|
||||
echo "Dependency on newer libc detected: $i"
|
||||
exit 1
|
||||
fi
|
||||
done
|
||||
|
||||
# Step 2: Other dynamic dependencies
|
||||
|
||||
for j in $(objdump -p "$1" | grep NEEDED | awk '{print $2}'); do
|
||||
PRESENT=0
|
||||
for k in ${ALLOWED_SHARED_LIBS[@]}; do
|
||||
if [[ "$k" == "$j" ]]; then
|
||||
PRESENT=1
|
||||
break
|
||||
fi
|
||||
done
|
||||
if ! [[ $PRESENT == 1 ]]; then
|
||||
echo "Unexpected shared object dependency detected: $j"
|
||||
exit 1
|
||||
fi
|
||||
done
|
@ -69,12 +69,6 @@ function renderResponse(response) {
|
||||
document.body.style.height = canvas.height + 10 + 'px';
|
||||
|
||||
let ctx = canvas.getContext('2d');
|
||||
|
||||
ctx.imageSmoothingEnabled = false;
|
||||
ctx.mozImageSmoothingEnabled = false;
|
||||
ctx.webkitImageSmoothingEnabled = false;
|
||||
ctx.msImageSmoothingEnabled = false;
|
||||
|
||||
let image = ctx.createImageData(canvas.width, canvas.height);
|
||||
let pixels = image.data;
|
||||
|
||||
@ -123,8 +117,6 @@ canvas.addEventListener('mousemove', event => {
|
||||
|
||||
let pixel = canvas.getContext('2d').getImageData(x, y, 1, 1).data;
|
||||
|
||||
console.log(pixel);
|
||||
|
||||
let info = document.getElementById('info');
|
||||
|
||||
info.innerText = `${date}, ${test}`;
|
||||
|
@ -1,34 +0,0 @@
|
||||
## Tool to upload results to CI ClickHouse
|
||||
|
||||
Currently allows to upload results from `junit_to_html` tool to ClickHouse CI
|
||||
|
||||
```
|
||||
usage: upload_test_results [-h] --sha SHA --pr PR --file FILE --type
|
||||
{suites,cases} [--user USER] --password PASSWORD
|
||||
[--ca-cert CA_CERT] [--host HOST] [--db DB]
|
||||
|
||||
Upload test result to CI ClickHouse.
|
||||
|
||||
optional arguments:
|
||||
-h, --help show this help message and exit
|
||||
--sha SHA sha of current commit
|
||||
--pr PR pr of current commit. 0 for master
|
||||
--file FILE file to upload
|
||||
--type {suites,cases}
|
||||
Export type
|
||||
--user USER user name
|
||||
--password PASSWORD password
|
||||
--ca-cert CA_CERT CA certificate path
|
||||
--host HOST CI ClickHouse host
|
||||
--db DB CI ClickHouse database name
|
||||
```
|
||||
|
||||
$ ./upload_test_results --sha "cf7eaee3301d4634acdacbfa308ddbe0cc6a061d" --pr "0" --file xyz/cases.jer --type cases --password $PASSWD
|
||||
|
||||
CI checks has single commit sha and pr identifier.
|
||||
While uploading your local results for testing purposes try to use correct sha and pr.
|
||||
|
||||
CA Certificate for ClickHouse CI can be obtained from Yandex.Cloud where CI database is hosted
|
||||
``` bash
|
||||
wget "https://storage.yandexcloud.net/cloud-certs/CA.pem" -O YandexInternalRootCA.crt
|
||||
```
|
@ -1,127 +0,0 @@
|
||||
#!/usr/bin/env python3
|
||||
import requests
|
||||
import argparse
|
||||
|
||||
# CREATE TABLE test_suites
|
||||
# (
|
||||
# sha String,
|
||||
# pr UInt16,
|
||||
# suite String,
|
||||
# errors UInt16,
|
||||
# failures UInt16,
|
||||
# hostname String,
|
||||
# skipped UInt16,
|
||||
# duration Double,
|
||||
# timestamp DateTime
|
||||
# ) ENGINE = MergeTree ORDER BY tuple(timestamp, suite);
|
||||
|
||||
QUERY_SUITES="INSERT INTO test_suites "\
|
||||
"SELECT '{sha}' AS sha, "\
|
||||
"{pr} AS pr, "\
|
||||
"suite, "\
|
||||
"errors, "\
|
||||
"failures, "\
|
||||
"hostname, "\
|
||||
"skipped, "\
|
||||
"duration, "\
|
||||
"timestamp "\
|
||||
"FROM input('"\
|
||||
"suite String, "\
|
||||
"errors UInt16, "\
|
||||
"failures UInt16, "\
|
||||
"hostname String, "\
|
||||
"skipped UInt16, "\
|
||||
"duration Double, "\
|
||||
"timestamp DateTime"\
|
||||
"') FORMAT JSONEachRow"
|
||||
|
||||
# CREATE TABLE test_cases
|
||||
# (
|
||||
# sha String,
|
||||
# pr UInt16,
|
||||
# hostname String,
|
||||
# suite String,
|
||||
# timestamp DateTime,
|
||||
# testname String,
|
||||
# classname String,
|
||||
# file String,
|
||||
# line UInt16,
|
||||
# duration Double,
|
||||
# suite_duration Double,
|
||||
# stderr String,
|
||||
# stdout String
|
||||
# ) ENGINE = MergeTree ORDER BY tuple(timestamp, testname);
|
||||
|
||||
QUERY_CASES="INSERT INTO test_cases "\
|
||||
"SELECT '{sha}' AS sha, "\
|
||||
"{pr} AS pr, "\
|
||||
"hostname, "\
|
||||
"suite, "\
|
||||
"timestamp, "\
|
||||
"testname, "\
|
||||
"classname, "\
|
||||
"file, "\
|
||||
"line, "\
|
||||
"duration, "\
|
||||
"suite_duration, "\
|
||||
"stderr,"\
|
||||
"stdout "\
|
||||
"FROM input('"\
|
||||
"hostname String, "\
|
||||
"suite String, "\
|
||||
"timestamp DateTime, "\
|
||||
"testname String, "\
|
||||
"classname String, "\
|
||||
"file String, "\
|
||||
"line UInt16, "\
|
||||
"duration Double, "\
|
||||
"suite_duration Double, "\
|
||||
"stderr String, "\
|
||||
"stdout String"\
|
||||
"') FORMAT JSONEachRow"
|
||||
|
||||
|
||||
def upload_request(sha, pr, file, q_type, user, password, ca_cert, host, db):
|
||||
with open(file) as upload_f:
|
||||
query = QUERY_SUITES if q_type=="suites" else QUERY_CASES
|
||||
query = query.format(sha=sha, pr=pr)
|
||||
url = 'https://{host}:8443/?database={db}&query={query}&date_time_input_format=best_effort'.format(
|
||||
host=host,
|
||||
db=db,
|
||||
query=query
|
||||
)
|
||||
data=upload_f
|
||||
auth = {
|
||||
'X-ClickHouse-User': user,
|
||||
'X-ClickHouse-Key': password,
|
||||
}
|
||||
|
||||
print(query);
|
||||
|
||||
res = requests.post(
|
||||
url,
|
||||
data=data,
|
||||
headers=auth,
|
||||
verify=ca_cert)
|
||||
res.raise_for_status()
|
||||
return res.text
|
||||
|
||||
if __name__ == "__main__":
|
||||
|
||||
parser = argparse.ArgumentParser(description='Upload test result to CI ClickHouse.')
|
||||
parser.add_argument('--sha', help='sha of current commit', type=str, required=True)
|
||||
parser.add_argument('--pr', help='pr of current commit. 0 for master', type=int, required=True)
|
||||
parser.add_argument('--file', help='file to upload', required=True)
|
||||
parser.add_argument('--type', help='Export type', choices=['suites', 'cases'] , required=True)
|
||||
parser.add_argument('--user', help='user name', type=str, default="clickhouse-ci")
|
||||
parser.add_argument('--password', help='password', type=str, required=True)
|
||||
parser.add_argument('--ca-cert', help='CA certificate path', type=str, default="/usr/local/share/ca-certificates/YandexInternalRootCA.crt")
|
||||
parser.add_argument('--host', help='CI ClickHouse host', type=str, default="c1a-ity5agjmuhyu6nu9.mdb.yandexcloud.net")
|
||||
parser.add_argument('--db', help='CI ClickHouse database name', type=str, default="clickhouse-ci")
|
||||
|
||||
args = parser.parse_args()
|
||||
|
||||
print((upload_request(args.sha, args.pr, args.file, args.type, args.user, args.password, args.ca_cert, args.host, args.db)))
|
||||
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
[
|
||||
{
|
||||
"system": "Xeon Gold 6266C, 3GHz, 4vCPU",
|
||||
"system_full": "Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB",
|
||||
"system": "Huawei Cloud c6.xlarge.4, 4vCPUs, 16 GiB",
|
||||
"system_full": "Huawei Cloud c6.xlarge.4, Xeon Gold 6266C, 3GHz, 4vCPU, 16GiB RAM, vda1 40GB",
|
||||
"cpu_vendor": "Intel",
|
||||
"cpu_model": "Xeon Gold 6266C",
|
||||
"time": "2021-12-23 00:00:00",
|
||||
|
45
website/blog/en/2021/tests-visualization.md
Normal file
45
website/blog/en/2021/tests-visualization.md
Normal file
@ -0,0 +1,45 @@
|
||||
---
|
||||
title: 'Decorating a Christmas Tree With the Help Of Flaky Tests'
|
||||
image: 'https://blog-images.clickhouse.com/en/2021/tests-visualization/tests.png'
|
||||
date: '2021-12-27'
|
||||
author: '[Alexey Milovidov](https://github.com/alexey-milovidov)'
|
||||
tags: ['tests', 'ci', 'flaky', 'christmas', 'visualization']
|
||||
---
|
||||
|
||||
Test suites and testing infrastructure are one of the main assets of ClickHouse. We have tons of functional, integration, unit, performance, stress and fuzz tests. Tests are run on a per commit basis and results are publicly available.
|
||||
|
||||
We also save the results of all test runs into the database in ClickHouse. We started collecting results in June 2020, and we have 1 777 608 240 records so far. Now we run around 5 to 9 million tests every day.
|
||||
|
||||
Tests are good (in general). A good test suite allows for fast development iterations, stable releases, and to accept more contributions from the community. We love tests. If there's something strange in ClickHouse, what are we gonna do? Write more tests.
|
||||
|
||||
Some tests can be flaky. The reasons for flakiness are uncountable - most of them are simple timing issues in the test script itself, but sometimes if a test has failed one of a thousand times it can uncover subtle logic errors in code.
|
||||
|
||||
The problem is how to deal with flaky tests. Some people suggest automatically muting the "annoying" flaky tests. Or adding automatic retries in case of failure. We believe that this is all wrong. Instead of trying to ignore flaky tests, we do the opposite: we put maximum effort into making the tests even more flaky!
|
||||
|
||||
Our recipes for flaky tests:
|
||||
— never mute or restart them; if the test failed once, always look and investigate the cause;
|
||||
— randomize the environment for every test run so the test will have more possible reasons to fail;
|
||||
— if new tests are added, run them 100 times and if at least one fails, do not merge the pull request;
|
||||
— if new tests are added, use them as a corpus for fuzzing - it will uncover corner cases even if author did not write tests for them;
|
||||
— [randomize thread scheduling](https://github.com/ClickHouse/ClickHouse/blob/master/src/Common/ThreadFuzzer.h) and add random sleeps and switching between CPU cores at random places and before and after mutex locks/unlocks;
|
||||
— run everything in parallel on slow machines;
|
||||
|
||||
Key point: to prevent flaky tests, we make our tests as flaky as possible.
|
||||
|
||||
## Nice Way To Visualize Flaky Tests
|
||||
|
||||
There is a test suite named "[functional stateless tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/queries/0_stateless)" that has 3772 tests. For every day since 2020-06-13 (561 days) and every test (3772 tests), I drew a picture of size 561x3772 where a pixel is green if all test runs finished successfully in the master branch during this day (for all commits and all combinations: release, debug+assertions, ASan, MSan, TSan, UBSan), and a pixel is red if at least one run failed. The pixel will be transparent if the test did not exist that day.
|
||||
|
||||
This visualization is a toy that I've made for fun:
|
||||
|
||||
![Visualization](https://blog-images.clickhouse.com/en/2021/tests-visualization/tree_half.png)
|
||||
|
||||
It looks like a Christmas Tree (you need a bit of imagination). If you have a different kind of imagination, you can see it as a green field with flowers.
|
||||
|
||||
The time is from left to right. The tests are numbered with non-unique numbers (new tests usually get larger numbers), and these numbers are on the vertical axis (newer tests on top).
|
||||
|
||||
If you see red dots in a horizontal line - it is a flaky test. If you see red dots in a vertical line - it means that one day we accidentally broke the master branch. If you see black horizontal lines or cuts in the tree - it means that the tests were added with some old numbers, most likely because some long living feature branch was merged. If you see black vertical lines - it means that some days tests were not run.
|
||||
|
||||
The velocity of adding new tests is represented by how tall and narrow the Christmas tree is. When we add a large number of tests, the tree grows with almost vertical slope.
|
||||
|
||||
The image is prepared by [HTML page](https://github.com/ClickHouse/ClickHouse/pull/33185) with some JavaScript that is querying a ClickHouse database directly and writing to a canvas. It took around ten seconds to build this picture. I also prepared an [interactive version](https://blog-images.clickhouse.com/en/2021/tests-visualization/demo.html) with already-saved data where you can play and find your favorite tests.
|
Loading…
Reference in New Issue
Block a user