Some small improvements

This commit is contained in:
Antonio Andelic 2024-06-06 09:45:07 +02:00
parent ce244e126d
commit f0e9703384
8 changed files with 60 additions and 27 deletions

View File

@ -300,11 +300,13 @@ String MonitorCommand::run()
print(ret, "server_state", keeper_info.getRole());
print(ret, "znode_count", state_machine.getNodesCount());
print(ret, "watch_count", state_machine.getTotalWatchesCount());
print(ret, "ephemerals_count", state_machine.getTotalEphemeralNodesCount());
print(ret, "approximate_data_size", state_machine.getApproximateDataSize());
print(ret, "key_arena_size", state_machine.getKeyArenaSize());
const auto & storage_stats = state_machine.getStorageStats();
print(ret, "znode_count", storage_stats.nodes_count.load(std::memory_order_relaxed));
print(ret, "watch_count", storage_stats.total_watches_count.load(std::memory_order_relaxed));
print(ret, "ephemerals_count", storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed));
print(ret, "approximate_data_size", storage_stats.approximate_data_size.load(std::memory_order_relaxed));
print(ret, "key_arena_size", 0);
print(ret, "latest_snapshot_size", state_machine.getLatestSnapshotSize());
#if defined(OS_LINUX) || defined(OS_DARWIN)
@ -386,6 +388,7 @@ String ServerStatCommand::run()
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats();
write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
@ -397,9 +400,9 @@ String ServerStatCommand::run()
write("Sent", toString(stats.getPacketsSent()));
write("Connections", toString(keeper_info.alive_connections_count));
write("Outstanding", toString(keeper_info.outstanding_requests_count));
write("Zxid", formatZxid(keeper_info.last_zxid));
write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed)));
write("Mode", keeper_info.getRole());
write("Node count", toString(keeper_info.total_nodes_count));
write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed)));
return buf.str();
}
@ -415,6 +418,7 @@ String StatCommand::run()
auto & stats = keeper_dispatcher.getKeeperConnectionStats();
Keeper4LWInfo keeper_info = keeper_dispatcher.getKeeper4LWInfo();
const auto & storage_stats = keeper_dispatcher.getStateMachine().getStorageStats();
write("ClickHouse Keeper version", String(VERSION_DESCRIBE) + "-" + VERSION_GITHASH);
@ -430,9 +434,9 @@ String StatCommand::run()
write("Sent", toString(stats.getPacketsSent()));
write("Connections", toString(keeper_info.alive_connections_count));
write("Outstanding", toString(keeper_info.outstanding_requests_count));
write("Zxid", formatZxid(keeper_info.last_zxid));
write("Zxid", formatZxid(storage_stats.last_zxid.load(std::memory_order_relaxed)));
write("Mode", keeper_info.getRole());
write("Node count", toString(keeper_info.total_nodes_count));
write("Node count", toString(storage_stats.nodes_count.load(std::memory_order_relaxed)));
return buf.str();
}

View File

@ -1,7 +1,5 @@
#pragma once
#include <string>
#include <base/types.h>
#include <Common/Exception.h>
@ -30,9 +28,6 @@ struct Keeper4LWInfo
uint64_t follower_count;
uint64_t synced_follower_count;
uint64_t total_nodes_count;
int64_t last_zxid;
String getRole() const
{
if (is_standalone)

View File

@ -38,15 +38,16 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM
is_follower = static_cast<size_t>(keeper_info.is_follower);
is_exceeding_mem_soft_limit = static_cast<size_t>(keeper_info.is_exceeding_mem_soft_limit);
zxid = keeper_info.last_zxid;
const auto & state_machine = keeper_dispatcher.getStateMachine();
znode_count = state_machine.getNodesCount();
watch_count = state_machine.getTotalWatchesCount();
ephemerals_count = state_machine.getTotalEphemeralNodesCount();
approximate_data_size = state_machine.getApproximateDataSize();
key_arena_size = state_machine.getKeyArenaSize();
session_with_watches = state_machine.getSessionsWithWatchesCount();
paths_watched = state_machine.getWatchedPathsCount();
const auto & storage_stats = state_machine.getStorageStats();
zxid = storage_stats.last_zxid.load(std::memory_order_relaxed);
znode_count = storage_stats.nodes_count.load(std::memory_order_relaxed);
watch_count = storage_stats.total_watches_count.load(std::memory_order_relaxed);
ephemerals_count = storage_stats.total_emphemeral_nodes_count.load(std::memory_order_relaxed);
approximate_data_size = storage_stats.approximate_data_size.load(std::memory_order_relaxed);
key_arena_size = 0;
session_with_watches = storage_stats.sessions_with_watches_count.load(std::memory_order_relaxed);
paths_watched = storage_stats.watched_paths_count.load(std::memory_order_relaxed);
# if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount();

View File

@ -1126,8 +1126,6 @@ Keeper4LWInfo KeeperServer::getPartiallyFilled4LWInfo() const
result.synced_follower_count = getSyncedFollowerCount();
}
result.is_exceeding_mem_soft_limit = isExceedingMemorySoftLimit();
result.total_nodes_count = getKeeperStateMachine()->getNodesCount();
result.last_zxid = getKeeperStateMachine()->getLastProcessedZxid();
return result;
}

