mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
merge master
This commit is contained in:
commit
aa97634d9b
@ -17,6 +17,8 @@ $ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clic
|
||||
|
||||
By default ClickHouse will be accessible only via docker network. See the [networking section below](#networking).
|
||||
|
||||
By default, starting above server instance will be run as default user without password.
|
||||
|
||||
### connect to it from a native client
|
||||
```bash
|
||||
$ docker run -it --rm --link some-clickhouse-server:clickhouse-server clickhouse/clickhouse-client --host clickhouse-server
|
||||
|
@ -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.
|
||||
|
||||
|
@ -55,13 +55,13 @@ The behavior of ClickHouse server for `ANY JOIN` operations depends on the [any_
|
||||
- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge)
|
||||
- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys)
|
||||
|
||||
## ON Section Conditions {on-section-conditions}
|
||||
## ON Section Conditions {#on-section-conditions}
|
||||
|
||||
An `ON` section can contain several conditions combined using the `AND` and `OR` operators. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query.
|
||||
|
||||
An `ON` section can contain several conditions combined using the `AND` operator. Conditions specifying join keys must refer both left and right tables and must use the equality operator. Other conditions may use other logical operators but they must refer either the left or the right table of a query.
|
||||
Rows are joined if the whole complex condition is met. If the conditions are not met, still rows may be included in the result depending on the `JOIN` type. Note that if the same conditions are placed in a `WHERE` section and they are not met, then rows are always filtered out from the result.
|
||||
|
||||
!!! note "Note"
|
||||
The `OR` operator inside an `ON` section is not supported yet.
|
||||
The `OR` operator inside the `ON` clause works using the hash join algorithm — for each `OR` argument with join keys for `JOIN`, a separate hash table is created, so memory consumption and query execution time grow linearly with an increase in the number of expressions `OR` of the `ON` clause.
|
||||
|
||||
!!! note "Note"
|
||||
If a condition refers columns from different tables, then only the equality operator (`=`) is supported so far.
|
||||
@ -109,7 +109,47 @@ Result:
|
||||
│ B │ Text B │ 15 │
|
||||
└──────┴────────┴────────┘
|
||||
```
|
||||
Query with `INNER` type of a join and condition with `OR`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a;
|
||||
|
||||
CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key;
|
||||
|
||||
INSERT INTO t1 SELECT number as a, -a as b from numbers(5);
|
||||
|
||||
INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5);
|
||||
|
||||
SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```
|
||||
┌─a─┬──b─┬─val─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
│ 1 │ -1 │ 1 │
|
||||
│ 2 │ -2 │ 2 │
|
||||
│ 3 │ -3 │ 3 │
|
||||
│ 4 │ -4 │ 4 │
|
||||
└───┴────┴─────┘
|
||||
```
|
||||
|
||||
Query with `INNER` type of a join and conditions with `OR` and `AND`:
|
||||
|
||||
``` sql
|
||||
SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3;
|
||||
```
|
||||
|
||||
Result:
|
||||
|
||||
```
|
||||
┌─a─┬──b─┬─val─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
│ 2 │ -2 │ 2 │
|
||||
│ 4 │ -4 │ 4 │
|
||||
└───┴────┴─────┘
|
||||
```
|
||||
## ASOF JOIN Usage {#asof-join-usage}
|
||||
|
||||
`ASOF JOIN` is useful when you need to join records that have no exact match.
|
||||
|
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}
|
||||
|
||||
|
@ -55,13 +55,13 @@ FROM <left_table>
|
||||
- [join_on_disk_max_files_to_merge](../../../operations/settings/settings.md#join_on_disk_max_files_to_merge)
|
||||
- [any_join_distinct_right_table_keys](../../../operations/settings/settings.md#any_join_distinct_right_table_keys)
|
||||
|
||||
## Условия в секции ON {on-section-conditions}
|
||||
## Условия в секции ON {#on-section-conditions}
|
||||
|
||||
Секция `ON` может содержать несколько условий, связанных операторами `AND` и `OR`. Условия, задающие ключи соединения, должны содержать столбцы левой и правой таблицы и должны использовать оператор равенства. Прочие условия могут использовать другие логические операторы, но в отдельном условии могут использоваться столбцы либо только левой, либо только правой таблицы.
|
||||
|
||||
Секция `ON` может содержать несколько условий, связанных оператором `AND`. Условия, задающие ключи соединения, должны содержать столбцы левой и правой таблицы и должны использовать оператор равенства. Прочие условия могут использовать другие логические операторы, но в отдельном условии могут использоваться столбцы либо только левой, либо только правой таблицы.
|
||||
Строки объединяются только тогда, когда всё составное условие выполнено. Если оно не выполнено, то строки могут попасть в результат в зависимости от типа `JOIN`. Обратите внимание, что если то же самое условие поместить в секцию `WHERE`, то строки, для которых оно не выполняется, никогда не попаду в результат.
|
||||
|
||||
!!! note "Примечание"
|
||||
Оператор `OR` внутри секции `ON` пока не поддерживается.
|
||||
Оператор `OR` внутри секции `ON` работает, используя алгоритм хеш-соединения — на каждый агрумент `OR` с ключами соединений для `JOIN` создается отдельная хеш-таблица, поэтому потребление памяти и время выполнения запроса растет линейно при увеличении количества выражений `OR` секции `ON`.
|
||||
|
||||
!!! note "Примечание"
|
||||
Если в условии использованы столбцы из разных таблиц, то пока поддерживается только оператор равенства (`=`).
|
||||
@ -110,6 +110,47 @@ SELECT name, text, scores FROM table_1 INNER JOIN table_2
|
||||
└──────┴────────┴────────┘
|
||||
```
|
||||
|
||||
Запрос с типом соединения `INNER` и условием с оператором `OR`:
|
||||
|
||||
``` sql
|
||||
CREATE TABLE t1 (`a` Int64, `b` Int64) ENGINE = MergeTree() ORDER BY a;
|
||||
|
||||
CREATE TABLE t2 (`key` Int32, `val` Int64) ENGINE = MergeTree() ORDER BY key;
|
||||
|
||||
INSERT INTO t1 SELECT number as a, -a as b from numbers(5);
|
||||
|
||||
INSERT INTO t2 SELECT if(number % 2 == 0, toInt64(number), -number) as key, number as val from numbers(5);
|
||||
|
||||
SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```
|
||||
┌─a─┬──b─┬─val─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
│ 1 │ -1 │ 1 │
|
||||
│ 2 │ -2 │ 2 │
|
||||
│ 3 │ -3 │ 3 │
|
||||
│ 4 │ -4 │ 4 │
|
||||
└───┴────┴─────┘
|
||||
```
|
||||
|
||||
Запрос с типом соединения `INNER` и условиями с операторами `OR` и `AND`:
|
||||
|
||||
``` sql
|
||||
SELECT a, b, val FROM t1 INNER JOIN t2 ON t1.a = t2.key OR t1.b = t2.key AND t2.val > 3;
|
||||
```
|
||||
|
||||
Результат:
|
||||
|
||||
```
|
||||
┌─a─┬──b─┬─val─┐
|
||||
│ 0 │ 0 │ 0 │
|
||||
│ 2 │ -2 │ 2 │
|
||||
│ 4 │ -4 │ 4 │
|
||||
└───┴────┴─────┘
|
||||
```
|
||||
## Использование ASOF JOIN {#asof-join-usage}
|
||||
|
||||
`ASOF JOIN` применим в том случае, когда необходимо объединять записи, которые не имеют точного совпадения.
|
||||
|
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
|
@ -46,7 +46,7 @@ void ClusterCopier::init()
|
||||
reloadTaskDescription();
|
||||
|
||||
task_cluster->loadTasks(*task_cluster_current_config);
|
||||
getContext()->setClustersConfig(task_cluster_current_config, task_cluster->clusters_prefix);
|
||||
getContext()->setClustersConfig(task_cluster_current_config, false, task_cluster->clusters_prefix);
|
||||
|
||||
/// Set up shards and their priority
|
||||
task_cluster->random_engine.seed(task_cluster->random_device());
|
||||
|
@ -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>
|
||||
@ -72,6 +73,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>
|
||||
@ -129,6 +131,11 @@ namespace CurrentMetrics
|
||||
extern const Metric MaxPushedDDLEntryID;
|
||||
}
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event MainConfigLoads;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
#if USE_JEMALLOC
|
||||
@ -346,16 +353,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 &)
|
||||
@ -533,6 +577,27 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
}
|
||||
}
|
||||
|
||||
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");
|
||||
@ -859,7 +924,7 @@ if (ThreadFuzzer::instance().isEffective())
|
||||
// in a lot of places. For now, disable updating log configuration without server restart.
|
||||
//setTextLog(global_context->getTextLog());
|
||||
updateLevels(*config, logger());
|
||||
global_context->setClustersConfig(config);
|
||||
global_context->setClustersConfig(config, has_zookeeper);
|
||||
global_context->setMacros(std::make_unique<Macros>(*config, "macros", log));
|
||||
global_context->setExternalAuthenticatorsConfig(*config);
|
||||
|
||||
@ -888,12 +953,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)
|
||||
|
||||
@ -1005,24 +1075,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"))
|
||||
{
|
||||
@ -1045,39 +1099,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.");
|
||||
@ -1085,14 +1146,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
|
||||
@ -1104,11 +1170,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();
|
||||
@ -1120,7 +1186,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);
|
||||
@ -1274,223 +1340,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();
|
||||
|
||||
{
|
||||
@ -1569,9 +1430,24 @@ 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.");
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
global_context->startClusterDiscovery();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Caught exception while starting cluster discovery");
|
||||
}
|
||||
|
||||
SCOPE_EXIT_SAFE({
|
||||
LOG_DEBUG(log, "Received termination signal.");
|
||||
@ -1580,10 +1456,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)
|
||||
@ -1596,7 +1475,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."
|
||||
@ -1632,4 +1511,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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -276,9 +276,11 @@
|
||||
M(ThreadPoolReaderPageCacheMissElapsedMicroseconds, "Time spent reading data inside the asynchronous job in ThreadPoolReader - when read was not done from page cache.") \
|
||||
\
|
||||
M(AsynchronousReadWaitMicroseconds, "Time spent in waiting for asynchronous reads.") \
|
||||
\
|
||||
M(ExternalDataSourceLocalCacheReadBytes, "Bytes read from local cache buffer in RemoteReadBufferCache")\
|
||||
|
||||
|
||||
\
|
||||
M(MainConfigLoads, "Number of times the main configuration was reloaded.") \
|
||||
\
|
||||
namespace ProfileEvents
|
||||
{
|
||||
|
||||
|
@ -75,6 +75,17 @@ std::string checkAndGetSuperdigest(const String & user_and_digest)
|
||||
return user_and_digest;
|
||||
}
|
||||
|
||||
int32_t getValueOrMaxInt32AndLogWarning(uint64_t value, const std::string & name, Poco::Logger * log)
|
||||
{
|
||||
if (value > std::numeric_limits<int32_t>::max())
|
||||
{
|
||||
LOG_WARNING(log, "Got {} value for setting '{}' which is bigger than int32_t max value, lowering value to {}.", value, name, std::numeric_limits<int32_t>::max());
|
||||
return std::numeric_limits<int32_t>::max();
|
||||
}
|
||||
|
||||
return static_cast<int32_t>(value);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
KeeperServer::KeeperServer(
|
||||
@ -134,18 +145,18 @@ void KeeperServer::startup()
|
||||
}
|
||||
|
||||
nuraft::raft_params params;
|
||||
params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds();
|
||||
params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds();
|
||||
params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds();
|
||||
|
||||
params.reserved_log_items_ = coordination_settings->reserved_log_items;
|
||||
params.snapshot_distance_ = coordination_settings->snapshot_distance;
|
||||
params.stale_log_gap_ = coordination_settings->stale_log_gap;
|
||||
params.fresh_log_gap_ = coordination_settings->fresh_log_gap;
|
||||
params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds();
|
||||
params.heart_beat_interval_ = getValueOrMaxInt32AndLogWarning(coordination_settings->heart_beat_interval_ms.totalMilliseconds(), "heart_beat_interval_ms", log);
|
||||
params.election_timeout_lower_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(), "election_timeout_lower_bound_ms", log);
|
||||
params.election_timeout_upper_bound_ = getValueOrMaxInt32AndLogWarning(coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(), "election_timeout_upper_bound_ms", log);
|
||||
params.reserved_log_items_ = getValueOrMaxInt32AndLogWarning(coordination_settings->reserved_log_items, "reserved_log_items", log);
|
||||
params.snapshot_distance_ = getValueOrMaxInt32AndLogWarning(coordination_settings->snapshot_distance, "snapshot_distance", log);
|
||||
params.stale_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->stale_log_gap, "stale_log_gap", log);
|
||||
params.fresh_log_gap_ = getValueOrMaxInt32AndLogWarning(coordination_settings->fresh_log_gap, "fresh_log_gap", log);
|
||||
params.client_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds(), "operation_timeout_ms", log);
|
||||
params.auto_forwarding_ = coordination_settings->auto_forwarding;
|
||||
params.auto_forwarding_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds() * 2;
|
||||
params.max_append_size_ = coordination_settings->max_requests_batch_size;
|
||||
params.auto_forwarding_req_timeout_ = std::max<uint64_t>(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, std::numeric_limits<int32_t>::max());
|
||||
params.auto_forwarding_req_timeout_ = getValueOrMaxInt32AndLogWarning(coordination_settings->operation_timeout_ms.totalMilliseconds() * 2, "operation_timeout_ms", log);
|
||||
params.max_append_size_ = getValueOrMaxInt32AndLogWarning(coordination_settings->max_requests_batch_size, "max_requests_batch_size", log);
|
||||
|
||||
params.return_method_ = nuraft::raft_params::async_handler;
|
||||
|
||||
|
@ -12,10 +12,7 @@ Connection::Connection(const ConnectionInfo & connection_info_, bool replication
|
||||
, log(&Poco::Logger::get("PostgreSQLReplicaConnection"))
|
||||
{
|
||||
if (replication)
|
||||
{
|
||||
connection_info = std::make_pair(
|
||||
fmt::format("{} replication=database", connection_info.first), connection_info.second);
|
||||
}
|
||||
connection_info = {fmt::format("{} replication=database", connection_info.connection_string), connection_info.host_port};
|
||||
}
|
||||
|
||||
void Connection::execWithRetry(const std::function<void(pqxx::nontransaction &)> & exec)
|
||||
@ -61,11 +58,14 @@ void Connection::updateConnection()
|
||||
{
|
||||
if (connection)
|
||||
connection->close();
|
||||
|
||||
/// Always throws if there is no connection.
|
||||
connection = std::make_unique<pqxx::connection>(connection_info.first);
|
||||
connection = std::make_unique<pqxx::connection>(connection_info.connection_string);
|
||||
|
||||
if (replication)
|
||||
connection->set_variable("default_transaction_isolation", "'repeatable read'");
|
||||
LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.second);
|
||||
|
||||
LOG_DEBUG(&Poco::Logger::get("PostgreSQLConnection"), "New connection to {}", connection_info.host_port);
|
||||
}
|
||||
|
||||
void Connection::connect()
|
||||
|
@ -8,19 +8,26 @@
|
||||
#include <Core/Types.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
/* Methods to work with PostgreSQL connection object.
|
||||
/** Methods to work with PostgreSQL connection object.
|
||||
* Should only be used in case there has to be a single connection object, which
|
||||
* is long-lived and there are no concurrent connection queries.
|
||||
* Now only use case - for replication handler for replication from PostgreSQL.
|
||||
* In all other integration engine use pool with failover.
|
||||
**/
|
||||
*/
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
||||
namespace pqxx
|
||||
{
|
||||
using ConnectionPtr = std::unique_ptr<pqxx::connection>;
|
||||
}
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
using ConnectionInfo = std::pair<String, String>;
|
||||
using ConnectionPtr = std::unique_ptr<pqxx::connection>;
|
||||
|
||||
struct ConnectionInfo
|
||||
{
|
||||
String connection_string;
|
||||
String host_port; /// For logs.
|
||||
};
|
||||
|
||||
class Connection : private boost::noncopyable
|
||||
{
|
||||
@ -33,14 +40,17 @@ public:
|
||||
|
||||
void connect();
|
||||
|
||||
void updateConnection();
|
||||
|
||||
void tryUpdateConnection();
|
||||
|
||||
const ConnectionInfo & getConnectionInfo() { return connection_info; }
|
||||
|
||||
private:
|
||||
void updateConnection();
|
||||
String getInfoForLog() const { return connection_info.host_port; }
|
||||
|
||||
ConnectionPtr connection;
|
||||
private:
|
||||
|
||||
pqxx::ConnectionPtr connection;
|
||||
ConnectionInfo connection_info;
|
||||
|
||||
bool replication;
|
||||
@ -48,6 +58,9 @@ private:
|
||||
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
using ConnectionPtr = std::unique_ptr<Connection>;
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -7,12 +7,12 @@
|
||||
#include <pqxx/pqxx>
|
||||
#include <Core/Types.h>
|
||||
#include <base/BorrowedObjectPool.h>
|
||||
#include "Connection.h"
|
||||
|
||||
|
||||
namespace postgres
|
||||
{
|
||||
|
||||
using ConnectionPtr = std::unique_ptr<pqxx::connection>;
|
||||
using Pool = BorrowedObjectPool<ConnectionPtr>;
|
||||
using PoolPtr = std::shared_ptr<Pool>;
|
||||
|
||||
@ -28,8 +28,12 @@ public:
|
||||
|
||||
pqxx::connection & get()
|
||||
{
|
||||
assert(connection != nullptr);
|
||||
return *connection;
|
||||
return connection->getRef();
|
||||
}
|
||||
|
||||
void update()
|
||||
{
|
||||
connection->updateConnection();
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -32,9 +32,9 @@ PoolWithFailover::PoolWithFailover(
|
||||
{
|
||||
for (const auto & replica_configuration : configurations)
|
||||
{
|
||||
auto connection_string = formatConnectionString(replica_configuration.database,
|
||||
replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password).first;
|
||||
replicas_with_priority[priority].emplace_back(connection_string, pool_size, getConnectionForLog(replica_configuration.host, replica_configuration.port));
|
||||
auto connection_info = formatConnectionString(replica_configuration.database,
|
||||
replica_configuration.host, replica_configuration.port, replica_configuration.username, replica_configuration.password);
|
||||
replicas_with_priority[priority].emplace_back(connection_info, pool_size);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -52,8 +52,8 @@ PoolWithFailover::PoolWithFailover(
|
||||
for (const auto & [host, port] : configuration.addresses)
|
||||
{
|
||||
LOG_DEBUG(&Poco::Logger::get("PostgreSQLPoolWithFailover"), "Adding address host: {}, port: {} to connection pool", host, port);
|
||||
auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password).first;
|
||||
replicas_with_priority[0].emplace_back(connection_string, pool_size, getConnectionForLog(host, port));
|
||||
auto connection_string = formatConnectionString(configuration.database, host, port, configuration.username, configuration.password);
|
||||
replicas_with_priority[0].emplace_back(connection_string, pool_size);
|
||||
}
|
||||
}
|
||||
|
||||
@ -83,16 +83,18 @@ ConnectionHolderPtr PoolWithFailover::get()
|
||||
try
|
||||
{
|
||||
/// Create a new connection or reopen an old connection if it became invalid.
|
||||
if (!connection || !connection->is_open())
|
||||
if (!connection)
|
||||
{
|
||||
connection = std::make_unique<pqxx::connection>(replica.connection_string);
|
||||
LOG_DEBUG(log, "New connection to {}:{}", connection->hostname(), connection->port());
|
||||
connection = std::make_unique<Connection>(replica.connection_info);
|
||||
LOG_DEBUG(log, "New connection to {}", connection->getInfoForLog());
|
||||
}
|
||||
|
||||
connection->connect();
|
||||
}
|
||||
catch (const pqxx::broken_connection & pqxx_error)
|
||||
{
|
||||
LOG_ERROR(log, "Connection error: {}", pqxx_error.what());
|
||||
error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.name_for_log << "` failed: " << pqxx_error.what() << "\n";
|
||||
error_message << "Try " << try_idx + 1 << ". Connection to `" << replica.connection_info.host_port << "` failed: " << pqxx_error.what() << "\n";
|
||||
|
||||
replica.pool->returnObject(std::move(connection));
|
||||
continue;
|
||||
|
@ -44,12 +44,11 @@ public:
|
||||
private:
|
||||
struct PoolHolder
|
||||
{
|
||||
String connection_string;
|
||||
ConnectionInfo connection_info;
|
||||
PoolPtr pool;
|
||||
String name_for_log;
|
||||
|
||||
PoolHolder(const String & connection_string_, size_t pool_size, const String & name_for_log_)
|
||||
: connection_string(connection_string_), pool(std::make_shared<Pool>(pool_size)), name_for_log(name_for_log_) {}
|
||||
PoolHolder(const ConnectionInfo & connection_info_, size_t pool_size)
|
||||
: connection_info(connection_info_), pool(std::make_shared<Pool>(pool_size)) {}
|
||||
};
|
||||
|
||||
/// Highest priority is 0, the bigger the number in map, the less the priority
|
||||
|
@ -17,7 +17,7 @@ ConnectionInfo formatConnectionString(String dbname, String host, UInt16 port, S
|
||||
<< " user=" << DB::quote << user
|
||||
<< " password=" << DB::quote << password
|
||||
<< " connect_timeout=10";
|
||||
return std::make_pair(out.str(), host + ':' + DB::toString(port));
|
||||
return {out.str(), host + ':' + DB::toString(port)};
|
||||
}
|
||||
|
||||
String getConnectionForLog(const String & host, UInt16 port)
|
||||
|
@ -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;
|
||||
|
@ -320,13 +320,29 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf
|
||||
if (old_config)
|
||||
{
|
||||
for (const auto & key : deleted_keys)
|
||||
impl.erase(key);
|
||||
{
|
||||
if (!automatic_clusters.contains(key))
|
||||
impl.erase(key);
|
||||
}
|
||||
}
|
||||
else
|
||||
impl.clear();
|
||||
{
|
||||
if (!automatic_clusters.empty())
|
||||
std::erase_if(impl, [this](const auto & e) { return automatic_clusters.contains(e.first); });
|
||||
else
|
||||
impl.clear();
|
||||
}
|
||||
|
||||
|
||||
for (const auto & key : new_config_keys)
|
||||
{
|
||||
if (new_config.has(config_prefix + "." + key + ".discovery"))
|
||||
{
|
||||
/// Handled in ClusterDiscovery
|
||||
automatic_clusters.insert(key);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (key.find('.') != String::npos)
|
||||
throw Exception("Cluster names with dots are not supported: '" + key + "'", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
|
||||
#include <map>
|
||||
#include <string>
|
||||
#include <unordered_set>
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
@ -295,12 +297,15 @@ public:
|
||||
|
||||
void updateClusters(const Poco::Util::AbstractConfiguration & new_config, const Settings & settings, const String & config_prefix, Poco::Util::AbstractConfiguration * old_config = nullptr);
|
||||
|
||||
public:
|
||||
using Impl = std::map<String, ClusterPtr>;
|
||||
|
||||
Impl getContainer() const;
|
||||
|
||||
protected:
|
||||
|
||||
/// setup outside of this class, stored to prevent deleting from impl on config update
|
||||
std::unordered_set<std::string> automatic_clusters;
|
||||
|
||||
Impl impl;
|
||||
mutable std::mutex mutex;
|
||||
};
|
||||
|
479
src/Interpreters/ClusterDiscovery.cpp
Normal file
479
src/Interpreters/ClusterDiscovery.cpp
Normal file
@ -0,0 +1,479 @@
|
||||
#include <algorithm>
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
||||
#include <unordered_map>
|
||||
#include <unordered_set>
|
||||
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <base/logger_useful.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/ZooKeeper/Types.h>
|
||||
#include <Common/setThreadName.h>
|
||||
|
||||
#include <Core/ServerUUID.h>
|
||||
|
||||
#include <Interpreters/Cluster.h>
|
||||
#include <Interpreters/ClusterDiscovery.h>
|
||||
#include <Interpreters/Context.h>
|
||||
|
||||
#include <Poco/Exception.h>
|
||||
#include <Poco/JSON/JSON.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <Poco/JSON/Parser.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
fs::path getShardsListPath(const String & zk_root)
|
||||
{
|
||||
return fs::path(zk_root + "/shards");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/*
|
||||
* Holds boolean flags for fixed set of keys.
|
||||
* Flags can be concurrently set from different threads, and consumer can wait for it.
|
||||
*/
|
||||
template <typename T>
|
||||
class ClusterDiscovery::ConcurrentFlags
|
||||
{
|
||||
public:
|
||||
template <typename It>
|
||||
ConcurrentFlags(It begin, It end)
|
||||
{
|
||||
for (auto it = begin; it != end; ++it)
|
||||
flags.emplace(*it, false);
|
||||
}
|
||||
|
||||
void set(const T & key)
|
||||
{
|
||||
auto it = flags.find(key);
|
||||
if (it == flags.end())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown value '{}'", key);
|
||||
it->second = true;
|
||||
any_need_update = true;
|
||||
cv.notify_one();
|
||||
}
|
||||
|
||||
/// waits unit at least one flag is set
|
||||
/// caller should handle all set flags (or set it again manually)
|
||||
/// note: keys of returen map should not be changed!
|
||||
/// @param finished - output parameter indicates that stop() was called
|
||||
std::unordered_map<T, std::atomic_bool> & wait(std::chrono::milliseconds timeout, bool & finished)
|
||||
{
|
||||
std::unique_lock<std::mutex> lk(mu);
|
||||
cv.wait_for(lk, timeout, [this]() -> bool { return any_need_update || stop_flag; });
|
||||
finished = stop_flag;
|
||||
|
||||
/// all set flags expected to be handled by caller
|
||||
any_need_update = false;
|
||||
return flags;
|
||||
}
|
||||
|
||||
void stop()
|
||||
{
|
||||
std::unique_lock<std::mutex> lk(mu);
|
||||
stop_flag = true;
|
||||
cv.notify_one();
|
||||
}
|
||||
|
||||
private:
|
||||
std::condition_variable cv;
|
||||
std::mutex mu;
|
||||
|
||||
/// flag indicates that update is required
|
||||
std::unordered_map<T, std::atomic_bool> flags;
|
||||
std::atomic_bool any_need_update = true;
|
||||
bool stop_flag = false;
|
||||
};
|
||||
|
||||
ClusterDiscovery::ClusterDiscovery(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
ContextPtr context_,
|
||||
const String & config_prefix)
|
||||
: context(Context::createCopy(context_))
|
||||
, current_node_name(toString(ServerUUID::get()))
|
||||
, log(&Poco::Logger::get("ClusterDiscovery"))
|
||||
{
|
||||
LOG_DEBUG(log, "Cluster discovery is enabled");
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_prefix, config_keys);
|
||||
|
||||
for (const auto & key : config_keys)
|
||||
{
|
||||
String prefix = config_prefix + "." + key + ".discovery";
|
||||
if (!config.has(prefix))
|
||||
continue;
|
||||
|
||||
clusters_info.emplace(
|
||||
key,
|
||||
ClusterInfo(
|
||||
/* name_= */ key,
|
||||
/* zk_root_= */ config.getString(prefix + ".path"),
|
||||
/* port= */ context->getTCPPort(),
|
||||
/* secure= */ config.getBool(prefix + ".secure", false),
|
||||
/* shard_id= */ config.getUInt(prefix + ".shard", 0)
|
||||
)
|
||||
);
|
||||
}
|
||||
clusters_to_update = std::make_shared<UpdateFlags>(config_keys.begin(), config_keys.end());
|
||||
}
|
||||
|
||||
/// List node in zookeper for cluster
|
||||
Strings ClusterDiscovery::getNodeNames(zkutil::ZooKeeperPtr & zk,
|
||||
const String & zk_root,
|
||||
const String & cluster_name,
|
||||
int * version,
|
||||
bool set_callback)
|
||||
{
|
||||
auto watch_callback = [cluster_name, clusters_to_update=clusters_to_update](auto) { clusters_to_update->set(cluster_name); };
|
||||
|
||||
Coordination::Stat stat;
|
||||
Strings nodes = zk->getChildrenWatch(getShardsListPath(zk_root), &stat, set_callback ? watch_callback : Coordination::WatchCallback{});
|
||||
if (version)
|
||||
*version = stat.cversion;
|
||||
return nodes;
|
||||
}
|
||||
|
||||
/// Reads node information from specified zookeeper nodes
|
||||
/// On error returns empty result
|
||||
ClusterDiscovery::NodesInfo ClusterDiscovery::getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids)
|
||||
{
|
||||
NodesInfo result;
|
||||
for (const auto & node_uuid : node_uuids)
|
||||
{
|
||||
String payload;
|
||||
bool ok = zk->tryGet(getShardsListPath(zk_root) / node_uuid, payload) &&
|
||||
NodeInfo::parse(payload, result[node_uuid]);
|
||||
if (!ok)
|
||||
{
|
||||
LOG_WARNING(log, "Can't get data from node '{}' in '{}'", node_uuid, zk_root);
|
||||
return {};
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Checks if cluster nodes set is changed.
|
||||
/// Returns true if update required.
|
||||
/// It performs only shallow check (set of nodes' uuids).
|
||||
/// So, if node's hostname are changed, then cluster won't be updated.
|
||||
bool ClusterDiscovery::needUpdate(const Strings & node_uuids, const NodesInfo & nodes)
|
||||
{
|
||||
bool has_difference = node_uuids.size() != nodes.size() ||
|
||||
std::any_of(node_uuids.begin(), node_uuids.end(), [&nodes] (auto u) { return !nodes.contains(u); });
|
||||
{
|
||||
/// Just to log updated nodes, suboptimal, but should be ok for expected update sizes
|
||||
std::set<String> new_names(node_uuids.begin(), node_uuids.end());
|
||||
std::set<String> old_names;
|
||||
for (const auto & [name, _] : nodes)
|
||||
old_names.emplace(name);
|
||||
|
||||
auto format_cluster_update = [](const std::set<String> & s1, const std::set<String> & s2)
|
||||
{
|
||||
std::vector<String> diff;
|
||||
std::set_difference(s1.begin(), s1.end(), s2.begin(), s2.end(), std::back_inserter(diff));
|
||||
|
||||
constexpr size_t max_to_show = 3;
|
||||
size_t sz = diff.size();
|
||||
bool need_crop = sz > max_to_show;
|
||||
if (need_crop)
|
||||
diff.resize(max_to_show);
|
||||
|
||||
if (sz == 0)
|
||||
return fmt::format("{} nodes", sz);
|
||||
return fmt::format("{} node{} [{}{}]", sz, sz != 1 ? "s" : "", fmt::join(diff, ", "), need_crop ? ",..." : "");
|
||||
};
|
||||
|
||||
LOG_DEBUG(log, "Cluster update: added {}, removed {}",
|
||||
format_cluster_update(new_names, old_names),
|
||||
format_cluster_update(old_names, new_names));
|
||||
}
|
||||
return has_difference;
|
||||
}
|
||||
|
||||
ClusterPtr ClusterDiscovery::makeCluster(const ClusterInfo & cluster_info)
|
||||
{
|
||||
std::vector<std::vector<String>> shards;
|
||||
{
|
||||
std::map<size_t, Strings> replica_adresses;
|
||||
|
||||
for (const auto & [_, node] : cluster_info.nodes_info)
|
||||
{
|
||||
if (cluster_info.current_node.secure != node.secure)
|
||||
{
|
||||
LOG_WARNING(log, "Node '{}' in cluster '{}' has different 'secure' value, skipping it", node.address, cluster_info.name);
|
||||
continue;
|
||||
}
|
||||
replica_adresses[node.shard_id].emplace_back(node.address);
|
||||
}
|
||||
|
||||
shards.reserve(replica_adresses.size());
|
||||
for (auto & [_, replicas] : replica_adresses)
|
||||
shards.emplace_back(std::move(replicas));
|
||||
}
|
||||
|
||||
bool secure = cluster_info.current_node.secure;
|
||||
auto cluster = std::make_shared<Cluster>(
|
||||
context->getSettings(),
|
||||
shards,
|
||||
/* username= */ context->getUserName(),
|
||||
/* password= */ "",
|
||||
/* clickhouse_port= */ secure ? context->getTCPPortSecure().value_or(DBMS_DEFAULT_SECURE_PORT) : context->getTCPPort(),
|
||||
/* treat_local_as_remote= */ false,
|
||||
/* treat_local_port_as_remote= */ context->getApplicationType() == Context::ApplicationType::LOCAL,
|
||||
/* secure= */ secure);
|
||||
return cluster;
|
||||
}
|
||||
|
||||
/// Reads data from zookeeper and tries to update cluster.
|
||||
/// Returns true on success (or no update required).
|
||||
bool ClusterDiscovery::updateCluster(ClusterInfo & cluster_info)
|
||||
{
|
||||
LOG_DEBUG(log, "Updating cluster '{}'", cluster_info.name);
|
||||
|
||||
auto zk = context->getZooKeeper();
|
||||
|
||||
int start_version;
|
||||
Strings node_uuids = getNodeNames(zk, cluster_info.zk_root, cluster_info.name, &start_version, false);
|
||||
auto & nodes_info = cluster_info.nodes_info;
|
||||
|
||||
if (std::find(node_uuids.begin(), node_uuids.end(), current_node_name) == node_uuids.end())
|
||||
{
|
||||
LOG_ERROR(log, "Can't find current node in cluster '{}', will register again", cluster_info.name);
|
||||
registerInZk(zk, cluster_info);
|
||||
nodes_info.clear();
|
||||
return false;
|
||||
}
|
||||
|
||||
if (!needUpdate(node_uuids, nodes_info))
|
||||
{
|
||||
LOG_DEBUG(log, "No update required for cluster '{}'", cluster_info.name);
|
||||
return true;
|
||||
}
|
||||
|
||||
nodes_info = getNodes(zk, cluster_info.zk_root, node_uuids);
|
||||
if (nodes_info.empty())
|
||||
{
|
||||
LOG_WARNING(log, "Can't get nodes info for '{}'", cluster_info.name);
|
||||
return false;
|
||||
}
|
||||
|
||||
int current_version;
|
||||
getNodeNames(zk, cluster_info.zk_root, cluster_info.name, ¤t_version, true);
|
||||
|
||||
if (current_version != start_version)
|
||||
{
|
||||
LOG_DEBUG(log, "Cluster '{}' configuration changed during update", cluster_info.name);
|
||||
nodes_info.clear();
|
||||
return false;
|
||||
}
|
||||
|
||||
LOG_DEBUG(log, "Updating system.clusters record for '{}' with {} nodes", cluster_info.name, cluster_info.nodes_info.size());
|
||||
|
||||
auto cluster = makeCluster(cluster_info);
|
||||
context->setCluster(cluster_info.name, cluster);
|
||||
return true;
|
||||
}
|
||||
|
||||
void ClusterDiscovery::registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info)
|
||||
{
|
||||
LOG_DEBUG(log, "Registering current node {} in cluster {}", current_node_name, info.name);
|
||||
|
||||
String node_path = getShardsListPath(info.zk_root) / current_node_name;
|
||||
zk->createAncestors(node_path);
|
||||
|
||||
zk->createOrUpdate(node_path, info.current_node.serialize(), zkutil::CreateMode::Ephemeral);
|
||||
LOG_DEBUG(log, "Current node {} registered in cluster {}", current_node_name, info.name);
|
||||
}
|
||||
|
||||
void ClusterDiscovery::initialUpdate()
|
||||
{
|
||||
auto zk = context->getZooKeeper();
|
||||
for (auto & [_, info] : clusters_info)
|
||||
{
|
||||
registerInZk(zk, info);
|
||||
if (!updateCluster(info))
|
||||
{
|
||||
LOG_WARNING(log, "Error on initial cluster '{}' update, will retry in background", info.name);
|
||||
clusters_to_update->set(info.name);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ClusterDiscovery::start()
|
||||
{
|
||||
if (clusters_info.empty())
|
||||
{
|
||||
LOG_DEBUG(log, "No defined clusters for discovery");
|
||||
return;
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
initialUpdate();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Caught exception in cluster discovery initialization");
|
||||
}
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
constexpr static std::chrono::milliseconds DEFAULT_BACKOFF_TIMEOUT = 10ms;
|
||||
|
||||
LOG_DEBUG(log, "Starting working thread");
|
||||
main_thread = ThreadFromGlobalPool([this]
|
||||
{
|
||||
std::chrono::milliseconds backoff_timeout = DEFAULT_BACKOFF_TIMEOUT;
|
||||
|
||||
bool finish = false;
|
||||
while (!finish)
|
||||
{
|
||||
try
|
||||
{
|
||||
finish = runMainThread([&backoff_timeout] { backoff_timeout = DEFAULT_BACKOFF_TIMEOUT; });
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/*
|
||||
* it can be zk error (will take new session) or other retriable error,
|
||||
* should not stop discovery forever
|
||||
*/
|
||||
tryLogCurrentException(log, "Caught exception in cluster discovery runMainThread");
|
||||
}
|
||||
std::this_thread::sleep_for(backoff_timeout);
|
||||
backoff_timeout = std::min(backoff_timeout * 2, std::chrono::milliseconds(3min));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/// Returns `true` on graceful shutdown (no restart required)
|
||||
bool ClusterDiscovery::runMainThread(std::function<void()> up_to_date_callback)
|
||||
{
|
||||
setThreadName("ClusterDiscover");
|
||||
LOG_DEBUG(log, "Worker thread started");
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
|
||||
constexpr auto force_update_interval = 2min;
|
||||
bool finished = false;
|
||||
while (!finished)
|
||||
{
|
||||
bool all_up_to_date = true;
|
||||
auto & clusters = clusters_to_update->wait(5s, finished);
|
||||
for (auto & [cluster_name, need_update] : clusters)
|
||||
{
|
||||
auto cluster_info_it = clusters_info.find(cluster_name);
|
||||
if (cluster_info_it == clusters_info.end())
|
||||
{
|
||||
LOG_ERROR(log, "Unknown cluster '{}'", cluster_name);
|
||||
continue;
|
||||
}
|
||||
auto & cluster_info = cluster_info_it->second;
|
||||
|
||||
if (!need_update.exchange(false))
|
||||
{
|
||||
/// force updating periodically
|
||||
bool force_update = cluster_info.watch.elapsedSeconds() > std::chrono::seconds(force_update_interval).count();
|
||||
if (!force_update)
|
||||
continue;
|
||||
}
|
||||
|
||||
if (updateCluster(cluster_info))
|
||||
{
|
||||
cluster_info.watch.restart();
|
||||
LOG_DEBUG(log, "Cluster '{}' updated successfully", cluster_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
all_up_to_date = false;
|
||||
/// no need to trigger convar, will retry after timeout in `wait`
|
||||
need_update = true;
|
||||
LOG_WARNING(log, "Cluster '{}' wasn't updated, will retry", cluster_name);
|
||||
}
|
||||
}
|
||||
|
||||
if (all_up_to_date)
|
||||
{
|
||||
up_to_date_callback();
|
||||
}
|
||||
}
|
||||
LOG_DEBUG(log, "Worker thread stopped");
|
||||
return finished;
|
||||
}
|
||||
|
||||
void ClusterDiscovery::shutdown()
|
||||
{
|
||||
LOG_DEBUG(log, "Shutting down");
|
||||
clusters_to_update->stop();
|
||||
|
||||
if (main_thread.joinable())
|
||||
main_thread.join();
|
||||
}
|
||||
|
||||
ClusterDiscovery::~ClusterDiscovery()
|
||||
{
|
||||
ClusterDiscovery::shutdown();
|
||||
}
|
||||
|
||||
bool ClusterDiscovery::NodeInfo::parse(const String & data, NodeInfo & result)
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::JSON::Parser parser;
|
||||
auto json = parser.parse(data).extract<Poco::JSON::Object::Ptr>();
|
||||
|
||||
size_t ver = json->optValue<size_t>("version", data_ver);
|
||||
if (ver == data_ver)
|
||||
{
|
||||
result.address = json->getValue<std::string>("address");
|
||||
result.secure = json->optValue<bool>("secure", false);
|
||||
result.shard_id = json->optValue<size_t>("shard_id", 0);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_ERROR(
|
||||
&Poco::Logger::get("ClusterDiscovery"),
|
||||
"Unsupported version '{}' of data in zk node '{}'",
|
||||
ver, data.size() < 1024 ? data : "[data too long]");
|
||||
}
|
||||
}
|
||||
catch (Poco::Exception & e)
|
||||
{
|
||||
LOG_WARNING(
|
||||
&Poco::Logger::get("ClusterDiscovery"),
|
||||
"Can't parse '{}' from node: {}",
|
||||
data.size() < 1024 ? data : "[data too long]", e.displayText());
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
String ClusterDiscovery::NodeInfo::serialize() const
|
||||
{
|
||||
Poco::JSON::Object json;
|
||||
json.set("version", data_ver);
|
||||
json.set("address", address);
|
||||
json.set("shard_id", shard_id);
|
||||
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(json, oss);
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
}
|
124
src/Interpreters/ClusterDiscovery.h
Normal file
124
src/Interpreters/ClusterDiscovery.h
Normal file
@ -0,0 +1,124 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Common/ZooKeeper/Common.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Interpreters/Cluster.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
#include <base/defines.h>
|
||||
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/*
|
||||
* Discover cluster nodes.
|
||||
*
|
||||
* Each node adds ephemernal node into specified path in zookeeper (each cluster have own path).
|
||||
* Also node subscribed for updates for these paths, and at each child node chanhe cluster updated.
|
||||
* When node goes down ephemernal node are destroyed, cluster configuration is updated on other node and gone node is removed from cluster.
|
||||
*/
|
||||
class ClusterDiscovery
|
||||
{
|
||||
|
||||
public:
|
||||
ClusterDiscovery(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
ContextPtr context_,
|
||||
const String & config_prefix = "remote_servers");
|
||||
|
||||
void start();
|
||||
|
||||
~ClusterDiscovery();
|
||||
|
||||
private:
|
||||
struct NodeInfo
|
||||
{
|
||||
/// versioning for format of data stored in zk
|
||||
static constexpr size_t data_ver = 1;
|
||||
|
||||
/// host:port
|
||||
String address;
|
||||
/// is secure tcp port user
|
||||
bool secure = false;
|
||||
/// shard number
|
||||
size_t shard_id = 0;
|
||||
|
||||
NodeInfo() = default;
|
||||
explicit NodeInfo(const String & address_, bool secure_, size_t shard_id_)
|
||||
: address(address_)
|
||||
, secure(secure_)
|
||||
, shard_id(shard_id_)
|
||||
{}
|
||||
|
||||
static bool parse(const String & data, NodeInfo & result);
|
||||
String serialize() const;
|
||||
};
|
||||
|
||||
// node uuid -> address ("host:port")
|
||||
using NodesInfo = std::unordered_map<String, NodeInfo>;
|
||||
|
||||
struct ClusterInfo
|
||||
{
|
||||
const String name;
|
||||
const String zk_root;
|
||||
NodesInfo nodes_info;
|
||||
|
||||
/// Track last update time
|
||||
Stopwatch watch;
|
||||
|
||||
NodeInfo current_node;
|
||||
|
||||
explicit ClusterInfo(const String & name_, const String & zk_root_, UInt16 port, bool secure, size_t shard_id)
|
||||
: name(name_)
|
||||
, zk_root(zk_root_)
|
||||
, current_node(getFQDNOrHostName() + ":" + toString(port), secure, shard_id)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
void initialUpdate();
|
||||
|
||||
void registerInZk(zkutil::ZooKeeperPtr & zk, ClusterInfo & info);
|
||||
|
||||
Strings getNodeNames(zkutil::ZooKeeperPtr & zk,
|
||||
const String & zk_root,
|
||||
const String & cluster_name,
|
||||
int * version = nullptr,
|
||||
bool set_callback = true);
|
||||
|
||||
NodesInfo getNodes(zkutil::ZooKeeperPtr & zk, const String & zk_root, const Strings & node_uuids);
|
||||
|
||||
ClusterPtr makeCluster(const ClusterInfo & cluster_info);
|
||||
|
||||
bool needUpdate(const Strings & node_uuids, const NodesInfo & nodes);
|
||||
bool updateCluster(ClusterInfo & cluster_info);
|
||||
|
||||
bool runMainThread(std::function<void()> up_to_date_callback);
|
||||
void shutdown();
|
||||
|
||||
/// cluster name -> cluster info (zk root, set of nodes)
|
||||
std::unordered_map<String, ClusterInfo> clusters_info;
|
||||
|
||||
ContextMutablePtr context;
|
||||
|
||||
String current_node_name;
|
||||
|
||||
template <typename T> class ConcurrentFlags;
|
||||
using UpdateFlags = ConcurrentFlags<std::string>;
|
||||
|
||||
/// Cluster names to update.
|
||||
/// The `shared_ptr` is used because it's passed to watch callback.
|
||||
/// It prevents accessing to invalid object after ClusterDiscovery is destroyed.
|
||||
std::shared_ptr<UpdateFlags> clusters_to_update;
|
||||
|
||||
ThreadFromGlobalPool main_thread;
|
||||
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
}
|
@ -86,6 +86,7 @@
|
||||
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
|
||||
#include <Interpreters/SynonymsExtensions.h>
|
||||
#include <Interpreters/Lemmatizers.h>
|
||||
#include <Interpreters/ClusterDiscovery.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
@ -254,6 +255,7 @@ struct ContextSharedPart
|
||||
std::shared_ptr<Clusters> clusters;
|
||||
ConfigurationPtr clusters_config; /// Stores updated configs
|
||||
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
|
||||
std::unique_ptr<ClusterDiscovery> cluster_discovery;
|
||||
|
||||
std::shared_ptr<AsynchronousInsertQueue> async_insert_queue;
|
||||
std::map<String, UInt16> server_ports;
|
||||
@ -2195,11 +2197,22 @@ std::shared_ptr<Clusters> Context::getClusters() const
|
||||
return shared->clusters;
|
||||
}
|
||||
|
||||
void Context::startClusterDiscovery()
|
||||
{
|
||||
if (!shared->cluster_discovery)
|
||||
return;
|
||||
shared->cluster_discovery->start();
|
||||
}
|
||||
|
||||
|
||||
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
|
||||
void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name)
|
||||
void Context::setClustersConfig(const ConfigurationPtr & config, bool enable_discovery, const String & config_name)
|
||||
{
|
||||
std::lock_guard lock(shared->clusters_mutex);
|
||||
if (config->getBool("allow_experimental_cluster_discovery", false) && enable_discovery && !shared->cluster_discovery)
|
||||
{
|
||||
shared->cluster_discovery = std::make_unique<ClusterDiscovery>(*config, getGlobalContext());
|
||||
}
|
||||
|
||||
/// Do not update clusters if this part of config wasn't changed.
|
||||
if (shared->clusters && isSameConfiguration(*config, *shared->clusters_config, config_name))
|
||||
@ -2209,7 +2222,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String &
|
||||
shared->clusters_config = config;
|
||||
|
||||
if (!shared->clusters)
|
||||
shared->clusters = std::make_unique<Clusters>(*shared->clusters_config, settings, config_name);
|
||||
shared->clusters = std::make_shared<Clusters>(*shared->clusters_config, settings, config_name);
|
||||
else
|
||||
shared->clusters->updateClusters(*shared->clusters_config, settings, config_name, old_clusters_config);
|
||||
}
|
||||
|
@ -750,7 +750,10 @@ public:
|
||||
std::shared_ptr<Clusters> getClusters() const;
|
||||
std::shared_ptr<Cluster> getCluster(const std::string & cluster_name) const;
|
||||
std::shared_ptr<Cluster> tryGetCluster(const std::string & cluster_name) const;
|
||||
void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers");
|
||||
void setClustersConfig(const ConfigurationPtr & config, bool enable_discovery = false, const String & config_name = "remote_servers");
|
||||
|
||||
void startClusterDiscovery();
|
||||
|
||||
/// Sets custom cluster, but doesn't update configuration
|
||||
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
|
||||
void reloadClusterConfig() const;
|
||||
|
@ -469,7 +469,7 @@ void InterpreterSystemQuery::restoreReplica()
|
||||
{
|
||||
getContext()->checkAccess(AccessType::SYSTEM_RESTORE_REPLICA, table_id);
|
||||
|
||||
const zkutil::ZooKeeperPtr& zookeeper = getContext()->getZooKeeper();
|
||||
const zkutil::ZooKeeperPtr & zookeeper = getContext()->getZooKeeper();
|
||||
|
||||
if (zookeeper->expired())
|
||||
throw Exception(ErrorCodes::NO_ZOOKEEPER,
|
||||
|
@ -74,7 +74,17 @@ template<typename T>
|
||||
void PostgreSQLSource<T>::onStart()
|
||||
{
|
||||
if (!tx)
|
||||
tx = std::make_shared<T>(connection_holder->get());
|
||||
{
|
||||
try
|
||||
{
|
||||
tx = std::make_shared<T>(connection_holder->get());
|
||||
}
|
||||
catch (const pqxx::broken_connection &)
|
||||
{
|
||||
connection_holder->update();
|
||||
tx = std::make_shared<T>(connection_holder->get());
|
||||
}
|
||||
}
|
||||
|
||||
stream = std::make_unique<pqxx::stream_from>(*tx, pqxx::from_query, std::string_view(query_str));
|
||||
}
|
||||
|
@ -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
|
||||
};
|
||||
|
@ -45,7 +45,8 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co
|
||||
// get an error when trying to get the info about DB from ZK.
|
||||
// Just ignore these inaccessible databases. A good example of a
|
||||
// failing test is `01526_client_start_and_exit`.
|
||||
try {
|
||||
try
|
||||
{
|
||||
writeCluster(res_columns, {name_and_database.first, replicated->getCluster()});
|
||||
}
|
||||
catch (...)
|
||||
|
@ -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);
|
||||
|
||||
|
@ -50,6 +50,7 @@ ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(ContextPtr c
|
||||
|
||||
if (!columns)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table structure not returned");
|
||||
|
||||
return ColumnsDescription{*columns};
|
||||
}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -1490,6 +1490,18 @@ class ClickHouseCluster:
|
||||
|
||||
common_opts = ['--verbose', 'up', '-d']
|
||||
|
||||
images_pull_cmd = self.base_cmd + ['pull']
|
||||
# sometimes dockerhub/proxy can be flaky
|
||||
for i in range(5):
|
||||
try:
|
||||
run_and_check(images_pull_cmd)
|
||||
break
|
||||
except Exception as ex:
|
||||
if i == 4:
|
||||
raise ex
|
||||
logging.info("Got exception pulling images: %s", ex)
|
||||
time.sleep(i * 3)
|
||||
|
||||
if self.with_zookeeper_secure and self.base_zookeeper_cmd:
|
||||
logging.debug('Setup ZooKeeper Secure')
|
||||
logging.debug(f'Creating internal ZooKeeper dirs: {self.zookeeper_dirs_to_create}')
|
||||
@ -2065,7 +2077,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 = {}
|
||||
@ -2079,12 +2092,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
|
||||
|
23
tests/integration/test_cluster_discovery/config/config.xml
Normal file
23
tests/integration/test_cluster_discovery/config/config.xml
Normal file
@ -0,0 +1,23 @@
|
||||
<clickhouse>
|
||||
<allow_experimental_cluster_discovery>1</allow_experimental_cluster_discovery>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
<two_shards>
|
||||
<!-- just to check that there's no conflict between automatic and manual clusters -->
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</two_shards>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -0,0 +1,24 @@
|
||||
<clickhouse>
|
||||
<allow_experimental_cluster_discovery>1</allow_experimental_cluster_discovery>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
<shard>1</shard>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
<two_shards>
|
||||
<!-- just to check that there's no conflict between automatic and manual clusters -->
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</two_shards>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
@ -0,0 +1,24 @@
|
||||
<clickhouse>
|
||||
<allow_experimental_cluster_discovery>1</allow_experimental_cluster_discovery>
|
||||
<remote_servers>
|
||||
<test_auto_cluster>
|
||||
<discovery>
|
||||
<path>/clickhouse/discovery/test_auto_cluster</path>
|
||||
<shard>3</shard>
|
||||
</discovery>
|
||||
</test_auto_cluster>
|
||||
<two_shards>
|
||||
<!-- just to check that there's no conflict between automatic and manual clusters -->
|
||||
<shard>
|
||||
<replica>
|
||||
<host>node1</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
<replica>
|
||||
<host>node2</host>
|
||||
<port>9000</port>
|
||||
</replica>
|
||||
</shard>
|
||||
</two_shards>
|
||||
</remote_servers>
|
||||
</clickhouse>
|
81
tests/integration/test_cluster_discovery/test.py
Normal file
81
tests/integration/test_cluster_discovery/test.py
Normal file
@ -0,0 +1,81 @@
|
||||
import pytest
|
||||
|
||||
import functools
|
||||
import time
|
||||
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
|
||||
shard_configs = {
|
||||
i: f'config/config_shard{i}.xml'
|
||||
for i in [1, 3]
|
||||
}
|
||||
|
||||
nodes = [
|
||||
cluster.add_instance(
|
||||
f'node{i}',
|
||||
main_configs=[shard_configs.get(i, 'config/config.xml')],
|
||||
stay_alive=True,
|
||||
with_zookeeper=True
|
||||
) for i in range(5)
|
||||
]
|
||||
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def start_cluster():
|
||||
try:
|
||||
cluster.start()
|
||||
yield cluster
|
||||
finally:
|
||||
cluster.shutdown()
|
||||
|
||||
|
||||
def check_on_cluster(nodes, expected, *, what, cluster_name='test_auto_cluster', msg=None, retries=5):
|
||||
"""
|
||||
Select data from `system.clusters` on specified nodes and check the result
|
||||
"""
|
||||
assert 1 <= retries <= 6
|
||||
|
||||
for retry in range(1, retries + 1):
|
||||
nodes_res = {
|
||||
node.name: int(node.query(f"SELECT {what} FROM system.clusters WHERE cluster = '{cluster_name}'"))
|
||||
for node in nodes
|
||||
}
|
||||
if all(actual == expected for actual in nodes_res.values()):
|
||||
break
|
||||
|
||||
if retry != retries:
|
||||
time.sleep(2 ** retry)
|
||||
else:
|
||||
msg = msg or f"Wrong '{what}' result"
|
||||
raise Exception(f'{msg}: {nodes_res}, expected: {expected} (after {retries} retries)')
|
||||
|
||||
|
||||
def test_cluster_discovery_startup_and_stop(start_cluster):
|
||||
"""
|
||||
Start cluster, check nodes count in system.clusters,
|
||||
then stop/start some nodes and check that it (dis)appeared in cluster.
|
||||
"""
|
||||
|
||||
check_nodes_count = functools.partial(check_on_cluster, what='count()', msg='Wrong nodes count in cluster')
|
||||
check_shard_num = functools.partial(check_on_cluster, what='count(DISTINCT shard_num)', msg='Wrong shard_num count in cluster')
|
||||
|
||||
total_shards = len(shard_configs) + 1
|
||||
check_nodes_count([nodes[0], nodes[2]], len(nodes))
|
||||
check_shard_num([nodes[0], nodes[2]], total_shards)
|
||||
|
||||
nodes[1].stop_clickhouse(kill=True)
|
||||
check_nodes_count([nodes[0], nodes[2]], len(nodes) - 1)
|
||||
check_shard_num([nodes[0], nodes[2]], total_shards - 1)
|
||||
|
||||
nodes[3].stop_clickhouse()
|
||||
check_nodes_count([nodes[0], nodes[2]], len(nodes) - 2)
|
||||
|
||||
nodes[1].start_clickhouse()
|
||||
check_nodes_count([nodes[0], nodes[2]], len(nodes) - 1)
|
||||
|
||||
nodes[3].start_clickhouse()
|
||||
check_nodes_count([nodes[0], nodes[2]], len(nodes))
|
||||
|
||||
check_nodes_count([nodes[1], nodes[2]], 2, cluster_name='two_shards', retries=1)
|
@ -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'
|
||||
|
@ -50,7 +50,6 @@ def skip_if_jemalloc_disabled():
|
||||
if output != b'ON' and output != b'1':
|
||||
pytest.skip(f'Compiled w/o jemalloc (USE_JEMALLOC={output})')
|
||||
|
||||
|
||||
# Ensure that clickhouse works even when number of online CPUs
|
||||
# (_SC_NPROCESSORS_ONLN) is smaller then available (_SC_NPROCESSORS_CONF).
|
||||
#
|
||||
|
@ -49,12 +49,16 @@ def test_merge_doesnt_work_without_zookeeper(start_cluster):
|
||||
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)")
|
||||
node1.query("INSERT INTO test_table VALUES ('2018-10-01', 4), ('2018-10-02', 5), ('2018-10-03', 6)")
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "2\n"
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table' and active") == "2\n"
|
||||
|
||||
with PartitionManager() as pm:
|
||||
node1.query("OPTIMIZE TABLE test_table FINAL")
|
||||
pm.drop_instance_zk_connections(node1)
|
||||
time.sleep(10) # > old_parts_lifetime
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "3\n"
|
||||
# unfortunately we can be too fast and delete node before partition with ZK
|
||||
if node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "1\n":
|
||||
print("We were too fast and deleted parts before partition with ZK")
|
||||
else:
|
||||
time.sleep(10) # > old_parts_lifetime
|
||||
assert node1.query("SELECT count(*) from system.parts where table = 'test_table'") == "3\n"
|
||||
|
||||
assert_eq_with_retry(node1, "SELECT count(*) from system.parts where table = 'test_table' and active = 1", "1")
|
||||
|
@ -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.
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user