mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge pull request #43998 from evillique/make_system_replicas_parallel
Make `system.replicas` parallel
This commit is contained in:
commit
9d5ec474a3
@ -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 &)
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user