View File

@ -802,10 +802,9 @@ uint64_t KeeperStateMachine::getLastProcessedZxid() const
return storage->getZXID();
}
uint64_t KeeperStateMachine::getNodesCount() const
const KeeperStorage::Stats & KeeperStateMachine::getStorageStats() const TSA_NO_THREAD_SAFETY_ANALYSIS
{
std::lock_guard lock(storage_and_responses_lock);
return storage->getNodesCount();
return storage->getStorageStats();
}
uint64_t KeeperStateMachine::getTotalWatchesCount() const

View File

@ -111,6 +111,7 @@ public:
/// Introspection functions for 4lw commands
uint64_t getLastProcessedZxid() const;
const KeeperStorage::Stats & getStorageStats() const;
uint64_t getNodesCount() const;
uint64_t getTotalWatchesCount() const;
uint64_t getWatchedPathsCount() const;

View File

@ -2514,6 +2514,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(
results.push_back(ResponseForSession{session_id, response});
}
updateStats();
return results;
}
@ -2660,6 +2661,23 @@ void KeeperStorage::dumpSessionsAndEphemerals(WriteBufferFromOwnString & buf) co
}
}
void KeeperStorage::updateStats()
{
stats.nodes_count.store(getNodesCount(), std::memory_order_relaxed);
stats.approximate_data_size.store(getApproximateDataSize(), std::memory_order_relaxed);
stats.total_watches_count.store(getTotalWatchesCount(), std::memory_order_relaxed);
stats.watched_paths_count.store(getWatchedPathsCount(), std::memory_order_relaxed);
stats.sessions_with_watches_count.store(getSessionsWithWatchesCount(), std::memory_order_relaxed);
stats.session_with_ephemeral_nodes_count.store(getSessionWithEphemeralNodesCount(), std::memory_order_relaxed);
stats.total_emphemeral_nodes_count.store(getTotalEphemeralNodesCount(), std::memory_order_relaxed);
stats.last_zxid.store(getZXID(), std::memory_order_relaxed);
}
const KeeperStorage::Stats & KeeperStorage::getStorageStats() const
{
return stats;
}
uint64_t KeeperStorage::getTotalWatchesCount() const
{
uint64_t ret = 0;

View File

@ -539,6 +539,23 @@ public:
/// Get all dead sessions
std::vector<int64_t> getDeadSessions() const { return session_expiry_queue.getExpiredSessions(); }
struct Stats
{
std::atomic<uint64_t> nodes_count = 0;
std::atomic<uint64_t> approximate_data_size = 0;
std::atomic<uint64_t> total_watches_count = 0;
std::atomic<uint64_t> watched_paths_count = 0;
std::atomic<uint64_t> sessions_with_watches_count = 0;
std::atomic<uint64_t> session_with_ephemeral_nodes_count = 0;
std::atomic<uint64_t> total_emphemeral_nodes_count = 0;
std::atomic<int64_t> last_zxid = 0;
};
Stats stats;
void updateStats();
const Stats & getStorageStats() const;
/// Introspection functions mostly used in 4-letter commands
uint64_t getNodesCount() const { return container.size(); }