mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge branch 'master' into filesystemCacheSizeLimitMetric
This commit is contained in:
commit
6f71adcca2
1
.gitignore
vendored
1
.gitignore
vendored
@ -161,6 +161,7 @@ tests/queries/0_stateless/test_*
|
||||
tests/queries/0_stateless/*.binary
|
||||
tests/queries/0_stateless/*.generated-expect
|
||||
tests/queries/0_stateless/*.expect.history
|
||||
tests/integration/**/_gen
|
||||
|
||||
# rust
|
||||
/rust/**/target
|
||||
|
@ -448,7 +448,7 @@ inline char * find_last_not_symbols_or_null(char * begin, char * end)
|
||||
/// See https://github.com/boostorg/algorithm/issues/63
|
||||
/// And https://bugs.llvm.org/show_bug.cgi?id=41141
|
||||
template <char... symbols, typename To>
|
||||
inline void splitInto(To & to, const std::string & what, bool token_compress = false)
|
||||
inline To & splitInto(To & to, std::string_view what, bool token_compress = false)
|
||||
{
|
||||
const char * pos = what.data();
|
||||
const char * end = pos + what.size();
|
||||
@ -464,4 +464,6 @@ inline void splitInto(To & to, const std::string & what, bool token_compress = f
|
||||
else
|
||||
pos = delimiter_or_end;
|
||||
}
|
||||
|
||||
return to;
|
||||
}
|
||||
|
9
base/base/move_extend.h
Normal file
9
base/base/move_extend.h
Normal file
@ -0,0 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
/// Extend @p to by moving elements from @p from to @p to end
|
||||
/// @return @p to iterator to first of moved elements.
|
||||
template <class To, class From>
|
||||
typename To::iterator moveExtend(To & to, From && from)
|
||||
{
|
||||
return to.insert(to.end(), std::make_move_iterator(from.begin()), std::make_move_iterator(from.end()));
|
||||
}
|
2
contrib/NuRaft
vendored
2
contrib/NuRaft
vendored
@ -1 +1 @@
|
||||
Subproject commit 491eaf592d950e0e37accbe8b3f217e068c9fecf
|
||||
Subproject commit eb1572129c71beb2156dcdaadc3fb136954aed96
|
@ -135,4 +135,5 @@ ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'
|
||||
|
||||
EXPOSE 2375
|
||||
ENTRYPOINT ["dockerd-entrypoint.sh"]
|
||||
CMD ["sh", "-c", "pytest $PYTEST_OPTS"]
|
||||
# To pass additional arguments (i.e. list of tests) use PYTEST_ADDOPTS
|
||||
CMD ["sh", "-c", "pytest"]
|
||||
|
@ -537,6 +537,8 @@ Possible values:
|
||||
|
||||
The first phase of a grace join reads the right table and splits it into N buckets depending on the hash value of key columns (initially, N is `grace_hash_join_initial_buckets`). This is done in a way to ensure that each bucket can be processed independently. Rows from the first bucket are added to an in-memory hash table while the others are saved to disk. If the hash table grows beyond the memory limit (e.g., as set by [`max_bytes_in_join`](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join)), the number of buckets is increased and the assigned bucket for each row. Any rows which don’t belong to the current bucket are flushed and reassigned.
|
||||
|
||||
Supports `INNER/LEFT/RIGHT/FULL ALL/ANY JOIN`.
|
||||
|
||||
- hash
|
||||
|
||||
[Hash join algorithm](https://en.wikipedia.org/wiki/Hash_join) is used. The most generic implementation that supports all combinations of kind and strictness and multiple join keys that are combined with `OR` in the `JOIN ON` section.
|
||||
|
@ -205,7 +205,7 @@ The optional keyword `EXTENDED` currently has no effect, it only exists for MySQ
|
||||
|
||||
The optional keyword `FULL` causes the output to include the collation, comment and privilege columns.
|
||||
|
||||
`SHOW COLUMNS` produces a result table with the following structure:
|
||||
The statement produces a result table with the following structure:
|
||||
- field - The name of the column (String)
|
||||
- type - The column data type (String)
|
||||
- null - If the column data type is Nullable (UInt8)
|
||||
@ -272,6 +272,10 @@ SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2
|
||||
|
||||
Displays a list of primary and data skipping indexes of a table.
|
||||
|
||||
This statement mostly exists for compatibility with MySQL. System tables [system.tables](../../operations/system-tables/tables.md) (for
|
||||
primary keys) and [system.data_skipping_indices](../../operations/system-tables/data_skipping_indices.md) (for data skipping indices)
|
||||
provide equivalent information but in a fashion more native to ClickHouse.
|
||||
|
||||
```sql
|
||||
SHOW [EXTENDED] {INDEX | INDEXES | INDICES | KEYS } {FROM | IN} <table> [{FROM | IN} <db>] [WHERE <expr>] [INTO OUTFILE <filename>] [FORMAT <format>]
|
||||
```
|
||||
@ -281,22 +285,22 @@ equivalent. If no database is specified, the query assumes the current database
|
||||
|
||||
The optional keyword `EXTENDED` currently has no effect, it only exists for MySQL compatibility.
|
||||
|
||||
`SHOW INDEX` produces a result table with the following structure:
|
||||
- table - The name of the table (String)
|
||||
- non_unique - 0 if the index cannot contain duplicates, 1 otherwise (UInt8)
|
||||
- key_name - The name of the index, `PRIMARY` if the index is a primary key index (String)
|
||||
- seq_in_index - Currently unused
|
||||
- column_name - Currently unused
|
||||
- collation - The sorting of the column in the index, `A` if ascending, `D` if descending, `NULL` if unsorted (Nullable(String))
|
||||
- cardinality - Currently unused
|
||||
- sub_part - Currently unused
|
||||
- packed - Currently unused
|
||||
The statement produces a result table with the following structure:
|
||||
- table - The name of the table. (String)
|
||||
- non_unique - Always `1` as ClickHouse does not support uniqueness constraints. (UInt8)
|
||||
- key_name - The name of the index, `PRIMARY` if the index is a primary key index. (String)
|
||||
- seq_in_index - For a primary key index, the position of the column starting from `1`. For a data skipping index: always `1`. (UInt8)
|
||||
- column_name - For a primary key index, the name of the column. For a data skipping index: `''` (empty string), see field "expression". (String)
|
||||
- collation - The sorting of the column in the index: `A` if ascending, `D` if descending, `NULL` if unsorted. (Nullable(String))
|
||||
- cardinality - An estimation of the index cardinality (number of unique values in the index). Currently always 0. (UInt64)
|
||||
- sub_part - Always `NULL` because ClickHouse does not support index prefixes like MySQL. (Nullable(String))
|
||||
- packed - Always `NULL` because ClickHouse does not support packed indexes (like MySQL). (Nullable(String))
|
||||
- null - Currently unused
|
||||
- index_type - The index type, e.g. `primary`, `minmax`, `bloom_filter` etc. (String)
|
||||
- comment - Currently unused
|
||||
- index_comment - Currently unused
|
||||
- visible - If the index is visible to the optimizer, always `YES` (String)
|
||||
- expression - The index expression (String)
|
||||
- index_type - The index type, e.g. `PRIMARY`, `MINMAX`, `BLOOM_FILTER` etc. (String)
|
||||
- comment - Additional information about the index, currently always `''` (empty string). (String)
|
||||
- index_comment - `''` (empty string) because indexes in ClickHouse cannot have a `COMMENT` field (like in MySQL). (String)
|
||||
- visible - If the index is visible to the optimizer, always `YES`. (String)
|
||||
- expression - For a data skipping index, the index expression. For a primary key index: `''` (empty string). (String)
|
||||
|
||||
**Examples**
|
||||
|
||||
@ -310,11 +314,12 @@ Result:
|
||||
|
||||
``` text
|
||||
┌─table─┬─non_unique─┬─key_name─┬─seq_in_index─┬─column_name─┬─collation─┬─cardinality─┬─sub_part─┬─packed─┬─null─┬─index_type───┬─comment─┬─index_comment─┬─visible─┬─expression─┐
|
||||
│ tbl │ 0 │ blf_idx │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ bloom_filter │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ YES │ d, b │
|
||||
│ tbl │ 0 │ mm1_idx │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ minmax │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ YES │ a, c, d │
|
||||
│ tbl │ 0 │ mm2_idx │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ minmax │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ YES │ c, d, e │
|
||||
│ tbl │ 0 │ PRIMARY │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ A │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ primary │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ YES │ c, a │
|
||||
│ tbl │ 0 │ set_idx │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ set │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ YES │ e │
|
||||
│ tbl │ 1 │ blf_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ BLOOM_FILTER │ │ │ YES │ d, b │
|
||||
│ tbl │ 1 │ mm1_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ a, c, d │
|
||||
│ tbl │ 1 │ mm2_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ MINMAX │ │ │ YES │ c, d, e │
|
||||
│ tbl │ 1 │ PRIMARY │ 1 │ c │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │
|
||||
│ tbl │ 1 │ PRIMARY │ 2 │ a │ A │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ PRIMARY │ │ │ YES │ │
|
||||
│ tbl │ 1 │ set_idx │ 1 │ 1 │ ᴺᵁᴸᴸ │ 0 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ SET │ │ │ YES │ e │
|
||||
└───────┴────────────┴──────────┴──────────────┴─────────────┴───────────┴─────────────┴──────────┴────────┴──────┴──────────────┴─────────┴───────────────┴─────────┴────────────┘
|
||||
```
|
||||
|
||||
|
@ -29,6 +29,7 @@ EnvironmentFile=-/etc/default/clickhouse
|
||||
LimitCORE=infinity
|
||||
LimitNOFILE=500000
|
||||
CapabilityBoundingSet=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE
|
||||
AmbientCapabilities=CAP_NET_ADMIN CAP_IPC_LOCK CAP_SYS_NICE CAP_NET_BIND_SERVICE
|
||||
|
||||
[Install]
|
||||
# ClickHouse should not start from the rescue shell (rescue.target).
|
||||
|
@ -34,6 +34,8 @@ add_dependencies(clickhouse-keeper-lib clickhouse_keeper_configs)
|
||||
if (BUILD_STANDALONE_KEEPER)
|
||||
# Straight list of all required sources
|
||||
set(CLICKHOUSE_KEEPER_STANDALONE_SOURCES
|
||||
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/KeeperReconfiguration.cpp
|
||||
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/RaftServerConfig.cpp
|
||||
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/ACLMap.cpp
|
||||
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/Changelog.cpp
|
||||
${CMAKE_CURRENT_SOURCE_DIR}/../../src/Coordination/CoordinationSettings.cpp
|
||||
|
@ -1601,13 +1601,7 @@ try
|
||||
global_context->setSystemZooKeeperLogAfterInitializationIfNeeded();
|
||||
/// Build loggers before tables startup to make log messages from tables
|
||||
/// attach available in system.text_log
|
||||
{
|
||||
String level_str = config().getString("text_log.level", "");
|
||||
int level = level_str.empty() ? INT_MAX : Poco::Logger::parseLevel(level_str);
|
||||
setTextLog(global_context->getTextLog(), level);
|
||||
|
||||
buildLoggers(config(), logger());
|
||||
}
|
||||
buildLoggers(config(), logger());
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||
attachSystemTablesServer(global_context, *database_catalog.getSystemDatabase(), has_zookeeper);
|
||||
attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA));
|
||||
|
@ -125,6 +125,7 @@
|
||||
M(ZooKeeperMulti, "Number of 'multi' requests to ZooKeeper (compound transactions).") \
|
||||
M(ZooKeeperCheck, "Number of 'check' requests to ZooKeeper. Usually they don't make sense in isolation, only as part of a complex transaction.") \
|
||||
M(ZooKeeperSync, "Number of 'sync' requests to ZooKeeper. These requests are rarely needed or usable.") \
|
||||
M(ZooKeeperReconfig, "Number of 'reconfig' requests to ZooKeeper.") \
|
||||
M(ZooKeeperClose, "Number of times connection with ZooKeeper has been closed voluntary.") \
|
||||
M(ZooKeeperWatchResponse, "Number of times watch notification has been received from ZooKeeper.") \
|
||||
M(ZooKeeperUserExceptions, "Number of exceptions while working with ZooKeeper related to the data (no node, bad version or similar).") \
|
||||
@ -503,6 +504,7 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(KeeperCreateRequest, "Number of create requests")\
|
||||
M(KeeperRemoveRequest, "Number of remove requests")\
|
||||
M(KeeperSetRequest, "Number of set requests")\
|
||||
M(KeeperReconfigRequest, "Number of reconfig requests")\
|
||||
M(KeeperCheckRequest, "Number of check requests")\
|
||||
M(KeeperMultiRequest, "Number of multi requests")\
|
||||
M(KeeperMultiReadRequest, "Number of multi read requests")\
|
||||
|
@ -38,43 +38,30 @@ namespace
|
||||
|
||||
ISystemLog::~ISystemLog() = default;
|
||||
|
||||
void ISystemLog::stopFlushThread()
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
|
||||
if (!saving_thread || !saving_thread->joinable())
|
||||
return;
|
||||
|
||||
if (is_shutdown)
|
||||
return;
|
||||
|
||||
is_shutdown = true;
|
||||
|
||||
/// Tell thread to shutdown.
|
||||
flush_event.notify_all();
|
||||
}
|
||||
|
||||
saving_thread->join();
|
||||
}
|
||||
|
||||
void ISystemLog::startup()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
saving_thread = std::make_unique<ThreadFromGlobalPool>([this] { savingThreadFunction(); });
|
||||
}
|
||||
|
||||
static thread_local bool recursive_add_call = false;
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::add(const LogElement & element)
|
||||
SystemLogQueue<LogElement>::SystemLogQueue(
|
||||
const String & table_name_,
|
||||
size_t flush_interval_milliseconds_,
|
||||
bool turn_off_logger_)
|
||||
: log(&Poco::Logger::get("SystemLogQueue (" + table_name_ + ")"))
|
||||
, flush_interval_milliseconds(flush_interval_milliseconds_)
|
||||
{
|
||||
if (turn_off_logger_)
|
||||
log->setLevel(0);
|
||||
}
|
||||
|
||||
static thread_local bool recursive_push_call = false;
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogQueue<LogElement>::push(const LogElement & element)
|
||||
{
|
||||
/// It is possible that the method will be called recursively.
|
||||
/// Better to drop these events to avoid complications.
|
||||
if (recursive_add_call)
|
||||
if (recursive_push_call)
|
||||
return;
|
||||
recursive_add_call = true;
|
||||
SCOPE_EXIT({ recursive_add_call = false; });
|
||||
recursive_push_call = true;
|
||||
SCOPE_EXIT({ recursive_push_call = false; });
|
||||
|
||||
/// Memory can be allocated while resizing on queue.push_back.
|
||||
/// The size of allocation can be in order of a few megabytes.
|
||||
@ -137,10 +124,16 @@ void SystemLogBase<LogElement>::add(const LogElement & element)
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::flush(bool force)
|
||||
{
|
||||
uint64_t this_thread_requested_offset = notifyFlushImpl(force);
|
||||
uint64_t this_thread_requested_offset = queue->notifyFlush(force);
|
||||
if (this_thread_requested_offset == uint64_t(-1))
|
||||
return;
|
||||
|
||||
queue->waitFlush(this_thread_requested_offset);
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogQueue<LogElement>::waitFlush(uint64_t expected_flushed_up_to)
|
||||
{
|
||||
// Use an arbitrary timeout to avoid endless waiting. 60s proved to be
|
||||
// too fast for our parallel functional tests, probably because they
|
||||
// heavily load the disk.
|
||||
@ -148,7 +141,7 @@ void SystemLogBase<LogElement>::flush(bool force)
|
||||
std::unique_lock lock(mutex);
|
||||
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&]
|
||||
{
|
||||
return flushed_up_to >= this_thread_requested_offset && !is_force_prepare_tables;
|
||||
return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables;
|
||||
});
|
||||
|
||||
if (!result)
|
||||
@ -159,10 +152,7 @@ void SystemLogBase<LogElement>::flush(bool force)
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::notifyFlush(bool force) { notifyFlushImpl(force); }
|
||||
|
||||
template <typename LogElement>
|
||||
uint64_t SystemLogBase<LogElement>::notifyFlushImpl(bool force)
|
||||
uint64_t SystemLogQueue<LogElement>::notifyFlush(bool should_prepare_tables_anyway)
|
||||
{
|
||||
uint64_t this_thread_requested_offset;
|
||||
|
||||
@ -175,7 +165,7 @@ uint64_t SystemLogBase<LogElement>::notifyFlushImpl(bool force)
|
||||
|
||||
// Publish our flush request, taking care not to overwrite the requests
|
||||
// made by other threads.
|
||||
is_force_prepare_tables |= force;
|
||||
is_force_prepare_tables |= should_prepare_tables_anyway;
|
||||
requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset);
|
||||
|
||||
flush_event.notify_all();
|
||||
@ -185,7 +175,77 @@ uint64_t SystemLogBase<LogElement>::notifyFlushImpl(bool force)
|
||||
return this_thread_requested_offset;
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogQueue<LogElement>::confirm(uint64_t to_flush_end)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
flushed_up_to = to_flush_end;
|
||||
is_force_prepare_tables = false;
|
||||
flush_event.notify_all();
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::pop(std::vector<LogElement>& output, bool& should_prepare_tables_anyway, bool& exit_this_thread)
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
flush_event.wait_for(lock,
|
||||
std::chrono::milliseconds(flush_interval_milliseconds),
|
||||
[&] ()
|
||||
{
|
||||
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
|
||||
}
|
||||
);
|
||||
|
||||
queue_front_index += queue.size();
|
||||
// Swap with existing array from previous flush, to save memory
|
||||
// allocations.
|
||||
output.resize(0);
|
||||
queue.swap(output);
|
||||
|
||||
should_prepare_tables_anyway = is_force_prepare_tables;
|
||||
|
||||
exit_this_thread = is_shutdown;
|
||||
return queue_front_index;
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogQueue<LogElement>::shutdown()
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
is_shutdown = true;
|
||||
/// Tell thread to shutdown.
|
||||
flush_event.notify_all();
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
SystemLogBase<LogElement>::SystemLogBase(
|
||||
const String& table_name_,
|
||||
size_t flush_interval_milliseconds_,
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue_)
|
||||
: queue(queue_ ? queue_ : std::make_shared<SystemLogQueue<LogElement>>(table_name_, flush_interval_milliseconds_))
|
||||
{
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::startup()
|
||||
{
|
||||
std::lock_guard lock(thread_mutex);
|
||||
saving_thread = std::make_unique<ThreadFromGlobalPool>([this] { savingThreadFunction(); });
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::add(const LogElement & element)
|
||||
{
|
||||
queue->push(element);
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLogBase<LogElement>::notifyFlush(bool force) { queue->notifyFlush(force); }
|
||||
|
||||
#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase<ELEMENT>;
|
||||
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE)
|
||||
|
||||
#define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue<ELEMENT>;
|
||||
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE)
|
||||
|
||||
}
|
||||
|
@ -55,33 +55,88 @@ public:
|
||||
virtual void prepareTable() = 0;
|
||||
|
||||
/// Start the background thread.
|
||||
virtual void startup();
|
||||
virtual void startup() = 0;
|
||||
|
||||
/// Stop the background flush thread before destructor. No more data will be written.
|
||||
virtual void shutdown() = 0;
|
||||
|
||||
virtual void stopFlushThread() = 0;
|
||||
|
||||
virtual ~ISystemLog();
|
||||
|
||||
virtual void savingThreadFunction() = 0;
|
||||
|
||||
protected:
|
||||
std::mutex thread_mutex;
|
||||
std::unique_ptr<ThreadFromGlobalPool> saving_thread;
|
||||
|
||||
bool is_shutdown = false;
|
||||
};
|
||||
|
||||
template <typename LogElement>
|
||||
class SystemLogQueue
|
||||
{
|
||||
using Index = uint64_t;
|
||||
|
||||
public:
|
||||
SystemLogQueue(
|
||||
const String & table_name_,
|
||||
size_t flush_interval_milliseconds_,
|
||||
bool turn_off_logger_ = false);
|
||||
|
||||
void shutdown();
|
||||
|
||||
// producer methods
|
||||
void push(const LogElement & element);
|
||||
Index notifyFlush(bool should_prepare_tables_anyway);
|
||||
void waitFlush(Index expected_flushed_up_to);
|
||||
|
||||
// consumer methods
|
||||
Index pop(std::vector<LogElement>& output, bool& should_prepare_tables_anyway, bool& exit_this_thread);
|
||||
void confirm(Index to_flush_end);
|
||||
|
||||
private:
|
||||
/// Data shared between callers of add()/flush()/shutdown(), and the saving thread
|
||||
std::mutex mutex;
|
||||
|
||||
bool is_shutdown = false;
|
||||
std::condition_variable flush_event;
|
||||
Poco::Logger * log;
|
||||
|
||||
void stopFlushThread();
|
||||
// Queue is bounded. But its size is quite large to not block in all normal cases.
|
||||
std::vector<LogElement> queue;
|
||||
// An always-incrementing index of the first message currently in the queue.
|
||||
// We use it to give a global sequential index to every message, so that we
|
||||
// can wait until a particular message is flushed. This is used to implement
|
||||
// synchronous log flushing for SYSTEM FLUSH LOGS.
|
||||
Index queue_front_index = 0;
|
||||
// A flag that says we must create the tables even if the queue is empty.
|
||||
bool is_force_prepare_tables = false;
|
||||
// Requested to flush logs up to this index, exclusive
|
||||
Index requested_flush_up_to = 0;
|
||||
// Flushed log up to this index, exclusive
|
||||
Index flushed_up_to = 0;
|
||||
// Logged overflow message at this queue front index
|
||||
Index logged_queue_full_at_index = -1;
|
||||
|
||||
bool is_shutdown = false;
|
||||
|
||||
std::condition_variable flush_event;
|
||||
const size_t flush_interval_milliseconds;
|
||||
};
|
||||
|
||||
|
||||
template <typename LogElement>
|
||||
class SystemLogBase : public ISystemLog
|
||||
{
|
||||
public:
|
||||
using Self = SystemLogBase;
|
||||
|
||||
SystemLogBase(
|
||||
const String& table_name_,
|
||||
size_t flush_interval_milliseconds_,
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue_ = nullptr);
|
||||
|
||||
void startup() override;
|
||||
|
||||
/** Append a record into log.
|
||||
* Writing to table will be done asynchronously and in case of failure, record could be lost.
|
||||
*/
|
||||
@ -98,27 +153,6 @@ public:
|
||||
static const char * getDefaultOrderBy() { return "event_date, event_time"; }
|
||||
|
||||
protected:
|
||||
Poco::Logger * log;
|
||||
|
||||
// Queue is bounded. But its size is quite large to not block in all normal cases.
|
||||
std::vector<LogElement> queue;
|
||||
// An always-incrementing index of the first message currently in the queue.
|
||||
// We use it to give a global sequential index to every message, so that we
|
||||
// can wait until a particular message is flushed. This is used to implement
|
||||
// synchronous log flushing for SYSTEM FLUSH LOGS.
|
||||
uint64_t queue_front_index = 0;
|
||||
// A flag that says we must create the tables even if the queue is empty.
|
||||
bool is_force_prepare_tables = false;
|
||||
// Requested to flush logs up to this index, exclusive
|
||||
uint64_t requested_flush_up_to = 0;
|
||||
// Flushed log up to this index, exclusive
|
||||
uint64_t flushed_up_to = 0;
|
||||
// Logged overflow message at this queue front index
|
||||
uint64_t logged_queue_full_at_index = -1;
|
||||
|
||||
private:
|
||||
uint64_t notifyFlushImpl(bool force);
|
||||
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -350,6 +350,29 @@ struct SyncResponse : virtual Response
|
||||
size_t bytesSize() const override { return path.size(); }
|
||||
};
|
||||
|
||||
struct ReconfigRequest : virtual Request
|
||||
{
|
||||
String joining;
|
||||
String leaving;
|
||||
String new_members;
|
||||
int32_t version;
|
||||
|
||||
String getPath() const final { return keeper_config_path; }
|
||||
|
||||
size_t bytesSize() const final
|
||||
{
|
||||
return joining.size() + leaving.size() + new_members.size() + sizeof(version);
|
||||
}
|
||||
};
|
||||
|
||||
struct ReconfigResponse : virtual Response
|
||||
{
|
||||
String value;
|
||||
Stat stat;
|
||||
|
||||
size_t bytesSize() const override { return value.size() + sizeof(stat); }
|
||||
};
|
||||
|
||||
struct MultiRequest : virtual Request
|
||||
{
|
||||
Requests requests;
|
||||
@ -395,9 +418,9 @@ using SetCallback = std::function<void(const SetResponse &)>;
|
||||
using ListCallback = std::function<void(const ListResponse &)>;
|
||||
using CheckCallback = std::function<void(const CheckResponse &)>;
|
||||
using SyncCallback = std::function<void(const SyncResponse &)>;
|
||||
using ReconfigCallback = std::function<void(const ReconfigResponse &)>;
|
||||
using MultiCallback = std::function<void(const MultiResponse &)>;
|
||||
|
||||
|
||||
/// For watches.
|
||||
enum State
|
||||
{
|
||||
@ -526,6 +549,13 @@ public:
|
||||
const String & path,
|
||||
SyncCallback callback) = 0;
|
||||
|
||||
virtual void reconfig(
|
||||
std::string_view joining,
|
||||
std::string_view leaving,
|
||||
std::string_view new_members,
|
||||
int32_t version,
|
||||
ReconfigCallback callback) = 0;
|
||||
|
||||
virtual void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback) = 0;
|
||||
@ -539,3 +569,11 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
template <> struct fmt::formatter<Coordination::Error> : fmt::formatter<std::string_view>
|
||||
{
|
||||
constexpr auto format(Coordination::Error code, auto & ctx)
|
||||
{
|
||||
return formatter<string_view>::format(Coordination::errorMessage(code), ctx);
|
||||
}
|
||||
};
|
||||
|
@ -3,12 +3,8 @@
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
#include <functional>
|
||||
|
||||
|
||||
namespace Coordination
|
||||
{
|
||||
|
||||
@ -147,6 +143,14 @@ struct TestKeeperSyncRequest final : SyncRequest, TestKeeperRequest
|
||||
std::pair<ResponsePtr, Undo> process(TestKeeper::Container & container, int64_t zxid) const override;
|
||||
};
|
||||
|
||||
struct TestKeeperReconfigRequest final : ReconfigRequest, TestKeeperRequest
|
||||
{
|
||||
TestKeeperReconfigRequest() = default;
|
||||
explicit TestKeeperReconfigRequest(const ReconfigRequest & base) : ReconfigRequest(base) {}
|
||||
ResponsePtr createResponse() const override;
|
||||
std::pair<ResponsePtr, Undo> process(TestKeeper::Container & container, int64_t zxid) const override;
|
||||
};
|
||||
|
||||
struct TestKeeperMultiRequest final : MultiRequest, TestKeeperRequest
|
||||
{
|
||||
explicit TestKeeperMultiRequest(const Requests & generic_requests)
|
||||
@ -226,15 +230,7 @@ std::pair<ResponsePtr, Undo> TestKeeperCreateRequest::process(TestKeeper::Contai
|
||||
std::string path_created = path;
|
||||
|
||||
if (is_sequential)
|
||||
{
|
||||
auto seq_num = it->second.seq_num;
|
||||
|
||||
std::stringstream seq_num_str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
seq_num_str.exceptions(std::ios::failbit);
|
||||
seq_num_str << std::setw(10) << std::setfill('0') << seq_num;
|
||||
|
||||
path_created += seq_num_str.str();
|
||||
}
|
||||
path_created += fmt::format("{:0>10}", it->second.seq_num);
|
||||
|
||||
/// Increment sequential number even if node is not sequential
|
||||
++it->second.seq_num;
|
||||
@ -446,6 +442,17 @@ std::pair<ResponsePtr, Undo> TestKeeperSyncRequest::process(TestKeeper::Containe
|
||||
return { std::make_shared<SyncResponse>(std::move(response)), {} };
|
||||
}
|
||||
|
||||
std::pair<ResponsePtr, Undo> TestKeeperReconfigRequest::process(TestKeeper::Container &, int64_t) const
|
||||
{
|
||||
// In TestKeeper we assume data is stored on one server, so this is a dummy implementation to
|
||||
// satisfy IKeeper interface.
|
||||
// We can't even check the validity of input data, neither can we create the /keeper/config znode
|
||||
// as we don't know the id of current "server".
|
||||
ReconfigResponse response;
|
||||
response.error = Error::ZOK;
|
||||
return { std::make_shared<ReconfigResponse>(std::move(response)), {} };
|
||||
}
|
||||
|
||||
std::pair<ResponsePtr, Undo> TestKeeperMultiRequest::process(TestKeeper::Container & container, int64_t zxid) const
|
||||
{
|
||||
MultiResponse response;
|
||||
@ -505,6 +512,7 @@ ResponsePtr TestKeeperSetRequest::createResponse() const { return std::make_shar
|
||||
ResponsePtr TestKeeperListRequest::createResponse() const { return std::make_shared<ListResponse>(); }
|
||||
ResponsePtr TestKeeperCheckRequest::createResponse() const { return std::make_shared<CheckResponse>(); }
|
||||
ResponsePtr TestKeeperSyncRequest::createResponse() const { return std::make_shared<SyncResponse>(); }
|
||||
ResponsePtr TestKeeperReconfigRequest::createResponse() const { return std::make_shared<ReconfigResponse>(); }
|
||||
ResponsePtr TestKeeperMultiRequest::createResponse() const { return std::make_shared<MultiResponse>(); }
|
||||
|
||||
|
||||
@ -828,6 +836,28 @@ void TestKeeper::sync(
|
||||
pushRequest(std::move(request_info));
|
||||
}
|
||||
|
||||
void TestKeeper::reconfig(
|
||||
std::string_view joining,
|
||||
std::string_view leaving,
|
||||
std::string_view new_members,
|
||||
int32_t version,
|
||||
ReconfigCallback callback)
|
||||
{
|
||||
TestKeeperReconfigRequest req;
|
||||
req.joining = joining;
|
||||
req.leaving = leaving;
|
||||
req.new_members = new_members;
|
||||
req.version = version;
|
||||
|
||||
pushRequest({
|
||||
.request = std::make_shared<TestKeeperReconfigRequest>(std::move(req)),
|
||||
.callback = [callback](const Response & response)
|
||||
{
|
||||
callback(dynamic_cast<const ReconfigResponse &>(response));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
void TestKeeper::multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback)
|
||||
|
@ -87,6 +87,13 @@ public:
|
||||
const String & path,
|
||||
SyncCallback callback) override;
|
||||
|
||||
void reconfig(
|
||||
std::string_view joining,
|
||||
std::string_view leaving,
|
||||
std::string_view new_members,
|
||||
int32_t version,
|
||||
ReconfigCallback callback) final;
|
||||
|
||||
void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback) override;
|
||||
|
@ -77,13 +77,14 @@ void ZooKeeper::init(ZooKeeperArgs args_)
|
||||
auto & host_string = host.host;
|
||||
try
|
||||
{
|
||||
bool secure = startsWith(host_string, "secure://");
|
||||
const bool secure = startsWith(host_string, "secure://");
|
||||
|
||||
if (secure)
|
||||
host_string.erase(0, strlen("secure://"));
|
||||
|
||||
LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, Poco::Net::SocketAddress{host_string}.toString());
|
||||
nodes.emplace_back(Coordination::ZooKeeper::Node{Poco::Net::SocketAddress{host_string}, secure});
|
||||
const Poco::Net::SocketAddress host_socket_addr{host_string};
|
||||
LOG_TEST(log, "Adding ZooKeeper host {} ({})", host_string, host_socket_addr.toString());
|
||||
nodes.emplace_back(Coordination::ZooKeeper::Node{host_socket_addr, secure});
|
||||
}
|
||||
catch (const Poco::Net::HostNotFoundException & e)
|
||||
{
|
||||
@ -193,12 +194,7 @@ std::vector<ShuffleHost> ZooKeeper::shuffleHosts() const
|
||||
shuffle_hosts.emplace_back(shuffle_host);
|
||||
}
|
||||
|
||||
::sort(
|
||||
shuffle_hosts.begin(), shuffle_hosts.end(),
|
||||
[](const ShuffleHost & lhs, const ShuffleHost & rhs)
|
||||
{
|
||||
return ShuffleHost::compare(lhs, rhs);
|
||||
});
|
||||
::sort(shuffle_hosts.begin(), shuffle_hosts.end(), ShuffleHost::compare);
|
||||
|
||||
return shuffle_hosts;
|
||||
}
|
||||
@ -233,7 +229,7 @@ Coordination::Error ZooKeeper::getChildrenImpl(const std::string & path, Strings
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::List), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::List, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -300,7 +296,7 @@ Coordination::Error ZooKeeper::createImpl(const std::string & path, const std::s
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Create), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Create, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -415,7 +411,7 @@ Coordination::Error ZooKeeper::removeImpl(const std::string & path, int32_t vers
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Remove), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Remove, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -447,7 +443,7 @@ Coordination::Error ZooKeeper::existsImpl(const std::string & path, Coordination
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Exists), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Exists, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -481,7 +477,7 @@ Coordination::Error ZooKeeper::getImpl(const std::string & path, std::string & r
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Get), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Get, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -553,7 +549,7 @@ Coordination::Error ZooKeeper::setImpl(const std::string & path, const std::stri
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Set), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Set, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -605,7 +601,7 @@ Coordination::Error ZooKeeper::multiImpl(const Coordination::Requests & requests
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Multi), requests[0]->getPath()));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Multi, requests[0]->getPath()));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -639,7 +635,7 @@ Coordination::Error ZooKeeper::syncImpl(const std::string & path, std::string &
|
||||
|
||||
if (future_result.wait_for(std::chrono::milliseconds(args.operation_timeout_ms)) != std::future_status::ready)
|
||||
{
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", toString(Coordination::OpNum::Sync), path));
|
||||
impl->finalize(fmt::format("Operation timeout on {} {}", Coordination::OpNum::Sync, path));
|
||||
return Coordination::Error::ZOPERATIONTIMEOUT;
|
||||
}
|
||||
else
|
||||
@ -1251,7 +1247,7 @@ size_t getFailedOpIndex(Coordination::Error exception_code, const Coordination::
|
||||
if (!Coordination::isUserError(exception_code))
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR,
|
||||
"There are no failed OPs because '{}' is not valid response code for that",
|
||||
std::string(Coordination::errorMessage(exception_code)));
|
||||
exception_code);
|
||||
|
||||
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is no failed OpResult");
|
||||
}
|
||||
|
@ -36,7 +36,7 @@ std::string ZooKeeperRequest::toString() const
|
||||
"OpNum = {}\n"
|
||||
"Additional info:\n{}",
|
||||
xid,
|
||||
Coordination::toString(getOpNum()),
|
||||
getOpNum(),
|
||||
toStringImpl());
|
||||
}
|
||||
|
||||
@ -76,6 +76,41 @@ void ZooKeeperSyncResponse::writeImpl(WriteBuffer & out) const
|
||||
Coordination::write(path, out);
|
||||
}
|
||||
|
||||
void ZooKeeperReconfigRequest::writeImpl(WriteBuffer & out) const
|
||||
{
|
||||
Coordination::write(joining, out);
|
||||
Coordination::write(leaving, out);
|
||||
Coordination::write(new_members, out);
|
||||
Coordination::write(version, out);
|
||||
}
|
||||
|
||||
void ZooKeeperReconfigRequest::readImpl(ReadBuffer & in)
|
||||
{
|
||||
Coordination::read(joining, in);
|
||||
Coordination::read(leaving, in);
|
||||
Coordination::read(new_members, in);
|
||||
Coordination::read(version, in);
|
||||
}
|
||||
|
||||
std::string ZooKeeperReconfigRequest::toStringImpl() const
|
||||
{
|
||||
return fmt::format(
|
||||
"joining = {}\nleaving = {}\nnew_members = {}\nversion = {}",
|
||||
joining, leaving, new_members, version);
|
||||
}
|
||||
|
||||
void ZooKeeperReconfigResponse::readImpl(ReadBuffer & in)
|
||||
{
|
||||
Coordination::read(value, in);
|
||||
Coordination::read(stat, in);
|
||||
}
|
||||
|
||||
void ZooKeeperReconfigResponse::writeImpl(WriteBuffer & out) const
|
||||
{
|
||||
Coordination::write(value, out);
|
||||
Coordination::write(stat, out);
|
||||
}
|
||||
|
||||
void ZooKeeperWatchResponse::readImpl(ReadBuffer & in)
|
||||
{
|
||||
Coordination::read(type, in);
|
||||
@ -664,6 +699,7 @@ ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTi
|
||||
ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperExistsResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperGetResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSetResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperReconfigRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperReconfigResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperListResponse>()); }
|
||||
ZooKeeperResponsePtr ZooKeeperSimpleListRequest::makeResponse() const { return setTime(std::make_shared<ZooKeeperSimpleListResponse>()); }
|
||||
|
||||
@ -861,7 +897,8 @@ void ZooKeeperMultiResponse::fillLogElements(LogElements & elems, size_t idx) co
|
||||
void ZooKeeperRequestFactory::registerRequest(OpNum op_num, Creator creator)
|
||||
{
|
||||
if (!op_num_to_request.try_emplace(op_num, creator).second)
|
||||
throw Coordination::Exception("Request type " + toString(op_num) + " already registered", Coordination::Error::ZRUNTIMEINCONSISTENCY);
|
||||
throw Coordination::Exception(Coordination::Error::ZRUNTIMEINCONSISTENCY,
|
||||
"Request type {} already registered", op_num);
|
||||
}
|
||||
|
||||
std::shared_ptr<ZooKeeperRequest> ZooKeeperRequest::read(ReadBuffer & in)
|
||||
@ -916,7 +953,7 @@ ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const
|
||||
{
|
||||
auto it = op_num_to_request.find(op_num);
|
||||
if (it == op_num_to_request.end())
|
||||
throw Exception("Unknown operation type " + toString(op_num), Error::ZBADARGUMENTS);
|
||||
throw Exception(Error::ZBADARGUMENTS, "Unknown operation type {}", op_num);
|
||||
|
||||
return it->second();
|
||||
}
|
||||
@ -960,6 +997,7 @@ ZooKeeperRequestFactory::ZooKeeperRequestFactory()
|
||||
registerZooKeeperRequest<OpNum::SimpleList, ZooKeeperSimpleListRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::List, ZooKeeperListRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::Check, ZooKeeperCheckRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::Reconfig, ZooKeeperReconfigRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::Multi, ZooKeeperMultiRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::MultiRead, ZooKeeperMultiRequest>(*this);
|
||||
registerZooKeeperRequest<OpNum::SessionID, ZooKeeperSessionIDRequest>(*this);
|
||||
|
@ -117,6 +117,35 @@ struct ZooKeeperSyncResponse final : SyncResponse, ZooKeeperResponse
|
||||
OpNum getOpNum() const override { return OpNum::Sync; }
|
||||
};
|
||||
|
||||
struct ZooKeeperReconfigRequest final : ZooKeeperRequest
|
||||
{
|
||||
String joining;
|
||||
String leaving;
|
||||
String new_members;
|
||||
int64_t version; // kazoo sends a 64bit integer in this request
|
||||
|
||||
String getPath() const override { return keeper_config_path; }
|
||||
OpNum getOpNum() const override { return OpNum::Reconfig; }
|
||||
void writeImpl(WriteBuffer & out) const override;
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
std::string toStringImpl() const override;
|
||||
ZooKeeperResponsePtr makeResponse() const override;
|
||||
bool isReadRequest() const override { return false; }
|
||||
|
||||
size_t bytesSize() const override
|
||||
{
|
||||
return ZooKeeperRequest::bytesSize() + joining.size() + leaving.size() + new_members.size()
|
||||
+ sizeof(version);
|
||||
}
|
||||
};
|
||||
|
||||
struct ZooKeeperReconfigResponse final : ReconfigResponse, ZooKeeperResponse
|
||||
{
|
||||
void readImpl(ReadBuffer & in) override;
|
||||
void writeImpl(WriteBuffer & out) const override;
|
||||
OpNum getOpNum() const override { return OpNum::Reconfig; }
|
||||
};
|
||||
|
||||
struct ZooKeeperHeartbeatResponse final : ZooKeeperResponse
|
||||
{
|
||||
void readImpl(ReadBuffer &) override {}
|
||||
|
@ -19,6 +19,7 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
|
||||
static_cast<int32_t>(OpNum::Heartbeat),
|
||||
static_cast<int32_t>(OpNum::List),
|
||||
static_cast<int32_t>(OpNum::Check),
|
||||
static_cast<int32_t>(OpNum::Reconfig),
|
||||
static_cast<int32_t>(OpNum::Multi),
|
||||
static_cast<int32_t>(OpNum::MultiRead),
|
||||
static_cast<int32_t>(OpNum::Auth),
|
||||
@ -29,55 +30,6 @@ static const std::unordered_set<int32_t> VALID_OPERATIONS =
|
||||
static_cast<int32_t>(OpNum::CheckNotExists),
|
||||
};
|
||||
|
||||
std::string toString(OpNum op_num)
|
||||
{
|
||||
switch (op_num)
|
||||
{
|
||||
case OpNum::Close:
|
||||
return "Close";
|
||||
case OpNum::Error:
|
||||
return "Error";
|
||||
case OpNum::Create:
|
||||
return "Create";
|
||||
case OpNum::Remove:
|
||||
return "Remove";
|
||||
case OpNum::Exists:
|
||||
return "Exists";
|
||||
case OpNum::Get:
|
||||
return "Get";
|
||||
case OpNum::Set:
|
||||
return "Set";
|
||||
case OpNum::SimpleList:
|
||||
return "SimpleList";
|
||||
case OpNum::List:
|
||||
return "List";
|
||||
case OpNum::Check:
|
||||
return "Check";
|
||||
case OpNum::Multi:
|
||||
return "Multi";
|
||||
case OpNum::MultiRead:
|
||||
return "MultiRead";
|
||||
case OpNum::Sync:
|
||||
return "Sync";
|
||||
case OpNum::Heartbeat:
|
||||
return "Heartbeat";
|
||||
case OpNum::Auth:
|
||||
return "Auth";
|
||||
case OpNum::SessionID:
|
||||
return "SessionID";
|
||||
case OpNum::SetACL:
|
||||
return "SetACL";
|
||||
case OpNum::GetACL:
|
||||
return "GetACL";
|
||||
case OpNum::FilteredList:
|
||||
return "FilteredList";
|
||||
case OpNum::CheckNotExists:
|
||||
return "CheckNotExists";
|
||||
}
|
||||
int32_t raw_op = static_cast<int32_t>(op_num);
|
||||
throw Exception("Operation " + std::to_string(raw_op) + " is unknown", Error::ZUNIMPLEMENTED);
|
||||
}
|
||||
|
||||
OpNum getOpNum(int32_t raw_op_num)
|
||||
{
|
||||
if (!VALID_OPERATIONS.contains(raw_op_num))
|
||||
|
@ -31,6 +31,7 @@ enum class OpNum : int32_t
|
||||
List = 12,
|
||||
Check = 13,
|
||||
Multi = 14,
|
||||
Reconfig = 16,
|
||||
MultiRead = 22,
|
||||
Auth = 100,
|
||||
|
||||
@ -41,7 +42,6 @@ enum class OpNum : int32_t
|
||||
SessionID = 997, /// Special internal request
|
||||
};
|
||||
|
||||
std::string toString(OpNum op_num);
|
||||
OpNum getOpNum(int32_t raw_op_num);
|
||||
|
||||
static constexpr int32_t ZOOKEEPER_PROTOCOL_VERSION = 0;
|
||||
|
@ -35,6 +35,7 @@ namespace ProfileEvents
|
||||
extern const Event ZooKeeperRemove;
|
||||
extern const Event ZooKeeperExists;
|
||||
extern const Event ZooKeeperMulti;
|
||||
extern const Event ZooKeeperReconfig;
|
||||
extern const Event ZooKeeperGet;
|
||||
extern const Event ZooKeeperSet;
|
||||
extern const Event ZooKeeperList;
|
||||
@ -571,7 +572,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
|
||||
|
||||
if (err != Error::ZOK)
|
||||
throw Exception(Error::ZMARSHALLINGERROR, "Error received in reply to auth request. Code: {}. Message: {}",
|
||||
static_cast<int32_t>(err), errorMessage(err));
|
||||
static_cast<int32_t>(err), err);
|
||||
}
|
||||
|
||||
void ZooKeeper::sendThread()
|
||||
@ -697,7 +698,7 @@ void ZooKeeper::receiveThread()
|
||||
if (earliest_operation)
|
||||
{
|
||||
throw Exception(Error::ZOPERATIONTIMEOUT, "Operation timeout (no response in {} ms) for request {} for path: {}",
|
||||
args.operation_timeout_ms, toString(earliest_operation->request->getOpNum()), earliest_operation->request->getPath());
|
||||
args.operation_timeout_ms, earliest_operation->request->getOpNum(), earliest_operation->request->getPath());
|
||||
}
|
||||
waited_us += max_wait_us;
|
||||
if (waited_us >= args.session_timeout_ms * 1000)
|
||||
@ -738,7 +739,7 @@ void ZooKeeper::receiveEvent()
|
||||
if (xid == PING_XID)
|
||||
{
|
||||
if (err != Error::ZOK)
|
||||
throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", errorMessage(err));
|
||||
throw Exception(Error::ZRUNTIMEINCONSISTENCY, "Received error in heartbeat response: {}", err);
|
||||
|
||||
response = std::make_shared<ZooKeeperHeartbeatResponse>();
|
||||
}
|
||||
@ -1195,7 +1196,6 @@ void ZooKeeper::create(
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperCreate);
|
||||
}
|
||||
|
||||
|
||||
void ZooKeeper::remove(
|
||||
const String & path,
|
||||
int32_t version,
|
||||
@ -1335,6 +1335,26 @@ void ZooKeeper::sync(
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperSync);
|
||||
}
|
||||
|
||||
void ZooKeeper::reconfig(
|
||||
std::string_view joining,
|
||||
std::string_view leaving,
|
||||
std::string_view new_members,
|
||||
int32_t version,
|
||||
ReconfigCallback callback)
|
||||
{
|
||||
ZooKeeperReconfigRequest request;
|
||||
request.joining = joining;
|
||||
request.leaving = leaving;
|
||||
request.new_members = new_members;
|
||||
request.version = version;
|
||||
|
||||
RequestInfo request_info;
|
||||
request_info.request = std::make_shared<ZooKeeperReconfigRequest>(std::move(request));
|
||||
request_info.callback = [callback](const Response & response) { callback(dynamic_cast<const ReconfigResponse &>(response)); };
|
||||
|
||||
pushRequest(std::move(request_info));
|
||||
ProfileEvents::increment(ProfileEvents::ZooKeeperReconfig);
|
||||
}
|
||||
|
||||
void ZooKeeper::multi(
|
||||
const Requests & requests,
|
||||
|
@ -178,6 +178,13 @@ public:
|
||||
const String & path,
|
||||
SyncCallback callback) override;
|
||||
|
||||
void reconfig(
|
||||
std::string_view joining,
|
||||
std::string_view leaving,
|
||||
std::string_view new_members,
|
||||
int32_t version,
|
||||
ReconfigCallback callback) final;
|
||||
|
||||
void multi(
|
||||
const Requests & requests,
|
||||
MultiCallback callback) override;
|
||||
|
@ -1,5 +1,4 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
@ -14,8 +13,8 @@ enum class KeeperApiVersion : uint8_t
|
||||
WITH_CHECK_NOT_EXISTS,
|
||||
};
|
||||
|
||||
const std::string keeper_system_path = "/keeper";
|
||||
const std::string keeper_api_version_path = keeper_system_path + "/api_version";
|
||||
const std::string keeper_api_feature_flags_path = keeper_system_path + "/feature_flags";
|
||||
|
||||
const String keeper_system_path = "/keeper";
|
||||
const String keeper_api_version_path = keeper_system_path + "/api_version";
|
||||
const String keeper_api_feature_flags_path = keeper_system_path + "/feature_flags";
|
||||
const String keeper_config_path = keeper_system_path + "/config";
|
||||
}
|
||||
|
@ -32,8 +32,9 @@ KeeperContext::KeeperContext(bool standalone_keeper_)
|
||||
system_nodes_with_data[keeper_api_version_path] = toString(static_cast<uint8_t>(KeeperApiVersion::WITH_MULTI_READ));
|
||||
}
|
||||
|
||||
void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config)
|
||||
void KeeperContext::initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_)
|
||||
{
|
||||
dispatcher = dispatcher_;
|
||||
digest_enabled = config.getBool("keeper_server.digest_enabled", false);
|
||||
ignore_system_path_on_startup = config.getBool("keeper_server.ignore_system_path_on_startup", false);
|
||||
|
||||
|
@ -1,10 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <Coordination/KeeperFeatureFlags.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Disks/DiskSelector.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <cstdint>
|
||||
#include <memory>
|
||||
@ -12,6 +10,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class KeeperDispatcher;
|
||||
|
||||
class KeeperContext
|
||||
{
|
||||
public:
|
||||
@ -24,7 +24,7 @@ public:
|
||||
SHUTDOWN
|
||||
};
|
||||
|
||||
void initialize(const Poco::Util::AbstractConfiguration & config);
|
||||
void initialize(const Poco::Util::AbstractConfiguration & config, KeeperDispatcher * dispatcher_);
|
||||
|
||||
Phase getServerState() const;
|
||||
void setServerState(Phase server_state_);
|
||||
@ -51,6 +51,9 @@ public:
|
||||
const KeeperFeatureFlags & getFeatureFlags() const;
|
||||
|
||||
void dumpConfiguration(WriteBufferFromOwnString & buf) const;
|
||||
|
||||
constexpr KeeperDispatcher * getDispatcher() const { return dispatcher; }
|
||||
|
||||
private:
|
||||
/// local disk defined using path or disk name
|
||||
using Storage = std::variant<DiskPtr, std::string>;
|
||||
@ -85,8 +88,8 @@ private:
|
||||
std::unordered_map<std::string, std::string> system_nodes_with_data;
|
||||
|
||||
KeeperFeatureFlags feature_flags;
|
||||
KeeperDispatcher * dispatcher{nullptr};
|
||||
};
|
||||
|
||||
using KeeperContextPtr = std::shared_ptr<KeeperContext>;
|
||||
|
||||
}
|
||||
|
@ -38,6 +38,8 @@ namespace ProfileEvents
|
||||
extern const Event MemoryAllocatorPurgeTimeMicroseconds;
|
||||
}
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -80,6 +82,7 @@ void KeeperDispatcher::requestThread()
|
||||
/// requests into a batch we must check that the new request is not read request. Otherwise we have to
|
||||
/// process all already accumulated write requests, wait them synchronously and only after that process
|
||||
/// read request. So reads are some kind of "separator" for writes.
|
||||
/// Also there is a special reconfig request also being a separator.
|
||||
try
|
||||
{
|
||||
if (requests_queue->tryPop(request, max_wait))
|
||||
@ -92,10 +95,13 @@ void KeeperDispatcher::requestThread()
|
||||
size_t current_batch_bytes_size = 0;
|
||||
|
||||
bool has_read_request = false;
|
||||
bool has_reconfig_request = false;
|
||||
|
||||
/// If new request is not read request or we must to process it through quorum.
|
||||
/// If new request is not read request or reconfig request we must process it through quorum.
|
||||
/// Otherwise we will process it locally.
|
||||
if (coordination_settings->quorum_reads || !request.request->isReadRequest())
|
||||
if (request.request->getOpNum() == Coordination::OpNum::Reconfig)
|
||||
has_reconfig_request = true;
|
||||
else if (coordination_settings->quorum_reads || !request.request->isReadRequest())
|
||||
{
|
||||
current_batch_bytes_size += request.request->bytesSize();
|
||||
current_batch.emplace_back(request);
|
||||
@ -113,6 +119,11 @@ void KeeperDispatcher::requestThread()
|
||||
std::lock_guard lock(read_request_queue_mutex);
|
||||
read_request_queue[last_request.session_id][last_request.request->xid].push_back(request);
|
||||
}
|
||||
else if (request.request->getOpNum() == Coordination::OpNum::Reconfig)
|
||||
{
|
||||
has_reconfig_request = true;
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
current_batch_bytes_size += request.request->bytesSize();
|
||||
@ -128,6 +139,7 @@ void KeeperDispatcher::requestThread()
|
||||
/// TODO: Deprecate max_requests_quick_batch_size and use only max_requests_batch_size and max_requests_batch_bytes_size
|
||||
size_t max_quick_batch_size = coordination_settings->max_requests_quick_batch_size;
|
||||
while (!shutdown_called && !has_read_request &&
|
||||
!has_reconfig_request &&
|
||||
current_batch.size() < max_quick_batch_size && current_batch_bytes_size < max_batch_bytes_size &&
|
||||
try_get_request())
|
||||
;
|
||||
@ -140,8 +152,10 @@ void KeeperDispatcher::requestThread()
|
||||
};
|
||||
|
||||
/// Waiting until previous append will be successful, or batch is big enough
|
||||
while (!shutdown_called && !has_read_request && !prev_result_done() &&
|
||||
current_batch.size() <= max_batch_size && current_batch_bytes_size < max_batch_bytes_size)
|
||||
while (!shutdown_called && !has_read_request &&
|
||||
!has_reconfig_request && !prev_result_done() &&
|
||||
current_batch.size() <= max_batch_size
|
||||
&& current_batch_bytes_size < max_batch_bytes_size)
|
||||
{
|
||||
try_get_request();
|
||||
}
|
||||
@ -165,7 +179,8 @@ void KeeperDispatcher::requestThread()
|
||||
|
||||
if (result)
|
||||
{
|
||||
if (has_read_request) /// If we will execute read request next, than we have to process result now
|
||||
/// If we will execute read or reconfig next, we have to process result now
|
||||
if (has_read_request || has_reconfig_request)
|
||||
forceWaitAndProcessResult(result, current_batch);
|
||||
}
|
||||
else
|
||||
@ -179,6 +194,9 @@ void KeeperDispatcher::requestThread()
|
||||
prev_result = result;
|
||||
}
|
||||
|
||||
if (has_reconfig_request)
|
||||
server->getKeeperStateMachine()->reconfigure(request);
|
||||
|
||||
/// Read request always goes after write batch (last request)
|
||||
if (has_read_request)
|
||||
{
|
||||
@ -335,7 +353,7 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
|
||||
snapshot_s3.startup(config, macros);
|
||||
|
||||
keeper_context = std::make_shared<KeeperContext>(standalone_keeper);
|
||||
keeper_context->initialize(config);
|
||||
keeper_context->initialize(config, this);
|
||||
|
||||
server = std::make_unique<KeeperServer>(
|
||||
configuration_and_settings,
|
||||
@ -392,7 +410,10 @@ void KeeperDispatcher::initialize(const Poco::Util::AbstractConfiguration & conf
|
||||
|
||||
/// Start it after keeper server start
|
||||
session_cleaner_thread = ThreadFromGlobalPool([this] { sessionCleanerTask(); });
|
||||
update_configuration_thread = ThreadFromGlobalPool([this] { updateConfigurationThread(); });
|
||||
|
||||
update_configuration_thread = reconfigEnabled()
|
||||
? ThreadFromGlobalPool([this] { clusterUpdateThread(); })
|
||||
: ThreadFromGlobalPool([this] { clusterUpdateWithReconfigDisabledThread(); });
|
||||
|
||||
LOG_DEBUG(log, "Dispatcher initialized");
|
||||
}
|
||||
@ -429,7 +450,7 @@ void KeeperDispatcher::shutdown()
|
||||
if (snapshot_thread.joinable())
|
||||
snapshot_thread.join();
|
||||
|
||||
update_configuration_queue.finish();
|
||||
cluster_update_queue.finish();
|
||||
if (update_configuration_thread.joinable())
|
||||
update_configuration_thread.join();
|
||||
}
|
||||
@ -615,7 +636,7 @@ void KeeperDispatcher::addErrorResponses(const KeeperStorage::RequestsForSession
|
||||
"Could not push error response xid {} zxid {} error message {} to responses queue",
|
||||
response->xid,
|
||||
response->zxid,
|
||||
errorMessage(error));
|
||||
error);
|
||||
}
|
||||
}
|
||||
|
||||
@ -660,7 +681,7 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms)
|
||||
{
|
||||
if (response->getOpNum() != Coordination::OpNum::SessionID)
|
||||
promise->set_exception(std::make_exception_ptr(Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Incorrect response of type {} instead of SessionID response", Coordination::toString(response->getOpNum()))));
|
||||
"Incorrect response of type {} instead of SessionID response", response->getOpNum())));
|
||||
|
||||
auto session_id_response = dynamic_cast<const Coordination::ZooKeeperSessionIDResponse &>(*response);
|
||||
if (session_id_response.internal_id != internal_id)
|
||||
@ -692,17 +713,12 @@ int64_t KeeperDispatcher::getSessionID(int64_t session_timeout_ms)
|
||||
return future.get();
|
||||
}
|
||||
|
||||
|
||||
void KeeperDispatcher::updateConfigurationThread()
|
||||
void KeeperDispatcher::clusterUpdateWithReconfigDisabledThread()
|
||||
{
|
||||
while (true)
|
||||
while (!shutdown_called)
|
||||
{
|
||||
if (shutdown_called)
|
||||
return;
|
||||
|
||||
try
|
||||
{
|
||||
using namespace std::chrono_literals;
|
||||
if (!server->checkInit())
|
||||
{
|
||||
LOG_INFO(log, "Server still not initialized, will not apply configuration until initialization finished");
|
||||
@ -717,11 +733,10 @@ void KeeperDispatcher::updateConfigurationThread()
|
||||
continue;
|
||||
}
|
||||
|
||||
ConfigUpdateAction action;
|
||||
if (!update_configuration_queue.pop(action))
|
||||
ClusterUpdateAction action;
|
||||
if (!cluster_update_queue.pop(action))
|
||||
break;
|
||||
|
||||
|
||||
/// We must wait this update from leader or apply it ourself (if we are leader)
|
||||
bool done = false;
|
||||
while (!done)
|
||||
@ -734,15 +749,13 @@ void KeeperDispatcher::updateConfigurationThread()
|
||||
|
||||
if (isLeader())
|
||||
{
|
||||
server->applyConfigurationUpdate(action);
|
||||
server->applyConfigUpdateWithReconfigDisabled(action);
|
||||
done = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
done = server->waitConfigurationUpdate(action);
|
||||
if (!done)
|
||||
LOG_INFO(log, "Cannot wait for configuration update, maybe we become leader, or maybe update is invalid, will try to wait one more time");
|
||||
}
|
||||
else if (done = server->waitForConfigUpdateWithReconfigDisabled(action); !done)
|
||||
LOG_INFO(log,
|
||||
"Cannot wait for configuration update, maybe we became leader "
|
||||
"or maybe update is invalid, will try to wait one more time");
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
@ -752,6 +765,41 @@ void KeeperDispatcher::updateConfigurationThread()
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperDispatcher::clusterUpdateThread()
|
||||
{
|
||||
while (!shutdown_called)
|
||||
{
|
||||
ClusterUpdateAction action;
|
||||
if (!cluster_update_queue.pop(action))
|
||||
return;
|
||||
|
||||
if (server->applyConfigUpdate(action))
|
||||
LOG_DEBUG(log, "Processing config update {}: accepted", action);
|
||||
else // TODO (myrrc) sleep a random amount? sleep less?
|
||||
{
|
||||
(void)cluster_update_queue.pushFront(action);
|
||||
LOG_DEBUG(log, "Processing config update {}: declined, backoff", action);
|
||||
std::this_thread::sleep_for(50ms);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void KeeperDispatcher::pushClusterUpdates(ClusterUpdateActions && actions)
|
||||
{
|
||||
if (shutdown_called) return;
|
||||
for (auto && action : actions)
|
||||
{
|
||||
if (!cluster_update_queue.push(std::move(action)))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot push configuration update");
|
||||
LOG_DEBUG(log, "Processing config update {}: pushed", action);
|
||||
}
|
||||
}
|
||||
|
||||
bool KeeperDispatcher::reconfigEnabled() const
|
||||
{
|
||||
return server->reconfigEnabled();
|
||||
}
|
||||
|
||||
bool KeeperDispatcher::isServerActive() const
|
||||
{
|
||||
return checkInit() && hasLeader() && !server->isRecovering();
|
||||
@ -759,20 +807,25 @@ bool KeeperDispatcher::isServerActive() const
|
||||
|
||||
void KeeperDispatcher::updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion<Macros>::Version & macros)
|
||||
{
|
||||
auto diff = server->getConfigurationDiff(config);
|
||||
auto diff = server->getRaftConfigurationDiff(config);
|
||||
|
||||
if (diff.empty())
|
||||
LOG_TRACE(log, "Configuration update triggered, but nothing changed for RAFT");
|
||||
LOG_TRACE(log, "Configuration update triggered, but nothing changed for Raft");
|
||||
else if (reconfigEnabled())
|
||||
LOG_WARNING(log,
|
||||
"Raft configuration changed, but keeper_server.enable_reconfiguration is on. "
|
||||
"This update will be ignored. Use \"reconfig\" instead");
|
||||
else if (diff.size() > 1)
|
||||
LOG_WARNING(log, "Configuration changed for more than one server ({}) from cluster, it's strictly not recommended", diff.size());
|
||||
LOG_WARNING(log,
|
||||
"Configuration changed for more than one server ({}) from cluster, "
|
||||
"it's strictly not recommended", diff.size());
|
||||
else
|
||||
LOG_DEBUG(log, "Configuration change size ({})", diff.size());
|
||||
|
||||
for (auto & change : diff)
|
||||
{
|
||||
bool push_result = update_configuration_queue.push(change);
|
||||
if (!push_result)
|
||||
throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue");
|
||||
}
|
||||
if (!reconfigEnabled())
|
||||
for (auto & change : diff)
|
||||
if (!cluster_update_queue.push(change))
|
||||
throw Exception(ErrorCodes::SYSTEM_ERROR, "Cannot push configuration update to queue");
|
||||
|
||||
snapshot_s3.updateS3Configuration(config, macros);
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ private:
|
||||
|
||||
using RequestsQueue = ConcurrentBoundedQueue<KeeperStorage::RequestForSession>;
|
||||
using SessionToResponseCallback = std::unordered_map<int64_t, ZooKeeperResponseCallback>;
|
||||
using UpdateConfigurationQueue = ConcurrentBoundedQueue<ConfigUpdateAction>;
|
||||
using ClusterUpdateQueue = ConcurrentBoundedQueue<ClusterUpdateAction>;
|
||||
|
||||
/// Size depends on coordination settings
|
||||
std::unique_ptr<RequestsQueue> requests_queue;
|
||||
@ -39,7 +39,7 @@ private:
|
||||
SnapshotsQueue snapshots_queue{1};
|
||||
|
||||
/// More than 1k updates is definitely misconfiguration.
|
||||
UpdateConfigurationQueue update_configuration_queue{1000};
|
||||
ClusterUpdateQueue cluster_update_queue{1000};
|
||||
|
||||
std::atomic<bool> shutdown_called{false};
|
||||
|
||||
@ -91,8 +91,10 @@ private:
|
||||
void sessionCleanerTask();
|
||||
/// Thread create snapshots in the background
|
||||
void snapshotThread();
|
||||
/// Thread apply or wait configuration changes from leader
|
||||
void updateConfigurationThread();
|
||||
|
||||
// TODO (myrrc) this should be removed once "reconfig" is stabilized
|
||||
void clusterUpdateWithReconfigDisabledThread();
|
||||
void clusterUpdateThread();
|
||||
|
||||
void setResponse(int64_t session_id, const Coordination::ZooKeeperResponsePtr & response);
|
||||
|
||||
@ -132,10 +134,9 @@ public:
|
||||
/// and achieved quorum
|
||||
bool isServerActive() const;
|
||||
|
||||
/// Registered in ConfigReloader callback. Add new configuration changes to
|
||||
/// update_configuration_queue. Keeper Dispatcher apply them asynchronously.
|
||||
/// 'macros' are used to substitute macros in endpoint of disks
|
||||
void updateConfiguration(const Poco::Util::AbstractConfiguration & config, const MultiVersion<Macros>::Version & macros);
|
||||
void pushClusterUpdates(ClusterUpdateActions && actions);
|
||||
bool reconfigEnabled() const;
|
||||
|
||||
/// Shutdown internal keeper parts (server, state machine, log storage, etc)
|
||||
void shutdown();
|
||||
|
91
src/Coordination/KeeperReconfiguration.cpp
Normal file
91
src/Coordination/KeeperReconfiguration.cpp
Normal file
@ -0,0 +1,91 @@
|
||||
#include "KeeperReconfiguration.h"
|
||||
#include <unordered_set>
|
||||
#include <base/find_symbols.h>
|
||||
#include <fmt/format.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining)
|
||||
{
|
||||
ClusterUpdateActions out;
|
||||
std::unordered_set<String> endpoints;
|
||||
|
||||
for (const auto & server : cfg->get_servers())
|
||||
endpoints.emplace(server->get_endpoint());
|
||||
|
||||
// We can either add new servers or change weight of existing ones.
|
||||
// It makes no sense having a server in _joining_ which is identical to existing one including
|
||||
// weight, so such requests are declined.
|
||||
for (const RaftServerConfig & update : parseRaftServers(joining))
|
||||
if (auto server_ptr = cfg->get_server(update.id))
|
||||
{
|
||||
if (update.endpoint != server_ptr->get_endpoint() || update.learner != server_ptr->is_learner()
|
||||
|| update.priority == server_ptr->get_priority())
|
||||
return {}; // can't change server endpoint/type due to NuRaft API limitations
|
||||
out.emplace_back(UpdateRaftServerPriority{.id = update.id, .priority = update.priority});
|
||||
}
|
||||
else if (endpoints.contains(update.endpoint))
|
||||
return {};
|
||||
else
|
||||
out.emplace_back(AddRaftServer{update});
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving)
|
||||
{
|
||||
std::vector<std::string_view> leaving_arr;
|
||||
splitInto<','>(leaving_arr, leaving);
|
||||
if (leaving_arr.size() >= cfg->get_servers().size())
|
||||
return {};
|
||||
|
||||
std::unordered_set<int32_t> remove_ids;
|
||||
ClusterUpdateActions out;
|
||||
|
||||
for (std::string_view leaving_server : leaving_arr)
|
||||
{
|
||||
int32_t id;
|
||||
if (!tryParse(id, leaving_server))
|
||||
return {};
|
||||
|
||||
if (remove_ids.contains(id))
|
||||
continue;
|
||||
|
||||
if (auto ptr = cfg->get_server(id))
|
||||
out.emplace_back(RemoveRaftServer{.id = id});
|
||||
else
|
||||
return {};
|
||||
|
||||
remove_ids.emplace(id);
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates)
|
||||
{
|
||||
RaftServers new_config;
|
||||
std::unordered_set<int32_t> remove_update_ids;
|
||||
|
||||
for (const auto & update : updates)
|
||||
{
|
||||
if (const auto * add = std::get_if<AddRaftServer>(&update))
|
||||
new_config.emplace_back(*add);
|
||||
else if (const auto * remove = std::get_if<RemoveRaftServer>(&update))
|
||||
remove_update_ids.insert(remove->id);
|
||||
else if (const auto * priority = std::get_if<UpdateRaftServerPriority>(&update))
|
||||
{
|
||||
remove_update_ids.insert(priority->id);
|
||||
new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)});
|
||||
}
|
||||
else
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
for (const auto & item : cfg->get_servers())
|
||||
if (!remove_update_ids.contains(item->get_id()))
|
||||
new_config.emplace_back(RaftServerConfig{*item});
|
||||
|
||||
return fmt::format("{}", fmt::join(new_config.begin(), new_config.end(), "\n"));
|
||||
}
|
||||
}
|
10
src/Coordination/KeeperReconfiguration.h
Normal file
10
src/Coordination/KeeperReconfiguration.h
Normal file
@ -0,0 +1,10 @@
|
||||
#pragma once
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Coordination/RaftServerConfig.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining);
|
||||
ClusterUpdateActions leavingToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view leaving);
|
||||
String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateActions & updates = {});
|
||||
}
|
@ -27,6 +27,7 @@
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Disks/DiskLocal.h>
|
||||
#include <fmt/chrono.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -40,6 +41,8 @@ namespace ErrorCodes
|
||||
extern const int INVALID_CONFIG_PARAMETER;
|
||||
}
|
||||
|
||||
using namespace std::chrono_literals;
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
@ -118,6 +121,7 @@ KeeperServer::KeeperServer(
|
||||
, is_recovering(config.getBool("keeper_server.force_recovery", false))
|
||||
, keeper_context{std::move(keeper_context_)}
|
||||
, create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true))
|
||||
, enable_reconfiguration(config.getBool("keeper_server.enable_reconfiguration", false))
|
||||
{
|
||||
if (coordination_settings->quorum_reads)
|
||||
LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower.");
|
||||
@ -450,7 +454,7 @@ void KeeperServer::shutdownRaftServer()
|
||||
size_t count = 0;
|
||||
while (asio_service->get_active_workers() != 0 && count < timeout * 100)
|
||||
{
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(10));
|
||||
std::this_thread::sleep_for(10ms);
|
||||
count++;
|
||||
}
|
||||
}
|
||||
@ -715,10 +719,12 @@ nuraft::cb_func::ReturnCode KeeperServer::callbackFunc(nuraft::cb_func::Type typ
|
||||
if (next_index < last_commited || next_index - last_commited <= 1)
|
||||
commited_store = true;
|
||||
|
||||
auto set_initialized = [this]()
|
||||
auto set_initialized = [this]
|
||||
{
|
||||
std::lock_guard lock(initialized_mutex);
|
||||
initialized_flag = true;
|
||||
{
|
||||
std::lock_guard lock(initialized_mutex);
|
||||
initialized_flag = true;
|
||||
}
|
||||
initialized_cv.notify_all();
|
||||
};
|
||||
|
||||
@ -783,9 +789,45 @@ std::vector<int64_t> KeeperServer::getDeadSessions()
|
||||
return state_machine->getDeadSessions();
|
||||
}
|
||||
|
||||
ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config)
|
||||
bool KeeperServer::applyConfigUpdate(const ClusterUpdateAction & action)
|
||||
{
|
||||
auto diff = state_manager->getConfigurationDiff(config);
|
||||
std::lock_guard _{server_write_mutex};
|
||||
|
||||
if (const auto * add = std::get_if<AddRaftServer>(&action))
|
||||
return raft_instance->get_srv_config(add->id) != nullptr
|
||||
|| raft_instance->add_srv(static_cast<nuraft::srv_config>(*add))->get_accepted();
|
||||
else if (const auto * remove = std::get_if<RemoveRaftServer>(&action))
|
||||
{
|
||||
if (remove->id == raft_instance->get_leader())
|
||||
{
|
||||
if (isLeader())
|
||||
raft_instance->yield_leadership();
|
||||
else
|
||||
raft_instance->request_leadership();
|
||||
return false;
|
||||
}
|
||||
|
||||
return raft_instance->get_srv_config(remove->id) == nullptr
|
||||
|| raft_instance->remove_srv(remove->id)->get_accepted();
|
||||
}
|
||||
else if (const auto * update = std::get_if<UpdateRaftServerPriority>(&action))
|
||||
{
|
||||
if (auto ptr = raft_instance->get_srv_config(update->id); ptr == nullptr)
|
||||
throw Exception(ErrorCodes::RAFT_ERROR,
|
||||
"Attempt to apply {} but server is not present in Raft",
|
||||
action);
|
||||
else if (ptr->get_priority() == update->priority)
|
||||
return true;
|
||||
|
||||
raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true);
|
||||
return true;
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
auto diff = state_manager->getRaftConfigurationDiff(config);
|
||||
|
||||
if (!diff.empty())
|
||||
{
|
||||
@ -796,160 +838,103 @@ ConfigUpdateActions KeeperServer::getConfigurationDiff(const Poco::Util::Abstrac
|
||||
return diff;
|
||||
}
|
||||
|
||||
void KeeperServer::applyConfigurationUpdate(const ConfigUpdateAction & task)
|
||||
void KeeperServer::applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action)
|
||||
{
|
||||
std::lock_guard lock{server_write_mutex};
|
||||
if (is_recovering)
|
||||
return;
|
||||
std::lock_guard _{server_write_mutex};
|
||||
if (is_recovering) return;
|
||||
constexpr auto sleep_time = 500ms;
|
||||
|
||||
size_t sleep_ms = 500;
|
||||
if (task.action_type == ConfigUpdateActionType::AddServer)
|
||||
LOG_INFO(log, "Will try to apply {}", action);
|
||||
|
||||
auto applied = [&] { LOG_INFO(log, "Applied {}", action); };
|
||||
auto not_leader = [&] { LOG_INFO(log, "Not leader anymore, aborting"); };
|
||||
auto backoff_on_refusal = [&](size_t i)
|
||||
{
|
||||
LOG_INFO(log, "Update was not accepted (try {}), backing off for {}", i + 1, sleep_time * (i + 1));
|
||||
std::this_thread::sleep_for(sleep_time * (i + 1));
|
||||
};
|
||||
|
||||
if (const auto * add = std::get_if<AddRaftServer>(&action))
|
||||
{
|
||||
LOG_INFO(log, "Will try to add server with id {}", task.server->get_id());
|
||||
bool added = false;
|
||||
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) != nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully added", task.server->get_id());
|
||||
added = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (raft_instance->get_srv_config(add->id) != nullptr)
|
||||
return applied();
|
||||
if (!isLeader())
|
||||
{
|
||||
LOG_INFO(log, "We are not leader anymore, will not try to add server {}", task.server->get_id());
|
||||
break;
|
||||
}
|
||||
|
||||
auto result = raft_instance->add_srv(*task.server);
|
||||
if (!result->get_accepted())
|
||||
LOG_INFO(
|
||||
log,
|
||||
"Command to add server {} was not accepted for the {} time, will sleep for {} ms and retry",
|
||||
task.server->get_id(),
|
||||
i + 1,
|
||||
sleep_ms * (i + 1));
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
|
||||
return not_leader();
|
||||
if (!raft_instance->add_srv(static_cast<nuraft::srv_config>(*add))->get_accepted())
|
||||
backoff_on_refusal(i);
|
||||
}
|
||||
if (!added)
|
||||
throw Exception(
|
||||
ErrorCodes::RAFT_ERROR,
|
||||
"Configuration change to add server (id {}) was not accepted by RAFT after all {} retries",
|
||||
task.server->get_id(),
|
||||
coordination_settings->configuration_change_tries_count);
|
||||
}
|
||||
else if (task.action_type == ConfigUpdateActionType::RemoveServer)
|
||||
else if (const auto * remove = std::get_if<RemoveRaftServer>(&action))
|
||||
{
|
||||
LOG_INFO(log, "Will try to remove server with id {}", task.server->get_id());
|
||||
|
||||
bool removed = false;
|
||||
if (task.server->get_id() == state_manager->server_id())
|
||||
if (remove->id == state_manager->server_id())
|
||||
{
|
||||
LOG_INFO(
|
||||
log,
|
||||
"Trying to remove leader node (ourself), so will yield leadership and some other node (new leader) will try remove us. "
|
||||
LOG_INFO(log,
|
||||
"Trying to remove leader node (ourself), so will yield leadership and some other node "
|
||||
"(new leader) will try to remove us. "
|
||||
"Probably you will have to run SYSTEM RELOAD CONFIG on the new leader node");
|
||||
|
||||
raft_instance->yield_leadership();
|
||||
return;
|
||||
return raft_instance->yield_leadership();
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) == nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully removed", task.server->get_id());
|
||||
removed = true;
|
||||
break;
|
||||
}
|
||||
|
||||
if (raft_instance->get_srv_config(remove->id) == nullptr)
|
||||
return applied();
|
||||
if (!isLeader())
|
||||
{
|
||||
LOG_INFO(log, "We are not leader anymore, will not try to remove server {}", task.server->get_id());
|
||||
break;
|
||||
}
|
||||
|
||||
auto result = raft_instance->remove_srv(task.server->get_id());
|
||||
if (!result->get_accepted())
|
||||
LOG_INFO(
|
||||
log,
|
||||
"Command to remove server {} was not accepted for the {} time, will sleep for {} ms and retry",
|
||||
task.server->get_id(),
|
||||
i + 1,
|
||||
sleep_ms * (i + 1));
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
|
||||
return not_leader();
|
||||
if (!raft_instance->remove_srv(remove->id)->get_accepted())
|
||||
backoff_on_refusal(i);
|
||||
}
|
||||
if (!removed)
|
||||
throw Exception(
|
||||
ErrorCodes::RAFT_ERROR,
|
||||
"Configuration change to remove server (id {}) was not accepted by RAFT after all {} retries",
|
||||
task.server->get_id(),
|
||||
coordination_settings->configuration_change_tries_count);
|
||||
}
|
||||
else if (task.action_type == ConfigUpdateActionType::UpdatePriority)
|
||||
raft_instance->set_priority(task.server->get_id(), task.server->get_priority());
|
||||
else
|
||||
LOG_WARNING(log, "Unknown configuration update type {}", static_cast<uint64_t>(task.action_type));
|
||||
else if (const auto * update = std::get_if<UpdateRaftServerPriority>(&action))
|
||||
{
|
||||
raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true);
|
||||
return;
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::RAFT_ERROR,
|
||||
"Configuration change {} was not accepted by Raft after {} retries",
|
||||
action, coordination_settings->configuration_change_tries_count);
|
||||
}
|
||||
|
||||
|
||||
bool KeeperServer::waitConfigurationUpdate(const ConfigUpdateAction & task)
|
||||
bool KeeperServer::waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action)
|
||||
{
|
||||
if (is_recovering)
|
||||
return false;
|
||||
if (is_recovering) return false;
|
||||
constexpr auto sleep_time = 500ms;
|
||||
|
||||
size_t sleep_ms = 500;
|
||||
if (task.action_type == ConfigUpdateActionType::AddServer)
|
||||
LOG_INFO(log, "Will try to wait for {}", action);
|
||||
|
||||
auto applied = [&] { LOG_INFO(log, "Applied {}", action); return true; };
|
||||
auto became_leader = [&] { LOG_INFO(log, "Became leader, aborting"); return false; };
|
||||
auto backoff = [&](size_t i) { std::this_thread::sleep_for(sleep_time * (i + 1)); };
|
||||
|
||||
if (const auto* add = std::get_if<AddRaftServer>(&action))
|
||||
{
|
||||
LOG_INFO(log, "Will try to wait server with id {} to be added", task.server->get_id());
|
||||
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) != nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully added by leader", task.server->get_id());
|
||||
return true;
|
||||
}
|
||||
|
||||
if (raft_instance->get_srv_config(add->id) != nullptr)
|
||||
return applied();
|
||||
if (isLeader())
|
||||
{
|
||||
LOG_INFO(log, "We are leader now, probably we will have to add server {}", task.server->get_id());
|
||||
return false;
|
||||
}
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
|
||||
return became_leader();
|
||||
backoff(i);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
else if (task.action_type == ConfigUpdateActionType::RemoveServer)
|
||||
else if (const auto* remove = std::get_if<RemoveRaftServer>(&action))
|
||||
{
|
||||
LOG_INFO(log, "Will try to wait remove of server with id {}", task.server->get_id());
|
||||
|
||||
for (size_t i = 0; i < coordination_settings->configuration_change_tries_count && !is_recovering; ++i)
|
||||
{
|
||||
if (raft_instance->get_srv_config(task.server->get_id()) == nullptr)
|
||||
{
|
||||
LOG_INFO(log, "Server with id {} was successfully removed by leader", task.server->get_id());
|
||||
return true;
|
||||
}
|
||||
|
||||
if (raft_instance->get_srv_config(remove->id) == nullptr)
|
||||
return applied();
|
||||
if (isLeader())
|
||||
{
|
||||
LOG_INFO(log, "We are leader now, probably we will have to remove server {}", task.server->get_id());
|
||||
return false;
|
||||
}
|
||||
|
||||
std::this_thread::sleep_for(std::chrono::milliseconds(sleep_ms * (i + 1)));
|
||||
return became_leader();
|
||||
backoff(i);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
else if (task.action_type == ConfigUpdateActionType::UpdatePriority)
|
||||
else if (std::holds_alternative<UpdateRaftServerPriority>(action))
|
||||
return true;
|
||||
else
|
||||
LOG_WARNING(log, "Unknown configuration update type {}", static_cast<uint64_t>(task.action_type));
|
||||
return true;
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Coordination/Keeper4LWInfo.h>
|
||||
#include <Coordination/KeeperContext.h>
|
||||
#include <Coordination/RaftServerConfig.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -28,9 +29,10 @@ private:
|
||||
nuraft::ptr<KeeperStateManager> state_manager;
|
||||
|
||||
struct KeeperRaftServer;
|
||||
nuraft::ptr<KeeperRaftServer> raft_instance;
|
||||
nuraft::ptr<KeeperRaftServer> raft_instance; // TSA_GUARDED_BY(server_write_mutex);
|
||||
nuraft::ptr<nuraft::asio_service> asio_service;
|
||||
std::vector<nuraft::ptr<nuraft::rpc_listener>> asio_listeners;
|
||||
|
||||
// because some actions can be applied
|
||||
// when we are sure that there are no requests currently being
|
||||
// processed (e.g. recovery) we do all write actions
|
||||
@ -65,6 +67,7 @@ private:
|
||||
std::shared_ptr<KeeperContext> keeper_context;
|
||||
|
||||
const bool create_snapshot_on_exit;
|
||||
const bool enable_reconfiguration;
|
||||
|
||||
public:
|
||||
KeeperServer(
|
||||
@ -84,6 +87,7 @@ public:
|
||||
void putLocalReadRequest(const KeeperStorage::RequestForSession & request);
|
||||
|
||||
bool isRecovering() const { return is_recovering; }
|
||||
bool reconfigEnabled() const { return enable_reconfiguration; }
|
||||
|
||||
/// Put batch of requests into Raft and get result of put. Responses will be set separately into
|
||||
/// responses_queue.
|
||||
@ -122,17 +126,12 @@ public:
|
||||
|
||||
int getServerID() const { return server_id; }
|
||||
|
||||
/// Get configuration diff between current configuration in RAFT and in XML file
|
||||
ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config);
|
||||
bool applyConfigUpdate(const ClusterUpdateAction& action);
|
||||
|
||||
/// Apply action for configuration update. Actually call raft_instance->remove_srv or raft_instance->add_srv.
|
||||
/// Synchronously check for update results with retries.
|
||||
void applyConfigurationUpdate(const ConfigUpdateAction & task);
|
||||
|
||||
|
||||
/// Wait configuration update for action. Used by followers.
|
||||
/// Return true if update was successfully received.
|
||||
bool waitConfigurationUpdate(const ConfigUpdateAction & task);
|
||||
// TODO (myrrc) these functions should be removed once "reconfig" is stabilized
|
||||
void applyConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action);
|
||||
bool waitForConfigUpdateWithReconfigDisabled(const ClusterUpdateAction& action);
|
||||
ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config);
|
||||
|
||||
uint64_t createSnapshot();
|
||||
|
||||
|
@ -2,24 +2,27 @@
|
||||
#include <future>
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
#include <Coordination/KeeperStateMachine.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <Coordination/KeeperReconfiguration.h>
|
||||
#include <Coordination/ReadBufferFromNuraftBuffer.h>
|
||||
#include <Coordination/WriteBufferFromNuraftBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/errnoToString.h>
|
||||
#include <base/move_extend.h>
|
||||
#include <sys/mman.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperCommon.h>
|
||||
#include <Common/ZooKeeper/ZooKeeperIO.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include "Coordination/KeeperStorage.h"
|
||||
|
||||
#include <Disks/DiskLocal.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event KeeperCommits;
|
||||
extern const Event KeeperReconfigRequest;
|
||||
extern const Event KeeperCommitsFailed;
|
||||
extern const Event KeeperSnapshotCreations;
|
||||
extern const Event KeeperSnapshotCreationsFailed;
|
||||
@ -146,7 +149,7 @@ void assertDigest(
|
||||
"Digest for nodes is not matching after {} request of type '{}'.\nExpected digest - {}, actual digest - {} (digest "
|
||||
"{}). Keeper will terminate to avoid inconsistencies.\nExtra information about the request:\n{}",
|
||||
committing ? "committing" : "preprocessing",
|
||||
Coordination::toString(request.getOpNum()),
|
||||
request.getOpNum(),
|
||||
first.value,
|
||||
second.value,
|
||||
first.version,
|
||||
@ -261,7 +264,8 @@ std::shared_ptr<KeeperStorage::RequestForSession> KeeperStateMachine::parseReque
|
||||
|
||||
bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & request_for_session)
|
||||
{
|
||||
if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
const auto op_num = request_for_session.request->getOpNum();
|
||||
if (op_num == Coordination::OpNum::SessionID || op_num == Coordination::OpNum::Reconfig)
|
||||
return true;
|
||||
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
@ -291,14 +295,105 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req
|
||||
return true;
|
||||
}
|
||||
|
||||
void KeeperStateMachine::reconfigure(const KeeperStorage::RequestForSession& request_for_session)
|
||||
{
|
||||
std::lock_guard _(storage_and_responses_lock);
|
||||
KeeperStorage::ResponseForSession response = processReconfiguration(request_for_session);
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
LOG_WARNING(log,
|
||||
"Failed to push response with session id {} to the queue, probably because of shutdown",
|
||||
response.session_id);
|
||||
}
|
||||
}
|
||||
|
||||
KeeperStorage::ResponseForSession KeeperStateMachine::processReconfiguration(
|
||||
const KeeperStorage::RequestForSession & request_for_session)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperReconfigRequest);
|
||||
|
||||
const auto & request = static_cast<const Coordination::ZooKeeperReconfigRequest&>(*request_for_session.request);
|
||||
const int64_t session_id = request_for_session.session_id;
|
||||
const int64_t zxid = request_for_session.zxid;
|
||||
|
||||
using enum Coordination::Error;
|
||||
auto bad_request = [&](Coordination::Error code = ZBADARGUMENTS) -> KeeperStorage::ResponseForSession
|
||||
{
|
||||
auto res = std::make_shared<Coordination::ZooKeeperReconfigResponse>();
|
||||
res->xid = request.xid;
|
||||
res->zxid = zxid;
|
||||
res->error = code;
|
||||
return { session_id, std::move(res) };
|
||||
};
|
||||
|
||||
if (!storage->checkACL(keeper_config_path, Coordination::ACL::Write, session_id, true))
|
||||
return bad_request(ZNOAUTH);
|
||||
|
||||
KeeperDispatcher& dispatcher = *keeper_context->getDispatcher();
|
||||
if (!dispatcher.reconfigEnabled())
|
||||
return bad_request(ZUNIMPLEMENTED);
|
||||
if (request.version != -1)
|
||||
return bad_request(ZBADVERSION);
|
||||
|
||||
const bool has_new_members = !request.new_members.empty();
|
||||
const bool has_joining = !request.joining.empty();
|
||||
const bool has_leaving = !request.leaving.empty();
|
||||
const bool incremental_reconfig = (has_joining || has_leaving) && !has_new_members;
|
||||
if (!incremental_reconfig)
|
||||
return bad_request();
|
||||
|
||||
const ClusterConfigPtr config = getClusterConfig();
|
||||
if (!config) // Server can be uninitialized yet
|
||||
return bad_request();
|
||||
|
||||
ClusterUpdateActions updates;
|
||||
|
||||
if (has_joining)
|
||||
{
|
||||
if (auto join_updates = joiningToClusterUpdates(config, request.joining); !join_updates.empty())
|
||||
moveExtend(updates, std::move(join_updates));
|
||||
else
|
||||
return bad_request();
|
||||
}
|
||||
|
||||
if (has_leaving)
|
||||
{
|
||||
if (auto leave_updates = leavingToClusterUpdates(config, request.leaving); !leave_updates.empty())
|
||||
moveExtend(updates, std::move(leave_updates));
|
||||
else
|
||||
return bad_request();
|
||||
}
|
||||
|
||||
auto response = std::make_shared<Coordination::ZooKeeperReconfigResponse>();
|
||||
response->xid = request.xid;
|
||||
response->zxid = zxid;
|
||||
response->error = Coordination::Error::ZOK;
|
||||
response->value = serializeClusterConfig(config, updates);
|
||||
|
||||
dispatcher.pushClusterUpdates(std::move(updates));
|
||||
return { session_id, std::move(response) };
|
||||
}
|
||||
|
||||
nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, nuraft::buffer & data)
|
||||
{
|
||||
auto request_for_session = parseRequest(data, true);
|
||||
if (!request_for_session->zxid)
|
||||
request_for_session->zxid = log_idx;
|
||||
|
||||
/// Special processing of session_id request
|
||||
if (request_for_session->request->getOpNum() == Coordination::OpNum::SessionID)
|
||||
auto try_push = [this](const KeeperStorage::ResponseForSession& response)
|
||||
{
|
||||
if (!responses_queue.push(response))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
LOG_WARNING(log,
|
||||
"Failed to push response with session id {} to the queue, probably because of shutdown",
|
||||
response.session_id);
|
||||
}
|
||||
};
|
||||
|
||||
const auto op_num = request_for_session->request->getOpNum();
|
||||
if (op_num == Coordination::OpNum::SessionID)
|
||||
{
|
||||
const Coordination::ZooKeeperSessionIDRequest & session_id_request
|
||||
= dynamic_cast<const Coordination::ZooKeeperSessionIDRequest &>(*request_for_session->request);
|
||||
@ -309,21 +404,16 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
KeeperStorage::ResponseForSession response_for_session;
|
||||
response_for_session.session_id = -1;
|
||||
response_for_session.response = response;
|
||||
{
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
session_id = storage->getSessionID(session_id_request.session_timeout_ms);
|
||||
LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms);
|
||||
response->session_id = session_id;
|
||||
if (!responses_queue.push(response_for_session))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
LOG_WARNING(log, "Failed to push response with session id {} to the queue, probably because of shutdown", session_id);
|
||||
}
|
||||
}
|
||||
|
||||
std::lock_guard lock(storage_and_responses_lock);
|
||||
session_id = storage->getSessionID(session_id_request.session_timeout_ms);
|
||||
LOG_DEBUG(log, "Session ID response {} with timeout {}", session_id, session_id_request.session_timeout_ms);
|
||||
response->session_id = session_id;
|
||||
try_push(response_for_session);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (request_for_session->request->getOpNum() == Coordination::OpNum::Close)
|
||||
if (op_num == Coordination::OpNum::Close)
|
||||
{
|
||||
std::lock_guard lock(request_cache_mutex);
|
||||
parsed_request_cache.erase(request_for_session->session_id);
|
||||
@ -333,14 +423,7 @@ nuraft::ptr<nuraft::buffer> KeeperStateMachine::commit(const uint64_t log_idx, n
|
||||
KeeperStorage::ResponsesForSessions responses_for_sessions
|
||||
= storage->processRequest(request_for_session->request, request_for_session->session_id, request_for_session->zxid);
|
||||
for (auto & response_for_session : responses_for_sessions)
|
||||
if (!responses_queue.push(response_for_session))
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::KeeperCommitsFailed);
|
||||
LOG_WARNING(
|
||||
log,
|
||||
"Failed to push response with session id {} to the queue, probably because of shutdown",
|
||||
response_for_session.session_id);
|
||||
}
|
||||
try_push(response_for_session);
|
||||
|
||||
if (keeper_context->digestEnabled() && request_for_session->digest)
|
||||
assertDigest(*request_for_session->digest, storage->getNodesDigest(true), *request_for_session->request, true);
|
||||
@ -782,5 +865,4 @@ void KeeperStateMachine::recalculateStorageStats()
|
||||
storage->recalculateStats();
|
||||
LOG_INFO(log, "Done recalculating storage stats");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -12,7 +12,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using ResponsesQueue = ConcurrentBoundedQueue<KeeperStorage::ResponseForSession>;
|
||||
using SnapshotsQueue = ConcurrentBoundedQueue<CreateSnapshotTask>;
|
||||
|
||||
@ -67,7 +66,9 @@ public:
|
||||
// (can happen in case of exception during preprocessing)
|
||||
void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing);
|
||||
|
||||
void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing);
|
||||
void rollbackRequestNoLock(
|
||||
const KeeperStorage::RequestForSession & request_for_session,
|
||||
bool allow_missing) TSA_NO_THREAD_SAFETY_ANALYSIS;
|
||||
|
||||
uint64_t last_commit_index() override { return last_committed_idx; }
|
||||
|
||||
@ -87,8 +88,13 @@ public:
|
||||
int read_logical_snp_obj(
|
||||
nuraft::snapshot & s, void *& user_snp_ctx, uint64_t obj_id, nuraft::ptr<nuraft::buffer> & data_out, bool & is_last_obj) override;
|
||||
|
||||
/// just for test
|
||||
KeeperStorage & getStorage() { return *storage; }
|
||||
// This should be used only for tests or keeper-data-dumper because it violates
|
||||
// TSA -- we can't acquire the lock outside of this class or return a storage under lock
|
||||
// in a reasonable way.
|
||||
KeeperStorage & getStorageUnsafe() TSA_NO_THREAD_SAFETY_ANALYSIS
|
||||
{
|
||||
return *storage;
|
||||
}
|
||||
|
||||
void shutdownStorage();
|
||||
|
||||
@ -122,6 +128,9 @@ public:
|
||||
uint64_t getLatestSnapshotBufSize() const;
|
||||
|
||||
void recalculateStorageStats();
|
||||
|
||||
void reconfigure(const KeeperStorage::RequestForSession& request_for_session);
|
||||
|
||||
private:
|
||||
CommitCallback commit_callback;
|
||||
/// In our state machine we always have a single snapshot which is stored
|
||||
@ -133,7 +142,7 @@ private:
|
||||
CoordinationSettingsPtr coordination_settings;
|
||||
|
||||
/// Main state machine logic
|
||||
KeeperStoragePtr storage;
|
||||
KeeperStoragePtr storage TSA_PT_GUARDED_BY(storage_and_responses_lock);
|
||||
|
||||
/// Save/Load and Serialize/Deserialize logic for snapshots.
|
||||
KeeperSnapshotManager snapshot_manager;
|
||||
@ -178,6 +187,9 @@ private:
|
||||
KeeperContextPtr keeper_context;
|
||||
|
||||
KeeperSnapshotManagerS3 * snapshot_manager_s3;
|
||||
};
|
||||
|
||||
KeeperStorage::ResponseForSession processReconfiguration(
|
||||
const KeeperStorage::RequestForSession& request_for_session)
|
||||
TSA_REQUIRES(storage_and_responses_lock);
|
||||
};
|
||||
}
|
||||
|
@ -451,7 +451,7 @@ nuraft::ptr<nuraft::srv_state> KeeperStateManager::read_state()
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const
|
||||
ClusterUpdateActions KeeperStateManager::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const
|
||||
{
|
||||
auto new_configuration_wrapper = parseServersConfiguration(config, true);
|
||||
|
||||
@ -465,14 +465,14 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A
|
||||
old_ids[old_server->get_id()] = old_server;
|
||||
}
|
||||
|
||||
ConfigUpdateActions result;
|
||||
ClusterUpdateActions result;
|
||||
|
||||
/// First of all add new servers
|
||||
for (const auto & [new_id, server_config] : new_ids)
|
||||
{
|
||||
auto old_server_it = old_ids.find(new_id);
|
||||
if (old_server_it == old_ids.end())
|
||||
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::AddServer, server_config});
|
||||
result.emplace_back(AddRaftServer{RaftServerConfig{*server_config}});
|
||||
else
|
||||
{
|
||||
const auto & old_endpoint = old_server_it->second->get_endpoint();
|
||||
@ -491,10 +491,8 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A
|
||||
|
||||
/// After that remove old ones
|
||||
for (auto [old_id, server_config] : old_ids)
|
||||
{
|
||||
if (!new_ids.contains(old_id))
|
||||
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::RemoveServer, server_config});
|
||||
}
|
||||
result.emplace_back(RemoveRaftServer{old_id});
|
||||
|
||||
{
|
||||
std::lock_guard lock(configuration_wrapper_mutex);
|
||||
@ -507,7 +505,10 @@ ConfigUpdateActions KeeperStateManager::getConfigurationDiff(const Poco::Util::A
|
||||
{
|
||||
if (old_server->get_priority() != new_server->get_priority())
|
||||
{
|
||||
result.emplace_back(ConfigUpdateAction{ConfigUpdateActionType::UpdatePriority, new_server});
|
||||
result.emplace_back(UpdateRaftServerPriority{
|
||||
.id = new_server->get_id(),
|
||||
.priority = new_server->get_priority()
|
||||
});
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -7,31 +7,13 @@
|
||||
#include <libnuraft/nuraft.hxx>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include "Coordination/KeeperStateMachine.h"
|
||||
#include "Coordination/RaftServerConfig.h"
|
||||
#include <Coordination/KeeperSnapshotManager.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using KeeperServerConfigPtr = nuraft::ptr<nuraft::srv_config>;
|
||||
|
||||
/// When our configuration changes the following action types
|
||||
/// can happen
|
||||
enum class ConfigUpdateActionType
|
||||
{
|
||||
RemoveServer,
|
||||
AddServer,
|
||||
UpdatePriority,
|
||||
};
|
||||
|
||||
/// Action to update configuration
|
||||
struct ConfigUpdateAction
|
||||
{
|
||||
ConfigUpdateActionType action_type;
|
||||
KeeperServerConfigPtr server;
|
||||
};
|
||||
|
||||
using ConfigUpdateActions = std::vector<ConfigUpdateAction>;
|
||||
|
||||
/// Responsible for managing our and cluster configuration
|
||||
class KeeperStateManager : public nuraft::state_mgr
|
||||
{
|
||||
@ -74,7 +56,11 @@ public:
|
||||
|
||||
int32_t server_id() override { return my_server_id; }
|
||||
|
||||
nuraft::ptr<nuraft::srv_config> get_srv_config() const { return configuration_wrapper.config; } /// NOLINT
|
||||
nuraft::ptr<nuraft::srv_config> get_srv_config() const
|
||||
{
|
||||
std::lock_guard lk(configuration_wrapper_mutex);
|
||||
return configuration_wrapper.config;
|
||||
}
|
||||
|
||||
void system_exit(const int exit_code) override; /// NOLINT
|
||||
|
||||
@ -106,8 +92,8 @@ public:
|
||||
/// Read all log entries in log store from the begging and return latest config (with largest log_index)
|
||||
ClusterConfigPtr getLatestConfigFromLogStore() const;
|
||||
|
||||
/// Get configuration diff between proposed XML and current state in RAFT
|
||||
ConfigUpdateActions getConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const;
|
||||
// TODO (myrrc) This should be removed once "reconfig" is stabilized
|
||||
ClusterUpdateActions getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config) const;
|
||||
|
||||
private:
|
||||
const String & getOldServerStatePath();
|
||||
@ -133,7 +119,7 @@ private:
|
||||
std::string config_prefix;
|
||||
|
||||
mutable std::mutex configuration_wrapper_mutex;
|
||||
KeeperConfigurationWrapper configuration_wrapper;
|
||||
KeeperConfigurationWrapper configuration_wrapper TSA_GUARDED_BY(configuration_wrapper_mutex);
|
||||
|
||||
nuraft::ptr<KeeperLogStore> log_store;
|
||||
|
||||
|
@ -20,10 +20,10 @@
|
||||
|
||||
#include <Coordination/pathUtils.h>
|
||||
#include <Coordination/KeeperConstants.h>
|
||||
#include <Coordination/KeeperReconfiguration.h>
|
||||
#include <Coordination/KeeperStorage.h>
|
||||
#include <Coordination/KeeperDispatcher.h>
|
||||
|
||||
#include <sstream>
|
||||
#include <iomanip>
|
||||
#include <mutex>
|
||||
#include <functional>
|
||||
#include <base/defines.h>
|
||||
@ -53,7 +53,6 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
String getSHA1(const String & userdata)
|
||||
{
|
||||
Poco::SHA1Engine engine;
|
||||
@ -1060,7 +1059,8 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
|
||||
ProfileEvents::increment(ProfileEvents::KeeperGetRequest);
|
||||
Coordination::ZooKeeperGetRequest & request = dynamic_cast<Coordination::ZooKeeperGetRequest &>(*zk_request);
|
||||
|
||||
if (request.path == Coordination::keeper_api_feature_flags_path)
|
||||
if (request.path == Coordination::keeper_api_feature_flags_path
|
||||
|| request.path == Coordination::keeper_config_path)
|
||||
return {};
|
||||
|
||||
if (!storage.uncommitted_state.getNode(request.path))
|
||||
@ -1085,6 +1085,14 @@ struct KeeperStorageGetRequestProcessor final : public KeeperStorageRequestProce
|
||||
}
|
||||
}
|
||||
|
||||
if (request.path == Coordination::keeper_config_path)
|
||||
{
|
||||
response.data = serializeClusterConfig(
|
||||
storage.keeper_context->getDispatcher()->getStateMachine().getClusterConfig());
|
||||
response.error = Coordination::Error::ZOK;
|
||||
return response_ptr;
|
||||
}
|
||||
|
||||
auto & container = storage.container;
|
||||
auto node_it = container.find(request.path);
|
||||
if (node_it == container.end())
|
||||
@ -1784,7 +1792,7 @@ struct KeeperStorageMultiRequestProcessor final : public KeeperStorageRequestPro
|
||||
throw DB::Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Illegal command as part of multi ZooKeeper request {}",
|
||||
Coordination::toString(sub_zk_request->getOpNum()));
|
||||
sub_zk_request->getOpNum());
|
||||
}
|
||||
}
|
||||
|
||||
@ -1975,7 +1983,7 @@ public:
|
||||
{
|
||||
auto request_it = op_num_to_request.find(zk_request->getOpNum());
|
||||
if (request_it == op_num_to_request.end())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", toString(zk_request->getOpNum()));
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknown operation type {}", zk_request->getOpNum());
|
||||
|
||||
return request_it->second(zk_request);
|
||||
}
|
||||
|
97
src/Coordination/RaftServerConfig.cpp
Normal file
97
src/Coordination/RaftServerConfig.cpp
Normal file
@ -0,0 +1,97 @@
|
||||
#include "RaftServerConfig.h"
|
||||
#include <unordered_set>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <base/find_symbols.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
RaftServerConfig::RaftServerConfig(const nuraft::srv_config & cfg) noexcept
|
||||
: id(cfg.get_id()), endpoint(cfg.get_endpoint()), learner(cfg.is_learner()), priority(cfg.get_priority())
|
||||
{
|
||||
}
|
||||
|
||||
RaftServerConfig::operator nuraft::srv_config() const noexcept
|
||||
{
|
||||
return {id, 0, endpoint, "", learner, priority};
|
||||
}
|
||||
|
||||
std::optional<RaftServerConfig> RaftServerConfig::parse(std::string_view server) noexcept
|
||||
{
|
||||
std::vector<std::string_view> parts;
|
||||
splitInto<';', '='>(parts, server);
|
||||
|
||||
const bool with_id_endpoint = parts.size() == 2;
|
||||
const bool with_server_type = parts.size() == 3;
|
||||
const bool with_priority = parts.size() == 4;
|
||||
if (!with_id_endpoint && !with_server_type && !with_priority)
|
||||
return std::nullopt;
|
||||
|
||||
const std::string_view id_str = parts[0];
|
||||
if (!id_str.starts_with("server."))
|
||||
return std::nullopt;
|
||||
|
||||
Int32 id;
|
||||
if (!tryParse(id, std::next(id_str.begin(), 7)))
|
||||
return std::nullopt;
|
||||
if (id <= 0)
|
||||
return std::nullopt;
|
||||
|
||||
const std::string_view endpoint = parts[1];
|
||||
const size_t port_delimiter = endpoint.find_last_of(':');
|
||||
if (port_delimiter == std::string::npos)
|
||||
return {};
|
||||
const std::string_view port = endpoint.substr(port_delimiter + 1);
|
||||
|
||||
uint16_t port_tmp;
|
||||
if (!tryParse(port_tmp, port))
|
||||
return std::nullopt;
|
||||
|
||||
RaftServerConfig out{id, endpoint};
|
||||
|
||||
if (with_id_endpoint)
|
||||
return out;
|
||||
|
||||
if (parts[2] != "learner" && parts[2] != "participant")
|
||||
return std::nullopt;
|
||||
out.learner = parts[2] == "learner";
|
||||
if (with_server_type)
|
||||
return out;
|
||||
|
||||
const std::string_view priority = parts[3];
|
||||
if (!tryParse(out.priority, priority))
|
||||
return std::nullopt;
|
||||
if (out.priority < 0)
|
||||
return std::nullopt;
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
RaftServers parseRaftServers(std::string_view servers)
|
||||
{
|
||||
std::vector<std::string_view> server_arr;
|
||||
std::unordered_set<int32_t> ids;
|
||||
std::unordered_set<String> endpoints;
|
||||
RaftServers out;
|
||||
|
||||
for (auto & server : splitInto<','>(server_arr, servers))
|
||||
{
|
||||
if (auto maybe_server = RaftServerConfig::parse(server))
|
||||
{
|
||||
String endpoint = maybe_server->endpoint;
|
||||
if (endpoints.contains(endpoint))
|
||||
return {};
|
||||
const int id = maybe_server->id;
|
||||
if (ids.contains(id))
|
||||
return {};
|
||||
|
||||
out.emplace_back(std::move(*maybe_server));
|
||||
endpoints.emplace(std::move(endpoint));
|
||||
ids.emplace(id);
|
||||
}
|
||||
else
|
||||
return {};
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
}
|
78
src/Coordination/RaftServerConfig.h
Normal file
78
src/Coordination/RaftServerConfig.h
Normal file
@ -0,0 +1,78 @@
|
||||
#pragma once
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <fmt/core.h>
|
||||
#include <libnuraft/srv_config.hxx>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
// default- and copy-constructible version of nuraft::srv_config
|
||||
struct RaftServerConfig
|
||||
{
|
||||
int id;
|
||||
String endpoint;
|
||||
bool learner;
|
||||
int priority;
|
||||
|
||||
constexpr RaftServerConfig() = default;
|
||||
constexpr RaftServerConfig(int id_, std::string_view endpoint_, bool learner_ = false, int priority_ = 1)
|
||||
: id(id_), endpoint(endpoint_), learner(learner_), priority(priority_)
|
||||
{
|
||||
}
|
||||
|
||||
constexpr bool operator==(const RaftServerConfig &) const = default;
|
||||
explicit RaftServerConfig(const nuraft::srv_config & cfg) noexcept;
|
||||
explicit operator nuraft::srv_config() const noexcept;
|
||||
|
||||
/// Parse server in format "server.id=host:port[;learner][;priority]"
|
||||
static std::optional<RaftServerConfig> parse(std::string_view server) noexcept;
|
||||
};
|
||||
|
||||
using RaftServers = std::vector<RaftServerConfig>;
|
||||
/// Parse comma-delimited servers. Check for duplicate endpoints and ids.
|
||||
/// @returns {} on parsing or validation error.
|
||||
RaftServers parseRaftServers(std::string_view servers);
|
||||
|
||||
struct AddRaftServer : RaftServerConfig
|
||||
{
|
||||
};
|
||||
|
||||
struct RemoveRaftServer
|
||||
{
|
||||
int id;
|
||||
};
|
||||
|
||||
struct UpdateRaftServerPriority
|
||||
{
|
||||
int id;
|
||||
int priority;
|
||||
};
|
||||
|
||||
using ClusterUpdateAction = std::variant<AddRaftServer, RemoveRaftServer, UpdateRaftServerPriority>;
|
||||
using ClusterUpdateActions = std::vector<ClusterUpdateAction>;
|
||||
}
|
||||
|
||||
template <>
|
||||
struct fmt::formatter<DB::RaftServerConfig> : fmt::formatter<string_view>
|
||||
{
|
||||
constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx)
|
||||
{
|
||||
return fmt::format_to(
|
||||
ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct fmt::formatter<DB::ClusterUpdateAction> : fmt::formatter<string_view>
|
||||
{
|
||||
constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx)
|
||||
{
|
||||
if (const auto * add = std::get_if<DB::AddRaftServer>(&action))
|
||||
return fmt::format_to(ctx.out(), "(Add server {})", add->id);
|
||||
if (const auto * remove = std::get_if<DB::RemoveRaftServer>(&action))
|
||||
return fmt::format_to(ctx.out(), "(Remove server {})", remove->id);
|
||||
if (const auto * update = std::get_if<DB::UpdateRaftServerPriority>(&action))
|
||||
return fmt::format_to(ctx.out(), "(Change server {} priority to {})", update->id, update->priority);
|
||||
UNREACHABLE();
|
||||
}
|
||||
};
|
@ -84,6 +84,47 @@ protected:
|
||||
}
|
||||
};
|
||||
|
||||
TEST_P(CoordinationTest, RaftServerConfigParse)
|
||||
{
|
||||
auto parse = Coordination::RaftServerConfig::parse;
|
||||
using Cfg = std::optional<DB::RaftServerConfig>;
|
||||
|
||||
EXPECT_EQ(parse(""), std::nullopt);
|
||||
EXPECT_EQ(parse("="), std::nullopt);
|
||||
EXPECT_EQ(parse("=;"), std::nullopt);
|
||||
EXPECT_EQ(parse("=;;"), std::nullopt);
|
||||
EXPECT_EQ(parse("=:80"), std::nullopt);
|
||||
EXPECT_EQ(parse("server."), std::nullopt);
|
||||
EXPECT_EQ(parse("server.=:80"), std::nullopt);
|
||||
EXPECT_EQ(parse("server.-5=1:2"), std::nullopt);
|
||||
EXPECT_EQ(parse("server.1=host;-123"), std::nullopt);
|
||||
EXPECT_EQ(parse("server.1=host:999"), (Cfg{{1, "host:999"}}));
|
||||
EXPECT_EQ(parse("server.1=host:999;learner"), (Cfg{{1, "host:999", true}}));
|
||||
EXPECT_EQ(parse("server.1=host:999;participant"), (Cfg{{1, "host:999", false}}));
|
||||
EXPECT_EQ(parse("server.1=host:999;learner;25"), (Cfg{{1, "host:999", true, 25}}));
|
||||
|
||||
EXPECT_EQ(parse("server.1=127.0.0.1:80"), (Cfg{{1, "127.0.0.1:80"}}));
|
||||
EXPECT_EQ(
|
||||
parse("server.1=2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"),
|
||||
(Cfg{{1, "2001:0db8:85a3:0000:0000:8a2e:0370:7334:80"}}));
|
||||
}
|
||||
|
||||
TEST_P(CoordinationTest, RaftServerClusterConfigParse)
|
||||
{
|
||||
auto parse = Coordination::parseRaftServers;
|
||||
using Cfg = DB::RaftServerConfig;
|
||||
using Servers = DB::RaftServers;
|
||||
|
||||
EXPECT_EQ(parse(""), Servers{});
|
||||
EXPECT_EQ(parse(","), Servers{});
|
||||
EXPECT_EQ(parse("1,2"), Servers{});
|
||||
EXPECT_EQ(parse("server.1=host:80,server.1=host2:80"), Servers{});
|
||||
EXPECT_EQ(parse("server.1=host:80,server.2=host:80"), Servers{});
|
||||
EXPECT_EQ(
|
||||
parse("server.1=host:80,server.2=host:81"),
|
||||
(Servers{Cfg{1, "host:80"}, Cfg{2, "host:81"}}));
|
||||
}
|
||||
|
||||
TEST_P(CoordinationTest, BuildTest)
|
||||
{
|
||||
DB::InMemoryLogStore store;
|
||||
@ -1575,8 +1616,8 @@ void testLogAndStateMachine(
|
||||
restore_machine->commit(i, changelog.entry_at(i)->get_buf());
|
||||
}
|
||||
|
||||
auto & source_storage = state_machine->getStorage();
|
||||
auto & restored_storage = restore_machine->getStorage();
|
||||
auto & source_storage = state_machine->getStorageUnsafe();
|
||||
auto & restored_storage = restore_machine->getStorageUnsafe();
|
||||
|
||||
EXPECT_EQ(source_storage.container.size(), restored_storage.container.size());
|
||||
for (size_t i = 1; i < total_logs + 1; ++i)
|
||||
@ -1678,7 +1719,7 @@ TEST_P(CoordinationTest, TestEphemeralNodeRemove)
|
||||
auto entry_c = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), request_c);
|
||||
state_machine->pre_commit(1, entry_c->get_buf());
|
||||
state_machine->commit(1, entry_c->get_buf());
|
||||
const auto & storage = state_machine->getStorage();
|
||||
const auto & storage = state_machine->getStorageUnsafe();
|
||||
|
||||
EXPECT_EQ(storage.ephemerals.size(), 1);
|
||||
std::shared_ptr<ZooKeeperRemoveRequest> request_d = std::make_shared<ZooKeeperRemoveRequest>();
|
||||
@ -1727,7 +1768,7 @@ TEST_P(CoordinationTest, TestCreateNodeWithAuthSchemeForAclWhenAuthIsPrecommitte
|
||||
auto create_entry = getLogEntryFromZKRequest(0, 1, state_machine->getNextZxid(), create_req);
|
||||
state_machine->pre_commit(2, create_entry->get_buf());
|
||||
|
||||
const auto & uncommitted_state = state_machine->getStorage().uncommitted_state;
|
||||
const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state;
|
||||
ASSERT_TRUE(uncommitted_state.nodes.contains(node_path));
|
||||
|
||||
// commit log entries
|
||||
@ -1790,7 +1831,7 @@ TEST_P(CoordinationTest, TestSetACLWithAuthSchemeForAclWhenAuthIsPrecommitted)
|
||||
state_machine->commit(2, create_entry->get_buf());
|
||||
state_machine->commit(3, set_acl_entry->get_buf());
|
||||
|
||||
const auto & uncommitted_state = state_machine->getStorage().uncommitted_state;
|
||||
const auto & uncommitted_state = state_machine->getStorageUnsafe().uncommitted_state;
|
||||
auto node = uncommitted_state.getNode(node_path);
|
||||
|
||||
ASSERT_NE(node, nullptr);
|
||||
|
@ -129,6 +129,7 @@ class IColumn;
|
||||
M(Bool, optimize_move_to_prewhere_if_final, false, "If query has `FINAL`, the optimization `move_to_prewhere` is not always correct and it is enabled only if both settings `optimize_move_to_prewhere` and `optimize_move_to_prewhere_if_final` are turned on", 0) \
|
||||
M(Bool, move_all_conditions_to_prewhere, true, "Move all viable conditions from WHERE to PREWHERE", 0) \
|
||||
M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \
|
||||
M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \
|
||||
\
|
||||
M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \
|
||||
M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \
|
||||
@ -577,7 +578,7 @@ class IColumn;
|
||||
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
|
||||
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
|
||||
M(Bool, optimize_use_projections, true, "Automatically choose projections to perform SELECT query", 0) ALIAS(allow_experimental_projection_optimization) \
|
||||
M(Bool, optimize_use_implicit_projections, false, "Automatically choose implicit projections to perform SELECT query", 0) \
|
||||
M(Bool, optimize_use_implicit_projections, true, "Automatically choose implicit projections to perform SELECT query", 0) \
|
||||
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
|
||||
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
|
||||
M(Bool, async_query_sending_for_remote, true, "Asynchronously create connections and send query to shards in remote query", 0) \
|
||||
|
@ -80,7 +80,6 @@ namespace SettingsChangesHistory
|
||||
/// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972)
|
||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"23.7", {{"optimize_use_implicit_projections", true, false, "Disable implicit projections due to unexpected results."}}},
|
||||
{"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."},
|
||||
{"http_receive_timeout", 180, 30, "See http_send_timeout."}}},
|
||||
{"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."},
|
||||
|
@ -32,7 +32,7 @@ namespace
|
||||
/// We need a unique name for a created custom disk, but it needs to be the same
|
||||
/// after table is reattached or server is restarted, so take a hash of the disk
|
||||
/// configuration serialized ast as a disk name suffix.
|
||||
auto disk_setting_string = serializeAST(function, true);
|
||||
auto disk_setting_string = serializeAST(function);
|
||||
disk_name = DiskSelector::TMP_INTERNAL_DISK_PREFIX
|
||||
+ toString(sipHash128(disk_setting_string.data(), disk_setting_string.size()));
|
||||
}
|
||||
|
@ -49,6 +49,9 @@ struct CountSubstringsImpl
|
||||
/// FIXME: suboptimal
|
||||
memset(&res[0], 0, res.size() * sizeof(res[0]));
|
||||
|
||||
if (needle.empty())
|
||||
return; // Return all zeros
|
||||
|
||||
/// Current index in the array of strings.
|
||||
size_t i = 0;
|
||||
|
||||
@ -223,16 +226,19 @@ struct CountSubstringsImpl
|
||||
const char * needle_beg = reinterpret_cast<const char *>(&needle_data[prev_needle_offset]);
|
||||
size_t needle_size = needle_offsets[i] - prev_needle_offset - 1;
|
||||
|
||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size);
|
||||
|
||||
const UInt8 * end = reinterpret_cast<const UInt8 *>(haystack.data() + haystack.size());
|
||||
const UInt8 * beg = reinterpret_cast<const UInt8 *>(Impl::advancePos(haystack.data(), reinterpret_cast<const char *>(end), start - 1));
|
||||
|
||||
const UInt8 * pos;
|
||||
while ((pos = searcher.search(beg, end)) < end)
|
||||
if (needle_size > 0)
|
||||
{
|
||||
++res[i];
|
||||
beg = pos + needle_size;
|
||||
typename Impl::SearcherInSmallHaystack searcher = Impl::createSearcherInSmallHaystack(needle_beg, needle_size);
|
||||
|
||||
const UInt8 * end = reinterpret_cast<const UInt8 *>(haystack.data() + haystack.size());
|
||||
const UInt8 * beg = reinterpret_cast<const UInt8 *>(Impl::advancePos(haystack.data(), reinterpret_cast<const char *>(end), start - 1));
|
||||
|
||||
const UInt8 * pos;
|
||||
while ((pos = searcher.search(beg, end)) < end)
|
||||
{
|
||||
++res[i];
|
||||
beg = pos + needle_size;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -976,7 +976,15 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
|
||||
if (node.name == "indexHint")
|
||||
{
|
||||
if (data.only_consts)
|
||||
{
|
||||
/// We need to collect constants inside `indexHint` for index analysis.
|
||||
if (node.arguments)
|
||||
{
|
||||
for (const auto & arg : node.arguments->children)
|
||||
visit(arg, data);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
/// Here we create a separate DAG for indexHint condition.
|
||||
/// It will be used only for index analysis.
|
||||
|
@ -870,13 +870,12 @@ void FileCache::loadMetadata()
|
||||
}
|
||||
|
||||
size_t total_size = 0;
|
||||
for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()};
|
||||
key_prefix_it != fs::directory_iterator();)
|
||||
for (auto key_prefix_it = fs::directory_iterator{metadata.getBaseDirectory()}; key_prefix_it != fs::directory_iterator();
|
||||
key_prefix_it++)
|
||||
{
|
||||
const fs::path key_prefix_directory = key_prefix_it->path();
|
||||
key_prefix_it++;
|
||||
|
||||
if (!fs::is_directory(key_prefix_directory))
|
||||
if (!key_prefix_it->is_directory())
|
||||
{
|
||||
if (key_prefix_directory.filename() != "status")
|
||||
{
|
||||
@ -887,19 +886,19 @@ void FileCache::loadMetadata()
|
||||
continue;
|
||||
}
|
||||
|
||||
if (fs::is_empty(key_prefix_directory))
|
||||
fs::directory_iterator key_it{key_prefix_directory};
|
||||
if (key_it == fs::directory_iterator{})
|
||||
{
|
||||
LOG_DEBUG(log, "Removing empty key prefix directory: {}", key_prefix_directory.string());
|
||||
fs::remove(key_prefix_directory);
|
||||
continue;
|
||||
}
|
||||
|
||||
for (fs::directory_iterator key_it{key_prefix_directory}; key_it != fs::directory_iterator();)
|
||||
for (/* key_it already initialized to verify emptiness */; key_it != fs::directory_iterator(); key_it++)
|
||||
{
|
||||
const fs::path key_directory = key_it->path();
|
||||
++key_it;
|
||||
|
||||
if (!fs::is_directory(key_directory))
|
||||
if (!key_it->is_directory())
|
||||
{
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
@ -908,7 +907,7 @@ void FileCache::loadMetadata()
|
||||
continue;
|
||||
}
|
||||
|
||||
if (fs::is_empty(key_directory))
|
||||
if (fs::directory_iterator{key_directory} == fs::directory_iterator{})
|
||||
{
|
||||
LOG_DEBUG(log, "Removing empty key directory: {}", key_directory.string());
|
||||
fs::remove(key_directory);
|
||||
|
@ -60,7 +60,7 @@ public:
|
||||
|
||||
/// The SELECT query as plain string, displayed in SYSTEM.QUERY_CACHE. Stored explicitly, i.e. not constructed from the AST, for the
|
||||
/// sole reason that QueryCache-related SETTINGS are pruned from the AST (see removeQueryCacheSettings()) which will look ugly in
|
||||
/// the SYSTEM.QUERY_CACHE.
|
||||
/// SYSTEM.QUERY_CACHE.
|
||||
const String query_string;
|
||||
|
||||
/// Ctor to construct a Key for writing into query cache.
|
||||
|
@ -302,7 +302,8 @@ void GraceHashJoin::initBuckets()
|
||||
bool GraceHashJoin::isSupported(const std::shared_ptr<TableJoin> & table_join)
|
||||
{
|
||||
bool is_asof = (table_join->strictness() == JoinStrictness::Asof);
|
||||
return !is_asof && isInnerOrLeft(table_join->kind()) && table_join->oneDisjunct();
|
||||
auto kind = table_join->kind();
|
||||
return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct();
|
||||
}
|
||||
|
||||
GraceHashJoin::~GraceHashJoin() = default;
|
||||
@ -322,7 +323,6 @@ bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) con
|
||||
/// One row can't be split, avoid loop
|
||||
if (total_rows < 2)
|
||||
return false;
|
||||
|
||||
bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes);
|
||||
|
||||
if (has_overflow)
|
||||
@ -494,17 +494,30 @@ bool GraceHashJoin::alwaysReturnsEmptySet() const
|
||||
return hash_join_is_empty;
|
||||
}
|
||||
|
||||
IBlocksStreamPtr GraceHashJoin::getNonJoinedBlocks(const Block &, const Block &, UInt64) const
|
||||
/// Each bucket are handled by the following steps
|
||||
/// 1. build hash_join by the right side blocks.
|
||||
/// 2. join left side with the hash_join,
|
||||
/// 3. read right non-joined blocks from hash_join.
|
||||
/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted.
|
||||
///
|
||||
/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform,
|
||||
/// only one processor could take the non-joined blocks from right stream, and ensure all rows from
|
||||
/// left stream have been emitted before this.
|
||||
IBlocksStreamPtr
|
||||
GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const
|
||||
{
|
||||
/// We do no support returning non joined blocks here.
|
||||
/// TODO: They _should_ be reported by getDelayedBlocks instead
|
||||
return nullptr;
|
||||
return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_);
|
||||
}
|
||||
|
||||
class GraceHashJoin::DelayedBlocks : public IBlocksStream
|
||||
{
|
||||
public:
|
||||
explicit DelayedBlocks(size_t current_bucket_, Buckets buckets_, InMemoryJoinPtr hash_join_, const Names & left_key_names_, const Names & right_key_names_)
|
||||
explicit DelayedBlocks(
|
||||
size_t current_bucket_,
|
||||
Buckets buckets_,
|
||||
InMemoryJoinPtr hash_join_,
|
||||
const Names & left_key_names_,
|
||||
const Names & right_key_names_)
|
||||
: current_bucket(current_bucket_)
|
||||
, buckets(std::move(buckets_))
|
||||
, hash_join(std::move(hash_join_))
|
||||
@ -522,12 +535,15 @@ public:
|
||||
|
||||
do
|
||||
{
|
||||
// One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform.
|
||||
// There is a lock inside left_reader.read() .
|
||||
block = left_reader.read();
|
||||
if (!block)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
// block comes from left_reader, need to join with right table to get the result.
|
||||
Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets);
|
||||
block = std::move(blocks[current_idx]);
|
||||
|
||||
|
@ -13,7 +13,6 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class TableJoin;
|
||||
class HashJoin;
|
||||
|
||||
@ -79,7 +78,7 @@ public:
|
||||
bool supportTotals() const override { return false; }
|
||||
|
||||
IBlocksStreamPtr
|
||||
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
|
||||
getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size) const override;
|
||||
|
||||
/// Open iterator over joined blocks.
|
||||
/// Must be called after all @joinBlock calls.
|
||||
|
@ -40,20 +40,20 @@ SELECT *
|
||||
FROM (
|
||||
(SELECT
|
||||
name AS table,
|
||||
0 AS non_unique,
|
||||
1 AS non_unique,
|
||||
'PRIMARY' AS key_name,
|
||||
NULL AS seq_in_index,
|
||||
NULL AS column_name,
|
||||
row_number() over (order by column_name) AS seq_in_index,
|
||||
arrayJoin(splitByString(', ', primary_key)) AS column_name,
|
||||
'A' AS collation,
|
||||
NULL AS cardinality,
|
||||
0 AS cardinality,
|
||||
NULL AS sub_part,
|
||||
NULL AS packed,
|
||||
NULL AS null,
|
||||
'primary' AS index_type,
|
||||
NULL AS comment,
|
||||
NULL AS index_comment,
|
||||
'PRIMARY' AS index_type,
|
||||
'' AS comment,
|
||||
'' AS index_comment,
|
||||
'YES' AS visible,
|
||||
primary_key AS expression
|
||||
'' AS expression
|
||||
FROM system.tables
|
||||
WHERE
|
||||
database = '{0}'
|
||||
@ -61,18 +61,18 @@ FROM (
|
||||
UNION ALL (
|
||||
SELECT
|
||||
table AS table,
|
||||
0 AS non_unique,
|
||||
1 AS non_unique,
|
||||
name AS key_name,
|
||||
NULL AS seq_in_index,
|
||||
NULL AS column_name,
|
||||
1 AS seq_in_index,
|
||||
'' AS column_name,
|
||||
NULL AS collation,
|
||||
NULL AS cardinality,
|
||||
0 AS cardinality,
|
||||
NULL AS sub_part,
|
||||
NULL AS packed,
|
||||
NULL AS null,
|
||||
type AS index_type,
|
||||
NULL AS comment,
|
||||
NULL AS index_comment,
|
||||
upper(type) AS index_type,
|
||||
'' AS comment,
|
||||
'' AS index_comment,
|
||||
'YES' AS visible,
|
||||
expr AS expression
|
||||
FROM system.data_skipping_indices
|
||||
@ -80,12 +80,27 @@ FROM (
|
||||
database = '{0}'
|
||||
AND table = '{1}'))
|
||||
{2}
|
||||
ORDER BY index_type, expression;)", database, table, where_expression);
|
||||
ORDER BY index_type, expression, column_name, seq_in_index;)", database, table, where_expression);
|
||||
|
||||
/// Sorting is strictly speaking not necessary but 1. it is convenient for users, 2. SQL currently does not allow to
|
||||
/// sort the output of SHOW INDEXES otherwise (SELECT * FROM (SHOW INDEXES ...) ORDER BY ...) is rejected) and 3. some
|
||||
/// SQL tests can take advantage of this.
|
||||
|
||||
/// Note about compatibility of fields 'column_name', 'seq_in_index' and 'expression' with MySQL:
|
||||
/// MySQL has non-functional and functional indexes.
|
||||
/// - Non-functional indexes only reference columns, e.g. 'col1, col2'. In this case, `SHOW INDEX` produces as many result rows as there
|
||||
/// are indexed columns. 'column_name' and 'seq_in_index' (an ascending integer 1, 2, ...) are filled, 'expression' is empty.
|
||||
/// - Functional indexes can reference arbitrary expressions, e.g. 'col1 + 1, concat(col2, col3)'. 'SHOW INDEX' produces a single row
|
||||
/// with `column_name` and `seq_in_index` empty and `expression` filled with the entire index expression. Only non-primary-key indexes
|
||||
/// can be functional indexes.
|
||||
/// Above SELECT tries to emulate that. Caveats:
|
||||
/// 1. The primary key index sub-SELECT assumes the primary key expression is non-functional. Non-functional primary key indexes in
|
||||
/// ClickHouse are possible but quiete obscure. In MySQL they are not possible at all.
|
||||
/// 2. Related to 1.: Poor man's tuple parsing with splitByString() in the PK sub-SELECT messes up for functional primary key index
|
||||
/// expressions where the comma is not only used as separator between tuple components, e.g. in 'col1 + 1, concat(col2, col3)'.
|
||||
/// 3. The data skipping index sub-SELECT assumes the index expression is functional. 3rd party tools that expect MySQL semantics from
|
||||
/// SHOW INDEX will probably not care as MySQL has no skipping indexes and they only use the result to figure out the primary key.
|
||||
|
||||
return rewritten_query;
|
||||
}
|
||||
|
||||
|
@ -337,6 +337,11 @@ std::shared_ptr<TableJoin> JoinedTables::makeTableJoin(const ASTSelectQuery & se
|
||||
LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' was not found", dictionary_name);
|
||||
return nullptr;
|
||||
}
|
||||
if (dictionary->getSpecialKeyType() == DictionarySpecialKeyType::Range)
|
||||
{
|
||||
LOG_TRACE(&Poco::Logger::get("JoinedTables"), "Can't use dictionary join: dictionary '{}' is a range dictionary", dictionary_name);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
auto dictionary_kv = std::dynamic_pointer_cast<const IKeyValueEntity>(dictionary);
|
||||
table_join->setStorageJoin(dictionary_kv);
|
||||
|
@ -332,15 +332,16 @@ SystemLog<LogElement>::SystemLog(
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const String & storage_def_,
|
||||
size_t flush_interval_milliseconds_)
|
||||
: WithContext(context_)
|
||||
size_t flush_interval_milliseconds_,
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue_)
|
||||
: Base(database_name_ + "." + table_name_, flush_interval_milliseconds_, queue_)
|
||||
, WithContext(context_)
|
||||
, log(&Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"))
|
||||
, table_id(database_name_, table_name_)
|
||||
, storage_def(storage_def_)
|
||||
, create_query(serializeAST(*getCreateTableQuery()))
|
||||
, flush_interval_milliseconds(flush_interval_milliseconds_)
|
||||
{
|
||||
assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE);
|
||||
log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")");
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
@ -353,6 +354,26 @@ void SystemLog<LogElement>::shutdown()
|
||||
table->flushAndShutdown();
|
||||
}
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLog<LogElement>::stopFlushThread()
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(thread_mutex);
|
||||
|
||||
if (!saving_thread || !saving_thread->joinable())
|
||||
return;
|
||||
|
||||
if (is_shutdown)
|
||||
return;
|
||||
|
||||
is_shutdown = true;
|
||||
queue->shutdown();
|
||||
}
|
||||
|
||||
saving_thread->join();
|
||||
}
|
||||
|
||||
|
||||
template <typename LogElement>
|
||||
void SystemLog<LogElement>::savingThreadFunction()
|
||||
{
|
||||
@ -370,27 +391,7 @@ void SystemLog<LogElement>::savingThreadFunction()
|
||||
// Should we prepare table even if there are no new messages.
|
||||
bool should_prepare_tables_anyway = false;
|
||||
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
flush_event.wait_for(lock,
|
||||
std::chrono::milliseconds(flush_interval_milliseconds),
|
||||
[&] ()
|
||||
{
|
||||
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
|
||||
}
|
||||
);
|
||||
|
||||
queue_front_index += queue.size();
|
||||
to_flush_end = queue_front_index;
|
||||
// Swap with existing array from previous flush, to save memory
|
||||
// allocations.
|
||||
to_flush.resize(0);
|
||||
queue.swap(to_flush);
|
||||
|
||||
should_prepare_tables_anyway = is_force_prepare_tables;
|
||||
|
||||
exit_this_thread = is_shutdown;
|
||||
}
|
||||
to_flush_end = queue->pop(to_flush, should_prepare_tables_anyway, exit_this_thread);
|
||||
|
||||
if (to_flush.empty())
|
||||
{
|
||||
@ -399,9 +400,7 @@ void SystemLog<LogElement>::savingThreadFunction()
|
||||
prepareTable();
|
||||
LOG_TRACE(log, "Table created (force)");
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
is_force_prepare_tables = false;
|
||||
flush_event.notify_all();
|
||||
queue->confirm(to_flush_end);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -473,12 +472,7 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
flushed_up_to = to_flush_end;
|
||||
is_force_prepare_tables = false;
|
||||
flush_event.notify_all();
|
||||
}
|
||||
queue->confirm(to_flush_end);
|
||||
|
||||
LOG_TRACE(log, "Flushed system log up to offset {}", to_flush_end);
|
||||
}
|
||||
@ -618,7 +612,6 @@ ASTPtr SystemLog<LogElement>::getCreateTableQuery()
|
||||
return create;
|
||||
}
|
||||
|
||||
|
||||
#define INSTANTIATE_SYSTEM_LOG(ELEMENT) template class SystemLog<ELEMENT>;
|
||||
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG)
|
||||
|
||||
|
@ -108,32 +108,34 @@ public:
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const String & storage_def_,
|
||||
size_t flush_interval_milliseconds_);
|
||||
size_t flush_interval_milliseconds_,
|
||||
std::shared_ptr<SystemLogQueue<LogElement>> queue_ = nullptr);
|
||||
|
||||
/** Append a record into log.
|
||||
* Writing to table will be done asynchronously and in case of failure, record could be lost.
|
||||
*/
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
void stopFlushThread() override;
|
||||
|
||||
protected:
|
||||
using ISystemLog::mutex;
|
||||
Poco::Logger * log;
|
||||
|
||||
using ISystemLog::is_shutdown;
|
||||
using ISystemLog::flush_event;
|
||||
using ISystemLog::stopFlushThread;
|
||||
using Base::log;
|
||||
using ISystemLog::saving_thread;
|
||||
using ISystemLog::thread_mutex;
|
||||
using Base::queue;
|
||||
using Base::queue_front_index;
|
||||
using Base::is_force_prepare_tables;
|
||||
using Base::requested_flush_up_to;
|
||||
using Base::flushed_up_to;
|
||||
using Base::logged_queue_full_at_index;
|
||||
|
||||
private:
|
||||
|
||||
|
||||
/* Saving thread data */
|
||||
const StorageID table_id;
|
||||
const String storage_def;
|
||||
String create_query;
|
||||
String old_create_query;
|
||||
bool is_prepared = false;
|
||||
const size_t flush_interval_milliseconds;
|
||||
|
||||
/** Creates new table if it does not exist.
|
||||
* Renames old table if its structure is not suitable.
|
||||
|
@ -223,10 +223,10 @@ public:
|
||||
{
|
||||
/// When join_algorithm = 'default' (not specified by user) we use hash or direct algorithm.
|
||||
/// It's behaviour that was initially supported by clickhouse.
|
||||
bool is_enbaled_by_default = val == JoinAlgorithm::DEFAULT
|
||||
bool is_enabled_by_default = val == JoinAlgorithm::DEFAULT
|
||||
|| val == JoinAlgorithm::HASH
|
||||
|| val == JoinAlgorithm::DIRECT;
|
||||
if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enbaled_by_default)
|
||||
if (join_algorithm.isSet(JoinAlgorithm::DEFAULT) && is_enabled_by_default)
|
||||
return true;
|
||||
return join_algorithm.isSet(val);
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ TextLog::TextLog(ContextPtr context_, const String & database_name_,
|
||||
const String & table_name_, const String & storage_def_,
|
||||
size_t flush_interval_milliseconds_)
|
||||
: SystemLog<TextLogElement>(context_, database_name_, table_name_,
|
||||
storage_def_, flush_interval_milliseconds_)
|
||||
storage_def_, flush_interval_milliseconds_, getLogQueue(flush_interval_milliseconds_))
|
||||
{
|
||||
// SystemLog methods may write text logs, so we disable logging for the text
|
||||
// log table to avoid recursion.
|
||||
|
@ -40,12 +40,20 @@ struct TextLogElement
|
||||
class TextLog : public SystemLog<TextLogElement>
|
||||
{
|
||||
public:
|
||||
using Queue = SystemLogQueue<TextLogElement>;
|
||||
|
||||
TextLog(
|
||||
ContextPtr context_,
|
||||
const String & database_name_,
|
||||
const String & table_name_,
|
||||
const String & storage_def_,
|
||||
size_t flush_interval_milliseconds_);
|
||||
|
||||
static std::shared_ptr<Queue> getLogQueue(size_t flush_interval_milliseconds)
|
||||
{
|
||||
static std::shared_ptr<Queue> queue = std::make_shared<Queue>("text_log", flush_interval_milliseconds, true);
|
||||
return queue;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -518,7 +518,7 @@ void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log, const String
|
||||
|
||||
static String getCleanQueryAst(const ASTPtr q, ContextPtr context)
|
||||
{
|
||||
String res = serializeAST(*q, true);
|
||||
String res = serializeAST(*q);
|
||||
if (auto * masker = SensitiveDataMasker::getInstance())
|
||||
masker->wipeSensitiveData(res);
|
||||
|
||||
|
@ -73,6 +73,7 @@ NamesAndTypesList ZooKeeperLogElement::getNamesAndTypes()
|
||||
{"Create", static_cast<Int16>(Coordination::OpNum::Create)},
|
||||
{"Remove", static_cast<Int16>(Coordination::OpNum::Remove)},
|
||||
{"Exists", static_cast<Int16>(Coordination::OpNum::Exists)},
|
||||
{"Reconfig", static_cast<Int16>(Coordination::OpNum::Reconfig)},
|
||||
{"Get", static_cast<Int16>(Coordination::OpNum::Get)},
|
||||
{"Set", static_cast<Int16>(Coordination::OpNum::Set)},
|
||||
{"GetACL", static_cast<Int16>(Coordination::OpNum::GetACL)},
|
||||
|
@ -45,21 +45,11 @@ static std::string renderFileNameTemplate(time_t now, const std::string & file_p
|
||||
}
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
void Loggers::setTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
|
||||
{
|
||||
text_log = log;
|
||||
text_log_max_priority = max_priority;
|
||||
}
|
||||
constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500;
|
||||
#endif
|
||||
|
||||
void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger /*_root*/, const std::string & cmd_name)
|
||||
{
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
if (split)
|
||||
if (auto log = text_log.lock())
|
||||
split->addTextLog(log, text_log_max_priority);
|
||||
#endif
|
||||
|
||||
auto current_logger = config.getString("logger", "");
|
||||
if (config_logger.has_value() && *config_logger == current_logger)
|
||||
return;
|
||||
@ -276,6 +266,16 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log
|
||||
}
|
||||
}
|
||||
}
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
if (config.has("text_log"))
|
||||
{
|
||||
String text_log_level_str = config.getString("text_log.level", "trace");
|
||||
int text_log_level = Poco::Logger::parseLevel(text_log_level_str);
|
||||
size_t flush_interval_milliseconds = config.getUInt64("text_log.flush_interval_milliseconds",
|
||||
DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS);
|
||||
split->addTextLog(DB::TextLog::getLogQueue(flush_interval_milliseconds), text_log_level);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
void Loggers::updateLevels(Poco::Util::AbstractConfiguration & config, Poco::Logger & logger)
|
||||
|
@ -7,12 +7,6 @@
|
||||
#include <Poco/Util/Application.h>
|
||||
#include "OwnSplitChannel.h"
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
namespace DB
|
||||
{
|
||||
class TextLog;
|
||||
}
|
||||
#endif
|
||||
|
||||
namespace Poco::Util
|
||||
{
|
||||
@ -29,9 +23,6 @@ public:
|
||||
/// Close log files. On next log write files will be reopened.
|
||||
void closeLogs(Poco::Logger & logger);
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
void setTextLog(std::shared_ptr<DB::TextLog> log, int max_priority);
|
||||
#endif
|
||||
|
||||
private:
|
||||
Poco::AutoPtr<Poco::FileChannel> log_file;
|
||||
@ -41,10 +32,6 @@ private:
|
||||
/// Previous value of logger element in config. It is used to reinitialize loggers whenever the value changed.
|
||||
std::optional<std::string> config_logger;
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
std::weak_ptr<DB::TextLog> text_log;
|
||||
int text_log_max_priority = -1;
|
||||
#endif
|
||||
|
||||
Poco::AutoPtr<DB::OwnSplitChannel> split;
|
||||
};
|
||||
|
@ -135,13 +135,10 @@ void OwnSplitChannel::logSplit(const Poco::Message & msg)
|
||||
elem.source_line = msg.getSourceLine();
|
||||
elem.message_format_string = msg.getFormatString();
|
||||
|
||||
std::shared_ptr<TextLog> text_log_locked{};
|
||||
{
|
||||
std::lock_guard lock(text_log_mutex);
|
||||
text_log_locked = text_log.lock();
|
||||
}
|
||||
std::shared_ptr<SystemLogQueue<TextLogElement>> text_log_locked{};
|
||||
text_log_locked = text_log.lock();
|
||||
if (text_log_locked)
|
||||
text_log_locked->add(elem);
|
||||
text_log_locked->push(elem);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
@ -153,10 +150,9 @@ void OwnSplitChannel::addChannel(Poco::AutoPtr<Poco::Channel> channel, const std
|
||||
}
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
void OwnSplitChannel::addTextLog(std::shared_ptr<DB::TextLog> log, int max_priority)
|
||||
void OwnSplitChannel::addTextLog(std::shared_ptr<SystemLogQueue<TextLogElement>> log_queue, int max_priority)
|
||||
{
|
||||
std::lock_guard lock(text_log_mutex);
|
||||
text_log = log;
|
||||
text_log = log_queue;
|
||||
text_log_max_priority.store(max_priority, std::memory_order_relaxed);
|
||||
}
|
||||
#endif
|
||||
|
@ -10,7 +10,9 @@
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
namespace DB
|
||||
{
|
||||
class TextLog;
|
||||
template <typename> class SystemLogQueue;
|
||||
struct TextLogElement;
|
||||
using TextLogQueue = SystemLogQueue<TextLogElement>;
|
||||
}
|
||||
#endif
|
||||
|
||||
@ -31,7 +33,7 @@ public:
|
||||
void addChannel(Poco::AutoPtr<Poco::Channel> channel, const std::string & name);
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
void addTextLog(std::shared_ptr<DB::TextLog> log, int max_priority);
|
||||
void addTextLog(std::shared_ptr<DB::TextLogQueue> log_queue, int max_priority);
|
||||
#endif
|
||||
|
||||
void setLevel(const std::string & name, int level);
|
||||
@ -45,10 +47,8 @@ private:
|
||||
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;
|
||||
std::map<std::string, ExtendedChannelPtrPair> channels;
|
||||
|
||||
std::mutex text_log_mutex;
|
||||
|
||||
#ifndef WITHOUT_TEXT_LOG
|
||||
std::weak_ptr<DB::TextLog> text_log;
|
||||
std::weak_ptr<DB::TextLogQueue> text_log;
|
||||
std::atomic<int> text_log_max_priority = -1;
|
||||
#endif
|
||||
};
|
||||
|
@ -170,7 +170,9 @@ size_t IAST::checkDepthImpl(size_t max_depth) const
|
||||
String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets) const
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
format({buf, one_line, show_secrets});
|
||||
FormatSettings settings(buf, one_line);
|
||||
settings.show_secrets = show_secrets;
|
||||
format(settings);
|
||||
return wipeSensitiveDataAndCutToLength(buf.str(), max_length);
|
||||
}
|
||||
|
||||
|
@ -191,27 +191,39 @@ public:
|
||||
struct FormatSettings
|
||||
{
|
||||
WriteBuffer & ostr;
|
||||
bool hilite = false;
|
||||
bool one_line;
|
||||
bool always_quote_identifiers = false;
|
||||
IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks;
|
||||
bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys).
|
||||
bool hilite;
|
||||
bool always_quote_identifiers;
|
||||
IdentifierQuotingStyle identifier_quoting_style;
|
||||
bool show_secrets; /// Show secret parts of the AST (e.g. passwords, encryption keys).
|
||||
char nl_or_ws; /// Newline or whitespace.
|
||||
|
||||
// Newline or whitespace.
|
||||
char nl_or_ws;
|
||||
|
||||
FormatSettings(WriteBuffer & ostr_, bool one_line_, bool show_secrets_ = true)
|
||||
: ostr(ostr_), one_line(one_line_), show_secrets(show_secrets_)
|
||||
explicit FormatSettings(
|
||||
WriteBuffer & ostr_,
|
||||
bool one_line_,
|
||||
bool hilite_ = false,
|
||||
bool always_quote_identifiers_ = false,
|
||||
IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks,
|
||||
bool show_secrets_ = true)
|
||||
: ostr(ostr_)
|
||||
, one_line(one_line_)
|
||||
, hilite(hilite_)
|
||||
, always_quote_identifiers(always_quote_identifiers_)
|
||||
, identifier_quoting_style(identifier_quoting_style_)
|
||||
, show_secrets(show_secrets_)
|
||||
, nl_or_ws(one_line ? ' ' : '\n')
|
||||
{
|
||||
nl_or_ws = one_line ? ' ' : '\n';
|
||||
}
|
||||
|
||||
FormatSettings(WriteBuffer & ostr_, const FormatSettings & other)
|
||||
: ostr(ostr_), hilite(other.hilite), one_line(other.one_line),
|
||||
always_quote_identifiers(other.always_quote_identifiers), identifier_quoting_style(other.identifier_quoting_style),
|
||||
show_secrets(other.show_secrets)
|
||||
: ostr(ostr_)
|
||||
, one_line(other.one_line)
|
||||
, hilite(other.hilite)
|
||||
, always_quote_identifiers(other.always_quote_identifiers)
|
||||
, identifier_quoting_style(other.identifier_quoting_style)
|
||||
, show_secrets(other.show_secrets)
|
||||
, nl_or_ws(other.nl_or_ws)
|
||||
{
|
||||
nl_or_ws = one_line ? ' ' : '\n';
|
||||
}
|
||||
|
||||
void writeIdentifier(const String & name) const;
|
||||
|
@ -6,16 +6,15 @@ namespace DB
|
||||
|
||||
void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite, bool one_line, bool show_secrets)
|
||||
{
|
||||
IAST::FormatSettings settings(buf, one_line, show_secrets);
|
||||
settings.hilite = hilite;
|
||||
|
||||
IAST::FormatSettings settings(buf, one_line, hilite);
|
||||
settings.show_secrets = show_secrets;
|
||||
ast.format(settings);
|
||||
}
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line)
|
||||
String serializeAST(const IAST & ast)
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(ast, buf, false, one_line);
|
||||
formatAST(ast, buf, false, true);
|
||||
return buf.str();
|
||||
}
|
||||
|
||||
|
@ -8,12 +8,13 @@ namespace DB
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
/** Takes a syntax tree and turns it back into text.
|
||||
* In case of INSERT query, the data will be missing.
|
||||
*/
|
||||
/// Takes a syntax tree and turns it into text.
|
||||
/// Intended for pretty-printing (multi-line + hiliting).
|
||||
/// In case of INSERT query, the data will be missing.
|
||||
void formatAST(const IAST & ast, WriteBuffer & buf, bool hilite = true, bool one_line = false, bool show_secrets = true);
|
||||
|
||||
String serializeAST(const IAST & ast, bool one_line = true);
|
||||
/// Like formatAST() but intended for serialization w/o pretty-printing (single-line, no hiliting).
|
||||
String serializeAST(const IAST & ast);
|
||||
|
||||
inline WriteBuffer & operator<<(WriteBuffer & buf, const IAST & ast)
|
||||
{
|
||||
|
@ -19,9 +19,7 @@ std::string getInsertQuery(const std::string & db_name, const std::string & tabl
|
||||
query.columns->children.emplace_back(std::make_shared<ASTIdentifier>(column.name));
|
||||
|
||||
WriteBufferFromOwnString buf;
|
||||
IAST::FormatSettings settings(buf, true);
|
||||
settings.always_quote_identifiers = true;
|
||||
settings.identifier_quoting_style = quoting;
|
||||
IAST::FormatSettings settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true, /*identifier_quoting_style*/ quoting);
|
||||
query.IAST::format(settings);
|
||||
return buf.str();
|
||||
}
|
||||
|
@ -65,7 +65,10 @@ TEST_P(ParserTest, parseQuery)
|
||||
if (std::string("CREATE USER or ALTER USER query") != parser->getName()
|
||||
&& std::string("ATTACH access entity query") != parser->getName())
|
||||
{
|
||||
EXPECT_EQ(expected_ast, serializeAST(*ast->clone(), false));
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast->clone(), buf, false, false);
|
||||
String formatted_ast = buf.str();
|
||||
EXPECT_EQ(expected_ast, formatted_ast);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -76,7 +79,10 @@ TEST_P(ParserTest, parseQuery)
|
||||
}
|
||||
else
|
||||
{
|
||||
EXPECT_TRUE(std::regex_match(serializeAST(*ast->clone(), false), std::regex(expected_ast)));
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast->clone(), buf, false, false);
|
||||
String formatted_ast = buf.str();
|
||||
EXPECT_TRUE(std::regex_match(formatted_ast, std::regex(expected_ast)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -155,7 +155,7 @@ TEST(ParserDictionaryDDL, AttributesWithMultipleProperties)
|
||||
|
||||
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
|
||||
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
|
||||
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression, true), "(rand() % 100) * 77");
|
||||
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression), "(rand() % 100) * 77");
|
||||
|
||||
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
|
||||
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, true);
|
||||
@ -201,7 +201,7 @@ TEST(ParserDictionaryDDL, CustomAttributePropertiesOrder)
|
||||
|
||||
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
|
||||
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->expression, nullptr);
|
||||
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression, true), "(rand() % 100) * 77");
|
||||
EXPECT_EQ(serializeAST(*attributes_children[2]->as<ASTDictionaryAttributeDeclaration>()->expression), "(rand() % 100) * 77");
|
||||
|
||||
EXPECT_EQ(attributes_children[0]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, false);
|
||||
EXPECT_EQ(attributes_children[1]->as<ASTDictionaryAttributeDeclaration>()->hierarchical, true);
|
||||
@ -288,7 +288,7 @@ TEST(ParserDictionaryDDL, Formatting)
|
||||
ParserCreateDictionaryQuery parser;
|
||||
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 0);
|
||||
ASTCreateQuery * create = ast->as<ASTCreateQuery>();
|
||||
auto str = serializeAST(*create, true);
|
||||
auto str = serializeAST(*create);
|
||||
EXPECT_EQ(str, "CREATE DICTIONARY test.dict5 (`key_column1` UInt64 DEFAULT 1 HIERARCHICAL INJECTIVE, `key_column2` String DEFAULT '', `second_column` UInt8 EXPRESSION intDiv(50, rand() % 1000), `third_column` UInt8) PRIMARY KEY key_column1, key_column2 SOURCE(MYSQL(HOST 'localhost' PORT 9000 USER 'default' REPLICA (HOST '127.0.0.1' PRIORITY 1) PASSWORD '')) LIFETIME(MIN 1 MAX 10) LAYOUT(CACHE(SIZE_IN_CELLS 50)) RANGE(MIN second_column MAX third_column)");
|
||||
}
|
||||
|
||||
@ -303,7 +303,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery)
|
||||
EXPECT_TRUE(drop1->is_dictionary);
|
||||
EXPECT_EQ(drop1->getDatabase(), "test");
|
||||
EXPECT_EQ(drop1->getTable(), "dict1");
|
||||
auto str1 = serializeAST(*drop1, true);
|
||||
auto str1 = serializeAST(*drop1);
|
||||
EXPECT_EQ(input1, str1);
|
||||
|
||||
String input2 = "DROP DICTIONARY IF EXISTS dict2";
|
||||
@ -314,7 +314,7 @@ TEST(ParserDictionaryDDL, ParseDropQuery)
|
||||
EXPECT_TRUE(drop2->is_dictionary);
|
||||
EXPECT_EQ(drop2->getDatabase(), "");
|
||||
EXPECT_EQ(drop2->getTable(), "dict2");
|
||||
auto str2 = serializeAST(*drop2, true);
|
||||
auto str2 = serializeAST(*drop2);
|
||||
EXPECT_EQ(input2, str2);
|
||||
}
|
||||
|
||||
|
@ -51,8 +51,7 @@ void compare(const String & expected, const String & query)
|
||||
ASTPtr ast = parseQuery(parser, query, 0, 0);
|
||||
|
||||
WriteBufferFromOwnString write_buffer;
|
||||
IAST::FormatSettings settings(write_buffer, true);
|
||||
settings.hilite = true;
|
||||
IAST::FormatSettings settings(write_buffer, true, true);
|
||||
ast->format(settings);
|
||||
|
||||
ASSERT_PRED2(HiliteComparator::are_equal_with_hilites_removed, expected, write_buffer.str());
|
||||
|
@ -542,7 +542,8 @@ void trySetStorageInTableJoin(const QueryTreeNodePtr & table_expression, std::sh
|
||||
if (!table_join->isEnabledAlgorithm(JoinAlgorithm::DIRECT))
|
||||
return;
|
||||
|
||||
if (auto storage_dictionary = std::dynamic_pointer_cast<StorageDictionary>(storage); storage_dictionary)
|
||||
if (auto storage_dictionary = std::dynamic_pointer_cast<StorageDictionary>(storage);
|
||||
storage_dictionary && storage_dictionary->getDictionary()->getSpecialKeyType() != DictionarySpecialKeyType::Range)
|
||||
table_join->setStorageJoin(std::dynamic_pointer_cast<const IKeyValueEntity>(storage_dictionary->getDictionary()));
|
||||
else if (auto storage_key_value = std::dynamic_pointer_cast<IKeyValueEntity>(storage); storage_key_value)
|
||||
table_join->setStorageJoin(storage_key_value);
|
||||
|
@ -8,7 +8,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag)
|
||||
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity)
|
||||
{
|
||||
using Parents = std::set<const ActionsDAG::Node *>;
|
||||
std::unordered_map<const ActionsDAG::Node *, Parents> inner_parents;
|
||||
@ -75,7 +75,12 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG
|
||||
}
|
||||
/// A node from found match may be nullptr.
|
||||
/// It means that node is visited, but no match was found.
|
||||
frame.mapped_children.push_back(it->second.node);
|
||||
if (it->second.monotonicity)
|
||||
/// Ignore a match with monotonicity.
|
||||
frame.mapped_children.push_back(nullptr);
|
||||
else
|
||||
frame.mapped_children.push_back(it->second.node);
|
||||
|
||||
}
|
||||
|
||||
if (frame.mapped_children.size() < frame.node->children.size())
|
||||
@ -182,7 +187,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG
|
||||
}
|
||||
}
|
||||
|
||||
if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity())
|
||||
if (!match.node && check_monotonicity && frame.node->function_base->hasInformationAboutMonotonicity())
|
||||
{
|
||||
size_t num_const_args = 0;
|
||||
const ActionsDAG::Node * monotonic_child = nullptr;
|
||||
|
@ -39,5 +39,5 @@ struct MatchedTrees
|
||||
using Matches = std::unordered_map<const ActionsDAG::Node *, Match>;
|
||||
};
|
||||
|
||||
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag);
|
||||
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag, bool check_monotonicity = true);
|
||||
}
|
||||
|
@ -287,7 +287,7 @@ ActionsDAGPtr analyzeAggregateProjection(
|
||||
{
|
||||
auto proj_index = buildDAGIndex(*info.before_aggregation);
|
||||
|
||||
MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag);
|
||||
MatchedTrees::Matches matches = matchTrees(*info.before_aggregation, *query.dag, false /* check_monotonicity */);
|
||||
|
||||
// for (const auto & [node, match] : matches)
|
||||
// {
|
||||
@ -497,6 +497,9 @@ AggregateProjectionCandidates getAggregateProjectionCandidates(
|
||||
|
||||
// LOG_TRACE(&Poco::Logger::get("optimizeUseProjections"), "Projection sample block 2 {}", block.dumpStructure());
|
||||
|
||||
// minmax_count_projection cannot be used used when there is no data to process, because
|
||||
// it will produce incorrect result during constant aggregation.
|
||||
// See https://github.com/ClickHouse/ClickHouse/issues/36728
|
||||
if (block)
|
||||
{
|
||||
MinMaxProjectionCandidate minmax;
|
||||
|
@ -86,9 +86,7 @@ static String formattedAST(const ASTPtr & ast)
|
||||
return {};
|
||||
|
||||
WriteBufferFromOwnString buf;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true);
|
||||
ast_format_settings.hilite = false;
|
||||
ast_format_settings.always_quote_identifiers = true;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true);
|
||||
ast->format(ast_format_settings);
|
||||
return buf.str();
|
||||
}
|
||||
|
@ -73,7 +73,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
|
||||
"Constraint expression returns nullable column that contains null value",
|
||||
backQuote(constraint_ptr->name),
|
||||
table_id.getNameForLogs(),
|
||||
serializeAST(*(constraint_ptr->expr), true));
|
||||
serializeAST(*(constraint_ptr->expr)));
|
||||
|
||||
result_column = nested_column;
|
||||
}
|
||||
@ -116,7 +116,7 @@ void CheckConstraintsTransform::onConsume(Chunk chunk)
|
||||
backQuote(constraint_ptr->name),
|
||||
table_id.getNameForLogs(),
|
||||
rows_written + row_idx + 1,
|
||||
serializeAST(*(constraint_ptr->expr), true),
|
||||
serializeAST(*(constraint_ptr->expr)),
|
||||
column_values_msg);
|
||||
}
|
||||
}
|
||||
|
@ -35,9 +35,20 @@ FinishSortingTransform::FinishSortingTransform(
|
||||
"Can't finish sorting. SortDescription "
|
||||
"of already sorted stream is not prefix of SortDescription needed to sort");
|
||||
|
||||
/// Remove constants from description_sorted_.
|
||||
SortDescription description_sorted_without_constants;
|
||||
description_sorted_without_constants.reserve(description_sorted_.size());
|
||||
size_t num_columns = const_columns_to_remove.size();
|
||||
for (const auto & column_description : description_sorted_)
|
||||
{
|
||||
auto pos = header.getPositionByName(column_description.column_name);
|
||||
|
||||
if (pos < num_columns && !const_columns_to_remove[pos])
|
||||
description_sorted_without_constants.push_back(column_description);
|
||||
}
|
||||
/// The target description is modified in SortingTransform constructor.
|
||||
/// To avoid doing the same actions with description_sorted just copy it from prefix of target description.
|
||||
for (const auto & column_sort_desc : description_sorted_)
|
||||
for (const auto & column_sort_desc : description_sorted_without_constants)
|
||||
description_with_positions.emplace_back(column_sort_desc, header_without_constants.getPositionByName(column_sort_desc.column_name));
|
||||
}
|
||||
|
||||
|
@ -189,7 +189,6 @@ void JoiningTransform::transform(Chunk & chunk)
|
||||
}
|
||||
else
|
||||
block = readExecute(chunk);
|
||||
|
||||
auto num_rows = block.rows();
|
||||
chunk.setColumns(block.getColumns(), num_rows);
|
||||
}
|
||||
@ -311,8 +310,11 @@ void FillingRightJoinSideTransform::work()
|
||||
}
|
||||
|
||||
|
||||
DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(Block output_header)
|
||||
: IProcessor(InputPorts{Block()}, OutputPorts{output_header})
|
||||
DelayedJoinedBlocksWorkerTransform::DelayedJoinedBlocksWorkerTransform(
|
||||
Block output_header_,
|
||||
NonJoinedStreamBuilder non_joined_stream_builder_)
|
||||
: IProcessor(InputPorts{Block()}, OutputPorts{output_header_})
|
||||
, non_joined_stream_builder(std::move(non_joined_stream_builder_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -365,6 +367,7 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare()
|
||||
|
||||
if (!data.chunk.hasChunkInfo())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform must have chunk info");
|
||||
|
||||
task = std::dynamic_pointer_cast<const DelayedBlocksTask>(data.chunk.getChunkInfo());
|
||||
}
|
||||
else
|
||||
@ -372,7 +375,8 @@ IProcessor::Status DelayedJoinedBlocksWorkerTransform::prepare()
|
||||
input.setNotNeeded();
|
||||
}
|
||||
|
||||
if (task->finished)
|
||||
// When delayed_blocks is nullptr, it means that all buckets have been joined.
|
||||
if (!task->delayed_blocks)
|
||||
{
|
||||
input.close();
|
||||
output.finish();
|
||||
@ -387,11 +391,21 @@ void DelayedJoinedBlocksWorkerTransform::work()
|
||||
if (!task)
|
||||
return;
|
||||
|
||||
Block block = task->delayed_blocks->next();
|
||||
|
||||
Block block;
|
||||
/// All joined and non-joined rows from left stream are emitted, only right non-joined rows are left
|
||||
if (!task->delayed_blocks->isFinished())
|
||||
{
|
||||
block = task->delayed_blocks->next();
|
||||
if (!block)
|
||||
block = nextNonJoinedBlock();
|
||||
}
|
||||
else
|
||||
{
|
||||
block = nextNonJoinedBlock();
|
||||
}
|
||||
if (!block)
|
||||
{
|
||||
task.reset();
|
||||
resetTask();
|
||||
return;
|
||||
}
|
||||
|
||||
@ -400,6 +414,30 @@ void DelayedJoinedBlocksWorkerTransform::work()
|
||||
output_chunk.setColumns(block.getColumns(), rows);
|
||||
}
|
||||
|
||||
void DelayedJoinedBlocksWorkerTransform::resetTask()
|
||||
{
|
||||
task.reset();
|
||||
non_joined_delayed_stream = nullptr;
|
||||
}
|
||||
|
||||
Block DelayedJoinedBlocksWorkerTransform::nextNonJoinedBlock()
|
||||
{
|
||||
// Before read from non-joined stream, all blocks in left file reader must have been joined.
|
||||
// For example, in HashJoin, it may return invalid mismatch rows from non-joined stream before
|
||||
// the all blocks in left file reader have been finished, since the used flags are incomplete.
|
||||
// To make only one processor could read from non-joined stream seems be a easy way.
|
||||
if (!non_joined_delayed_stream && task && task->left_delayed_stream_finish_counter->isLast())
|
||||
{
|
||||
non_joined_delayed_stream = non_joined_stream_builder();
|
||||
}
|
||||
|
||||
if (non_joined_delayed_stream)
|
||||
{
|
||||
return non_joined_delayed_stream->next();
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
DelayedJoinedBlocksTransform::DelayedJoinedBlocksTransform(size_t num_streams, JoinPtr join_)
|
||||
: IProcessor(InputPorts{}, OutputPorts(num_streams, Block()))
|
||||
, join(std::move(join_))
|
||||
@ -433,6 +471,9 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare()
|
||||
|
||||
if (finished)
|
||||
{
|
||||
// Since have memory limit, cannot handle all buckets parallelly by different
|
||||
// DelayedJoinedBlocksWorkerTransform. So send the same task to all outputs.
|
||||
// Wait for all DelayedJoinedBlocksWorkerTransform be idle before getting next bucket.
|
||||
for (auto & output : outputs)
|
||||
{
|
||||
if (output.isFinished())
|
||||
@ -448,10 +489,14 @@ IProcessor::Status DelayedJoinedBlocksTransform::prepare()
|
||||
|
||||
if (delayed_blocks)
|
||||
{
|
||||
// This counter is used to ensure that only the last DelayedJoinedBlocksWorkerTransform
|
||||
// could read right non-joined blocks from the join.
|
||||
auto left_delayed_stream_finished_counter = std::make_shared<JoiningTransform::FinishCounter>(outputs.size());
|
||||
for (auto & output : outputs)
|
||||
{
|
||||
Chunk chunk;
|
||||
chunk.setChunkInfo(std::make_shared<DelayedBlocksTask>(delayed_blocks));
|
||||
auto task = std::make_shared<DelayedBlocksTask>(delayed_blocks, left_delayed_stream_finished_counter);
|
||||
chunk.setChunkInfo(task);
|
||||
output.push(std::move(chunk));
|
||||
}
|
||||
delayed_blocks = nullptr;
|
||||
|
@ -115,12 +115,16 @@ class DelayedBlocksTask : public ChunkInfo
|
||||
{
|
||||
public:
|
||||
|
||||
explicit DelayedBlocksTask() : finished(true) {}
|
||||
explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_) : delayed_blocks(std::move(delayed_blocks_)) {}
|
||||
DelayedBlocksTask() = default;
|
||||
explicit DelayedBlocksTask(IBlocksStreamPtr delayed_blocks_, JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter_)
|
||||
: delayed_blocks(std::move(delayed_blocks_))
|
||||
, left_delayed_stream_finish_counter(left_delayed_stream_finish_counter_)
|
||||
{
|
||||
}
|
||||
|
||||
IBlocksStreamPtr delayed_blocks = nullptr;
|
||||
JoiningTransform::FinishCounterPtr left_delayed_stream_finish_counter = nullptr;
|
||||
|
||||
bool finished = false;
|
||||
};
|
||||
|
||||
using DelayedBlocksTaskPtr = std::shared_ptr<const DelayedBlocksTask>;
|
||||
@ -147,7 +151,10 @@ private:
|
||||
class DelayedJoinedBlocksWorkerTransform : public IProcessor
|
||||
{
|
||||
public:
|
||||
explicit DelayedJoinedBlocksWorkerTransform(Block output_header);
|
||||
using NonJoinedStreamBuilder = std::function<IBlocksStreamPtr()>;
|
||||
explicit DelayedJoinedBlocksWorkerTransform(
|
||||
Block output_header_,
|
||||
NonJoinedStreamBuilder non_joined_stream_builder_);
|
||||
|
||||
String getName() const override { return "DelayedJoinedBlocksWorkerTransform"; }
|
||||
|
||||
@ -157,8 +164,12 @@ public:
|
||||
private:
|
||||
DelayedBlocksTaskPtr task;
|
||||
Chunk output_chunk;
|
||||
/// For building a block stream to access the non-joined rows.
|
||||
NonJoinedStreamBuilder non_joined_stream_builder;
|
||||
IBlocksStreamPtr non_joined_delayed_stream = nullptr;
|
||||
|
||||
bool finished = false;
|
||||
void resetTask();
|
||||
Block nextNonJoinedBlock();
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -491,7 +491,10 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
|
||||
if (delayed_root)
|
||||
{
|
||||
// Process delayed joined blocks when all JoiningTransform are finished.
|
||||
auto delayed = std::make_shared<DelayedJoinedBlocksWorkerTransform>(joined_header);
|
||||
auto delayed = std::make_shared<DelayedJoinedBlocksWorkerTransform>(
|
||||
joined_header,
|
||||
[left_header, joined_header, max_block_size, join]()
|
||||
{ return join->getNonJoinedBlocks(left_header, joined_header, max_block_size); });
|
||||
if (delayed->getInputs().size() != 1 || delayed->getOutputs().size() != 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "DelayedJoinedBlocksWorkerTransform should have one input and one output");
|
||||
|
||||
|
@ -35,7 +35,7 @@ String ConstraintsDescription::toString() const
|
||||
for (const auto & constraint : constraints)
|
||||
list.children.push_back(constraint);
|
||||
|
||||
return serializeAST(list, true);
|
||||
return serializeAST(list);
|
||||
}
|
||||
|
||||
ConstraintsDescription ConstraintsDescription::parse(const String & str)
|
||||
|
@ -67,7 +67,8 @@ struct HudiMetadataParser<Configuration, MetadataReadHelper>::Impl
|
||||
{
|
||||
auto key_file = std::filesystem::path(key);
|
||||
Strings file_parts;
|
||||
splitInto<'_'>(file_parts, key_file.stem());
|
||||
const String stem = key_file.stem();
|
||||
splitInto<'_'>(file_parts, stem);
|
||||
if (file_parts.size() != 3)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format for file: {}", key);
|
||||
|
||||
|
@ -151,7 +151,7 @@ String IndicesDescription::toString() const
|
||||
for (const auto & index : *this)
|
||||
list.children.push_back(index.definition_ast);
|
||||
|
||||
return serializeAST(list, true);
|
||||
return serializeAST(list);
|
||||
}
|
||||
|
||||
|
||||
|
@ -62,9 +62,10 @@ ColumnsDescription StorageMeiliSearch::getTableStructureFromData(const MeiliSear
|
||||
String convertASTtoStr(ASTPtr ptr)
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
IAST::FormatSettings settings(out, true);
|
||||
settings.identifier_quoting_style = IdentifierQuotingStyle::BackticksMySQL;
|
||||
settings.always_quote_identifiers = IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None;
|
||||
IAST::FormatSettings settings(
|
||||
out, /*one_line*/ true, /*hilite*/ false,
|
||||
/*always_quote_identifiers*/ IdentifierQuotingStyle::BackticksMySQL != IdentifierQuotingStyle::None,
|
||||
/*identifier_quoting_style*/ IdentifierQuotingStyle::BackticksMySQL);
|
||||
ptr->format(settings);
|
||||
return out.str();
|
||||
}
|
||||
|
@ -501,7 +501,7 @@ public:
|
||||
|
||||
mutable std::atomic<DataPartRemovalState> removal_state = DataPartRemovalState::NOT_ATTEMPTED;
|
||||
|
||||
mutable std::atomic<time_t> last_removal_attemp_time = 0;
|
||||
mutable std::atomic<time_t> last_removal_attempt_time = 0;
|
||||
|
||||
protected:
|
||||
|
||||
|
@ -230,7 +230,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
/// the fast replica is not overloaded because amount of executing merges doesn't affect the ability to acquire locks for new merges.
|
||||
///
|
||||
/// So here we trying to solve it with the simplest solution -- sleep random time up to 500ms for 1GB part and up to 7 seconds for 300GB part.
|
||||
/// It can sound too much, but we are trying to aquite these locks in background tasks which can be scheduled each 5 seconds or so.
|
||||
/// It can sound too much, but we are trying to acquire these locks in background tasks which can be scheduled each 5 seconds or so.
|
||||
double start_to_sleep_seconds = std::logf(storage_settings_ptr->zero_copy_merge_mutation_min_parts_size_sleep_before_lock.value);
|
||||
uint64_t right_border_to_sleep_ms = static_cast<uint64_t>((std::log(estimated_space_for_merge) - start_to_sleep_seconds + 0.5) * 1000);
|
||||
uint64_t time_to_sleep_milliseconds = std::min<uint64_t>(10000UL, std::uniform_int_distribution<uint64_t>(1, 1 + right_border_to_sleep_ms)(rng));
|
||||
@ -245,7 +245,11 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare()
|
||||
|
||||
if (!zero_copy_lock || !zero_copy_lock->isLocked())
|
||||
{
|
||||
LOG_DEBUG(log, "Merge of part {} started by some other replica, will wait it and fetch merged part", entry.new_part_name);
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Merge of part {} started by some other replica, will wait for it and fetch merged part. Number of tries {}",
|
||||
entry.new_part_name,
|
||||
entry.num_tries);
|
||||
storage.watchZeroCopyLock(entry.new_part_name, disk);
|
||||
/// Don't check for missing part -- it's missing because other replica still not
|
||||
/// finished merge.
|
||||
|
@ -465,9 +465,10 @@ void MergeTreeData::checkProperties(
|
||||
const StorageInMemoryMetadata & new_metadata,
|
||||
const StorageInMemoryMetadata & old_metadata,
|
||||
bool attach,
|
||||
bool allow_empty_sorting_key,
|
||||
ContextPtr local_context) const
|
||||
{
|
||||
if (!new_metadata.sorting_key.definition_ast)
|
||||
if (!new_metadata.sorting_key.definition_ast && !allow_empty_sorting_key)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ORDER BY cannot be empty");
|
||||
|
||||
KeyDescription new_sorting_key = new_metadata.sorting_key;
|
||||
@ -580,6 +581,9 @@ void MergeTreeData::checkProperties(
|
||||
if (projections_names.find(projection.name) != projections_names.end())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection with name {} already exists", backQuote(projection.name));
|
||||
|
||||
/// We cannot alter a projection so far. So here we do not try to find a projection in old metadata.
|
||||
bool is_aggregate = projection.type == ProjectionDescription::Type::Aggregate;
|
||||
checkProperties(*projection.metadata, *projection.metadata, attach, is_aggregate, local_context);
|
||||
projections_names.insert(projection.name);
|
||||
}
|
||||
}
|
||||
@ -593,7 +597,7 @@ void MergeTreeData::setProperties(
|
||||
bool attach,
|
||||
ContextPtr local_context)
|
||||
{
|
||||
checkProperties(new_metadata, old_metadata, attach, local_context);
|
||||
checkProperties(new_metadata, old_metadata, attach, false, local_context);
|
||||
setInMemoryMetadata(new_metadata);
|
||||
}
|
||||
|
||||
@ -2187,7 +2191,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force)
|
||||
|
||||
const DataPartPtr & part = *it;
|
||||
|
||||
part->last_removal_attemp_time.store(time_now, std::memory_order_relaxed);
|
||||
part->last_removal_attempt_time.store(time_now, std::memory_order_relaxed);
|
||||
|
||||
/// Do not remove outdated part if it may be visible for some transaction
|
||||
if (!part->version.canBeRemoved())
|
||||
@ -3286,7 +3290,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
}
|
||||
}
|
||||
|
||||
checkProperties(new_metadata, old_metadata, false, local_context);
|
||||
checkProperties(new_metadata, old_metadata, false, false, local_context);
|
||||
checkTTLExpressions(new_metadata, old_metadata);
|
||||
|
||||
if (!columns_to_check_conversion.empty())
|
||||
@ -7027,7 +7031,9 @@ std::optional<ProjectionCandidate> MergeTreeData::getQueryProcessingStageWithAgg
|
||||
max_added_blocks.get(),
|
||||
query_context);
|
||||
|
||||
// minmax_count_projection should not be used when there is no data to process.
|
||||
// minmax_count_projection cannot be used used when there is no data to process, because
|
||||
// it will produce incorrect result during constant aggregation.
|
||||
// See https://github.com/ClickHouse/ClickHouse/issues/36728
|
||||
if (!query_info.minmax_count_projection_block)
|
||||
return;
|
||||
|
||||
|
@ -1229,7 +1229,7 @@ protected:
|
||||
/// The same for clearOldTemporaryDirectories.
|
||||
std::mutex clear_old_temporary_directories_mutex;
|
||||
|
||||
void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr) const;
|
||||
void checkProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach, bool allow_empty_sorting_key, ContextPtr local_context) const;
|
||||
|
||||
void setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach = false, ContextPtr local_context = nullptr);
|
||||
|
||||
|
@ -61,7 +61,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP
|
||||
{
|
||||
auto out = disk->writeFile(std::filesystem::path(path_prefix) / file_name, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, settings);
|
||||
*out << "format version: 1\n"
|
||||
<< "create time: " << LocalDateTime(create_time) << "\n";
|
||||
<< "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n";
|
||||
*out << "commands: ";
|
||||
commands.writeText(*out, /* with_pure_metadata_commands = */ false);
|
||||
*out << "\n";
|
||||
|
@ -22,6 +22,33 @@ namespace DB
|
||||
/// This is used to assume that condition is likely to have good selectivity.
|
||||
static constexpr auto threshold = 2;
|
||||
|
||||
static NameToIndexMap fillNamesPositions(const Names & names)
|
||||
{
|
||||
NameToIndexMap names_positions;
|
||||
|
||||
for (size_t position = 0; position < names.size(); ++position)
|
||||
{
|
||||
const auto & name = names[position];
|
||||
names_positions[name] = position;
|
||||
}
|
||||
|
||||
return names_positions;
|
||||
}
|
||||
|
||||
/// Find minimal position of any of the column in primary key.
|
||||
static Int64 findMinPosition(const NameSet & condition_table_columns, const NameToIndexMap & primary_key_positions)
|
||||
{
|
||||
Int64 min_position = std::numeric_limits<Int64>::max() - 1;
|
||||
|
||||
for (const auto & column : condition_table_columns)
|
||||
{
|
||||
auto it = primary_key_positions.find(column);
|
||||
if (it != primary_key_positions.end())
|
||||
min_position = std::min(min_position, static_cast<Int64>(it->second));
|
||||
}
|
||||
|
||||
return min_position;
|
||||
}
|
||||
|
||||
MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
||||
std::unordered_map<std::string, UInt64> column_sizes_,
|
||||
@ -35,6 +62,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
||||
, supported_columns{supported_columns_}
|
||||
, sorting_key_names{NameSet(
|
||||
metadata_snapshot->getSortingKey().column_names.begin(), metadata_snapshot->getSortingKey().column_names.end())}
|
||||
, primary_key_names_positions(fillNamesPositions(metadata_snapshot->getPrimaryKey().column_names))
|
||||
, log{log_}
|
||||
, column_sizes{std::move(column_sizes_)}
|
||||
{
|
||||
@ -60,6 +88,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons
|
||||
where_optimizer_context.context = context;
|
||||
where_optimizer_context.array_joined_names = determineArrayJoinedNames(select);
|
||||
where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere;
|
||||
where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere;
|
||||
where_optimizer_context.is_final = select.final();
|
||||
|
||||
RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/);
|
||||
@ -89,6 +118,7 @@ std::optional<MergeTreeWhereOptimizer::FilterActionsOptimizeResult> MergeTreeWhe
|
||||
where_optimizer_context.context = context;
|
||||
where_optimizer_context.array_joined_names = {};
|
||||
where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere;
|
||||
where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere;
|
||||
where_optimizer_context.is_final = is_final;
|
||||
|
||||
RPNBuilderTreeContext tree_context(context);
|
||||
@ -234,6 +264,14 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree
|
||||
if (cond.viable)
|
||||
cond.good = isConditionGood(node, table_columns);
|
||||
|
||||
if (where_optimizer_context.move_primary_key_columns_to_end_of_prewhere)
|
||||
{
|
||||
/// Consider all conditions good with this setting enabled.
|
||||
cond.good = cond.viable;
|
||||
/// Find min position in PK of any column that is used in this condition.
|
||||
cond.min_position_in_primary_key = findMinPosition(cond.table_columns, primary_key_names_positions);
|
||||
}
|
||||
|
||||
res.emplace_back(std::move(cond));
|
||||
}
|
||||
}
|
||||
|
@ -72,9 +72,14 @@ private:
|
||||
/// Does the condition presumably have good selectivity?
|
||||
bool good = false;
|
||||
|
||||
/// Does the condition contain primary key column?
|
||||
/// If so, it is better to move it further to the end of PREWHERE chain depending on minimal position in PK of any
|
||||
/// column in this condition because this condition have bigger chances to be already satisfied by PK analysis.
|
||||
Int64 min_position_in_primary_key = std::numeric_limits<Int64>::max() - 1;
|
||||
|
||||
auto tuple() const
|
||||
{
|
||||
return std::make_tuple(!viable, !good, columns_size, table_columns.size());
|
||||
return std::make_tuple(!viable, !good, -min_position_in_primary_key, columns_size, table_columns.size());
|
||||
}
|
||||
|
||||
/// Is condition a better candidate for moving to PREWHERE?
|
||||
@ -91,6 +96,7 @@ private:
|
||||
ContextPtr context;
|
||||
NameSet array_joined_names;
|
||||
bool move_all_conditions_to_prewhere = false;
|
||||
bool move_primary_key_columns_to_end_of_prewhere = false;
|
||||
bool is_final = false;
|
||||
};
|
||||
|
||||
@ -141,6 +147,7 @@ private:
|
||||
const Names queried_columns;
|
||||
const std::optional<NameSet> supported_columns;
|
||||
const NameSet sorting_key_names;
|
||||
const NameToIndexMap primary_key_names_positions;
|
||||
Poco::Logger * log;
|
||||
std::unordered_map<std::string, UInt64> column_sizes;
|
||||
UInt64 total_size_of_queried_columns = 0;
|
||||
|
@ -154,8 +154,12 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare()
|
||||
|
||||
if (!zero_copy_lock || !zero_copy_lock->isLocked())
|
||||
{
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Mutation of part {} started by some other replica, will wait for it and mutated merged part. Number of tries {}",
|
||||
entry.new_part_name,
|
||||
entry.num_tries);
|
||||
storage.watchZeroCopyLock(entry.new_part_name, disk);
|
||||
LOG_DEBUG(log, "Mutation of part {} started by some other replica, will wait it and mutated merged part", entry.new_part_name);
|
||||
|
||||
return PrepareResult{
|
||||
.prepared_successfully = false,
|
||||
|
@ -174,7 +174,7 @@ bool ReplicatedMergeMutateTaskBase::executeImpl()
|
||||
|
||||
part_log_writer = prepare_result.part_log_writer;
|
||||
|
||||
/// Avoid resheduling, execute fetch here, in the same thread.
|
||||
/// Avoid rescheduling, execute fetch here, in the same thread.
|
||||
if (!prepare_result.prepared_successfully)
|
||||
return execute_fetch(prepare_result.need_to_check_missing_part_in_fetch);
|
||||
|
||||
|
@ -492,7 +492,7 @@ size_t ReplicatedMergeTreeCleanupThread::clearOldBlocks(const String & blocks_di
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, Coordination::errorMessage(rc));
|
||||
LOG_WARNING(log, "Error while deleting ZooKeeper path `{}`: {}, ignoring.", path, rc);
|
||||
}
|
||||
first_outdated_block++;
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ void ReplicatedMergeTreeLogEntryData::writeText(WriteBuffer & out) const
|
||||
format_version = std::max<UInt8>(format_version, FORMAT_WITH_LOG_ENTRY_ID);
|
||||
|
||||
out << "format version: " << format_version << "\n"
|
||||
<< "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n"
|
||||
<< "create_time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n"
|
||||
<< "source replica: " << source_replica << '\n'
|
||||
<< "block_id: " << escape << block_id << '\n';
|
||||
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const
|
||||
{
|
||||
out << "format version: 1\n"
|
||||
<< "create time: " << LocalDateTime(create_time ? create_time : time(nullptr)) << "\n"
|
||||
<< "create time: " << LocalDateTime(create_time ? create_time : time(nullptr), DateLUT::serverTimezoneInstance()) << "\n"
|
||||
<< "source replica: " << source_replica << "\n"
|
||||
<< "block numbers count: " << block_numbers.size() << "\n";
|
||||
|
||||
|
@ -497,7 +497,7 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
|
||||
if (code != Coordination::Error::ZOK)
|
||||
LOG_ERROR(log, "Couldn't set value of nodes for insert times "
|
||||
"({}/min_unprocessed_insert_time, max_processed_insert_time): {}. "
|
||||
"This shouldn't happen often.", replica_path, Coordination::errorMessage(code));
|
||||
"This shouldn't happen often.", replica_path, code);
|
||||
}
|
||||
}
|
||||
|
||||
@ -554,7 +554,7 @@ void ReplicatedMergeTreeQueue::removeProcessedEntry(zkutil::ZooKeeperPtr zookeep
|
||||
|
||||
auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / entry->znode_name);
|
||||
if (code != Coordination::Error::ZOK)
|
||||
LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, Coordination::errorMessage(code));
|
||||
LOG_ERROR(log, "Couldn't remove {}/queue/{}: {}. This shouldn't happen often.", replica_path, entry->znode_name, code);
|
||||
|
||||
updateTimesInZooKeeper(zookeeper, min_unprocessed_insert_time_changed, max_processed_insert_time_changed);
|
||||
}
|
||||
@ -1147,7 +1147,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange(
|
||||
|
||||
auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / znode_name);
|
||||
if (code != Coordination::Error::ZOK)
|
||||
LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), Coordination::errorMessage(code));
|
||||
LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / znode_name).string(), code);
|
||||
|
||||
updateStateOnQueueEntryRemoval(
|
||||
*it, /* is_successful = */ false,
|
||||
@ -1370,13 +1370,27 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry(
|
||||
if (data_settings->allow_remote_fs_zero_copy_replication)
|
||||
{
|
||||
auto disks = storage.getDisks();
|
||||
bool only_s3_storage = true;
|
||||
DiskPtr disk_with_zero_copy = nullptr;
|
||||
for (const auto & disk : disks)
|
||||
if (!disk->supportZeroCopyReplication())
|
||||
only_s3_storage = false;
|
||||
{
|
||||
if (disk->supportZeroCopyReplication())
|
||||
{
|
||||
disk_with_zero_copy = disk;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// Technically speaking if there are more than one disk that could store the part (a local hot + cloud cold)
|
||||
/// It would be possible for the merge to happen concurrently with other replica if the other replica is doing
|
||||
/// a merge using zero-copy and the cloud storage, and the local replica uses the local storage instead
|
||||
/// The question is, is it worth keep retrying to do the merge over and over for the opportunity to do
|
||||
/// double the work? Probably not
|
||||
/// So what we do is that, even if hot merge could happen, check the zero copy lock anyway.
|
||||
/// Keep in mind that for the zero copy lock check to happen (via existing_zero_copy_locks) we need to
|
||||
/// have failed first because of it and added it via watchZeroCopyLock. Considering we've already tried to
|
||||
/// use cloud storage and zero-copy replication, the most likely scenario is that we'll try again
|
||||
String replica_to_execute_merge;
|
||||
if (!disks.empty() && only_s3_storage && storage.checkZeroCopyLockExists(entry.new_part_name, disks[0], replica_to_execute_merge))
|
||||
if (disk_with_zero_copy && storage.checkZeroCopyLockExists(entry.new_part_name, disk_with_zero_copy, replica_to_execute_merge))
|
||||
{
|
||||
constexpr auto fmt_string = "Not executing merge/mutation for the part {}, waiting for {} to execute it and will fetch after.";
|
||||
out_postpone_reason = fmt::format(fmt_string, entry.new_part_name, replica_to_execute_merge);
|
||||
|
@ -729,7 +729,7 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
retries_ctl.setUserError(
|
||||
ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR,
|
||||
"Insert failed due to zookeeper error. Please retry. Reason: {}",
|
||||
Coordination::errorMessage(write_part_info_keeper_error));
|
||||
write_part_info_keeper_error);
|
||||
}
|
||||
|
||||
retries_ctl.stopRetries();
|
||||
@ -1039,7 +1039,7 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
retries_ctl.setUserError(
|
||||
ErrorCodes::UNKNOWN_STATUS_OF_INSERT,
|
||||
"Unknown status, client must retry. Reason: {}",
|
||||
Coordination::errorMessage(multi_code));
|
||||
multi_code);
|
||||
return;
|
||||
}
|
||||
else if (Coordination::isUserError(multi_code))
|
||||
@ -1115,7 +1115,7 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
"Unexpected logical error while adding block {} with ID '{}': {}, path {}",
|
||||
block_number,
|
||||
toString(block_id),
|
||||
Coordination::errorMessage(multi_code),
|
||||
multi_code,
|
||||
failed_op_path);
|
||||
}
|
||||
}
|
||||
@ -1128,7 +1128,7 @@ std::pair<std::vector<String>, bool> ReplicatedMergeTreeSinkImpl<async_insert>::
|
||||
"Unexpected ZooKeeper error while adding block {} with ID '{}': {}",
|
||||
block_number,
|
||||
toString(block_id),
|
||||
Coordination::errorMessage(multi_code));
|
||||
multi_code);
|
||||
}
|
||||
},
|
||||
[&zookeeper]() { zookeeper->cleanupEphemeralNodes(); });
|
||||
|
@ -298,6 +298,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context)
|
||||
SelectQueryOptions{
|
||||
type == ProjectionDescription::Type::Normal ? QueryProcessingStage::FetchColumns
|
||||
: QueryProcessingStage::WithMergeableState}
|
||||
.ignoreASTOptimizations()
|
||||
.ignoreSettingConstraints())
|
||||
.buildQueryPipeline();
|
||||
builder.resize(1);
|
||||
@ -324,7 +325,7 @@ String ProjectionsDescription::toString() const
|
||||
for (const auto & projection : projections)
|
||||
list.children.push_back(projection.definition_ast);
|
||||
|
||||
return serializeAST(list, true);
|
||||
return serializeAST(list);
|
||||
}
|
||||
|
||||
ProjectionsDescription ProjectionsDescription::parse(const String & str, const ColumnsDescription & columns, ContextPtr query_context)
|
||||
|
@ -906,8 +906,7 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteBetweenDistribu
|
||||
String new_query_str;
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true);
|
||||
ast_format_settings.always_quote_identifiers = true;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers_=*/ true);
|
||||
new_query->IAST::format(ast_format_settings);
|
||||
new_query_str = buf.str();
|
||||
}
|
||||
@ -968,8 +967,7 @@ std::optional<QueryPipeline> StorageDistributed::distributedWriteFromClusterStor
|
||||
String new_query_str;
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true);
|
||||
ast_format_settings.always_quote_identifiers = true;
|
||||
IAST::FormatSettings ast_format_settings(buf, /*one_line*/ true, /*hilite*/ false, /*always_quote_identifiers*/ true);
|
||||
new_query->IAST::format(ast_format_settings);
|
||||
new_query_str = buf.str();
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user