Merge branch 'master' of https://github.com/ClickHouse/ClickHouse into gyuton-DOCSUP-8002-Edit-and-translate-to-Russian

This commit is contained in:
George 2021-03-30 11:55:49 +03:00
commit 9bd66638bd
174 changed files with 2349 additions and 1466 deletions

View File

@ -1,45 +1,28 @@
// https://stackoverflow.com/questions/1413445/reading-a-password-from-stdcin
#include <common/setTerminalEcho.h>
#include <common/errnoToString.h>
#include <stdexcept>
#include <cstring>
#include <string>
#ifdef WIN32
#include <windows.h>
#else
#include <termios.h>
#include <unistd.h>
#include <errno.h>
#endif
void setTerminalEcho(bool enable)
{
#ifdef WIN32
auto handle = GetStdHandle(STD_INPUT_HANDLE);
DWORD mode;
if (!GetConsoleMode(handle, &mode))
throw std::runtime_error(std::string("setTerminalEcho failed get: ") + std::to_string(GetLastError()));
/// Obtain terminal attributes,
/// toggle the ECHO flag
/// and set them back.
if (!enable)
mode &= ~ENABLE_ECHO_INPUT;
else
mode |= ENABLE_ECHO_INPUT;
struct termios tty{};
if (!SetConsoleMode(handle, mode))
throw std::runtime_error(std::string("setTerminalEcho failed set: ") + std::to_string(GetLastError()));
#else
struct termios tty;
if (tcgetattr(STDIN_FILENO, &tty))
if (0 != tcgetattr(STDIN_FILENO, &tty))
throw std::runtime_error(std::string("setTerminalEcho failed get: ") + errnoToString(errno));
if (!enable)
tty.c_lflag &= ~ECHO;
else
tty.c_lflag |= ECHO;
auto ret = tcsetattr(STDIN_FILENO, TCSANOW, &tty);
if (ret)
if (enable)
tty.c_lflag |= ECHO;
else
tty.c_lflag &= ~ECHO;
if (0 != tcsetattr(STDIN_FILENO, TCSANOW, &tty))
throw std::runtime_error(std::string("setTerminalEcho failed set: ") + errnoToString(errno));
#endif
}

File diff suppressed because it is too large Load Diff

View File

@ -70,7 +70,7 @@ function start_server
--path "$FASTTEST_DATA"
--user_files_path "$FASTTEST_DATA/user_files"
--top_level_domains_path "$FASTTEST_DATA/top_level_domains"
--test_keeper_server.log_storage_path "$FASTTEST_DATA/coordination"
--keeper_server.log_storage_path "$FASTTEST_DATA/coordination"
)
clickhouse-server "${opts[@]}" &>> "$FASTTEST_OUTPUT/server.log" &
server_pid=$!

View File

@ -1514,6 +1514,14 @@ FORMAT PrettyCompactMonoBlock
Default value: 0
## optimize_skip_unused_shards_limit {#optimize-skip-unused-shards-limit}
Limit for number of sharding key values, turns off `optimize_skip_unused_shards` if the limit is reached.
Too many values may require significant amount for processing, while the benefit is doubtful, since if you have huge number of values in `IN (...)`, then most likely the query will be sent to all shards anyway.
Default value: 1000
## optimize_skip_unused_shards {#optimize-skip-unused-shards}
Enables or disables skipping of unused shards for [SELECT](../../sql-reference/statements/select/index.md) queries that have sharding key condition in `WHERE/PREWHERE` (assuming that the data is distributed by sharding key, otherwise does nothing).

View File

@ -243,7 +243,7 @@ The function works according to the algorithm:
**Syntax**
``` sql
windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
windowFunnel(window, [mode, [mode, ... ]])(timestamp, cond1, cond2, ..., condN)
```
**Arguments**
@ -254,8 +254,10 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
**Parameters**
- `window` — Length of the sliding window. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`.
- `mode` — It is an optional argument.
- `'strict'` — When the `'strict'` is set, the windowFunnel() applies conditions only for the unique values.
- `mode` — It is an optional argument. One or more modes can be set.
- `'strict'` — If same condition holds for sequence of events then such non-unique events would be skipped.
- `'strict_order'` — Don't allow interventions of other events. E.g. in the case of `A->B->D->C`, it stops finding `A->B->C` at the `D` and the max event level is 2.
- `'strict_increase'` — Apply conditions only to events with strictly increasing timestamps.
**Returned value**

View File

@ -12,6 +12,8 @@ The search is case-sensitive by default in all these functions. There are separa
## position(haystack, needle), locate(haystack, needle) {#position}
Searches for the substring `needle` in the string `haystack`.
Returns the position (in bytes) of the found substring in the string, starting from 1.
For a case-insensitive search, use the function [positionCaseInsensitive](#positioncaseinsensitive).
@ -22,13 +24,20 @@ For a case-insensitive search, use the function [positionCaseInsensitive](#posit
position(haystack, needle[, start_pos])
```
``` sql
position(needle IN haystack)
```
Alias: `locate(haystack, needle[, start_pos])`.
!!! note "Note"
Syntax of `position(needle IN haystack)` provides SQL-compatibility, the function works the same way as to `position(haystack, needle)`.
**Arguments**
- `haystack` — String, in which substring will to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substring to be searched. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `start_pos` — Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md).
- `start_pos` Position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). Optional.
**Returned values**
@ -83,6 +92,36 @@ Result:
└───────────────────────────────┘
```
**Examples for POSITION(needle IN haystack) syntax**
Query:
```sql
SELECT 3 = position('c' IN 'abc');
```
Result:
```text
┌─equals(3, position('abc', 'c'))─┐
│ 1 │
└─────────────────────────────────┘
```
Query:
```sql
SELECT 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s);
```
Result:
```text
┌─equals(6, position(s, '/'))─┐
│ 1 │
└─────────────────────────────┘
```
## positionCaseInsensitive {#positioncaseinsensitive}
The same as [position](#position) returns the position (in bytes) of the found substring in the string, starting from 1. Use the function for a case-insensitive search.
@ -772,4 +811,3 @@ Result:
│ 2 │
└───────────────────────────────┘
```

View File

@ -243,7 +243,7 @@ SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t
**Синтаксис**
``` sql
windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
windowFunnel(window, [mode, [mode, ... ]])(timestamp, cond1, cond2, ..., condN)
```
**Аргументы**
@ -254,7 +254,10 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN)
**Параметры**
- `window` — ширина скользящего окна по времени. Единица измерения зависит от `timestamp` и может варьироваться. Должно соблюдаться условие `timestamp события cond2 <= timestamp события cond1 + window`.
- `mode` — необязательный параметр. Если установлено значение `'strict'`, то функция `windowFunnel()` применяет условия только для уникальных значений.
- `mode` — необязательный параметр. Может быть установленно несколько значений одновременно.
- `'strict'` — не учитывать подряд идущие повторяющиеся события.
- `'strict_order'` — запрещает посторонние события в искомой последовательности. Например, при поиске цепочки `A->B->C` в `A->B->D->C` поиск будет остановлен на `D` и функция вернет 2.
- `'strict_increase'` — условия прменяются только для событий со строго возрастающими временными метками.
**Возвращаемое значение**

View File

