Merge pull request #43998 from evillique/make_system_replicas_parallel

Make `system.replicas` parallel
This commit is contained in:
Alexander Tokmakov 2023-01-13 16:33:36 +03:00 committed by GitHub
commit 9d5ec474a3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 43 additions and 11 deletions

View File

@ -5877,7 +5877,24 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit
{
try
{
auto log_entries = zookeeper->getChildren(fs::path(zookeeper_path) / "log");
std::vector<std::string> paths;
paths.push_back(fs::path(zookeeper_path) / "log");
paths.push_back(fs::path(zookeeper_path) / "replicas");
auto children_result = zookeeper->getChildren(paths);
const auto & log_entries = children_result[0].names;
const auto & all_replicas = children_result[1].names;
paths.clear();
paths.push_back(fs::path(replica_path) / "log_pointer");
for (const String & replica : all_replicas)
paths.push_back(fs::path(zookeeper_path) / "replicas" / replica / "is_active");
auto get_result = zookeeper->tryGet(paths);
const auto & log_pointer_str = get_result[0].data;
if (get_result[0].error == Coordination::Error::ZNONODE)
throw zkutil::KeeperException(get_result[0].error);
if (!log_entries.empty())
{
@ -5885,17 +5902,14 @@ void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool wit
res.log_max_index = parse<UInt64>(last_log_entry.substr(strlen("log-")));
}
String log_pointer_str = zookeeper->get(fs::path(replica_path) / "log_pointer");
res.log_pointer = log_pointer_str.empty() ? 0 : parse<UInt64>(log_pointer_str);
auto all_replicas = zookeeper->getChildren(fs::path(zookeeper_path) / "replicas");
res.total_replicas = all_replicas.size();
for (const String & replica : all_replicas)
for (size_t i = 0, size = all_replicas.size(); i < size; ++i)
{
bool is_replica_active = zookeeper->exists(fs::path(zookeeper_path) / "replicas" / replica / "is_active");
bool is_replica_active = get_result[i + 1].error != Coordination::Error::ZNONODE;
res.active_replicas += static_cast<UInt8>(is_replica_active);
res.replica_is_active.emplace(replica, is_replica_active);
res.replica_is_active.emplace(all_replicas[i], is_replica_active);
}
}
catch (const Coordination::Exception &)

View File

@ -10,6 +10,7 @@
#include <Common/typeid_cast.h>
#include <Databases/IDatabase.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
namespace DB
@ -151,14 +152,31 @@ Pipe StorageSystemReplicas::read(
MutableColumns res_columns = storage_snapshot->metadata->getSampleBlock().cloneEmptyColumns();
for (size_t i = 0, size = col_database->size(); i < size; ++i)
size_t tables_size = col_database->size();
std::vector<ReplicatedTableStatus> statuses(tables_size);
size_t thread_pool_size = std::min(tables_size, static_cast<size_t>(getNumberOfPhysicalCPUCores()));
auto settings = context->getSettingsRef();
if (settings.max_threads != 0)
thread_pool_size = std::min(thread_pool_size, static_cast<size_t>(settings.max_threads));
ThreadPool thread_pool(thread_pool_size);
for (size_t i = 0; i < tables_size; ++i)
{
ReplicatedTableStatus status;
dynamic_cast<StorageReplicatedMergeTree &>(
thread_pool.scheduleOrThrowOnError([&, i=i]
{
dynamic_cast<StorageReplicatedMergeTree &>(
*replicated_tables
[(*col_database)[i].safeGet<const String &>()]
[(*col_table)[i].safeGet<const String &>()]).getStatus(status, with_zk_fields);
[(*col_table)[i].safeGet<const String &>()]).getStatus(statuses[i], with_zk_fields);
});
}
thread_pool.wait();
for (const auto & status: statuses)
{
size_t col_num = 3;
res_columns[col_num++]->insert(status.is_leader);
res_columns[col_num++]->insert(status.can_become_leader);