2022-11-09 12:37:42 +00:00
# include <Coordination/KeeperAsynchronousMetrics.h>
# include <Coordination/KeeperDispatcher.h>
# include <Common/getCurrentProcessFDCount.h>
# include <Common/getMaxFileDescriptorCount.h>
namespace DB
{
2022-11-09 14:51:41 +00:00
void updateKeeperInformation ( KeeperDispatcher & keeper_dispatcher , AsynchronousMetricValues & new_values )
2022-11-09 12:37:42 +00:00
{
# if USE_NURAFT
size_t is_leader = 0 ;
size_t is_follower = 0 ;
size_t is_observer = 0 ;
size_t is_standalone = 0 ;
size_t znode_count = 0 ;
size_t watch_count = 0 ;
size_t ephemerals_count = 0 ;
size_t approximate_data_size = 0 ;
size_t key_arena_size = 0 ;
size_t open_file_descriptor_count = 0 ;
2024-01-18 09:11:50 +00:00
std : : optional < size_t > max_file_descriptor_count = 0 ;
2022-11-09 12:37:42 +00:00
size_t followers = 0 ;
size_t synced_followers = 0 ;
size_t zxid = 0 ;
size_t session_with_watches = 0 ;
size_t paths_watched = 0 ;
2024-01-17 05:52:10 +00:00
size_t is_exceeding_mem_soft_limit = 0 ;
2022-11-09 12:37:42 +00:00
if ( keeper_dispatcher . isServerActive ( ) )
{
auto keeper_info = keeper_dispatcher . getKeeper4LWInfo ( ) ;
is_standalone = static_cast < size_t > ( keeper_info . is_standalone ) ;
is_leader = static_cast < size_t > ( keeper_info . is_leader ) ;
is_observer = static_cast < size_t > ( keeper_info . is_observer ) ;
is_follower = static_cast < size_t > ( keeper_info . is_follower ) ;
2024-01-17 05:52:10 +00:00
is_exceeding_mem_soft_limit = static_cast < size_t > ( keeper_info . is_exceeding_mem_soft_limit ) ;
2022-11-09 12:37:42 +00:00
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 ( ) ;
# if defined(__linux__) || defined(__APPLE__)
open_file_descriptor_count = getCurrentProcessFDCount ( ) ;
max_file_descriptor_count = getMaxFileDescriptorCount ( ) ;
# endif
if ( keeper_info . is_leader )
{
followers = keeper_info . follower_count ;
synced_followers = keeper_info . synced_follower_count ;
}
}
2022-11-15 12:00:02 +00:00
new_values [ " KeeperIsLeader " ] = { is_leader , " 1 if ClickHouse Keeper is a leader, 0 otherwise. " } ;
new_values [ " KeeperIsFollower " ] = { is_follower , " 1 if ClickHouse Keeper is a follower, 0 otherwise. " } ;
new_values [ " KeeperIsObserver " ] = { is_observer , " 1 if ClickHouse Keeper is an observer, 0 otherwise. " } ;
new_values [ " KeeperIsStandalone " ] = { is_standalone , " 1 if ClickHouse Keeper is in a standalone mode, 0 otherwise. " } ;
2024-01-17 05:52:10 +00:00
new_values [ " KeeperIsExceedingMemorySoftLimitHit " ] = { is_exceeding_mem_soft_limit , " 1 if ClickHouse Keeper is exceeding the memory soft limit, 0 otherwise. " } ;
2022-11-15 12:00:02 +00:00
new_values [ " KeeperZnodeCount " ] = { znode_count , " The number of nodes (data entries) in ClickHouse Keeper. " } ;
new_values [ " KeeperWatchCount " ] = { watch_count , " The number of watches in ClickHouse Keeper. " } ;
new_values [ " KeeperEphemeralsCount " ] = { ephemerals_count , " The number of ephemeral nodes in ClickHouse Keeper. " } ;
new_values [ " KeeperApproximateDataSize " ] = { approximate_data_size , " The approximate data size of ClickHouse Keeper, in bytes. " } ;
new_values [ " KeeperKeyArenaSize " ] = { key_arena_size , " The size in bytes of the memory arena for keys in ClickHouse Keeper. " } ;
2024-02-14 11:11:45 +00:00
/// TODO: value was incorrectly set to 0 previously for local snapshots
/// it needs to be fixed and it needs to be atomic to avoid deadlock
///new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." };
2022-11-15 12:00:02 +00:00
new_values [ " KeeperOpenFileDescriptorCount " ] = { open_file_descriptor_count , " The number of open file descriptors in ClickHouse Keeper. " } ;
2024-01-18 09:11:50 +00:00
if ( max_file_descriptor_count . has_value ( ) )
new_values [ " KeeperMaxFileDescriptorCount " ] = { * max_file_descriptor_count , " The maximum number of open file descriptors in ClickHouse Keeper. " } ;
else
new_values [ " KeeperMaxFileDescriptorCount " ] = { - 1 , " The maximum number of open file descriptors in ClickHouse Keeper. " } ;
2022-11-15 12:00:02 +00:00
new_values [ " KeeperFollowers " ] = { followers , " The number of followers of ClickHouse Keeper. " } ;
new_values [ " KeeperSyncedFollowers " ] = { synced_followers , " The number of followers of ClickHouse Keeper who are also in-sync. " } ;
new_values [ " KeeperZxid " ] = { zxid , " The current transaction id number (zxid) in ClickHouse Keeper. " } ;
new_values [ " KeeperSessionWithWatches " ] = { session_with_watches , " The number of client sessions of ClickHouse Keeper having watches. " } ;
new_values [ " KeeperPathsWatched " ] = { paths_watched , " The number of different paths watched by the clients of ClickHouse Keeper. " } ;
2022-11-09 14:51:41 +00:00
auto keeper_log_info = keeper_dispatcher . getKeeperLogInfo ( ) ;
2022-11-15 12:00:02 +00:00
new_values [ " KeeperLastLogIdx " ] = { keeper_log_info . last_log_idx , " Index of the last log stored in ClickHouse Keeper. " } ;
new_values [ " KeeperLastLogTerm " ] = { keeper_log_info . last_log_term , " Raft term of the last log stored in ClickHouse Keeper. " } ;
2022-11-09 14:51:41 +00:00
2022-11-15 12:00:02 +00:00
new_values [ " KeeperLastCommittedLogIdx " ] = { keeper_log_info . last_committed_log_idx , " Index of the last committed log in ClickHouse Keeper. " } ;
new_values [ " KeeperTargetCommitLogIdx " ] = { keeper_log_info . target_committed_log_idx , " Index until which logs can be committed in ClickHouse Keeper. " } ;
new_values [ " KeeperLastSnapshotIdx " ] = { keeper_log_info . last_snapshot_idx , " Index of the last log present in the last created snapshot. " } ;
2022-11-09 14:51:41 +00:00
2024-02-01 10:04:34 +00:00
new_values [ " KeeperLatestLogsCacheEntries " ] = { keeper_log_info . latest_logs_cache_entries , " Number of entries stored in the in-memory cache for latest logs " } ;
new_values [ " KeeperLatestLogsCacheSize " ] = { keeper_log_info . latest_logs_cache_size , " Total size of in-memory cache for latest logs " } ;
new_values [ " KeeperCommitLogsCacheEntries " ] = { keeper_log_info . commit_logs_cache_entries , " Number of entries stored in the in-memory cache for next logs to be committed " } ;
new_values [ " KeeperCommitLogsCacheSize " ] = { keeper_log_info . commit_logs_cache_size , " Total size of in-memory cache for next logs to be committed " } ;
2022-11-09 14:51:41 +00:00
auto & keeper_connection_stats = keeper_dispatcher . getKeeperConnectionStats ( ) ;
2022-11-15 12:00:02 +00:00
new_values [ " KeeperMinLatency " ] = { keeper_connection_stats . getMinLatency ( ) , " Minimal request latency of ClickHouse Keeper. " } ;
new_values [ " KeeperMaxLatency " ] = { keeper_connection_stats . getMaxLatency ( ) , " Maximum request latency of ClickHouse Keeper. " } ;
new_values [ " KeeperAvgLatency " ] = { keeper_connection_stats . getAvgLatency ( ) , " Average request latency of ClickHouse Keeper. " } ;
new_values [ " KeeperPacketsReceived " ] = { keeper_connection_stats . getPacketsReceived ( ) , " Number of packets received by ClickHouse Keeper. " } ;
new_values [ " KeeperPacketsSent " ] = { keeper_connection_stats . getPacketsSent ( ) , " Number of packets sent by ClickHouse Keeper. " } ;
2022-11-09 12:37:42 +00:00
# endif
}
KeeperAsynchronousMetrics : : KeeperAsynchronousMetrics (
2023-05-22 12:24:16 +00:00
ContextPtr context_ , int update_period_seconds , const ProtocolServerMetricsFunc & protocol_server_metrics_func_ )
: AsynchronousMetrics ( update_period_seconds , protocol_server_metrics_func_ ) , context ( std : : move ( context_ ) )
2022-11-09 12:37:42 +00:00
{
}
Fix data race on vptr (ctor/dtor vs virtual call) in AsynchronousMetrics
TSan report [1]:
Exception: Sanitizer assert found for instance ==================
WARNING: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) (pid=1)
Write of size 8 at 0x7ffce68f5680 by main thread:
0 DB::AsynchronousMetrics::~AsynchronousMetrics() build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 (clickhouse+0xf38fa6e) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
1 DB::ServerAsynchronousMetrics::~ServerAsynchronousMetrics() build_docker/./src/Interpreters/ServerAsynchronousMetrics.h:10:7 (clickhouse+0xf2b2f0f) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
2 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>>>> const&) build_docker/./programs/server/Server.cpp:1996:1 (clickhouse+0xf2b2f0f)
3 Poco::Util::Application::run() build_docker/./base/poco/Util/src/Application.cpp:315:8 (clickhouse+0x1d6c95fe) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
4 DB::Server::run() build_docker/./programs/server/Server.cpp:397:25 (clickhouse+0xf29e041) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
5 Poco::Util::ServerApplication::run(int, char**) build_docker/./base/poco/Util/src/ServerApplication.cpp:131:9 (clickhouse+0x1d6e9354) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
6 mainEntryClickHouseServer(int, char**) build_docker/./programs/server/Server.cpp:203:20 (clickhouse+0xf29b1a3) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
7 main build_docker/./programs/main.cpp:505:12 (clickhouse+0x72d7fa0) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
Previous read of size 8 at 0x7ffce68f5680 by thread T649:
0 DB::AsynchronousMetrics::update(std::__1::chrono::time_point<std::__1::chrono::system_clock, std::__1::chrono::duration<long long, std::__1::ratio<1l, 1000000l>>>) build_docker/./src/Common/AsynchronousMetrics.cpp:1559:5 (clickhouse+0xf38ebc5) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
1 DB::AsynchronousMetrics::run() build_docker/./src/Common/AsynchronousMetrics.cpp:354:13 (clickhouse+0xf390696) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
2 DB::AsynchronousMetrics::start()::$_0::operator()() const build_docker/./src/Common/AsynchronousMetrics.cpp:273:62 (clickhouse+0xf392eca) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
3 decltype(std::declval<DB::AsynchronousMetrics::start()::$_0&>()()) std::__1::__invoke[abi:v15000]<DB::AsynchronousMetrics::start()::$_0&>(DB::AsynchronousMetrics::start()::$_0&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf392eca)
4 decltype(auto) std::__1::__apply_tuple_impl[abi:v15000]<DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/tuple:1789:1 (clickhouse+0xf392eca)
5 decltype(auto) std::__1::apply[abi:v15000]<DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&>(DB::AsynchronousMetrics::start()::$_0&, std::__1::tuple<>&) build_docker/./contrib/llvm-project/libcxx/include/tuple:1798:1 (clickhouse+0xf392eca)
6 ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()::operator()() build_docker/./src/Common/ThreadPool.h:253:13 (clickhouse+0xf392eca)
7 decltype(std::declval<DB::AsynchronousMetrics::start()::$_0>()()) std::__1::__invoke[abi:v15000]<ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(DB::AsynchronousMetrics::start()::$_0&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf392eca)
8 void std::__1::__invoke_void_return_wrapper<void, true>::__call<ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&>(ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'()&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:479:9 (clickhouse+0xf392eca)
9 std::__1::__function::__default_alloc_func<ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>::operator()[abi:v15000]() build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:235:12 (clickhouse+0xf392eca)
10 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPoolImpl<true>::ThreadFromGlobalPoolImpl<DB::AsynchronousMetrics::start()::$_0>(DB::AsynchronousMetrics::start()::$_0&&)::'lambda'(), void ()>>(std::__1::__function::__policy_storage const*) build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:716:16 (clickhouse+0xf392eca)
11 std::__1::__function::__policy_func<void ()>::operator()[abi:v15000]() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:848:16 (clickhouse+0xf30682e) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
12 std::__1::function<void ()>::operator()() const build_docker/./contrib/llvm-project/libcxx/include/__functional/function.h:1187:12 (clickhouse+0xf30682e)
13 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) build_docker/./src/Common/ThreadPool.cpp:421:13 (clickhouse+0xf30682e)
14 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, Priority, std::__1::optional<unsigned long>, bool)::'lambda0'()::operator()() const build_docker/./src/Common/ThreadPool.cpp:183:73 (clickhouse+0xf30d3d1) (BuildId: d49dc55f80b62802f8721fdbcb07bdc88ac204a4)
15 decltype(std::declval<void>()()) std::__1::__invoke[abi:v15000]<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, Priority, std::__1::optional<unsigned long>, bool)::'lambda0'()>(void&&) build_docker/./contrib/llvm-project/libcxx/include/__functional/invoke.h:394:23 (clickhouse+0xf30d3d1)
16 void std::__1::__thread_execute[abi:v15000]<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct>>, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, Priority, std::__1::optional<unsigned long>, bool)::'lambda0'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, Priority, std::__1::optional<unsigned long>, bool)::'lambda0'()>&, std::__1::__tuple_indices<>) build_docker/./contrib/llvm-project/libcxx/include/thread:284:5 (clickhouse+0xf30d3d1)
17 void* std::__1::__thread_proxy[abi:v15000]<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct>>, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, Priority, std::__1::optional<unsigned long>, bool)::'lambda0'()>>(void*) build_docker/./contrib/llvm-project/libcxx/include/thread:295:5 (clickhouse+0xf30d3d1)
SUMMARY: ThreadSanitizer: data race on vptr (ctor/dtor vs virtual call) build_docker/./src/Common/AsynchronousMetrics.cpp:299:1 in DB::AsynchronousMetrics::~AsynchronousMetrics()
And the problem is real, see [2].
[1]: https://s3.amazonaws.com/clickhouse-test-reports/52717/a6cf2297ef07ca0b065d944c9e30a3986285331d/integration_tests__tsan__[2_6]/integration_run_parallel0_0.log
[2]: https://github.com/google/sanitizers/wiki/ThreadSanitizerPopularDataRaces#data-race-on-vptr
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-12-10 17:17:41 +00:00
KeeperAsynchronousMetrics : : ~ KeeperAsynchronousMetrics ( )
{
/// NOTE: stop() from base class is not enough, since this leads to leak on vptr
stop ( ) ;
}
2023-08-22 21:29:05 +00:00
void KeeperAsynchronousMetrics : : updateImpl ( TimePoint /*update_time*/ , TimePoint /*current_time*/ , bool /*force_update*/ , bool /*first_run*/ , AsynchronousMetricValues & new_values )
2022-11-09 12:37:42 +00:00
{
# if USE_NURAFT
{
2023-05-22 12:24:16 +00:00
auto keeper_dispatcher = context - > tryGetKeeperDispatcher ( ) ;
2022-11-09 12:37:42 +00:00
if ( keeper_dispatcher )
updateKeeperInformation ( * keeper_dispatcher , new_values ) ;
}
# endif
}
}