@ -7,7 +7,7 @@ toc_title: "Функции поиска в строках"
Во всех функциях, поиск регистрозависимый по умолчанию. Существуют варианты функций для регистронезависимого поиска.
## position(haystack, needle) {#position}
## position(haystack, needle), locate(haystack, needle) {#position}
Поиск подстроки `needle` в строке `haystack`.
@ -21,8 +21,15 @@ toc_title: "Функции поиска в строках"
position(haystack, needle[, start_pos])
```
``` sql
position(needle IN haystack)
```
Алиас: `locate(haystack, needle[, start_pos])`.
!!! note "Примечание"
Синтаксис `position(needle IN haystack)` обеспечивает совместимость с SQL, функция работает так же, как `position(haystack, needle)`.
**Аргументы**
- `haystack` — строка, по которой выполняется поиск. [Строка](../syntax.md#syntax-string-literal).
@ -70,6 +77,36 @@ SELECT position('Привет, мир!', '!');
└───────────────────────────────┘
```
**Примеры работы функции с синтаксисом POSITION(needle IN haystack)**
Запрос:
```sql
SELECT 1 = position('абв' IN 'абв');
```
Результат:
```text
┌─equals(1, position('абв', 'абв'))─┐
│ 1 │
└───────────────────────────────────┘
```
Запрос:
```sql
SELECT 0 = position('абв' IN '');
```
Результат:
```text
┌─equals(0, position('', 'абв'))─┐
│ 1 │
└────────────────────────────────┘
```
## positionCaseInsensitive {#positioncaseinsensitive}
Такая же, как и [position](#position), но работает без учета регистра. Возвращает позицию в байтах найденной подстроки в строке, начиная с 1.
@ -758,4 +795,3 @@ SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв');
│ 3 │
└────────────────────────────────────────────────────────────┘
```

View File

@ -23,7 +23,6 @@ nltk==3.5
nose==1.3.7
protobuf==3.14.0
numpy==1.19.2
Pygments==2.5.2
pymdown-extensions==8.0
python-slugify==4.0.1
PyYAML==5.4.1
@ -36,3 +35,4 @@ termcolor==1.1.0
tornado==6.1
Unidecode==1.1.1
urllib3==1.25.10
Pygments>=2.7.4

View File

@ -97,7 +97,7 @@
#endif
#if USE_NURAFT
# include <Server/NuKeeperTCPHandlerFactory.h>
# include <Server/KeeperTCPHandlerFactory.h>
#endif
namespace CurrentMetrics
@ -867,15 +867,15 @@ int Server::main(const std::vector<std::string> & /*args*/)
listen_try = true;
}
if (config().has("test_keeper_server"))
if (config().has("keeper_server"))
{
#if USE_NURAFT
/// Initialize test keeper RAFT. Do nothing if no nu_keeper_server in config.
global_context->initializeNuKeeperStorageDispatcher();
global_context->initializeKeeperStorageDispatcher();
for (const auto & listen_host : listen_hosts)
{
/// TCP NuKeeper
const char * port_name = "test_keeper_server.tcp_port";
/// TCP Keeper
const char * port_name = "keeper_server.tcp_port";
createServer(listen_host, port_name, listen_try, [&](UInt16 port)
{
Poco::Net::ServerSocket socket;
@ -885,9 +885,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
servers_to_start_before_tables->emplace_back(
port_name,
std::make_unique<Poco::Net::TCPServer>(
new NuKeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
new KeeperTCPHandlerFactory(*this), server_pool, socket, new Poco::Net::TCPServerParams));
LOG_INFO(log, "Listening for connections to NuKeeper (tcp): {}", address.toString());
LOG_INFO(log, "Listening for connections to Keeper (tcp): {}", address.toString());
});
}
#else
@ -934,7 +934,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
else
LOG_INFO(log, "Closed connections to servers for tables.");
global_context->shutdownNuKeeperStorageDispatcher();
global_context->shutdownKeeperStorageDispatcher();
}
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.

View File

@ -0,0 +1 @@
../../../tests/config/config.d/keeper_port.xml

View File

@ -1 +0,0 @@
../../../tests/config/config.d/test_keeper_port.xml

View File

@ -6,7 +6,6 @@
#include <DataTypes/DataTypeDateTime.h>
#include <ext/range.h>
#include "registerAggregateFunctions.h"
namespace DB

View File

@ -29,6 +29,7 @@ struct ComparePair final
};
static constexpr auto max_events = 32;
template <typename T>
struct AggregateFunctionWindowFunnelData
{
@ -46,7 +47,7 @@ struct AggregateFunctionWindowFunnelData
void add(T timestamp, UInt8 event)
{
// Since most events should have already been sorted by timestamp.
/// Since most events should have already been sorted by timestamp.
if (sorted && events_list.size() > 0)
{
if (events_list.back().first == timestamp)
@ -145,14 +146,20 @@ class AggregateFunctionWindowFunnel final
private:
UInt64 window;
UInt8 events_size;
UInt8 strict; // When the 'strict' is set, it applies conditions only for the not repeating values.
UInt8 strict_order; // When the 'strict_order' is set, it doesn't allow interventions of other events.
// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
/// When the 'strict' is set, it applies conditions only for the not repeating values.
bool strict;
// Loop through the entire events_list, update the event timestamp value
// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
// If found, returns the max event level, else return 0.
// The Algorithm complexity is O(n).
/// When the 'strict_order' is set, it doesn't allow interventions of other events.
/// In the case of 'A->B->D->C', it stops finding 'A->B->C' at the 'D' and the max event level is 2.
bool strict_order;
/// Applies conditions only to events with strictly increasing timestamps
bool strict_increase;
/// Loop through the entire events_list, update the event timestamp value
/// The level path must be 1---2---3---...---check_events_size, find the max event level that satisfied the path in the sliding window.
/// If found, returns the max event level, else return 0.
/// The Algorithm complexity is O(n).
UInt8 getEventLevel(Data & data) const
{
if (data.size() == 0)
@ -162,16 +169,13 @@ private:
data.sort();
/// events_timestamp stores the timestamp that latest i-th level event happen within time window after previous level event.
/// timestamp defaults to -1, which unsigned timestamp value never meet
/// there may be some bugs when UInt64 type timstamp overflows Int64, but it works on most cases.
std::vector<Int64> events_timestamp(events_size, -1);
/// events_timestamp stores the timestamp of the first and previous i-th level event happen within time window
std::vector<std::optional<std::pair<UInt64, UInt64>>> events_timestamp(events_size);
bool first_event = false;
for (const auto & pair : data.events_list)
{
const T & timestamp = pair.first;
const auto & event_idx = pair.second - 1;
if (strict_order && event_idx == -1)
{
if (first_event)
@ -181,31 +185,39 @@ private:
}
else if (event_idx == 0)
{
events_timestamp[0] = timestamp;
events_timestamp[0] = std::make_pair(timestamp, timestamp);
first_event = true;
}
else if (strict && events_timestamp[event_idx] >= 0)
else if (strict && events_timestamp[event_idx].has_value())
{
return event_idx + 1;
}
else if (strict_order && first_event && events_timestamp[event_idx - 1] == -1)
else if (strict_order && first_event && !events_timestamp[event_idx - 1].has_value())
{
for (size_t event = 0; event < events_timestamp.size(); ++event)
{
if (events_timestamp[event] == -1)
if (!events_timestamp[event].has_value())
return event;
}
}
else if (events_timestamp[event_idx - 1] >= 0 && timestamp <= events_timestamp[event_idx - 1] + window)
else if (events_timestamp[event_idx - 1].has_value())
{
events_timestamp[event_idx] = events_timestamp[event_idx - 1];
auto first_timestamp = events_timestamp[event_idx - 1]->first;
bool time_matched = timestamp <= first_timestamp + window;
if (strict_increase)
time_matched = time_matched && events_timestamp[event_idx - 1]->second < timestamp;
if (time_matched)
{
events_timestamp[event_idx] = std::make_pair(first_timestamp, timestamp);
if (event_idx + 1 == events_size)
return events_size;
}
}
}
for (size_t event = events_timestamp.size(); event > 0; --event)
{
if (events_timestamp[event - 1] >= 0)
if (events_timestamp[event - 1].has_value())
return event;
}
return 0;
@ -223,15 +235,18 @@ public:
events_size = arguments.size() - 1;
window = params.at(0).safeGet<UInt64>();
strict = 0;
strict_order = 0;
strict = false;
strict_order = false;
strict_increase = false;
for (size_t i = 1; i < params.size(); ++i)
{
String option = params.at(i).safeGet<String>();
if (option.compare("strict") == 0)
strict = 1;
else if (option.compare("strict_order") == 0)
strict_order = 1;
if (option == "strict")
strict = true;
else if (option == "strict_order")
strict_order = true;
else if (option == "strict_increase")
strict_increase = true;
else
throw Exception{"Aggregate function " + getName() + " doesn't support a parameter: " + option, ErrorCodes::BAD_ARGUMENTS};
}
@ -253,7 +268,7 @@ public:
{
bool has_event = false;
const auto timestamp = assert_cast<const ColumnVector<T> *>(columns[0])->getData()[row_num];
// reverse iteration and stable sorting are needed for events that are qualified by more than one condition.
/// reverse iteration and stable sorting are needed for events that are qualified by more than one condition.
for (auto i = events_size; i > 0; --i)
{
auto event = assert_cast<const ColumnVector<UInt8> *>(columns[i])->getData()[row_num];

View File

@ -13,6 +13,7 @@
#include <Common/NaNUtils.h>
#include <Poco/Exception.h>
namespace DB
{
namespace ErrorCodes
@ -162,6 +163,11 @@ public:
sorted = false;
}
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wclass-memaccess"
#endif
void write(DB::WriteBuffer & buf) const
{
size_t size = samples.size();
@ -169,8 +175,25 @@ public:
DB::writeIntBinary<size_t>(total_values, buf);
for (size_t i = 0; i < size; ++i)
DB::writePODBinary(samples[i], buf);
{
/// There was a mistake in this function.
/// Instead of correctly serializing the elements,
/// it was writing them with uninitialized padding.
/// Here we ensure that padding is zero without changing the protocol.
/// TODO: After implementation of "versioning aggregate function state",
/// change the serialization format.
Element elem;
memset(&elem, 0, sizeof(elem));
elem = samples[i];
DB::writePODBinary(elem, buf);
}
}
#if !__clang__
#pragma GCC diagnostic pop
#endif
private:
/// We allocate some memory on the stack to avoid allocations when there are many objects with a small number of elements.

View File

@ -88,9 +88,9 @@ public:
const String & user_, const String & password_,
const String & cluster_,
const String & cluster_secret_,
const String & client_name_ = "client",
Protocol::Compression compression_ = Protocol::Compression::Enable,
Protocol::Secure secure_ = Protocol::Secure::Disable,
const String & client_name_,
Protocol::Compression compression_,
Protocol::Secure secure_,
Poco::Timespan sync_request_timeout_ = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0))
:
host(host_), port(port_), default_database(default_database_),

View File

@ -56,9 +56,9 @@ public:
const String & password_,
const String & cluster_,
const String & cluster_secret_,
const String & client_name_ = "client",
Protocol::Compression compression_ = Protocol::Compression::Enable,
Protocol::Secure secure_ = Protocol::Secure::Disable,
const String & client_name_,
Protocol::Compression compression_,
Protocol::Secure secure_,
Int64 priority_ = 1)
: Base(max_connections_,
&Poco::Logger::get("ConnectionPool (" + host_ + ":" + toString(port_) + ")")),

View File

@ -13,6 +13,7 @@
#include <IO/ConnectionTimeouts.h>
namespace DB
{

View File

@ -1,40 +1,40 @@
#include <Coordination/NuKeeperLogStore.h>
#include <Coordination/KeeperLogStore.h>
namespace DB
{
NuKeeperLogStore::NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_)
: log(&Poco::Logger::get("NuKeeperLogStore"))
KeeperLogStore::KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_)
: log(&Poco::Logger::get("KeeperLogStore"))
, changelog(changelogs_path, rotate_interval_, log)
, force_sync(force_sync_)
{
}
size_t NuKeeperLogStore::start_index() const
size_t KeeperLogStore::start_index() const
{
std::lock_guard lock(changelog_lock);
return changelog.getStartIndex();
}
void NuKeeperLogStore::init(size_t last_commited_log_index, size_t logs_to_keep)
void KeeperLogStore::init(size_t last_commited_log_index, size_t logs_to_keep)
{
std::lock_guard lock(changelog_lock);
changelog.readChangelogAndInitWriter(last_commited_log_index, logs_to_keep);
}
size_t NuKeeperLogStore::next_slot() const
size_t KeeperLogStore::next_slot() const
{
std::lock_guard lock(changelog_lock);
return changelog.getNextEntryIndex();
}
nuraft::ptr<nuraft::log_entry> NuKeeperLogStore::last_entry() const
nuraft::ptr<nuraft::log_entry> KeeperLogStore::last_entry() const
{
std::lock_guard lock(changelog_lock);
return changelog.getLastEntry();
}
size_t NuKeeperLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
size_t KeeperLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
{
std::lock_guard lock(changelog_lock);
size_t idx = changelog.getNextEntryIndex();
@ -43,25 +43,25 @@ size_t NuKeeperLogStore::append(nuraft::ptr<nuraft::log_entry> & entry)
}
void NuKeeperLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry)
void KeeperLogStore::write_at(size_t index, nuraft::ptr<nuraft::log_entry> & entry)
{
std::lock_guard lock(changelog_lock);
changelog.writeAt(index, entry, force_sync);
}
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> NuKeeperLogStore::log_entries(size_t start, size_t end)
nuraft::ptr<std::vector<nuraft::ptr<nuraft::log_entry>>> KeeperLogStore::log_entries(size_t start, size_t end)
{
std::lock_guard lock(changelog_lock);
return changelog.getLogEntriesBetween(start, end);
}
nuraft::ptr<nuraft::log_entry> NuKeeperLogStore::entry_at(size_t index)
nuraft::ptr<nuraft::log_entry> KeeperLogStore::entry_at(size_t index)
{
std::lock_guard lock(changelog_lock);
return changelog.entryAt(index);
}
size_t NuKeeperLogStore::term_at(size_t index)
size_t KeeperLogStore::term_at(size_t index)
{
std::lock_guard lock(changelog_lock);
auto entry = changelog.entryAt(index);
@ -70,33 +70,33 @@ size_t NuKeeperLogStore::term_at(size_t index)
return 0;
}
nuraft::ptr<nuraft::buffer> NuKeeperLogStore::pack(size_t index, int32_t cnt)
nuraft::ptr<nuraft::buffer> KeeperLogStore::pack(size_t index, int32_t cnt)
{
std::lock_guard lock(changelog_lock);
return changelog.serializeEntriesToBuffer(index, cnt);
}
bool NuKeeperLogStore::compact(size_t last_log_index)
bool KeeperLogStore::compact(size_t last_log_index)
{
std::lock_guard lock(changelog_lock);
changelog.compact(last_log_index);
return true;
}
bool NuKeeperLogStore::flush()
bool KeeperLogStore::flush()
{
std::lock_guard lock(changelog_lock);
changelog.flush();
return true;
}
void NuKeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack)
void KeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack)
{
std::lock_guard lock(changelog_lock);
changelog.applyEntriesFromBuffer(index, pack, force_sync);
}
size_t NuKeeperLogStore::size() const
size_t KeeperLogStore::size() const
{
std::lock_guard lock(changelog_lock);
return changelog.size();

View File

@ -9,10 +9,10 @@
namespace DB
{
class NuKeeperLogStore : public nuraft::log_store
class KeeperLogStore : public nuraft::log_store
{
public:
NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_);
KeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_);
void init(size_t last_commited_log_index, size_t logs_to_keep);

View File

@ -1,7 +1,7 @@
#include <Coordination/NuKeeperServer.h>
#include <Coordination/KeeperServer.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
@ -18,7 +18,7 @@ namespace ErrorCodes
extern const int RAFT_ERROR;
}
NuKeeperServer::NuKeeperServer(
KeeperServer::KeeperServer(
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
@ -26,15 +26,18 @@ NuKeeperServer::NuKeeperServer(
SnapshotsQueue & snapshots_queue_)
: server_id(server_id_)
, coordination_settings(coordination_settings_)
, state_machine(nuraft::cs_new<NuKeeperStateMachine>(responses_queue_, snapshots_queue_, config.getString("test_keeper_server.snapshot_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/snapshots"), coordination_settings))
, state_manager(nuraft::cs_new<NuKeeperStateManager>(server_id, "test_keeper_server", config, coordination_settings))
, state_machine(nuraft::cs_new<KeeperStateMachine>(
responses_queue_, snapshots_queue_,
config.getString("keeper_server.snapshot_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/snapshots"),
coordination_settings))
, state_manager(nuraft::cs_new<KeeperStateManager>(server_id, "keeper_server", config, coordination_settings))
, responses_queue(responses_queue_)
{
if (coordination_settings->quorum_reads)
LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Quorum reads enabled, NuKeeper will work slower.");
LOG_WARNING(&Poco::Logger::get("KeeperServer"), "Quorum reads enabled, Keeper will work slower.");
}
void NuKeeperServer::startup()
void KeeperServer::startup()
{
state_machine->init();
@ -84,13 +87,13 @@ void NuKeeperServer::startup()
throw Exception(ErrorCodes::RAFT_ERROR, "Cannot allocate RAFT instance");
}
void NuKeeperServer::shutdown()
void KeeperServer::shutdown()
{
state_machine->shutdownStorage();
state_manager->flushLogStore();
auto timeout = coordination_settings->shutdown_timeout.totalSeconds();
if (!launcher.shutdown(timeout))
LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", timeout);
LOG_WARNING(&Poco::Logger::get("KeeperServer"), "Failed to shutdown RAFT server in {} seconds", timeout);
}
namespace
@ -106,7 +109,7 @@ nuraft::ptr<nuraft::buffer> getZooKeeperLogEntry(int64_t session_id, const Coord
}
void NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & request_for_session)
void KeeperServer::putRequest(const KeeperStorage::RequestForSession & request_for_session)
{
auto [session_id, request] = request_for_session;
if (!coordination_settings->quorum_reads && isLeaderAlive() && request->isReadRequest())
@ -123,29 +126,29 @@ void NuKeeperServer::putRequest(const NuKeeperStorage::RequestForSession & reque
auto result = raft_instance->append_entries(entries);
if (!result->get_accepted())
{
NuKeeperStorage::ResponsesForSessions responses;
KeeperStorage::ResponsesForSessions responses;
auto response = request->makeResponse();
response->xid = request->xid;
response->zxid = 0;
response->error = Coordination::Error::ZOPERATIONTIMEOUT;
responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response});
responses_queue.push(DB::KeeperStorage::ResponseForSession{session_id, response});
}
if (result->get_result_code() == nuraft::cmd_result_code::TIMEOUT)
{
NuKeeperStorage::ResponsesForSessions responses;
KeeperStorage::ResponsesForSessions responses;
auto response = request->makeResponse();
response->xid = request->xid;
response->zxid = 0;
response->error = Coordination::Error::ZOPERATIONTIMEOUT;
responses_queue.push(DB::NuKeeperStorage::ResponseForSession{session_id, response});
responses_queue.push(DB::KeeperStorage::ResponseForSession{session_id, response});
}
else if (result->get_result_code() != nuraft::cmd_result_code::OK)
throw Exception(ErrorCodes::RAFT_ERROR, "Requests result failed with code {} and message: '{}'", result->get_result_code(), result->get_result_str());
}
}
int64_t NuKeeperServer::getSessionID(int64_t session_timeout_ms)
int64_t KeeperServer::getSessionID(int64_t session_timeout_ms)
{
auto entry = nuraft::buffer::alloc(sizeof(int64_t));
/// Just special session request
@ -170,17 +173,17 @@ int64_t NuKeeperServer::getSessionID(int64_t session_timeout_ms)
return bs_resp.get_i64();
}
bool NuKeeperServer::isLeader() const
bool KeeperServer::isLeader() const
{
return raft_instance->is_leader();
}
bool NuKeeperServer::isLeaderAlive() const
bool KeeperServer::isLeaderAlive() const
{
return raft_instance->is_leader_alive();
}
nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */)
{
size_t last_commited = state_machine->last_commit_index();
size_t next_index = state_manager->getLogStore()->next_slot();
@ -240,7 +243,7 @@ nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type t
}
}
void NuKeeperServer::waitInit()
void KeeperServer::waitInit()
{
std::unique_lock lock(initialized_mutex);
int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds();
@ -248,7 +251,7 @@ void NuKeeperServer::waitInit()
throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization");
}
std::unordered_set<int64_t> NuKeeperServer::getDeadSessions()
std::unordered_set<int64_t> KeeperServer::getDeadSessions()
{
return state_machine->getDeadSessions();
}

View File

@ -2,25 +2,25 @@
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/KeeperStorage.h>
#include <Coordination/CoordinationSettings.h>
#include <unordered_map>
namespace DB
{
class NuKeeperServer
class KeeperServer
{
private:
int server_id;
CoordinationSettingsPtr coordination_settings;
nuraft::ptr<NuKeeperStateMachine> state_machine;
nuraft::ptr<KeeperStateMachine> state_machine;
nuraft::ptr<NuKeeperStateManager> state_manager;
nuraft::ptr<KeeperStateManager> state_manager;
nuraft::raft_launcher launcher;
@ -38,7 +38,7 @@ private:
nuraft::cb_func::ReturnCode callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * param);
public:
NuKeeperServer(
KeeperServer(
int server_id_,
const CoordinationSettingsPtr & coordination_settings_,
const Poco::Util::AbstractConfiguration & config,
@ -47,7 +47,7 @@ public:
void startup();
void putRequest(const NuKeeperStorage::RequestForSession & request);
void putRequest(const KeeperStorage::RequestForSession & request);
int64_t getSessionID(int64_t session_timeout_ms);

View File

@ -1,4 +1,4 @@
#include <Coordination/NuKeeperSnapshotManager.h>
#include <Coordination/KeeperSnapshotManager.h>
#include <IO/WriteHelpers.h>
#include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h>
@ -51,7 +51,7 @@ namespace
return "/";
}
void writeNode(const NuKeeperStorage::Node & node, WriteBuffer & out)
void writeNode(const KeeperStorage::Node & node, WriteBuffer & out)
{
writeBinary(node.data, out);
@ -81,7 +81,7 @@ namespace
writeBinary(node.seq_num, out);
}
void readNode(NuKeeperStorage::Node & node, ReadBuffer & in)
void readNode(KeeperStorage::Node & node, ReadBuffer & in)
{
readBinary(node.data, in);
@ -132,7 +132,7 @@ namespace
}
void NuKeeperStorageSnapshot::serialize(const NuKeeperStorageSnapshot & snapshot, WriteBuffer & out)
void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out)
{
writeBinary(static_cast<uint8_t>(snapshot.version), out);
serializeSnapshotMetadata(snapshot.snapshot_meta, out);
@ -159,7 +159,7 @@ void NuKeeperStorageSnapshot::serialize(const NuKeeperStorageSnapshot & snapshot
}
}
SnapshotMetadataPtr NuKeeperStorageSnapshot::deserialize(NuKeeperStorage & storage, ReadBuffer & in)
SnapshotMetadataPtr KeeperStorageSnapshot::deserialize(KeeperStorage & storage, ReadBuffer & in)
{
uint8_t version;
readBinary(version, in);
@ -180,7 +180,7 @@ SnapshotMetadataPtr NuKeeperStorageSnapshot::deserialize(NuKeeperStorage & stora
{
std::string path;
readBinary(path, in);
NuKeeperStorage::Node node;
KeeperStorage::Node node;
readNode(node, in);
storage.container.insertOrReplace(path, node);
if (node.stat.ephemeralOwner != 0)
@ -194,7 +194,7 @@ SnapshotMetadataPtr NuKeeperStorageSnapshot::deserialize(NuKeeperStorage & stora
if (itr.key != "/")
{
auto parent_path = parentPath(itr.key);
storage.container.updateValue(parent_path, [&path = itr.key] (NuKeeperStorage::Node & value) { value.children.insert(getBaseName(path)); });
storage.container.updateValue(parent_path, [&path = itr.key] (KeeperStorage::Node & value) { value.children.insert(getBaseName(path)); });
}
}
@ -214,7 +214,7 @@ SnapshotMetadataPtr NuKeeperStorageSnapshot::deserialize(NuKeeperStorage & stora
return result;
}
NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, size_t up_to_log_idx_)
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_)
: storage(storage_)
, snapshot_meta(std::make_shared<SnapshotMetadata>(up_to_log_idx_, 0, std::make_shared<nuraft::cluster_config>()))
, session_id(storage->session_id_counter)
@ -225,7 +225,7 @@ NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, siz
session_and_timeout = storage->getActiveSessions();
}
NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_)
KeeperStorageSnapshot::KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_)
: storage(storage_)
, snapshot_meta(snapshot_meta_)
, session_id(storage->session_id_counter)
@ -236,12 +236,12 @@ NuKeeperStorageSnapshot::NuKeeperStorageSnapshot(NuKeeperStorage * storage_, con
session_and_timeout = storage->getActiveSessions();
}
NuKeeperStorageSnapshot::~NuKeeperStorageSnapshot()
KeeperStorageSnapshot::~KeeperStorageSnapshot()
{
storage->disableSnapshotMode();
}
NuKeeperSnapshotManager::NuKeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, size_t storage_tick_time_)
KeeperSnapshotManager::KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, size_t storage_tick_time_)
: snapshots_path(snapshots_path_)
, snapshots_to_keep(snapshots_to_keep_)
, storage_tick_time(storage_tick_time_)
@ -266,7 +266,7 @@ NuKeeperSnapshotManager::NuKeeperSnapshotManager(const std::string & snapshots_p
}
std::string NuKeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx)
std::string KeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx)
{
ReadBufferFromNuraftBuffer reader(buffer);
@ -287,7 +287,7 @@ std::string NuKeeperSnapshotManager::serializeSnapshotBufferToDisk(nuraft::buffe
return new_snapshot_path;
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeLatestSnapshotBufferFromDisk()
{
while (!existing_snapshots.empty())
{
@ -307,7 +307,7 @@ nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeLatestSnapshotBu
return nullptr;
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::deserializeSnapshotBufferFromDisk(size_t up_to_log_idx) const
{
const std::string & snapshot_path = existing_snapshots.at(up_to_log_idx);
WriteBufferFromNuraftBuffer writer;
@ -316,26 +316,26 @@ nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::deserializeSnapshotBufferFr
return writer.getBuffer();
}
nuraft::ptr<nuraft::buffer> NuKeeperSnapshotManager::serializeSnapshotToBuffer(const NuKeeperStorageSnapshot & snapshot)
nuraft::ptr<nuraft::buffer> KeeperSnapshotManager::serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot)
{
WriteBufferFromNuraftBuffer writer;
CompressedWriteBuffer compressed_writer(writer);
NuKeeperStorageSnapshot::serialize(snapshot, compressed_writer);
KeeperStorageSnapshot::serialize(snapshot, compressed_writer);
compressed_writer.finalize();
return writer.getBuffer();
}
SnapshotMetaAndStorage NuKeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
SnapshotMetaAndStorage KeeperSnapshotManager::deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const
{
ReadBufferFromNuraftBuffer reader(buffer);
CompressedReadBuffer compressed_reader(reader);
auto storage = std::make_unique<NuKeeperStorage>(storage_tick_time);
auto snapshot_metadata = NuKeeperStorageSnapshot::deserialize(*storage, compressed_reader);
auto storage = std::make_unique<KeeperStorage>(storage_tick_time);
auto snapshot_metadata = KeeperStorageSnapshot::deserialize(*storage, compressed_reader);
return std::make_pair(snapshot_metadata, std::move(storage));
}
SnapshotMetaAndStorage NuKeeperSnapshotManager::restoreFromLatestSnapshot()
SnapshotMetaAndStorage KeeperSnapshotManager::restoreFromLatestSnapshot()
{
if (existing_snapshots.empty())
return {};
@ -346,13 +346,13 @@ SnapshotMetaAndStorage NuKeeperSnapshotManager::restoreFromLatestSnapshot()
return deserializeSnapshotFromBuffer(buffer);
}
void NuKeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
void KeeperSnapshotManager::removeOutdatedSnapshotsIfNeeded()
{
while (existing_snapshots.size() > snapshots_to_keep)
removeSnapshot(existing_snapshots.begin()->first);
}
void NuKeeperSnapshotManager::removeSnapshot(size_t log_idx)
void KeeperSnapshotManager::removeSnapshot(size_t log_idx)
{
auto itr = existing_snapshots.find(log_idx);
if (itr == existing_snapshots.end())

View File

@ -1,6 +1,6 @@
#pragma once
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/KeeperStorage.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
@ -15,42 +15,42 @@ enum SnapshotVersion : uint8_t
V0 = 0,
};
struct NuKeeperStorageSnapshot
struct KeeperStorageSnapshot
{
public:
NuKeeperStorageSnapshot(NuKeeperStorage * storage_, size_t up_to_log_idx_);
KeeperStorageSnapshot(KeeperStorage * storage_, size_t up_to_log_idx_);
NuKeeperStorageSnapshot(NuKeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_);
~NuKeeperStorageSnapshot();
KeeperStorageSnapshot(KeeperStorage * storage_, const SnapshotMetadataPtr & snapshot_meta_);
~KeeperStorageSnapshot();
static void serialize(const NuKeeperStorageSnapshot & snapshot, WriteBuffer & out);
static void serialize(const KeeperStorageSnapshot & snapshot, WriteBuffer & out);
static SnapshotMetadataPtr deserialize(NuKeeperStorage & storage, ReadBuffer & in);
static SnapshotMetadataPtr deserialize(KeeperStorage & storage, ReadBuffer & in);
NuKeeperStorage * storage;
KeeperStorage * storage;
SnapshotVersion version = SnapshotVersion::V0;
SnapshotMetadataPtr snapshot_meta;
int64_t session_id;
size_t snapshot_container_size;
NuKeeperStorage::Container::const_iterator begin;
KeeperStorage::Container::const_iterator begin;
SessionAndTimeout session_and_timeout;
};
using NuKeeperStorageSnapshotPtr = std::shared_ptr<NuKeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<void(NuKeeperStorageSnapshotPtr &&)>;
using KeeperStorageSnapshotPtr = std::shared_ptr<KeeperStorageSnapshot>;
using CreateSnapshotCallback = std::function<void(KeeperStorageSnapshotPtr &&)>;
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, NuKeeperStoragePtr>;
using SnapshotMetaAndStorage = std::pair<SnapshotMetadataPtr, KeeperStoragePtr>;
class NuKeeperSnapshotManager
class KeeperSnapshotManager
{
public:
NuKeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, size_t storage_tick_time_ = 500);
KeeperSnapshotManager(const std::string & snapshots_path_, size_t snapshots_to_keep_, size_t storage_tick_time_ = 500);
SnapshotMetaAndStorage restoreFromLatestSnapshot();
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const NuKeeperStorageSnapshot & snapshot);
static nuraft::ptr<nuraft::buffer> serializeSnapshotToBuffer(const KeeperStorageSnapshot & snapshot);
std::string serializeSnapshotBufferToDisk(nuraft::buffer & buffer, size_t up_to_log_idx);
SnapshotMetaAndStorage deserializeSnapshotFromBuffer(nuraft::ptr<nuraft::buffer> buffer) const;
@ -82,7 +82,7 @@ private:
struct CreateSnapshotTask
{
NuKeeperStorageSnapshotPtr snapshot;
KeeperStorageSnapshotPtr snapshot;
CreateSnapshotCallback create_snapshot;
};

View File

@ -1,9 +1,9 @@
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <IO/ReadHelpers.h>
#include <Common/ZooKeeper/ZooKeeperIO.h>
#include <Coordination/NuKeeperSnapshotManager.h>
#include <Coordination/KeeperSnapshotManager.h>
#include <future>
namespace DB
@ -14,10 +14,10 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
KeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
{
ReadBufferFromNuraftBuffer buffer(data);
NuKeeperStorage::RequestForSession request_for_session;
KeeperStorage::RequestForSession request_for_session;
readIntBinary(request_for_session.session_id, buffer);
int32_t length;
@ -36,17 +36,17 @@ NuKeeperStorage::RequestForSession parseRequest(nuraft::buffer & data)
return request_for_session;
}
NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_)
KeeperStateMachine::KeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_)
: coordination_settings(coordination_settings_)
, snapshot_manager(snapshots_path_, coordination_settings->snapshots_to_keep, coordination_settings->dead_session_check_period_ms.totalMicroseconds())
, responses_queue(responses_queue_)
, snapshots_queue(snapshots_queue_)
, last_committed_idx(0)
, log(&Poco::Logger::get("NuKeeperStateMachine"))
, log(&Poco::Logger::get("KeeperStateMachine"))
{
}
void NuKeeperStateMachine::init()
void KeeperStateMachine::init()
{
/// Do everything without mutexes, no other threads exist.
LOG_DEBUG(log, "Totally have {} snapshots", snapshot_manager.totalSnapshots());
@ -85,10 +85,10 @@ void NuKeeperStateMachine::init()
}
if (!storage)
storage = std::make_unique<NuKeeperStorage>(coordination_settings->dead_session_check_period_ms.totalMilliseconds());
storage = std::make_unique<KeeperStorage>(coordination_settings->dead_session_check_period_ms.totalMilliseconds());
}
nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const size_t log_idx, nuraft::buffer & data)
{
if (data.size() == sizeof(int64_t))
{
@ -109,7 +109,7 @@ nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, n
else
{
auto request_for_session = parseRequest(data);
NuKeeperStorage::ResponsesForSessions responses_for_sessions;
KeeperStorage::ResponsesForSessions responses_for_sessions;
{
std::lock_guard lock(storage_lock);
responses_for_sessions = storage->processRequest(request_for_session.request, request_for_session.session_id, log_idx);
@ -122,7 +122,7 @@ nuraft::ptr<nuraft::buffer> NuKeeperStateMachine::commit(const size_t log_idx, n
}
}
bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
bool KeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
{
LOG_DEBUG(log, "Applying snapshot {}", s.get_last_log_idx());
nuraft::ptr<nuraft::buffer> latest_snapshot_ptr;
@ -142,14 +142,14 @@ bool NuKeeperStateMachine::apply_snapshot(nuraft::snapshot & s)
return true;
}
nuraft::ptr<nuraft::snapshot> NuKeeperStateMachine::last_snapshot()
nuraft::ptr<nuraft::snapshot> KeeperStateMachine::last_snapshot()
{
/// Just return the latest snapshot.
std::lock_guard<std::mutex> lock(snapshots_lock);
return latest_snapshot_meta;
}
void NuKeeperStateMachine::create_snapshot(
void KeeperStateMachine::create_snapshot(
nuraft::snapshot & s,
nuraft::async_result<bool>::handler_type & when_done)
{
@ -160,10 +160,10 @@ void NuKeeperStateMachine::create_snapshot(
CreateSnapshotTask snapshot_task;
{
std::lock_guard lock(storage_lock);
snapshot_task.snapshot = std::make_shared<NuKeeperStorageSnapshot>(storage.get(), snapshot_meta_copy);
snapshot_task.snapshot = std::make_shared<KeeperStorageSnapshot>(storage.get(), snapshot_meta_copy);
}
snapshot_task.create_snapshot = [this, when_done] (NuKeeperStorageSnapshotPtr && snapshot)
snapshot_task.create_snapshot = [this, when_done] (KeeperStorageSnapshotPtr && snapshot)
{
nuraft::ptr<std::exception> exception(nullptr);
bool ret = true;
@ -203,7 +203,7 @@ void NuKeeperStateMachine::create_snapshot(
snapshots_queue.push(std::move(snapshot_task));
}
void NuKeeperStateMachine::save_logical_snp_obj(
void KeeperStateMachine::save_logical_snp_obj(
nuraft::snapshot & s,
size_t & obj_id,
nuraft::buffer & data,
@ -217,7 +217,7 @@ void NuKeeperStateMachine::save_logical_snp_obj(
if (obj_id == 0)
{
std::lock_guard lock(storage_lock);
NuKeeperStorageSnapshot snapshot(storage.get(), s.get_last_log_idx());
KeeperStorageSnapshot snapshot(storage.get(), s.get_last_log_idx());
cloned_buffer = snapshot_manager.serializeSnapshotToBuffer(snapshot);
}
else
@ -235,7 +235,7 @@ void NuKeeperStateMachine::save_logical_snp_obj(
std::shared_ptr<std::promise<void>> waiter = std::make_shared<std::promise<void>>();
auto future = waiter->get_future();
snapshot_task.snapshot = nullptr;
snapshot_task.create_snapshot = [this, waiter, cloned_buffer, log_idx = s.get_last_log_idx()] (NuKeeperStorageSnapshotPtr &&)
snapshot_task.create_snapshot = [this, waiter, cloned_buffer, log_idx = s.get_last_log_idx()] (KeeperStorageSnapshotPtr &&)
{
try
{
@ -261,7 +261,7 @@ void NuKeeperStateMachine::save_logical_snp_obj(
obj_id++;
}
int NuKeeperStateMachine::read_logical_snp_obj(
int KeeperStateMachine::read_logical_snp_obj(
nuraft::snapshot & s,
void* & /*user_snp_ctx*/,
ulong obj_id,
@ -289,9 +289,9 @@ int NuKeeperStateMachine::read_logical_snp_obj(
return 0;
}
void NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session)
void KeeperStateMachine::processReadRequest(const KeeperStorage::RequestForSession & request_for_session)
{
NuKeeperStorage::ResponsesForSessions responses;
KeeperStorage::ResponsesForSessions responses;
{
std::lock_guard lock(storage_lock);
responses = storage->processRequest(request_for_session.request, request_for_session.session_id, std::nullopt);
@ -300,13 +300,13 @@ void NuKeeperStateMachine::processReadRequest(const NuKeeperStorage::RequestForS
responses_queue.push(response);
}
std::unordered_set<int64_t> NuKeeperStateMachine::getDeadSessions()
std::unordered_set<int64_t> KeeperStateMachine::getDeadSessions()
{
std::lock_guard lock(storage_lock);
return storage->getDeadSessions();
}
void NuKeeperStateMachine::shutdownStorage()
void KeeperStateMachine::shutdownStorage()
{
std::lock_guard lock(storage_lock);
storage->finalize();

View File

@ -1,22 +1,22 @@
#pragma once
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/KeeperStorage.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <common/logger_useful.h>
#include <Coordination/ThreadSafeQueue.h>
#include <Coordination/CoordinationSettings.h>
#include <Coordination/NuKeeperSnapshotManager.h>
#include <Coordination/KeeperSnapshotManager.h>
namespace DB
{
using ResponsesQueue = ThreadSafeQueue<NuKeeperStorage::ResponseForSession>;
using ResponsesQueue = ThreadSafeQueue<KeeperStorage::ResponseForSession>;
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
class NuKeeperStateMachine : public nuraft::state_machine
class KeeperStateMachine : public nuraft::state_machine
{
public:
NuKeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_);
KeeperStateMachine(ResponsesQueue & responses_queue_, SnapshotsQueue & snapshots_queue_, const std::string & snapshots_path_, const CoordinationSettingsPtr & coordination_settings_);
void init();
@ -50,12 +50,12 @@ public:
nuraft::ptr<nuraft::buffer> & data_out,
bool & is_last_obj) override;
NuKeeperStorage & getStorage()
KeeperStorage & getStorage()
{
return *storage;
}
void processReadRequest(const NuKeeperStorage::RequestForSession & request_for_session);
void processReadRequest(const KeeperStorage::RequestForSession & request_for_session);
std::unordered_set<int64_t> getDeadSessions();
@ -68,9 +68,9 @@ private:
CoordinationSettingsPtr coordination_settings;
NuKeeperStoragePtr storage;
KeeperStoragePtr storage;
NuKeeperSnapshotManager snapshot_manager;
KeeperSnapshotManager snapshot_manager;
ResponsesQueue & responses_queue;

View File

@ -1,4 +1,4 @@
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/KeeperStateManager.h>
#include <Common/Exception.h>
namespace DB
@ -9,23 +9,23 @@ namespace ErrorCodes
extern const int RAFT_ERROR;
}
NuKeeperStateManager::NuKeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
KeeperStateManager::KeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path)
: my_server_id(server_id_)
, my_port(port)
, log_store(nuraft::cs_new<NuKeeperLogStore>(logs_path, 5000, false))
, log_store(nuraft::cs_new<KeeperLogStore>(logs_path, 5000, false))
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
{
auto peer_config = nuraft::cs_new<nuraft::srv_config>(my_server_id, host + ":" + std::to_string(port));
cluster_config->get_servers().push_back(peer_config);
}
NuKeeperStateManager::NuKeeperStateManager(
KeeperStateManager::KeeperStateManager(
int my_server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings)
: my_server_id(my_server_id_)
, log_store(nuraft::cs_new<NuKeeperLogStore>(
, log_store(nuraft::cs_new<KeeperLogStore>(
config.getString(config_prefix + ".log_storage_path", config.getString("path", DBMS_DEFAULT_PATH) + "coordination/logs"),
coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync))
, cluster_config(nuraft::cs_new<nuraft::cluster_config>())
@ -64,17 +64,17 @@ NuKeeperStateManager::NuKeeperStateManager(
throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without <start_as_follower>)");
}
void NuKeeperStateManager::loadLogStore(size_t last_commited_index, size_t logs_to_keep)
void KeeperStateManager::loadLogStore(size_t last_commited_index, size_t logs_to_keep)
{
log_store->init(last_commited_index, logs_to_keep);
}
void NuKeeperStateManager::flushLogStore()
void KeeperStateManager::flushLogStore()
{
log_store->flush();
}
void NuKeeperStateManager::save_config(const nuraft::cluster_config & config)
void KeeperStateManager::save_config(const nuraft::cluster_config & config)
{
// Just keep in memory in this example.
// Need to write to disk here, if want to make it durable.
@ -82,7 +82,7 @@ void NuKeeperStateManager::save_config(const nuraft::cluster_config & config)
cluster_config = nuraft::cluster_config::deserialize(*buf);
}
void NuKeeperStateManager::save_state(const nuraft::srv_state & state)
void KeeperStateManager::save_state(const nuraft::srv_state & state)
{
// Just keep in memory in this example.
// Need to write to disk here, if want to make it durable.

View File

@ -2,7 +2,7 @@
#include <Core/Types.h>
#include <string>
#include <Coordination/NuKeeperLogStore.h>
#include <Coordination/KeeperLogStore.h>
#include <Coordination/CoordinationSettings.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <Poco/Util/AbstractConfiguration.h>
@ -10,16 +10,16 @@
namespace DB
{
class NuKeeperStateManager : public nuraft::state_mgr
class KeeperStateManager : public nuraft::state_mgr
{
public:
NuKeeperStateManager(
KeeperStateManager(
int server_id_,
const std::string & config_prefix,
const Poco::Util::AbstractConfiguration & config,
const CoordinationSettingsPtr & coordination_settings);
NuKeeperStateManager(
KeeperStateManager(
int server_id_,
const std::string & host,
int port,
@ -52,7 +52,7 @@ public:
return start_as_follower_servers.count(my_server_id);
}
nuraft::ptr<NuKeeperLogStore> getLogStore() const { return log_store; }
nuraft::ptr<KeeperLogStore> getLogStore() const { return log_store; }
size_t getTotalServers() const { return total_servers; }
@ -61,7 +61,7 @@ private:
int my_port;
size_t total_servers{0};
std::unordered_set<int> start_as_follower_servers;
nuraft::ptr<NuKeeperLogStore> log_store;
nuraft::ptr<KeeperLogStore> log_store;
nuraft::ptr<nuraft::srv_config> my_server_config;
nuraft::ptr<nuraft::cluster_config> cluster_config;
nuraft::ptr<nuraft::srv_state> server_state;

View File

@ -1,4 +1,4 @@
#include <Coordination/NuKeeperStorage.h>
#include <Coordination/KeeperStorage.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/setThreadName.h>
#include <mutex>
@ -31,9 +31,9 @@ static std::string getBaseName(const String & path)
return std::string{&path[basename_start + 1], path.length() - basename_start - 1};
}
static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type)
static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type)
{
NuKeeperStorage::ResponsesForSessions result;
KeeperStorage::ResponsesForSessions result;
auto it = watches.find(path);
if (it != watches.end())
{
@ -44,7 +44,7 @@ static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & p
watch_response->type = event_type;
watch_response->state = Coordination::State::CONNECTED;
for (auto watcher_session : it->second)
result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_response});
result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_response});
watches.erase(it);
}
@ -60,14 +60,14 @@ static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & p
watch_list_response->type = Coordination::Event::CHILD;
watch_list_response->state = Coordination::State::CONNECTED;
for (auto watcher_session : it->second)
result.push_back(NuKeeperStorage::ResponseForSession{watcher_session, watch_list_response});
result.push_back(KeeperStorage::ResponseForSession{watcher_session, watch_list_response});
list_watches.erase(it);
}
return result;
}
NuKeeperStorage::NuKeeperStorage(int64_t tick_time_ms)
KeeperStorage::KeeperStorage(int64_t tick_time_ms)
: session_expiry_queue(tick_time_ms)
{
container.insert("/", Node());
@ -75,32 +75,32 @@ NuKeeperStorage::NuKeeperStorage(int64_t tick_time_ms)
using Undo = std::function<void()>;
struct NuKeeperStorageRequest
struct KeeperStorageRequest
{
Coordination::ZooKeeperRequestPtr zk_request;
explicit NuKeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
explicit KeeperStorageRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: zk_request(zk_request_)
{}
virtual std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0;
virtual NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & /*watches*/, NuKeeperStorage::Watches & /*list_watches*/) const { return {}; }
virtual std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const = 0;
virtual KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & /*watches*/, KeeperStorage::Watches & /*list_watches*/) const { return {}; }
virtual ~NuKeeperStorageRequest() = default;
virtual ~KeeperStorageRequest() = default;
};
struct NuKeeperStorageHeartbeatRequest final : public NuKeeperStorageRequest
struct KeeperStorageHeartbeatRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & /* container */, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
return {zk_request->makeResponse(), {}};
}
};
struct NuKeeperStorageSyncRequest final : public NuKeeperStorageRequest
struct KeeperStorageSyncRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & /* container */, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & /* container */, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
auto response = zk_request->makeResponse();
dynamic_cast<Coordination::ZooKeeperSyncResponse *>(response.get())->path = dynamic_cast<Coordination::ZooKeeperSyncRequest *>(zk_request.get())->path;
@ -108,16 +108,16 @@ struct NuKeeperStorageSyncRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
struct KeeperStorageCreateRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
using KeeperStorageRequest::KeeperStorageRequest;
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CREATED);
}
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Undo undo;
@ -143,7 +143,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
}
else
{
NuKeeperStorage::Node created_node;
KeeperStorage::Node created_node;
created_node.stat.czxid = zxid;
created_node.stat.mzxid = zxid;
created_node.stat.ctime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1);
@ -167,7 +167,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
}
auto child_path = getBaseName(path_created);
container.updateValue(parent_path, [child_path] (NuKeeperStorage::Node & parent)
container.updateValue(parent_path, [child_path] (KeeperStorage::Node & parent)
{
/// Increment sequential number even if node is not sequential
++parent.seq_num;
@ -188,7 +188,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
if (is_ephemeral)
ephemerals[session_id].erase(path_created);
container.updateValue(parent_path, [child_path] (NuKeeperStorage::Node & undo_parent)
container.updateValue(parent_path, [child_path] (KeeperStorage::Node & undo_parent)
{
--undo_parent.stat.cversion;
--undo_parent.stat.numChildren;
@ -205,10 +205,10 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageGetRequest final : public NuKeeperStorageRequest
struct KeeperStorageGetRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /* zxid */, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperGetResponse & response = dynamic_cast<Coordination::ZooKeeperGetResponse &>(*response_ptr);
@ -230,10 +230,10 @@ struct NuKeeperStorageGetRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
struct KeeperStorageRemoveRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t /*session_id*/) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & ephemerals, int64_t /*zxid*/, int64_t /*session_id*/) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperRemoveResponse & response = dynamic_cast<Coordination::ZooKeeperRemoveResponse &>(*response_ptr);
@ -265,7 +265,7 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
}
auto child_basename = getBaseName(it->key);
container.updateValue(parentPath(request.path), [&child_basename] (NuKeeperStorage::Node & parent)
container.updateValue(parentPath(request.path), [&child_basename] (KeeperStorage::Node & parent)
{
--parent.stat.numChildren;
++parent.stat.cversion;
@ -282,7 +282,7 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
ephemerals[prev_node.stat.ephemeralOwner].emplace(path);
container.insert(path, prev_node);
container.updateValue(parentPath(path), [&child_basename] (NuKeeperStorage::Node & parent)
container.updateValue(parentPath(path), [&child_basename] (KeeperStorage::Node & parent)
{
++parent.stat.numChildren;
--parent.stat.cversion;
@ -294,16 +294,16 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest
return { response_ptr, undo };
}
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::DELETED);
}
};
struct NuKeeperStorageExistsRequest final : public NuKeeperStorageRequest
struct KeeperStorageExistsRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperExistsResponse & response = dynamic_cast<Coordination::ZooKeeperExistsResponse &>(*response_ptr);
@ -324,10 +324,10 @@ struct NuKeeperStorageExistsRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
struct KeeperStorageSetRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & /* ephemerals */, int64_t zxid, int64_t /* session_id */) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperSetResponse & response = dynamic_cast<Coordination::ZooKeeperSetResponse &>(*response_ptr);
@ -343,7 +343,7 @@ struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
{
auto prev_node = it->value;
auto itr = container.updateValue(request.path, [zxid, request] (NuKeeperStorage::Node & value)
auto itr = container.updateValue(request.path, [zxid, request] (KeeperStorage::Node & value)
{
value.data = request.data;
value.stat.version++;
@ -353,7 +353,7 @@ struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
value.data = request.data;
});
container.updateValue(parentPath(request.path), [] (NuKeeperStorage::Node & parent)
container.updateValue(parentPath(request.path), [] (KeeperStorage::Node & parent)
{
parent.stat.cversion++;
});
@ -363,8 +363,8 @@ struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
undo = [prev_node, &container, path = request.path]
{
container.updateValue(path, [&prev_node] (NuKeeperStorage::Node & value) { value = prev_node; });
container.updateValue(parentPath(path), [] (NuKeeperStorage::Node & parent)
container.updateValue(path, [&prev_node] (KeeperStorage::Node & value) { value = prev_node; });
container.updateValue(parentPath(path), [] (KeeperStorage::Node & parent)
{
parent.stat.cversion--;
});
@ -378,16 +378,16 @@ struct NuKeeperStorageSetRequest final : public NuKeeperStorageRequest
return { response_ptr, undo };
}
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
{
return processWatchesImpl(zk_request->getPath(), watches, list_watches, Coordination::Event::CHANGED);
}
};
struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest
struct KeeperStorageListRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperListResponse & response = dynamic_cast<Coordination::ZooKeeperListResponse &>(*response_ptr);
@ -415,10 +415,10 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageCheckRequest final : public NuKeeperStorageRequest
struct KeeperStorageCheckRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & /* ephemerals */, int64_t /*zxid*/, int64_t /*session_id*/) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperCheckResponse & response = dynamic_cast<Coordination::ZooKeeperCheckResponse &>(*response_ptr);
@ -441,11 +441,11 @@ struct NuKeeperStorageCheckRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest
struct KeeperStorageMultiRequest final : public KeeperStorageRequest
{
std::vector<NuKeeperStorageRequestPtr> concrete_requests;
explicit NuKeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: NuKeeperStorageRequest(zk_request_)
std::vector<KeeperStorageRequestPtr> concrete_requests;
explicit KeeperStorageMultiRequest(const Coordination::ZooKeeperRequestPtr & zk_request_)
: KeeperStorageRequest(zk_request_)
{
Coordination::ZooKeeperMultiRequest & request = dynamic_cast<Coordination::ZooKeeperMultiRequest &>(*zk_request);
concrete_requests.reserve(request.requests.size());
@ -455,26 +455,26 @@ struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest
auto sub_zk_request = std::dynamic_pointer_cast<Coordination::ZooKeeperRequest>(sub_request);
if (sub_zk_request->getOpNum() == Coordination::OpNum::Create)
{
concrete_requests.push_back(std::make_shared<NuKeeperStorageCreateRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<KeeperStorageCreateRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Remove)
{
concrete_requests.push_back(std::make_shared<NuKeeperStorageRemoveRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<KeeperStorageRemoveRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Set)
{
concrete_requests.push_back(std::make_shared<NuKeeperStorageSetRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<KeeperStorageSetRequest>(sub_zk_request));
}
else if (sub_zk_request->getOpNum() == Coordination::OpNum::Check)
{
concrete_requests.push_back(std::make_shared<NuKeeperStorageCheckRequest>(sub_zk_request));
concrete_requests.push_back(std::make_shared<KeeperStorageCheckRequest>(sub_zk_request));
}
else
throw DB::Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal command as part of multi ZooKeeper request {}", sub_zk_request->getOpNum());
}
}
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container & container, NuKeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container & container, KeeperStorage::Ephemerals & ephemerals, int64_t zxid, int64_t session_id) const override
{
Coordination::ZooKeeperResponsePtr response_ptr = zk_request->makeResponse();
Coordination::ZooKeeperMultiResponse & response = dynamic_cast<Coordination::ZooKeeperMultiResponse &>(*response_ptr);
@ -527,9 +527,9 @@ struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest
}
}
NuKeeperStorage::ResponsesForSessions processWatches(NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches) const override
KeeperStorage::ResponsesForSessions processWatches(KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches) const override
{
NuKeeperStorage::ResponsesForSessions result;
KeeperStorage::ResponsesForSessions result;
for (const auto & generic_request : concrete_requests)
{
auto responses = generic_request->processWatches(watches, list_watches);
@ -539,16 +539,16 @@ struct NuKeeperStorageMultiRequest final : public NuKeeperStorageRequest
}
};
struct NuKeeperStorageCloseRequest final : public NuKeeperStorageRequest
struct KeeperStorageCloseRequest final : public KeeperStorageRequest
{
using NuKeeperStorageRequest::NuKeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(NuKeeperStorage::Container &, NuKeeperStorage::Ephemerals &, int64_t, int64_t) const override
using KeeperStorageRequest::KeeperStorageRequest;
std::pair<Coordination::ZooKeeperResponsePtr, Undo> process(KeeperStorage::Container &, KeeperStorage::Ephemerals &, int64_t, int64_t) const override
{
throw DB::Exception("Called process on close request", ErrorCodes::LOGICAL_ERROR);
}
};
void NuKeeperStorage::finalize()
void KeeperStorage::finalize()
{
if (finalized)
throw DB::Exception("Testkeeper storage already finalized", ErrorCodes::LOGICAL_ERROR);
@ -568,20 +568,20 @@ void NuKeeperStorage::finalize()
}
class NuKeeperWrapperFactory final : private boost::noncopyable
class KeeperWrapperFactory final : private boost::noncopyable
{
public:
using Creator = std::function<NuKeeperStorageRequestPtr(const Coordination::ZooKeeperRequestPtr &)>;
using Creator = std::function<KeeperStorageRequestPtr(const Coordination::ZooKeeperRequestPtr &)>;
using OpNumToRequest = std::unordered_map<Coordination::OpNum, Creator>;
static NuKeeperWrapperFactory & instance()
static KeeperWrapperFactory & instance()
{
static NuKeeperWrapperFactory factory;
static KeeperWrapperFactory factory;
return factory;
}
NuKeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const
KeeperStorageRequestPtr get(const Coordination::ZooKeeperRequestPtr & zk_request) const
{
auto it = op_num_to_request.find(zk_request->getOpNum());
if (it == op_num_to_request.end())
@ -598,37 +598,37 @@ public:
private:
OpNumToRequest op_num_to_request;
NuKeeperWrapperFactory();
KeeperWrapperFactory();
};
template<Coordination::OpNum num, typename RequestT>
void registerNuKeeperRequestWrapper(NuKeeperWrapperFactory & factory)
void registerKeeperRequestWrapper(KeeperWrapperFactory & factory)
{
factory.registerRequest(num, [] (const Coordination::ZooKeeperRequestPtr & zk_request) { return std::make_shared<RequestT>(zk_request); });
}
NuKeeperWrapperFactory::NuKeeperWrapperFactory()
KeeperWrapperFactory::KeeperWrapperFactory()
{
registerNuKeeperRequestWrapper<Coordination::OpNum::Heartbeat, NuKeeperStorageHeartbeatRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Sync, NuKeeperStorageSyncRequest>(*this);
//registerNuKeeperRequestWrapper<Coordination::OpNum::Auth, NuKeeperStorageAuthRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Close, NuKeeperStorageCloseRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Create, NuKeeperStorageCreateRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Remove, NuKeeperStorageRemoveRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Exists, NuKeeperStorageExistsRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Get, NuKeeperStorageGetRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Set, NuKeeperStorageSetRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::List, NuKeeperStorageListRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::SimpleList, NuKeeperStorageListRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Check, NuKeeperStorageCheckRequest>(*this);
registerNuKeeperRequestWrapper<Coordination::OpNum::Multi, NuKeeperStorageMultiRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Heartbeat, KeeperStorageHeartbeatRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Sync, KeeperStorageSyncRequest>(*this);
//registerKeeperRequestWrapper<Coordination::OpNum::Auth, KeeperStorageAuthRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Close, KeeperStorageCloseRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Create, KeeperStorageCreateRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Remove, KeeperStorageRemoveRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Exists, KeeperStorageExistsRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Get, KeeperStorageGetRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Set, KeeperStorageSetRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::List, KeeperStorageListRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::SimpleList, KeeperStorageListRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Check, KeeperStorageCheckRequest>(*this);
registerKeeperRequestWrapper<Coordination::OpNum::Multi, KeeperStorageMultiRequest>(*this);
}
NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, std::optional<int64_t> new_last_zxid)
KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordination::ZooKeeperRequestPtr & zk_request, int64_t session_id, std::optional<int64_t> new_last_zxid)
{
NuKeeperStorage::ResponsesForSessions results;
KeeperStorage::ResponsesForSessions results;
if (new_last_zxid)
{
if (zxid >= *new_last_zxid)
@ -645,7 +645,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
for (const auto & ephemeral_path : it->second)
{
container.erase(ephemeral_path);
container.updateValue(parentPath(ephemeral_path), [&ephemeral_path] (NuKeeperStorage::Node & parent)
container.updateValue(parentPath(ephemeral_path), [&ephemeral_path] (KeeperStorage::Node & parent)
{
--parent.stat.numChildren;
++parent.stat.cversion;
@ -669,7 +669,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
}
else if (zk_request->getOpNum() == Coordination::OpNum::Heartbeat)
{
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
KeeperStorageRequestPtr storage_request = KeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);
response->xid = zk_request->xid;
response->zxid = getZXID();
@ -678,7 +678,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
}
else
{
NuKeeperStorageRequestPtr storage_request = NuKeeperWrapperFactory::instance().get(zk_request);
KeeperStorageRequestPtr storage_request = KeeperWrapperFactory::instance().get(zk_request);
auto [response, _] = storage_request->process(container, ephemerals, zxid, session_id);
if (zk_request->has_watch)
@ -715,7 +715,7 @@ NuKeeperStorage::ResponsesForSessions NuKeeperStorage::processRequest(const Coor
}
void NuKeeperStorage::clearDeadWatches(int64_t session_id)
void KeeperStorage::clearDeadWatches(int64_t session_id)
{
auto watches_it = sessions_and_watchers.find(session_id);
if (watches_it != sessions_and_watchers.end())

View File

@ -14,15 +14,15 @@ namespace DB
{
using namespace DB;
struct NuKeeperStorageRequest;
using NuKeeperStorageRequestPtr = std::shared_ptr<NuKeeperStorageRequest>;
struct KeeperStorageRequest;
using KeeperStorageRequestPtr = std::shared_ptr<KeeperStorageRequest>;
using ResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr &)>;
using ChildrenSet = std::unordered_set<std::string>;
using SessionAndTimeout = std::unordered_map<int64_t, int64_t>;
struct NuKeeperStorageSnapshot;
struct KeeperStorageSnapshot;
class NuKeeperStorage
class KeeperStorage
{
public:
int64_t session_id_counter{1};
@ -80,7 +80,7 @@ public:
}
public:
NuKeeperStorage(int64_t tick_time_ms);
KeeperStorage(int64_t tick_time_ms);
int64_t getSessionID(int64_t session_timeout_ms)
{
@ -131,6 +131,6 @@ public:
}
};
using NuKeeperStoragePtr = std::unique_ptr<NuKeeperStorage>;
using KeeperStoragePtr = std::unique_ptr<KeeperStorage>;
}

View File

@ -1,4 +1,4 @@
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <Coordination/KeeperStorageDispatcher.h>
#include <Common/setThreadName.h>
namespace DB
@ -11,18 +11,18 @@ namespace ErrorCodes
extern const int TIMEOUT_EXCEEDED;
}
NuKeeperStorageDispatcher::NuKeeperStorageDispatcher()
KeeperStorageDispatcher::KeeperStorageDispatcher()
: coordination_settings(std::make_shared<CoordinationSettings>())
, log(&Poco::Logger::get("NuKeeperDispatcher"))
, log(&Poco::Logger::get("KeeperDispatcher"))
{
}
void NuKeeperStorageDispatcher::requestThread()
void KeeperStorageDispatcher::requestThread()
{
setThreadName("NuKeeperReqT");
setThreadName("KeeperReqT");
while (!shutdown_called)
{
NuKeeperStorage::RequestForSession request;
KeeperStorage::RequestForSession request;
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
@ -43,12 +43,12 @@ void NuKeeperStorageDispatcher::requestThread()
}
}
void NuKeeperStorageDispatcher::responseThread()
void KeeperStorageDispatcher::responseThread()
{
setThreadName("NuKeeperRspT");
setThreadName("KeeperRspT");
while (!shutdown_called)
{
NuKeeperStorage::ResponseForSession response_for_session;
KeeperStorage::ResponseForSession response_for_session;
UInt64 max_wait = UInt64(coordination_settings->operation_timeout_ms.totalMilliseconds());
@ -69,9 +69,9 @@ void NuKeeperStorageDispatcher::responseThread()
}
}
void NuKeeperStorageDispatcher::snapshotThread()
void KeeperStorageDispatcher::snapshotThread()
{
setThreadName("NuKeeperSnpT");
setThreadName("KeeperSnpT");
while (!shutdown_called)
{
CreateSnapshotTask task;
@ -91,7 +91,7 @@ void NuKeeperStorageDispatcher::snapshotThread()
}
}
void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
void KeeperStorageDispatcher::setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_writer = session_to_response_callback.find(session_id);
@ -104,7 +104,7 @@ void NuKeeperStorageDispatcher::setResponse(int64_t session_id, const Coordinati
session_to_response_callback.erase(session_writer);
}
bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id)
bool KeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id)
{
{
std::lock_guard lock(session_to_response_callback_mutex);
@ -112,7 +112,7 @@ bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestP
return false;
}
NuKeeperStorage::RequestForSession request_info;
KeeperStorage::RequestForSession request_info;
request_info.request = request;
request_info.session_id = session_id;
@ -125,18 +125,18 @@ bool NuKeeperStorageDispatcher::putRequest(const Coordination::ZooKeeperRequestP
return true;
}
void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config)
void KeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfiguration & config)
{
LOG_DEBUG(log, "Initializing storage dispatcher");
int myid = config.getInt("test_keeper_server.server_id");
int myid = config.getInt("keeper_server.server_id");
coordination_settings->loadFromConfig("test_keeper_server.coordination_settings", config);
coordination_settings->loadFromConfig("keeper_server.coordination_settings", config);
request_thread = ThreadFromGlobalPool([this] { requestThread(); });
responses_thread = ThreadFromGlobalPool([this] { responseThread(); });
snapshot_thread = ThreadFromGlobalPool([this] { snapshotThread(); });
server = std::make_unique<NuKeeperServer>(myid, coordination_settings, config, responses_queue, snapshots_queue);
server = std::make_unique<KeeperServer>(myid, coordination_settings, config, responses_queue, snapshots_queue);
try
{
LOG_DEBUG(log, "Waiting server to initialize");
@ -158,7 +158,7 @@ void NuKeeperStorageDispatcher::initialize(const Poco::Util::AbstractConfigurati
LOG_DEBUG(log, "Dispatcher initialized");
}
void NuKeeperStorageDispatcher::shutdown()
void KeeperStorageDispatcher::shutdown()
{
try
{
@ -191,7 +191,7 @@ void NuKeeperStorageDispatcher::shutdown()
if (server)
server->shutdown();
NuKeeperStorage::RequestForSession request_for_session;
KeeperStorage::RequestForSession request_for_session;
while (requests_queue.tryPop(request_for_session))
{
if (request_for_session.request)
@ -215,19 +215,19 @@ void NuKeeperStorageDispatcher::shutdown()
LOG_DEBUG(log, "Dispatcher shut down");
}
NuKeeperStorageDispatcher::~NuKeeperStorageDispatcher()
KeeperStorageDispatcher::~KeeperStorageDispatcher()
{
shutdown();
}
void NuKeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback)
void KeeperStorageDispatcher::registerSession(int64_t session_id, ZooKeeperResponseCallback callback)
{
std::lock_guard lock(session_to_response_callback_mutex);
if (!session_to_response_callback.try_emplace(session_id, callback).second)
throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Session with id {} already registered in dispatcher", session_id);
}
void NuKeeperStorageDispatcher::sessionCleanerTask()
void KeeperStorageDispatcher::sessionCleanerTask()
{
while (true)
{
@ -244,7 +244,7 @@ void NuKeeperStorageDispatcher::sessionCleanerTask()
LOG_INFO(log, "Found dead session {}, will try to close it", dead_session);
Coordination::ZooKeeperRequestPtr request = Coordination::ZooKeeperRequestFactory::instance().get(Coordination::OpNum::Close);
request->xid = Coordination::CLOSE_XID;
NuKeeperStorage::RequestForSession request_info;
KeeperStorage::RequestForSession request_info;
request_info.request = request;
request_info.session_id = dead_session;
{
@ -265,7 +265,7 @@ void NuKeeperStorageDispatcher::sessionCleanerTask()
}
}
void NuKeeperStorageDispatcher::finishSession(int64_t session_id)
void KeeperStorageDispatcher::finishSession(int64_t session_id)
{
std::lock_guard lock(session_to_response_callback_mutex);
auto session_it = session_to_response_callback.find(session_id);

View File

@ -13,7 +13,7 @@
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <functional>
#include <Coordination/NuKeeperServer.h>
#include <Coordination/KeeperServer.h>
#include <Coordination/CoordinationSettings.h>
@ -22,14 +22,14 @@ namespace DB
using ZooKeeperResponseCallback = std::function<void(const Coordination::ZooKeeperResponsePtr & response)>;
class NuKeeperStorageDispatcher
class KeeperStorageDispatcher
{
private:
std::mutex push_request_mutex;
CoordinationSettingsPtr coordination_settings;
using RequestsQueue = ConcurrentBoundedQueue<NuKeeperStorage::RequestForSession>;
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
RequestsQueue requests_queue{1};
@ -46,7 +46,7 @@ private:
ThreadFromGlobalPool session_cleaner_thread;
ThreadFromGlobalPool snapshot_thread;
std::unique_ptr<NuKeeperServer> server;
std::unique_ptr<KeeperServer> server;
Poco::Logger * log;
@ -58,13 +58,13 @@ private:
void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response);
public:
NuKeeperStorageDispatcher();
KeeperStorageDispatcher();
void initialize(const Poco::Util::AbstractConfiguration & config);
void shutdown();
~NuKeeperStorageDispatcher();
~KeeperStorageDispatcher();
bool putRequest(const Coordination::ZooKeeperRequestPtr & request, int64_t session_id);

View File

@ -1,24 +0,0 @@
#pragma once
#include <Common/ZooKeeper/ZooKeeperCommon.h>
namespace DB
{
struct NuKeeperRequest
{
int64_t session_id;
Coordination::ZooKeeperRequestPtr request;
};
using NuKeeperRequests = std::vector<NuKeeperRequest>;
struct NuKeeperResponse
{
int64_t session_id;
Coordination::ZooKeeperRequestPtr response;
};
using NuKeeperResponses = std::vector<NuKeeperResponse>;
}

View File

@ -9,10 +9,10 @@
#include <Poco/ConsoleChannel.h>
#include <Poco/Logger.h>
#include <Coordination/InMemoryLogStore.h>
#include <Coordination/NuKeeperStateManager.h>
#include <Coordination/NuKeeperSnapshotManager.h>
#include <Coordination/KeeperStateManager.h>
#include <Coordination/KeeperSnapshotManager.h>
#include <Coordination/SummingStateMachine.h>
#include <Coordination/NuKeeperStateMachine.h>
#include <Coordination/KeeperStateMachine.h>
#include <Coordination/LoggerWrapper.h>
#include <Coordination/WriteBufferFromNuraftBuffer.h>
#include <Coordination/ReadBufferFromNuraftBuffer.h>
@ -24,7 +24,7 @@
#include <common/logger_useful.h>
#include <libnuraft/nuraft.hxx> // Y_IGNORE
#include <thread>
#include <Coordination/NuKeeperLogStore.h>
#include <Coordination/KeeperLogStore.h>
#include <Coordination/Changelog.h>
#include <filesystem>
@ -102,7 +102,7 @@ struct SimpliestRaftServer
, port(port_)
, endpoint(hostname + ":" + std::to_string(port))
, state_machine(nuraft::cs_new<StateMachine>())
, state_manager(nuraft::cs_new<DB::NuKeeperStateManager>(server_id, hostname, port, logs_path))
, state_manager(nuraft::cs_new<DB::KeeperStateManager>(server_id, hostname, port, logs_path))
{
state_manager->loadLogStore(1, 0);
nuraft::raft_params params;
@ -153,7 +153,7 @@ struct SimpliestRaftServer
nuraft::ptr<StateMachine> state_machine;
// State manager.
nuraft::ptr<DB::NuKeeperStateManager> state_manager;
nuraft::ptr<DB::KeeperStateManager> state_manager;
// Raft launcher.
nuraft::raft_launcher launcher;
@ -207,7 +207,7 @@ DB::LogEntryPtr getLogEntry(const std::string & s, size_t term)
TEST(CoordinationTest, ChangelogTestSimple)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
auto entry = getLogEntry("hello world", 77);
changelog.append(entry);
@ -221,7 +221,7 @@ TEST(CoordinationTest, ChangelogTestSimple)
TEST(CoordinationTest, ChangelogTestFile)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
auto entry = getLogEntry("hello world", 77);
changelog.append(entry);
@ -242,7 +242,7 @@ TEST(CoordinationTest, ChangelogTestFile)
TEST(CoordinationTest, ChangelogReadWrite)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 1000, true);
DB::KeeperLogStore changelog("./logs", 1000, true);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
@ -250,7 +250,7 @@ TEST(CoordinationTest, ChangelogReadWrite)
changelog.append(entry);
}
EXPECT_EQ(changelog.size(), 10);
DB::NuKeeperLogStore changelog_reader("./logs", 1000, true);
DB::KeeperLogStore changelog_reader("./logs", 1000, true);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10);
EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term());
@ -269,7 +269,7 @@ TEST(CoordinationTest, ChangelogReadWrite)
TEST(CoordinationTest, ChangelogWriteAt)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 1000, true);
DB::KeeperLogStore changelog("./logs", 1000, true);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
@ -285,7 +285,7 @@ TEST(CoordinationTest, ChangelogWriteAt)
EXPECT_EQ(changelog.entry_at(7)->get_term(), 77);
EXPECT_EQ(changelog.next_slot(), 8);
DB::NuKeeperLogStore changelog_reader("./logs", 1000, true);
DB::KeeperLogStore changelog_reader("./logs", 1000, true);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), changelog.size());
@ -298,7 +298,7 @@ TEST(CoordinationTest, ChangelogWriteAt)
TEST(CoordinationTest, ChangelogTestAppendAfterRead)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 7; ++i)
{
@ -310,7 +310,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead)
EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin"));
EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin"));
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
DB::KeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 7);
@ -346,7 +346,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead)
TEST(CoordinationTest, ChangelogTestCompaction)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 3; ++i)
@ -387,7 +387,7 @@ TEST(CoordinationTest, ChangelogTestCompaction)
EXPECT_EQ(changelog.next_slot(), 8);
EXPECT_EQ(changelog.last_entry()->get_term(), 60);
/// And we able to read it
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
DB::KeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(7, 0);
EXPECT_EQ(changelog_reader.size(), 1);
EXPECT_EQ(changelog_reader.start_index(), 7);
@ -398,7 +398,7 @@ TEST(CoordinationTest, ChangelogTestCompaction)
TEST(CoordinationTest, ChangelogTestBatchOperations)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 100, true);
DB::KeeperLogStore changelog("./logs", 100, true);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
@ -410,7 +410,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations)
auto entries = changelog.pack(1, 5);
DB::NuKeeperLogStore apply_changelog("./logs", 100, true);
DB::KeeperLogStore apply_changelog("./logs", 100, true);
apply_changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
@ -440,7 +440,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations)
TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 100, true);
DB::KeeperLogStore changelog("./logs", 100, true);
changelog.init(1, 0);
for (size_t i = 0; i < 10; ++i)
{
@ -453,7 +453,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
auto entries = changelog.pack(5, 5);
ChangelogDirTest test1("./logs1");
DB::NuKeeperLogStore changelog_new("./logs1", 100, true);
DB::KeeperLogStore changelog_new("./logs1", 100, true);
changelog_new.init(1, 0);
EXPECT_EQ(changelog_new.size(), 0);
@ -472,7 +472,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
EXPECT_EQ(changelog_new.start_index(), 5);
EXPECT_EQ(changelog_new.next_slot(), 11);
DB::NuKeeperLogStore changelog_reader("./logs1", 100, true);
DB::KeeperLogStore changelog_reader("./logs1", 100, true);
changelog_reader.init(5, 0);
}
@ -480,7 +480,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty)
TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
@ -515,7 +515,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile)
EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin"));
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_read("./logs", 5, true);
DB::KeeperLogStore changelog_read("./logs", 5, true);
changelog_read.init(1, 0);
EXPECT_EQ(changelog_read.size(), 7);
EXPECT_EQ(changelog_read.start_index(), 1);
@ -526,7 +526,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile)
TEST(CoordinationTest, ChangelogTestWriteAtFileBorder)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
@ -561,7 +561,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder)
EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin"));
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_read("./logs", 5, true);
DB::KeeperLogStore changelog_read("./logs", 5, true);
changelog_read.init(1, 0);
EXPECT_EQ(changelog_read.size(), 11);
EXPECT_EQ(changelog_read.start_index(), 1);
@ -572,7 +572,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder)
TEST(CoordinationTest, ChangelogTestWriteAtAllFiles)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 33; ++i)
@ -611,7 +611,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtAllFiles)
TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
@ -630,7 +630,7 @@ TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead)
EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin"));
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
DB::KeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1, 0);
auto entry = getLogEntry("36_hello_world", 360);
@ -652,7 +652,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 5, true);
DB::KeeperLogStore changelog("./logs", 5, true);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
@ -672,7 +672,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
DB::WriteBufferFromFile plain_buf("./logs/changelog_11_15.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
plain_buf.truncate(0);
DB::NuKeeperLogStore changelog_reader("./logs", 5, true);
DB::KeeperLogStore changelog_reader("./logs", 5, true);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 10);
@ -701,7 +701,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate)
EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin"));
EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin"));
DB::NuKeeperLogStore changelog_reader2("./logs", 5, true);
DB::KeeperLogStore changelog_reader2("./logs", 5, true);
changelog_reader2.init(1, 0);
EXPECT_EQ(changelog_reader2.size(), 11);
EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777);
@ -711,7 +711,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 20, true);
DB::KeeperLogStore changelog("./logs", 20, true);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
@ -726,7 +726,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
DB::WriteBufferFromFile plain_buf("./logs/changelog_1_20.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY);
plain_buf.truncate(140);
DB::NuKeeperLogStore changelog_reader("./logs", 20, true);
DB::KeeperLogStore changelog_reader("./logs", 20, true);
changelog_reader.init(1, 0);
EXPECT_EQ(changelog_reader.size(), 2);
@ -739,7 +739,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2)
EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777);
DB::NuKeeperLogStore changelog_reader2("./logs", 20, true);
DB::KeeperLogStore changelog_reader2("./logs", 20, true);
changelog_reader2.init(1, 0);
EXPECT_EQ(changelog_reader2.size(), 3);
EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777);
@ -749,7 +749,7 @@ TEST(CoordinationTest, ChangelogTestLostFiles)
{
ChangelogDirTest test("./logs");
DB::NuKeeperLogStore changelog("./logs", 20, true);
DB::KeeperLogStore changelog("./logs", 20, true);
changelog.init(1, 0);
for (size_t i = 0; i < 35; ++i)
@ -763,7 +763,7 @@ TEST(CoordinationTest, ChangelogTestLostFiles)
fs::remove("./logs/changelog_1_20.bin");
DB::NuKeeperLogStore changelog_reader("./logs", 20, true);
DB::KeeperLogStore changelog_reader("./logs", 20, true);
/// It should print error message, but still able to start
changelog_reader.init(5, 0);
EXPECT_FALSE(fs::exists("./logs/changelog_1_20.bin"));
@ -862,9 +862,9 @@ TEST(CoordinationTest, SnapshotableHashMapTrySnapshot)
map_snp.disableSnapshotMode();
}
void addNode(DB::NuKeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0)
void addNode(DB::KeeperStorage & storage, const std::string & path, const std::string & data, int64_t ephemeral_owner=0)
{
using Node = DB::NuKeeperStorage::Node;
using Node = DB::KeeperStorage::Node;
Node node{};
node.data = data;
node.stat.ephemeralOwner = ephemeral_owner;
@ -874,9 +874,9 @@ void addNode(DB::NuKeeperStorage & storage, const std::string & path, const std:
TEST(CoordinationTest, TestStorageSnapshotSimple)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::KeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
DB::KeeperStorage storage(500);
addNode(storage, "/hello", "world", 1);
addNode(storage, "/hello/somepath", "somedata", 3);
storage.session_id_counter = 5;
@ -886,7 +886,7 @@ TEST(CoordinationTest, TestStorageSnapshotSimple)
storage.getSessionID(130);
storage.getSessionID(130);
DB::NuKeeperStorageSnapshot snapshot(&storage, 2);
DB::KeeperStorageSnapshot snapshot(&storage, 2);
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 2);
EXPECT_EQ(snapshot.session_id, 7);
@ -921,9 +921,9 @@ TEST(CoordinationTest, TestStorageSnapshotSimple)
TEST(CoordinationTest, TestStorageSnapshotMoreWrites)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::KeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
DB::KeeperStorage storage(500);
storage.getSessionID(130);
for (size_t i = 0; i < 50; ++i)
@ -931,7 +931,7 @@ TEST(CoordinationTest, TestStorageSnapshotMoreWrites)
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
DB::KeeperStorageSnapshot snapshot(&storage, 50);
EXPECT_EQ(snapshot.snapshot_meta->get_last_log_idx(), 50);
EXPECT_EQ(snapshot.snapshot_container_size, 51);
@ -961,9 +961,9 @@ TEST(CoordinationTest, TestStorageSnapshotMoreWrites)
TEST(CoordinationTest, TestStorageSnapshotManySnapshots)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::KeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
DB::KeeperStorage storage(500);
storage.getSessionID(130);
for (size_t j = 1; j <= 5; ++j)
@ -973,7 +973,7 @@ TEST(CoordinationTest, TestStorageSnapshotManySnapshots)
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
DB::NuKeeperStorageSnapshot snapshot(&storage, j * 50);
DB::KeeperStorageSnapshot snapshot(&storage, j * 50);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, j * 50);
EXPECT_TRUE(fs::exists(std::string{"./snapshots/snapshot_"} + std::to_string(j * 50) + ".bin"));
@ -999,15 +999,15 @@ TEST(CoordinationTest, TestStorageSnapshotManySnapshots)
TEST(CoordinationTest, TestStorageSnapshotMode)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
DB::KeeperSnapshotManager manager("./snapshots", 3);
DB::KeeperStorage storage(500);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
{
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
DB::KeeperStorageSnapshot snapshot(&storage, 50);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "wlrd_" + std::to_string(i));
@ -1050,14 +1050,14 @@ TEST(CoordinationTest, TestStorageSnapshotMode)
TEST(CoordinationTest, TestStorageSnapshotBroken)
{
ChangelogDirTest test("./snapshots");
DB::NuKeeperSnapshotManager manager("./snapshots", 3);
DB::NuKeeperStorage storage(500);
DB::KeeperSnapshotManager manager("./snapshots", 3);
DB::KeeperStorage storage(500);
for (size_t i = 0; i < 50; ++i)
{
addNode(storage, "/hello_" + std::to_string(i), "world_" + std::to_string(i));
}
{
DB::NuKeeperStorageSnapshot snapshot(&storage, 50);
DB::KeeperStorageSnapshot snapshot(&storage, 50);
auto buf = manager.serializeSnapshotToBuffer(snapshot);
manager.serializeSnapshotBufferToDisk(*buf, 50);
}
@ -1095,9 +1095,9 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, size
ResponsesQueue queue;
SnapshotsQueue snapshots_queue{1};
auto state_machine = std::make_shared<NuKeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings);
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings);
state_machine->init();
DB::NuKeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true);
DB::KeeperLogStore changelog("./logs", settings->rotate_log_storage_interval, true);
changelog.init(state_machine->last_commit_index() + 1, settings->reserved_log_items);
for (size_t i = 1; i < total_logs + 1; ++i)
{
@ -1132,11 +1132,11 @@ void testLogAndStateMachine(Coordination::CoordinationSettingsPtr settings, size
}
SnapshotsQueue snapshots_queue1{1};
auto restore_machine = std::make_shared<NuKeeperStateMachine>(queue, snapshots_queue1, "./snapshots", settings);
auto restore_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue1, "./snapshots", settings);
restore_machine->init();
EXPECT_EQ(restore_machine->last_commit_index(), total_logs - total_logs % settings->snapshot_distance);
DB::NuKeeperLogStore restore_changelog("./logs", settings->rotate_log_storage_interval, true);
DB::KeeperLogStore restore_changelog("./logs", settings->rotate_log_storage_interval, true);
restore_changelog.init(restore_machine->last_commit_index() + 1, settings->reserved_log_items);
EXPECT_EQ(restore_changelog.size(), std::min(settings->reserved_log_items + total_logs % settings->snapshot_distance, total_logs));
@ -1242,7 +1242,7 @@ TEST(CoordinationTest, TestEphemeralNodeRemove)
ResponsesQueue queue;
SnapshotsQueue snapshots_queue{1};
auto state_machine = std::make_shared<NuKeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings);
auto state_machine = std::make_shared<KeeperStateMachine>(queue, snapshots_queue, "./snapshots", settings);
state_machine->init();
std::shared_ptr<ZooKeeperCreateRequest> request_c = std::make_shared<ZooKeeperCreateRequest>();

View File

@ -116,6 +116,7 @@ class IColumn;
M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \
M(UInt64, distributed_group_by_no_merge, 0, "If 1, Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards. If 2 - same as 1 but also apply ORDER BY and LIMIT stages", 0) \
M(Bool, optimize_distributed_group_by_sharding_key, false, "Optimize GROUP BY sharding_key queries (by avoiding costly aggregation on the initiator server).", 0) \
M(UInt64, optimize_skip_unused_shards_limit, 1000, "Limit for number of sharding key values, turns off optimize_skip_unused_shards if the limit is reached", 0) \
M(Bool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
M(Bool, allow_nondeterministic_optimize_skip_unused_shards, false, "Allow non-deterministic functions (includes dictGet) in sharding_key for optimize_skip_unused_shards", 0) \
M(UInt64, force_optimize_skip_unused_shards, 0, "Throw an exception if unused shards cannot be skipped (1 - throw only if the table has the sharding key, 2 - always throw.", 0) \
@ -215,7 +216,7 @@ class IColumn;
\
M(Bool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \
M(UInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \
M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.", 0) \
M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \
M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \
M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \
\
@ -437,7 +438,9 @@ class IColumn;
M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \
M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \
M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \
M(Bool, database_replicated_ddl_output, true, "Return table with query execution status as a result of DDL query", 0) \
M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \
M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \
M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \
\
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\
@ -449,6 +452,7 @@ class IColumn;
M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \
M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \
M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \
M(Bool, database_replicated_ddl_output, true, "Obsolete setting, does nothing. Will be removed after 2021-09-08", 0) \
// End of COMMON_SETTINGS
// Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below.

View File

@ -102,4 +102,10 @@ IMPLEMENT_SETTING_ENUM(UnionMode, ErrorCodes::UNKNOWN_UNION,
{"ALL", UnionMode::ALL},
{"DISTINCT", UnionMode::DISTINCT}})
IMPLEMENT_SETTING_ENUM(DistributedDDLOutputMode, ErrorCodes::BAD_ARGUMENTS,
{{"none", DistributedDDLOutputMode::NONE},
{"throw", DistributedDDLOutputMode::THROW},
{"null_status_on_timeout", DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT},
{"never_throw", DistributedDDLOutputMode::NEVER_THROW}})
}

View File

@ -138,4 +138,15 @@ enum class UnionMode
DECLARE_SETTING_ENUM(UnionMode)
enum class DistributedDDLOutputMode
{
NONE,
THROW,
NULL_STATUS_ON_TIMEOUT,
NEVER_THROW,
};
DECLARE_SETTING_ENUM(DistributedDDLOutputMode)
}

View File

@ -231,8 +231,8 @@ void DatabaseOnDisk::createTable(
if (create.attach_short_syntax)
{
/// Metadata already exists, table was detached
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, true);
attachTable(table_name, table, getTableDataPath(create));
removeDetachedPermanentlyFlag(table_name, table_metadata_path);
return;
}
@ -270,12 +270,12 @@ void DatabaseOnDisk::createTable(
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path, context);
removeDetachedPermanentlyFlag(table_name, table_metadata_path);
removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, false);
}
/// If the table was detached permanently we will have a flag file with
/// .sql.detached extension, is not needed anymore since we attached the table back
void DatabaseOnDisk::removeDetachedPermanentlyFlag(const String & table_name, const String & table_metadata_path) const
void DatabaseOnDisk::removeDetachedPermanentlyFlag(const Context &, const String & table_name, const String & table_metadata_path, bool) const
{
try
{

View File

@ -94,11 +94,10 @@ protected:
virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path, const Context & query_context);
virtual void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const;
const String metadata_path;
const String data_path;
private:
void removeDetachedPermanentlyFlag(const String & table_name, const String & table_metadata_path) const;
};
}

View File

@ -18,6 +18,7 @@
#include <Interpreters/Cluster.h>
#include <common/getFQDNOrHostName.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
@ -105,7 +106,22 @@ std::pair<String, String> DatabaseReplicated::parseFullReplicaName(const String
ClusterPtr DatabaseReplicated::getCluster() const
{
/// TODO Maintain up-to-date Cluster and allow to use it in Distributed tables
std::lock_guard lock{mutex};
if (cluster)
return cluster;
cluster = getClusterImpl();
return cluster;
}
void DatabaseReplicated::setCluster(ClusterPtr && new_cluster)
{
std::lock_guard lock{mutex};
cluster = std::move(new_cluster);
}
ClusterPtr DatabaseReplicated::getClusterImpl() const
{
Strings hosts;
Strings host_ids;
@ -120,7 +136,7 @@ ClusterPtr DatabaseReplicated::getCluster() const
hosts = zookeeper->getChildren(zookeeper_path + "/replicas", &stat);
if (hosts.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No hosts found");
Int32 cver = stat.cversion;
Int32 cversion = stat.cversion;
std::sort(hosts.begin(), hosts.end());
std::vector<zkutil::ZooKeeper::FutureGet> futures;
@ -139,7 +155,9 @@ ClusterPtr DatabaseReplicated::getCluster() const
}
zookeeper->get(zookeeper_path + "/replicas", &stat);
if (success && cver == stat.version)
if (cversion != stat.cversion)
success = false;
if (success)
break;
}
if (!success)
@ -157,22 +175,23 @@ ClusterPtr DatabaseReplicated::getCluster() const
if (id == DROPPED_MARK)
continue;
auto [shard, replica] = parseFullReplicaName(hosts[i]);
auto pos = id.find(':');
String host = id.substr(0, pos);
auto pos = id.rfind(':');
String host_port = id.substr(0, pos);
if (shard != current_shard)
{
current_shard = shard;
if (!shards.back().empty())
shards.emplace_back();
}
shards.back().emplace_back(unescapeForFileName(host));
shards.back().emplace_back(unescapeForFileName(host_port));
}
/// TODO make it configurable
String username = "default";
String password;
String username = db_settings.cluster_username;
String password = db_settings.cluster_password;
UInt16 default_port = global_context.getTCPPort();
bool secure = db_settings.cluster_secure_connection;
return std::make_shared<Cluster>(global_context.getSettingsRef(), shards, username, password, global_context.getTCPPort(), false);
return std::make_shared<Cluster>(global_context.getSettingsRef(), shards, username, password, default_port, false, secure);
}
void DatabaseReplicated::tryConnectToZooKeeperAndInitDatabase(bool force_attach)
@ -253,11 +272,8 @@ bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperP
__builtin_unreachable();
}
void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper)
void DatabaseReplicated::createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper)
{
/// Write host name to replica_path, it will protect from multiple replicas with the same name
auto host_id = getHostID(global_context, db_uuid);
/// On replica creation add empty entry to log. Can be used to trigger some actions on other replicas (e.g. update cluster info).
DDLLogEntry entry{};
@ -266,11 +282,20 @@ void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPt
String counter_path = current_zookeeper->create(counter_prefix, "", zkutil::CreateMode::EphemeralSequential);
String query_path = query_path_prefix + counter_path.substr(counter_prefix.size());
ops.emplace_back(zkutil::makeCreateRequest(query_path, entry.toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(query_path + "/committed", getFullReplicaName(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1));
}
void DatabaseReplicated::createReplicaNodesInZooKeeper(const zkutil::ZooKeeperPtr & current_zookeeper)
{
/// Write host name to replica_path, it will protect from multiple replicas with the same name
auto host_id = getHostID(global_context, db_uuid);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(replica_path, host_id, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_ptr", "0", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(query_path, entry.toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest(counter_path, -1));
createEmptyLogEntry(ops, current_zookeeper);
current_zookeeper->multi(ops);
}
@ -294,7 +319,11 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const
/// Replicas will set correct name of current database in query context (database name can be different on replicas)
if (auto * ddl_query = query->as<ASTQueryWithTableAndOutput>())
{
if (ddl_query->database != getDatabaseName())
throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed");
ddl_query->database.clear();
}
if (const auto * query_alter = query->as<ASTAlterQuery>())
{
@ -305,23 +334,26 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const
}
}
if (auto * query_drop = query->as<ASTDropQuery>())
{
if (query_drop->kind == ASTDropQuery::Kind::Detach && query_context.getSettingsRef().database_replicated_always_detach_permanently)
query_drop->permanently = true;
if (query_drop->kind == ASTDropQuery::Kind::Detach && !query_drop->permanently)
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
"Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA or set "
"database_replicated_always_detach_permanently to 1");
}
LOG_DEBUG(log, "Proposing query: {}", queryToString(query));
/// TODO maybe write current settings to log entry?
DDLLogEntry entry;
entry.query = queryToString(query);
entry.initiator = ddl_worker->getCommonHostID();
entry.setSettingsIfRequired(query_context);
String node_path = ddl_worker->tryEnqueueAndExecuteEntry(entry, query_context);
BlockIO io;
if (query_context.getSettingsRef().distributed_ddl_task_timeout == 0)
return io;
Strings hosts_to_wait = getZooKeeper()->getChildren(zookeeper_path + "/replicas");
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, query_context, hosts_to_wait);
if (query_context.getSettingsRef().database_replicated_ddl_output)
io.in = std::move(stream);
return io;
return getDistributedDDLStatus(node_path, entry, query_context, hosts_to_wait);
}
static UUID getTableUUIDIfReplicated(const String & metadata, const Context & context)
@ -560,9 +592,11 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node
if (create.uuid == UUIDHelpers::Nil || create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER || !create.database.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Got unexpected query from {}: {}", node_name, query);
bool is_materialized_view_with_inner_table = create.is_materialized_view && create.to_table_id.empty();
create.database = getDatabaseName();
create.table = unescapeForFileName(node_name);
create.attach = false;
create.attach = is_materialized_view_with_inner_table;
return ast;
}
@ -570,8 +604,13 @@ ASTPtr DatabaseReplicated::parseQueryFromMetadataInZooKeeper(const String & node
void DatabaseReplicated::drop(const Context & context_)
{
auto current_zookeeper = getZooKeeper();
current_zookeeper->set(replica_path, DROPPED_MARK);
Coordination::Requests ops;
ops.emplace_back(zkutil::makeSetRequest(replica_path, DROPPED_MARK, -1));
createEmptyLogEntry(ops, current_zookeeper);
current_zookeeper->multi(ops);
DatabaseAtomic::drop(context_);
current_zookeeper->tryRemoveRecursive(replica_path);
/// TODO it may leave garbage in ZooKeeper if the last node lost connection here
if (current_zookeeper->tryRemove(zookeeper_path + "/replicas") == Coordination::Error::ZOK)
@ -598,7 +637,7 @@ void DatabaseReplicated::shutdown()
void DatabaseReplicated::dropTable(const Context & context, const String & table_name, bool no_delay)
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
assert(!ddl_worker->isCurrentlyActive() || txn || startsWith(table_name, ".inner_id."));
if (txn && txn->isInitialQuery())
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
@ -702,12 +741,28 @@ void DatabaseReplicated::detachTablePermanently(const Context & context, const S
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery())
{
/// We have to remove metadata from zookeeper, because we do not distinguish permanently detached tables
/// from attached tables when recovering replica.
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
txn->addOp(zkutil::makeRemoveRequest(metadata_zk_path, -1));
}
DatabaseAtomic::detachTablePermanently(context, table_name);
}
void DatabaseReplicated::removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const
{
auto txn = context.getZooKeeperMetadataTransaction();
assert(!ddl_worker->isCurrentlyActive() || txn);
if (txn && txn->isInitialQuery() && attach)
{
String metadata_zk_path = zookeeper_path + "/metadata/" + escapeForFileName(table_name);
String statement = readMetadataFile(table_name);
txn->addOp(zkutil::makeCreateRequest(metadata_zk_path, statement, zkutil::CreateMode::Persistent));
}
DatabaseAtomic::removeDetachedPermanentlyFlag(context, table_name, table_metadata_path, attach);
}
String DatabaseReplicated::readMetadataFile(const String & table_name) const
{
String statement;

View File

@ -45,6 +45,7 @@ public:
const ASTPtr & query) override;
void removeDictionary(const Context & context, const String & dictionary_name) override;
void detachTablePermanently(const Context & context, const String & table_name) override;
void removeDetachedPermanentlyFlag(const Context & context, const String & table_name, const String & table_metadata_path, bool attach) const override;
/// Try to execute DLL query on current host as initial query. If query is succeed,
/// then it will be executed on all replicas.
@ -76,6 +77,11 @@ private:
ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query);
String readMetadataFile(const String & table_name) const;
ClusterPtr getClusterImpl() const;
void setCluster(ClusterPtr && new_cluster);
void createEmptyLogEntry(Coordination::Requests & ops, const ZooKeeperPtr & current_zookeeper);
String zookeeper_path;
String shard_name;
String replica_name;
@ -86,6 +92,8 @@ private:
std::atomic_bool is_readonly = true;
std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker;
mutable ClusterPtr cluster;
};
}

View File

@ -11,6 +11,9 @@ class ASTStorage;
M(Float, max_broken_tables_ratio, 0.5, "Do not recover replica automatically if the ratio of staled tables to all tables is greater", 0) \
M(UInt64, max_replication_lag_to_enqueue, 10, "Replica will throw exception on attempt to execute query if its replication lag greater", 0) \
M(UInt64, wait_entry_commited_timeout_sec, 3600, "Replicas will try to cancel query if timeout exceed, but initiator host has not executed it yet", 0) \
M(String, cluster_username, "default", "Username to use when connecting to hosts of cluster", 0) \
M(String, cluster_password, "", "Password to use when connecting to hosts of cluster", 0) \
M(Bool, cluster_secure_connection, false, "Enable TLS when connecting to hosts of cluster", 0) \
DECLARE_SETTINGS_TRAITS(DatabaseReplicatedSettingsTraits, LIST_OF_DATABASE_REPLICATED_SETTINGS)

View File

@ -237,6 +237,8 @@ DDLTaskPtr DatabaseReplicatedDDLWorker::initAndCheckTask(const String & entry_na
if (task->entry.query.empty())
{
/// Some replica is added or removed, let's update cached cluster
database->setCluster(database->getClusterImpl());
out_reason = fmt::format("Entry {} is a dummy task", entry_name);
return {};
}

View File

@ -34,8 +34,8 @@ private:
void registerFunctionConnectionID(FunctionFactory & factory)
{
factory.registerFunction<FunctionConnectionID>();
factory.registerAlias("connection_id", "connectionID");
factory.registerFunction<FunctionConnectionID>(FunctionFactory::CaseInsensitive);
factory.registerAlias("connection_id", "connectionID", FunctionFactory::CaseInsensitive);
}
}

View File

@ -49,7 +49,7 @@ public:
void registerFunctionVersion(FunctionFactory & factory)
{
factory.registerFunction<FunctionVersion>();
factory.registerFunction<FunctionVersion>(FunctionFactory::CaseInsensitive);
}
}

View File

@ -116,7 +116,9 @@ Cluster::Address::Address(
const String & password_,
UInt16 clickhouse_port,
bool secure_,
Int64 priority_)
Int64 priority_,
UInt32 shard_index_,
UInt32 replica_index_)
: user(user_)
, password(password_)
{
@ -126,6 +128,8 @@ Cluster::Address::Address(
secure = secure_ ? Protocol::Secure::Enable : Protocol::Secure::Disable;
priority = priority_;
is_local = isLocal(clickhouse_port);
shard_index = shard_index_;
replica_index = replica_index_;
}
@ -491,7 +495,7 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
{
Addresses current;
for (const auto & replica : shard)
current.emplace_back(replica, username, password, clickhouse_port, secure, priority);
current.emplace_back(replica, username, password, clickhouse_port, secure, priority, current_shard_num, current.size() + 1);
addresses_with_failover.emplace_back(current);
@ -589,6 +593,7 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti
if (from.addresses_with_failover.empty())
throw Exception("Cluster is empty", ErrorCodes::LOGICAL_ERROR);
UInt32 shard_num = 0;
std::set<std::pair<String, int>> unique_hosts;
for (size_t shard_index : ext::range(0, from.shards_info.size()))
{
@ -599,6 +604,8 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti
continue; /// Duplicate host, skip.
ShardInfo info;
info.shard_num = ++shard_num;
if (address.is_local)
info.local_addresses.push_back(address);

View File

@ -97,6 +97,7 @@ public:
Int64 priority = 1;
Address() = default;
Address(
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix,
@ -104,13 +105,16 @@ public:
const String & cluster_secret_,
UInt32 shard_index_ = 0,
UInt32 replica_index_ = 0);
Address(
const String & host_port_,
const String & user_,
const String & password_,
UInt16 clickhouse_port,
bool secure_ = false,
Int64 priority_ = 1);
Int64 priority_ = 1,
UInt32 shard_index_ = 0,
UInt32 replica_index_ = 0);
/// Returns 'escaped_host_name:port'
String toString() const;

View File

@ -12,7 +12,7 @@
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
#include <Common/thread_local_rng.h>
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <Coordination/KeeperStorageDispatcher.h>
#include <Compression/ICompressionCodec.h>
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
@ -54,6 +54,7 @@
#include <Interpreters/SystemLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/UncompressedCache.h>
#include <IO/MMappedFileCache.h>
@ -313,7 +314,7 @@ struct ContextShared
#if USE_NURAFT
mutable std::mutex nu_keeper_storage_dispatcher_mutex;
mutable std::shared_ptr<NuKeeperStorageDispatcher> nu_keeper_storage_dispatcher;
mutable std::shared_ptr<KeeperStorageDispatcher> nu_keeper_storage_dispatcher;
#endif
mutable std::mutex auxiliary_zookeepers_mutex;
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
@ -1615,35 +1616,35 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
}
void Context::initializeNuKeeperStorageDispatcher() const
void Context::initializeKeeperStorageDispatcher() const
{
#if USE_NURAFT
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
if (shared->nu_keeper_storage_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize NuKeeper multiple times");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times");
const auto & config = getConfigRef();
if (config.has("test_keeper_server"))
if (config.has("keeper_server"))
{
shared->nu_keeper_storage_dispatcher = std::make_shared<NuKeeperStorageDispatcher>();
shared->nu_keeper_storage_dispatcher = std::make_shared<KeeperStorageDispatcher>();
shared->nu_keeper_storage_dispatcher->initialize(config);
}
#endif
}
#if USE_NURAFT
std::shared_ptr<NuKeeperStorageDispatcher> & Context::getNuKeeperStorageDispatcher() const
std::shared_ptr<KeeperStorageDispatcher> & Context::getKeeperStorageDispatcher() const
{
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
if (!shared->nu_keeper_storage_dispatcher)
throw Exception(ErrorCodes::LOGICAL_ERROR, "NuKeeper must be initialized before requests");
throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests");
return shared->nu_keeper_storage_dispatcher;
}
#endif
void Context::shutdownNuKeeperStorageDispatcher() const
void Context::shutdownKeeperStorageDispatcher() const
{
#if USE_NURAFT
std::lock_guard lock(shared->nu_keeper_storage_dispatcher_mutex);
@ -1800,11 +1801,14 @@ std::optional<UInt16> Context::getTCPPortSecure() const
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
{
auto res = getClusters().getCluster(cluster_name);
if (!res)
throw Exception("Requested cluster '" + cluster_name + "' not found", ErrorCodes::BAD_GET);
if (res)
return res;
res = tryGetReplicatedDatabaseCluster(cluster_name);
if (res)
return res;
throw Exception("Requested cluster '" + cluster_name + "' not found", ErrorCodes::BAD_GET);
}

View File

@ -111,7 +111,7 @@ class StoragePolicySelector;
using StoragePolicySelectorPtr = std::shared_ptr<const StoragePolicySelector>;
struct PartUUIDs;
using PartUUIDsPtr = std::shared_ptr<PartUUIDs>;
class NuKeeperStorageDispatcher;
class KeeperStorageDispatcher;
class IOutputFormat;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
@ -598,10 +598,10 @@ public:
std::shared_ptr<zkutil::ZooKeeper> getAuxiliaryZooKeeper(const String & name) const;
#if USE_NURAFT
std::shared_ptr<NuKeeperStorageDispatcher> & getNuKeeperStorageDispatcher() const;
std::shared_ptr<KeeperStorageDispatcher> & getKeeperStorageDispatcher() const;
#endif
void initializeNuKeeperStorageDispatcher() const;
void shutdownNuKeeperStorageDispatcher() const;
void initializeKeeperStorageDispatcher() const;
void shutdownKeeperStorageDispatcher() const;
/// Set auxiliary zookeepers configuration at server starting or configuration reloading.
void reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config);

View File

@ -10,6 +10,7 @@
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Databases/DatabaseReplicated.h>
@ -43,20 +44,47 @@ bool HostID::isLocalAddress(UInt16 clickhouse_port) const
}
}
void DDLLogEntry::assertVersion() const
{
constexpr UInt64 max_version = 2;
if (version == 0 || max_version < version)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}."
"Maximum supported version is {}", version, max_version);
}
void DDLLogEntry::setSettingsIfRequired(const Context & context)
{
version = context.getSettingsRef().distributed_ddl_entry_format_version;
if (version == 2)
settings.emplace(context.getSettingsRef().changes());
}
String DDLLogEntry::toString() const
{
WriteBufferFromOwnString wb;
Strings host_id_strings(hosts.size());
std::transform(hosts.begin(), hosts.end(), host_id_strings.begin(), HostID::applyToString);
auto version = CURRENT_VERSION;
wb << "version: " << version << "\n";
wb << "query: " << escape << query << "\n";
bool write_hosts = version == 1 || !hosts.empty();
if (write_hosts)
{
Strings host_id_strings(hosts.size());
std::transform(hosts.begin(), hosts.end(), host_id_strings.begin(), HostID::applyToString);
wb << "hosts: " << host_id_strings << "\n";
}
wb << "initiator: " << initiator << "\n";
bool write_settings = 1 <= version && settings && !settings->empty();
if (write_settings)
{
ASTSetQuery ast;
ast.is_standalone = false;
ast.changes = *settings;
wb << "settings: " << serializeAST(ast) << "\n";
}
return wb.str();
}
@ -64,26 +92,47 @@ void DDLLogEntry::parse(const String & data)
{
ReadBufferFromString rb(data);
int version;
rb >> "version: " >> version >> "\n";
if (version != CURRENT_VERSION)
throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown DDLLogEntry format version: {}", version);
assertVersion();
Strings host_id_strings;
rb >> "query: " >> escape >> query >> "\n";
if (version == 1)
{
rb >> "hosts: " >> host_id_strings >> "\n";
if (!rb.eof())
rb >> "initiator: " >> initiator >> "\n";
else
initiator.clear();
}
else if (version == 2)
{
if (!rb.eof() && *rb.position() == 'h')
rb >> "hosts: " >> host_id_strings >> "\n";
if (!rb.eof() && *rb.position() == 'i')
rb >> "initiator: " >> initiator >> "\n";
if (!rb.eof() && *rb.position() == 's')
{
String settings_str;
rb >> "settings: " >> settings_str >> "\n";
ParserSetQuery parser{true};
constexpr UInt64 max_size = 4096;
constexpr UInt64 max_depth = 16;
ASTPtr settings_ast = parseQuery(parser, settings_str, max_size, max_depth);
settings.emplace(std::move(settings_ast->as<ASTSetQuery>()->changes));
}
}
assertEOF(rb);
if (!host_id_strings.empty())
{
hosts.resize(host_id_strings.size());
std::transform(host_id_strings.begin(), host_id_strings.end(), hosts.begin(), HostID::fromString);
}
}
void DDLTaskBase::parseQueryFromEntry(const Context & context)
@ -102,6 +151,8 @@ std::unique_ptr<Context> DDLTaskBase::makeQueryContext(Context & from_context, c
query_context->makeQueryContext();
query_context->setCurrentQueryId(""); // generate random query_id
query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY;
if (entry.settings)
query_context->applySettingsChanges(*entry.settings);
return query_context;
}
@ -345,4 +396,11 @@ void ZooKeeperMetadataTransaction::commit()
state = COMMITTED;
}
ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name)
{
if (const auto * replicated_db = dynamic_cast<const DatabaseReplicated *>(DatabaseCatalog::instance().tryGetDatabase(cluster_name).get()))
return replicated_db->getCluster();
return {};
}
}

View File

@ -18,6 +18,7 @@ namespace DB
class ASTQueryWithOnCluster;
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
using ClusterPtr = std::shared_ptr<Cluster>;
class DatabaseReplicated;
class ZooKeeperMetadataTransaction;
@ -56,15 +57,16 @@ struct HostID
struct DDLLogEntry
{
UInt64 version = 1;
String query;
std::vector<HostID> hosts;
String initiator; // optional
std::optional<SettingsChanges> settings;
static constexpr int CURRENT_VERSION = 1;
void setSettingsIfRequired(const Context & context);
String toString() const;
void parse(const String & data);
void assertVersion() const;
};
struct DDLTaskBase
@ -192,4 +194,6 @@ public:
~ZooKeeperMetadataTransaction() { assert(isExecuted() || std::uncaught_exceptions()); }
};
ClusterPtr tryGetReplicatedDatabaseCluster(const String & cluster_name);
}

View File

@ -713,6 +713,19 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
}
}
static void generateUUIDForTable(ASTCreateQuery & create)
{
if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
/// If destination table (to_table_id) is not specified for materialized view,
/// then MV will create inner table. We should generate UUID of inner table here,
/// so it will be the same on all hosts if query in ON CLUSTER or database engine is Replicated.
bool need_uuid_for_inner_table = create.is_materialized_view && !create.to_table_id;
if (need_uuid_for_inner_table && create.to_inner_uuid == UUIDHelpers::Nil)
create.to_inner_uuid = UUIDHelpers::generateV4();
}
void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const
{
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
@ -744,18 +757,19 @@ void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const Data
kind_upper, create.table);
}
if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
generateUUIDForTable(create);
}
else
{
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
bool has_uuid = create.uuid != UUIDHelpers::Nil || create.to_inner_uuid != UUIDHelpers::Nil;
if (has_uuid && !is_on_cluster)
throw Exception(ErrorCodes::INCORRECT_QUERY,
"{} UUID specified, but engine of database {} is not Atomic", kind, create.database);
/// Ignore UUID if it's ON CLUSTER query
create.uuid = UUIDHelpers::Nil;
create.to_inner_uuid = UUIDHelpers::Nil;
}
if (create.replace_table)
@ -804,6 +818,17 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.attach && !create.storage && !create.columns_list)
{
auto database = DatabaseCatalog::instance().getDatabase(database_name);
if (database->getEngineName() == "Replicated")
{
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, create.table);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
create.database = database_name;
guard->releaseTableLock();
return typeid_cast<DatabaseReplicated *>(database.get())->tryEnqueueReplicatedDDL(query_ptr, context);
}
}
bool if_not_exists = create.if_not_exists;
// Table SQL definition is available even if the table is detached (even permanently)
@ -877,7 +902,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (need_add_to_database && database->getEngineName() == "Replicated")
{
auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
database = DatabaseCatalog::instance().getDatabase(create.database);
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
{
assertOrSetUUID(create, database);
@ -1136,8 +1160,7 @@ void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, cons
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
/// It will be ignored if database does not support UUIDs.
if (create.uuid == UUIDHelpers::Nil)
create.uuid = UUIDHelpers::generateV4();
generateUUIDForTable(create);
/// For cross-replication cluster we cannot use UUID in replica path.
String cluster_name_expanded = context.getMacros()->expand(cluster_name);

View File

@ -133,10 +133,6 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat
!is_drop_or_detach_database;
if (is_replicated_ddl_query)
{
if (query.kind == ASTDropQuery::Kind::Detach && !query.permanently)
throw Exception(ErrorCodes::INCORRECT_QUERY, "DETACH TABLE is not allowed for Replicated databases. "
"Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA");
if (query.kind == ASTDropQuery::Kind::Detach)
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
else if (query.kind == ASTDropQuery::Kind::Truncate)

View File

@ -82,6 +82,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
{
auto & create = create_query->as<ASTCreateQuery &>();
create.uuid = UUIDHelpers::Nil;
create.to_inner_uuid = UUIDHelpers::Nil;
}
WriteBufferFromOwnString buf;

View File

@ -166,9 +166,9 @@ namespace
return result;
}
Disjunction analyzeFunction(const ASTFunction * fn, const ExpressionActionsPtr & expr)
Disjunction analyzeFunction(const ASTFunction * fn, const ExpressionActionsPtr & expr, size_t & limit)
{
if (!fn)
if (!fn || !limit)
{
return {};
}
@ -182,6 +182,7 @@ namespace
const auto * identifier = left->as<ASTIdentifier>() ? left->as<ASTIdentifier>() : right->as<ASTIdentifier>();
const auto * literal = left->as<ASTLiteral>() ? left->as<ASTLiteral>() : right->as<ASTLiteral>();
--limit;
return analyzeEquals(identifier, literal, expr);
}
else if (fn->name == "in")
@ -192,6 +193,19 @@ namespace
Disjunction result;
auto add_dnf = [&](const auto &dnf)
{
if (dnf.size() > limit)
{
result.clear();
return false;
}
result.insert(result.end(), dnf.begin(), dnf.end());
limit -= dnf.size();
return true;
};
if (const auto * tuple_func = right->as<ASTFunction>(); tuple_func && tuple_func->name == "tuple")
{
const auto * tuple_elements = tuple_func->children.front()->as<ASTExpressionList>();
@ -205,7 +219,10 @@ namespace
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
if (!add_dnf(dnf))
{
return {};
}
}
}
else if (const auto * tuple_literal = right->as<ASTLiteral>();
@ -221,7 +238,10 @@ namespace
return {};
}
result.insert(result.end(), dnf.begin(), dnf.end());
if (!add_dnf(dnf))
{
return {};
}
}
}
else
@ -244,13 +264,14 @@ namespace
for (const auto & arg : args->children)
{
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr);
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr, limit);
if (dnf.empty())
{
return {};
}
/// limit accounted in analyzeFunction()
result.insert(result.end(), dnf.begin(), dnf.end());
}
@ -269,13 +290,14 @@ namespace
for (const auto & arg : args->children)
{
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr);
const auto dnf = analyzeFunction(arg->as<ASTFunction>(), expr, limit);
if (dnf.empty())
{
continue;
}
/// limit accounted in analyzeFunction()
result = andDNF(result, dnf);
}
@ -286,15 +308,15 @@ namespace
}
}
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr)
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit)
{
Blocks result;
if (const auto * fn = node->as<ASTFunction>())
{
const auto dnf = analyzeFunction(fn, target_expr);
const auto dnf = analyzeFunction(fn, target_expr, limit);
if (dnf.empty())
if (dnf.empty() || !limit)
{
return {};
}

View File

@ -46,10 +46,11 @@ ASTPtr evaluateConstantExpressionForDatabaseName(const ASTPtr & node, const Cont
/** Try to fold condition to countable set of constant values.
* @param node a condition that we try to fold.
* @param target_expr expression evaluated over a set of constants.
* @param limit limit for number of values
* @return optional blocks each with a single row and a single column for target expression,
* or empty blocks if condition is always false,
* or nothing if condition can't be folded to a set of constants.
*/
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr);
std::optional<Blocks> evaluateExpressionOverConstantCondition(const ASTPtr & node, const ExpressionActionsPtr & target_expr, size_t & limit);
}

View File

@ -13,6 +13,9 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataStreams/NullBlockOutputStream.h>
#include <DataStreams/copyData.h>
#include <filesystem>
namespace fs = std::filesystem;
@ -160,18 +163,32 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
entry.hosts = std::move(hosts);
entry.query = queryToString(query_ptr);
entry.initiator = ddl_worker.getCommonHostID();
entry.setSettingsIfRequired(context);
String node_path = ddl_worker.enqueueQuery(entry);
return getDistributedDDLStatus(node_path, entry, context);
}
BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, const Context & context, const std::optional<Strings> & hosts_to_wait)
{
BlockIO io;
if (context.getSettingsRef().distributed_ddl_task_timeout == 0)
return io;
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, context);
auto stream = std::make_shared<DDLQueryStatusInputStream>(node_path, entry, context, hosts_to_wait);
if (context.getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE)
{
/// Wait for query to finish, but ignore output
NullBlockOutputStream output{Block{}};
copyData(*stream, output);
}
else
{
io.in = std::move(stream);
}
return io;
}
DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path, const DDLLogEntry & entry, const Context & context_,
const std::optional<Strings> & hosts_to_wait)
: node_path(zk_node_path)
@ -179,11 +196,27 @@ DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path
, watch(CLOCK_MONOTONIC_COARSE)
, log(&Poco::Logger::get("DDLQueryStatusInputStream"))
{
if (context.getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::THROW ||
context.getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NONE)
throw_on_timeout = true;
else if (context.getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NULL_STATUS_ON_TIMEOUT ||
context.getSettingsRef().distributed_ddl_output_mode == DistributedDDLOutputMode::NEVER_THROW)
throw_on_timeout = false;
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown output mode");
auto maybe_make_nullable = [&](const DataTypePtr & type) -> DataTypePtr
{
if (throw_on_timeout)
return type;
return std::make_shared<DataTypeNullable>(type);
};
sample = Block{
{std::make_shared<DataTypeString>(), "host"},
{std::make_shared<DataTypeUInt16>(), "port"},
{std::make_shared<DataTypeInt64>(), "status"},
{std::make_shared<DataTypeString>(), "error"},
{maybe_make_nullable(std::make_shared<DataTypeInt64>()), "status"},
{maybe_make_nullable(std::make_shared<DataTypeString>()), "error"},
{std::make_shared<DataTypeUInt64>(), "num_hosts_remaining"},
{std::make_shared<DataTypeUInt64>(), "num_hosts_active"},
};
@ -192,6 +225,7 @@ DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path
{
waiting_hosts = NameSet(hosts_to_wait->begin(), hosts_to_wait->end());
by_hostname = false;
sample.erase("port");
}
else
{
@ -204,12 +238,29 @@ DDLQueryStatusInputStream::DDLQueryStatusInputStream(const String & zk_node_path
timeout_seconds = context.getSettingsRef().distributed_ddl_task_timeout;
}
std::pair<String, UInt16> DDLQueryStatusInputStream::parseHostAndPort(const String & host_id) const
{
String host = host_id;
UInt16 port = 0;
if (by_hostname)
{
auto host_and_port = Cluster::Address::fromString(host_id);
host = host_and_port.first;
port = host_and_port.second;
}
return {host, port};
}
Block DDLQueryStatusInputStream::readImpl()
{
Block res;
if (num_hosts_finished >= waiting_hosts.size())
bool all_hosts_finished = num_hosts_finished >= waiting_hosts.size();
/// Seems like num_hosts_finished cannot be strictly greater than waiting_hosts.size()
assert(num_hosts_finished <= waiting_hosts.size());
if (all_hosts_finished || timeout_exceeded)
{
if (first_exception)
bool throw_if_error_on_host = context.getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW;
if (first_exception && throw_if_error_on_host)
throw Exception(*first_exception);
return res;
@ -222,7 +273,8 @@ Block DDLQueryStatusInputStream::readImpl()
{
if (isCancelled())
{
if (first_exception)
bool throw_if_error_on_host = context.getSettingsRef().distributed_ddl_output_mode != DistributedDDLOutputMode::NEVER_THROW;
if (first_exception && throw_if_error_on_host)
throw Exception(*first_exception);
return res;
@ -233,11 +285,36 @@ Block DDLQueryStatusInputStream::readImpl()
size_t num_unfinished_hosts = waiting_hosts.size() - num_hosts_finished;
size_t num_active_hosts = current_active_hosts.size();
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED,
"Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. "
"There are {} unfinished hosts ({} of them are currently active), they are going to execute the query in background",
constexpr const char * msg_format = "Watching task {} is executing longer than distributed_ddl_task_timeout (={}) seconds. "
"There are {} unfinished hosts ({} of them are currently active), "
"they are going to execute the query in background";
if (throw_on_timeout)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, msg_format,
node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
timeout_exceeded = true;
LOG_INFO(log, msg_format, node_path, timeout_seconds, num_unfinished_hosts, num_active_hosts);
NameSet unfinished_hosts = waiting_hosts;
for (const auto & host_id : finished_hosts)
unfinished_hosts.erase(host_id);
/// Query is not finished on the rest hosts, so fill the corresponding rows with NULLs.
MutableColumns columns = sample.cloneEmptyColumns();
for (const String & host_id : unfinished_hosts)
{
auto [host, port] = parseHostAndPort(host_id);
size_t num = 0;
columns[num++]->insert(host);
if (by_hostname)
columns[num++]->insert(port);
columns[num++]->insert(Field{});
columns[num++]->insert(Field{});
columns[num++]->insert(num_unfinished_hosts);
columns[num++]->insert(num_active_hosts);
}
res = sample.cloneWithColumns(std::move(columns));
return res;
}
if (num_hosts_finished != 0 || try_number != 0)
@ -269,26 +346,21 @@ Block DDLQueryStatusInputStream::readImpl()
status.tryDeserializeText(status_data);
}
String host = host_id;
UInt16 port = 0;
if (by_hostname)
{
auto host_and_port = Cluster::Address::fromString(host_id);
host = host_and_port.first;
port = host_and_port.second;
}
auto [host, port] = parseHostAndPort(host_id);
if (status.code != 0 && first_exception == nullptr)
first_exception = std::make_unique<Exception>(status.code, "There was an error on [{}:{}]: {}", host, port, status.message);
++num_hosts_finished;
columns[0]->insert(host);
columns[1]->insert(port);
columns[2]->insert(status.code);
columns[3]->insert(status.message);
columns[4]->insert(waiting_hosts.size() - num_hosts_finished);
columns[5]->insert(current_active_hosts.size());
size_t num = 0;
columns[num++]->insert(host);
if (by_hostname)
columns[num++]->insert(port);
columns[num++]->insert(status.code);
columns[num++]->insert(status.message);
columns[num++]->insert(waiting_hosts.size() - num_hosts_finished);
columns[num++]->insert(current_active_hosts.size());
}
res = sample.cloneWithColumns(std::move(columns));
}

View File

@ -24,6 +24,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & conte
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, const AccessRightsElements & query_requires_access);
BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr, const Context & context, AccessRightsElements && query_requires_access);
BlockIO getDistributedDDLStatus(const String & node_path, const DDLLogEntry & entry, const Context & context, const std::optional<Strings> & hosts_to_wait = {});
class DDLQueryStatusInputStream final : public IBlockInputStream
{
@ -44,6 +45,8 @@ private:
Strings getNewAndUpdate(const Strings & current_list_of_finished_hosts);
std::pair<String, UInt16> parseHostAndPort(const String & host_id) const;
String node_path;
const Context & context;
Stopwatch watch;
@ -62,6 +65,8 @@ private:
Int64 timeout_seconds = 120;
bool by_hostname = true;
bool throw_on_timeout = true;
bool timeout_exceeded = false;
};
}

View File

@ -297,12 +297,20 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (to_table_id)
{
assert(is_materialized_view && to_inner_uuid == UUIDHelpers::Nil);
settings.ostr
<< (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "")
<< (!to_table_id.database_name.empty() ? backQuoteIfNeed(to_table_id.database_name) + "." : "")
<< backQuoteIfNeed(to_table_id.table_name);
}
if (to_inner_uuid != UUIDHelpers::Nil)
{
assert(is_materialized_view && !to_table_id);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO INNER UUID " << (settings.hilite ? hilite_none : "")
<< quoteString(toString(to_inner_uuid));
}
if (!as_table.empty())
{
settings.ostr

View File

@ -66,6 +66,7 @@ public:
ASTExpressionList * tables = nullptr;
StorageID to_table_id = StorageID::createEmpty(); /// For CREATE MATERIALIZED VIEW mv TO table.
UUID to_inner_uuid = UUIDHelpers::Nil; /// For materialized view with inner table
ASTStorage * storage = nullptr;
String as_database;
String as_table;

View File

@ -780,6 +780,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
ASTPtr table;
ASTPtr to_table;
ASTPtr to_inner_uuid;
ASTPtr columns_list;
ASTPtr storage;
ASTPtr as_database;
@ -830,9 +831,16 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return false;
}
// TO [db.]table
if (ParserKeyword{"TO"}.ignore(pos, expected))
if (ParserKeyword{"TO INNER UUID"}.ignore(pos, expected))
{
ParserLiteral literal_p;
if (!literal_p.parse(pos, to_inner_uuid, expected))
return false;
}
else if (ParserKeyword{"TO"}.ignore(pos, expected))
{
// TO [db.]table
if (!table_name_p.parse(pos, to_table, expected))
return false;
}
@ -883,6 +891,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (to_table)
query->to_table_id = getTableIdentifier(to_table);
if (to_inner_uuid)
query->to_inner_uuid = parseFromString<UUID>(to_inner_uuid->as<ASTLiteral>()->value.get<String>());
query->set(query->columns_list, columns_list);
query->set(query->storage, storage);

View File

@ -1,4 +1,4 @@
#include <Server/NuKeeperTCPHandler.h>
#include <Server/KeeperTCPHandler.h>
#if USE_NURAFT
@ -189,20 +189,20 @@ struct SocketInterruptablePollWrapper
#endif
};
NuKeeperTCPHandler::NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_)
KeeperTCPHandler::KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_)
: Poco::Net::TCPServerConnection(socket_)
, server(server_)
, log(&Poco::Logger::get("NuKeeperTCPHandler"))
, log(&Poco::Logger::get("KeeperTCPHandler"))
, global_context(server.context())
, nu_keeper_storage_dispatcher(global_context.getNuKeeperStorageDispatcher())
, operation_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000)
, session_timeout(0, global_context.getConfigRef().getUInt("test_keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000)
, nu_keeper_storage_dispatcher(global_context.getKeeperStorageDispatcher())
, operation_timeout(0, global_context.getConfigRef().getUInt("keeper_server.operation_timeout_ms", Coordination::DEFAULT_OPERATION_TIMEOUT_MS) * 1000)
, session_timeout(0, global_context.getConfigRef().getUInt("keeper_server.session_timeout_ms", Coordination::DEFAULT_SESSION_TIMEOUT_MS) * 1000)
, poll_wrapper(std::make_unique<SocketInterruptablePollWrapper>(socket_))
, responses(std::make_unique<ThreadSafeResponseQueue>())
{
}
void NuKeeperTCPHandler::sendHandshake(bool has_leader)
void KeeperTCPHandler::sendHandshake(bool has_leader)
{
Coordination::write(Coordination::SERVER_HANDSHAKE_LENGTH, *out);
if (has_leader)
@ -217,12 +217,12 @@ void NuKeeperTCPHandler::sendHandshake(bool has_leader)
out->next();
}
void NuKeeperTCPHandler::run()
void KeeperTCPHandler::run()
{
runImpl();
}
Poco::Timespan NuKeeperTCPHandler::receiveHandshake()
Poco::Timespan KeeperTCPHandler::receiveHandshake()
{
int32_t handshake_length;
int32_t protocol_version;
@ -254,7 +254,7 @@ Poco::Timespan NuKeeperTCPHandler::receiveHandshake()
}
void NuKeeperTCPHandler::runImpl()
void KeeperTCPHandler::runImpl()
{
setThreadName("TstKprHandler");
ThreadStatus thread_status;
@ -393,7 +393,7 @@ void NuKeeperTCPHandler::runImpl()
}
}
std::pair<Coordination::OpNum, Coordination::XID> NuKeeperTCPHandler::receiveRequest()
std::pair<Coordination::OpNum, Coordination::XID> KeeperTCPHandler::receiveRequest()
{
int32_t length;
Coordination::read(length, *in);

View File

@ -13,7 +13,7 @@
#include <Interpreters/Context.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h>
#include <Coordination/NuKeeperStorageDispatcher.h>
#include <Coordination/KeeperStorageDispatcher.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <IO/ReadBufferFromPocoSocket.h>
#include <Coordination/ThreadSafeQueue.h>
@ -29,16 +29,16 @@ using ThreadSafeResponseQueue = ThreadSafeQueue<Coordination::ZooKeeperResponseP
using ThreadSafeResponseQueuePtr = std::unique_ptr<ThreadSafeResponseQueue>;
class NuKeeperTCPHandler : public Poco::Net::TCPServerConnection
class KeeperTCPHandler : public Poco::Net::TCPServerConnection
{
public:
NuKeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_);
KeeperTCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_);
void run() override;
private:
IServer & server;
Poco::Logger * log;
Context global_context;
std::shared_ptr<NuKeeperStorageDispatcher> nu_keeper_storage_dispatcher;
std::shared_ptr<KeeperStorageDispatcher> nu_keeper_storage_dispatcher;
Poco::Timespan operation_timeout;
Poco::Timespan session_timeout;
int64_t session_id{-1};

View File

@ -1,6 +1,6 @@
#pragma once
#include <Server/NuKeeperTCPHandler.h>
#include <Server/KeeperTCPHandler.h>
#include <Poco/Net/TCPServerConnectionFactory.h>
#include <Poco/Net/NetException.h>
#include <common/logger_useful.h>
@ -9,7 +9,7 @@
namespace DB
{
class NuKeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
class KeeperTCPHandlerFactory : public Poco::Net::TCPServerConnectionFactory
{
private:
IServer & server;
@ -21,9 +21,9 @@ private:
void run() override {}
};
public:
NuKeeperTCPHandlerFactory(IServer & server_)
KeeperTCPHandlerFactory(IServer & server_)
: server(server_)
, log(&Poco::Logger::get("NuKeeperTCPHandlerFactory"))
, log(&Poco::Logger::get("KeeperTCPHandlerFactory"))
{
}
@ -31,8 +31,8 @@ public:
{
try
{
LOG_TRACE(log, "NuKeeper request. Address: {}", socket.peerAddress().toString());
return new NuKeeperTCPHandler(server, socket);
LOG_TRACE(log, "Keeper request. Address: {}", socket.peerAddress().toString());
return new KeeperTCPHandler(server, socket);
}
catch (const Poco::Net::NetException &)
{

View File

@ -22,10 +22,10 @@ SRCS(
HTTPHandler.cpp
HTTPHandlerFactory.cpp
InterserverIOHTTPHandler.cpp
KeeperTCPHandler.cpp
MySQLHandler.cpp
MySQLHandlerFactory.cpp
NotFoundHandler.cpp
NuKeeperTCPHandler.cpp
PostgreSQLHandler.cpp
PostgreSQLHandlerFactory.cpp
PrometheusMetricsWriter.cpp

View File

@ -83,6 +83,9 @@ struct Settings;
M(UInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \
M(UInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \
M(UInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \
M(Seconds, replicated_fetches_http_connection_timeout, 0, "HTTP connection timeout for part fetch requests. Inherited from default profile `http_connection_timeout` if not set explicitly.", 0) \
M(Seconds, replicated_fetches_http_send_timeout, 0, "HTTP send timeout for part fetch requests. Inherited from default profile `http_send_timeout` if not set explicitly.", 0) \
M(Seconds, replicated_fetches_http_receive_timeout, 0, "HTTP receive timeout for fetch part requests. Inherited from default profile `http_receive_timeout` if not set explicitly.", 0) \
M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \
M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \
M(Bool, detach_old_local_parts_when_cloning_replica, 1, "Do not remove old local parts when repairing lost replica.", 0) \

View File

@ -39,7 +39,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
, queried_columns{queried_columns_}
, sorting_key_names{NameSet(
metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())}
, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)}
, block_with_constants{KeyCondition::getBlockWithConstants(query_info.query->clone(), query_info.syntax_analyzer_result, context)}
, log{log_}
, column_sizes{std::move(column_sizes_)}
{

View File

@ -907,7 +907,24 @@ ClusterPtr StorageDistributed::skipUnusedShards(
}
replaceConstantExpressions(condition_ast, context, metadata_snapshot->getColumns().getAll(), shared_from_this(), metadata_snapshot);
const auto blocks = evaluateExpressionOverConstantCondition(condition_ast, sharding_key_expr);
size_t limit = context.getSettingsRef().optimize_skip_unused_shards_limit;
if (!limit || limit > SSIZE_MAX)
{
throw Exception("optimize_skip_unused_shards_limit out of range (0, {}]", ErrorCodes::ARGUMENT_OUT_OF_BOUND, SSIZE_MAX);
}
// To interpret limit==0 as limit is reached
++limit;
const auto blocks = evaluateExpressionOverConstantCondition(condition_ast, sharding_key_expr, limit);
if (!limit)
{
LOG_TRACE(log,
"Number of values for sharding key exceeds optimize_skip_unused_shards_limit={}, "
"try to increase it, but note that this may increase query processing time.",
context.getSettingsRef().optimize_skip_unused_shards_limit);
return nullptr;
}
// Can't get definite answer if we can skip any shards
if (!blocks)

View File

@ -85,7 +85,7 @@ StorageMaterializedView::StorageMaterializedView(
else if (attach_)
{
/// If there is an ATTACH request, then the internal table must already be created.
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID()));
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID()), query.to_inner_uuid);
}
else
{
@ -94,6 +94,7 @@ StorageMaterializedView::StorageMaterializedView(
auto manual_create_query = std::make_shared<ASTCreateQuery>();
manual_create_query->database = getStorageID().database_name;
manual_create_query->table = generateInnerTableName(getStorageID());
manual_create_query->uuid = query.to_inner_uuid;
auto new_columns_list = std::make_shared<ASTColumns>();
new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr());

View File

@ -2313,7 +2313,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry)
{
String source_replica_path = zookeeper_path + "/replicas/" + part_desc->replica;
ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host"));
auto timeouts = ConnectionTimeouts::getHTTPTimeouts(global_context);
auto timeouts = getFetchPartHTTPTimeouts(global_context);
auto [user, password] = global_context.getInterserverCredentials();
String interserver_scheme = global_context.getInterserverScheme();
@ -3246,6 +3247,23 @@ void StorageReplicatedMergeTree::exitLeaderElection()
leader_election = nullptr;
}
ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(const Context & context)
{
auto timeouts = ConnectionTimeouts::getHTTPTimeouts(context);
auto settings = getSettings();
if (settings->replicated_fetches_http_connection_timeout.changed)
timeouts.connection_timeout = settings->replicated_fetches_http_connection_timeout;
if (settings->replicated_fetches_http_send_timeout.changed)
timeouts.send_timeout = settings->replicated_fetches_http_send_timeout;
if (settings->replicated_fetches_http_receive_timeout.changed)
timeouts.receive_timeout = settings->replicated_fetches_http_receive_timeout;
return timeouts;
}
bool StorageReplicatedMergeTree::checkReplicaHavePart(const String & replica, const String & part_name)
{
auto zookeeper = getZooKeeper();
@ -3661,7 +3679,8 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora
else
{
address.fromString(zookeeper->get(source_replica_path + "/host"));
timeouts = ConnectionTimeouts::getHTTPTimeouts(global_context);
timeouts = getFetchPartHTTPTimeouts(global_context);
user_password = global_context.getInterserverCredentials();
interserver_scheme = global_context.getInterserverScheme();

View File

@ -507,6 +507,8 @@ private:
/// Exchange parts.
ConnectionTimeouts getFetchPartHTTPTimeouts(const Context & context);
/** Returns an empty string if no one has a part.
*/
String findReplicaHavingPart(const String & part_name, bool active);

View File

@ -359,6 +359,7 @@ protected:
{
auto & create = ast->as<ASTCreateQuery &>();
create.uuid = UUIDHelpers::Nil;
create.to_inner_uuid = UUIDHelpers::Nil;
}
if (columns_mask[src_index++])

View File

@ -16,5 +16,17 @@
</replica>
</shard>
</test_cluster_two_shards_different_databases>
<test_cluster_one_shard_two_replicas>
<shard>
<replica>
<host>127.0.0.1</host>
<port>9000</port>
</replica>
<replica>
<host>127.0.0.2</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_one_shard_two_replicas>
</remote_servers>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
@ -17,5 +17,5 @@
<port>44444</port>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -29,7 +29,7 @@ ln -sf $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/database_atomic.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/max_concurrent_queries.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/test_cluster_with_incorrect_pw.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/test_keeper_port.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/keeper_port.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/logging_no_rotate.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/

View File

@ -2,9 +2,11 @@
<profiles>
<default>
<allow_experimental_database_replicated>1</allow_experimental_database_replicated>
<database_replicated_ddl_output>0</database_replicated_ddl_output>
<distributed_ddl_output_mode>none</distributed_ddl_output_mode>
<database_replicated_initial_query_timeout_sec>30</database_replicated_initial_query_timeout_sec>
<distributed_ddl_task_timeout>30</distributed_ddl_task_timeout>
<database_replicated_always_detach_permanently>1</database_replicated_always_detach_permanently>
<distributed_ddl_entry_format_version>2</distributed_ddl_entry_format_version>
</default>
</profiles>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -19,5 +19,5 @@
<port>44444</port>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -7,7 +7,7 @@ import time
from multiprocessing.dummy import Pool
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=['configs/enable_test_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True)
node = cluster.add_instance('node', main_configs=['configs/enable_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True)
from kazoo.client import KazooClient, KazooState, KeeperState
def get_genuine_zk():

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -9,9 +9,9 @@ from helpers.network import PartitionManager
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
from kazoo.client import KazooClient, KazooState

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>2</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>3</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -37,5 +37,5 @@
<priority>1</priority>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -9,9 +9,9 @@ from helpers.network import PartitionManager
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node1 = cluster.add_instance('node1', main_configs=['configs/enable_keeper1.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
node2 = cluster.add_instance('node2', main_configs=['configs/enable_keeper2.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
node3 = cluster.add_instance('node3', main_configs=['configs/enable_keeper3.xml', 'configs/log_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
from kazoo.client import KazooClient, KazooState

View File

@ -1,5 +1,5 @@
<yandex>
<test_keeper_server>
<keeper_server>
<tcp_port>9181</tcp_port>
<server_id>1</server_id>
<log_storage_path>/var/lib/clickhouse/coordination/log</log_storage_path>
@ -18,5 +18,5 @@
<port>44444</port>
</server>
</raft_configuration>
</test_keeper_server>
</keeper_server>
</yandex>

View File

@ -10,7 +10,7 @@ from kazoo.client import KazooClient, KazooState
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=['configs/enable_test_keeper.xml', 'configs/logs_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True)
node = cluster.add_instance('node', main_configs=['configs/enable_keeper.xml', 'configs/logs_conf.xml', 'configs/use_keeper.xml'], stay_alive=True)
def random_string(length):

Some files were not shown because too many files have changed in this diff Show More