mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-17 20:02:05 +00:00
Merge pull request #52392 from azat/netlink-provider
Rename TaskStatsInfoGetter into NetlinkMetricsProvider
This commit is contained in:
commit
2467eaf32d
@ -3,7 +3,6 @@
|
||||
#include "CurrentThread.h"
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <Common/TaskStatsInfoGetter.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <base/getThreadId.h>
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include "TaskStatsInfoGetter.h"
|
||||
#include "NetlinkMetricsProvider.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
@ -200,7 +200,7 @@ bool checkPermissionsImpl()
|
||||
if (!res)
|
||||
return false;
|
||||
|
||||
/// Check that we can successfully initialize TaskStatsInfoGetter.
|
||||
/// Check that we can successfully initialize NetlinkMetricsProvider.
|
||||
/// It will ask about family id through Netlink.
|
||||
/// On some LXC containers we have capability but we still cannot use Netlink.
|
||||
/// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result.
|
||||
@ -208,7 +208,7 @@ bool checkPermissionsImpl()
|
||||
try
|
||||
{
|
||||
::taskstats stats{};
|
||||
TaskStatsInfoGetter().getStat(stats, static_cast<pid_t>(getThreadId()));
|
||||
NetlinkMetricsProvider().getStat(stats, static_cast<pid_t>(getThreadId()));
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
@ -244,14 +244,14 @@ UInt16 getFamilyId(int fd)
|
||||
}
|
||||
|
||||
|
||||
bool TaskStatsInfoGetter::checkPermissions()
|
||||
bool NetlinkMetricsProvider::checkPermissions()
|
||||
{
|
||||
static bool res = checkPermissionsImpl();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
TaskStatsInfoGetter::TaskStatsInfoGetter()
|
||||
NetlinkMetricsProvider::NetlinkMetricsProvider()
|
||||
{
|
||||
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
|
||||
if (netlink_socket_fd < 0)
|
||||
@ -293,7 +293,7 @@ TaskStatsInfoGetter::TaskStatsInfoGetter()
|
||||
}
|
||||
|
||||
|
||||
void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const
|
||||
void NetlinkMetricsProvider::getStat(::taskstats & out_stats, pid_t tid) const
|
||||
{
|
||||
NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid));
|
||||
|
||||
@ -318,7 +318,7 @@ void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid) const
|
||||
}
|
||||
|
||||
|
||||
TaskStatsInfoGetter::~TaskStatsInfoGetter()
|
||||
NetlinkMetricsProvider::~NetlinkMetricsProvider()
|
||||
{
|
||||
if (netlink_socket_fd >= 0)
|
||||
{
|
||||
@ -335,15 +335,15 @@ TaskStatsInfoGetter::~TaskStatsInfoGetter()
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool TaskStatsInfoGetter::checkPermissions()
|
||||
bool NetlinkMetricsProvider::checkPermissions()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
TaskStatsInfoGetter::TaskStatsInfoGetter() = default;
|
||||
TaskStatsInfoGetter::~TaskStatsInfoGetter() = default;
|
||||
NetlinkMetricsProvider::NetlinkMetricsProvider() = default;
|
||||
NetlinkMetricsProvider::~NetlinkMetricsProvider() = default;
|
||||
|
||||
void TaskStatsInfoGetter::getStat(::taskstats &, pid_t) const
|
||||
void NetlinkMetricsProvider::getStat(::taskstats &, pid_t) const
|
||||
{
|
||||
}
|
||||
|
@ -15,11 +15,11 @@ namespace DB
|
||||
///
|
||||
/// [1]: https://elixir.bootlin.com/linux/v5.18-rc4/source/kernel/tsacct.c#L101
|
||||
///
|
||||
class TaskStatsInfoGetter : private boost::noncopyable
|
||||
class NetlinkMetricsProvider : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
TaskStatsInfoGetter();
|
||||
~TaskStatsInfoGetter();
|
||||
NetlinkMetricsProvider();
|
||||
~NetlinkMetricsProvider();
|
||||
|
||||
void getStat(::taskstats & out_stats, pid_t tid) const;
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if defined(OS_LINUX)
|
||||
|
||||
#include "TaskStatsInfoGetter.h"
|
||||
#include "NetlinkMetricsProvider.h"
|
||||
#include "ProcfsMetricsProvider.h"
|
||||
#include "hasLinuxCapability.h"
|
||||
|
||||
@ -99,7 +99,7 @@ TasksStatsCounters::MetricsProvider TasksStatsCounters::findBestAvailableProvide
|
||||
static std::optional<MetricsProvider> provider =
|
||||
[]() -> MetricsProvider
|
||||
{
|
||||
if (TaskStatsInfoGetter::checkPermissions())
|
||||
if (NetlinkMetricsProvider::checkPermissions())
|
||||
{
|
||||
return MetricsProvider::Netlink;
|
||||
}
|
||||
@ -119,7 +119,7 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p
|
||||
switch (provider)
|
||||
{
|
||||
case MetricsProvider::Netlink:
|
||||
stats_getter = [metrics_provider = std::make_shared<TaskStatsInfoGetter>(), tid]()
|
||||
stats_getter = [metrics_provider = std::make_shared<NetlinkMetricsProvider>(), tid]()
|
||||
{
|
||||
::taskstats result{};
|
||||
metrics_provider->getStat(result, static_cast<pid_t>(tid));
|
||||
|
@ -114,7 +114,7 @@ std::future<IAsynchronousReader::Result> ThreadPoolReader::submit(Request reques
|
||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
||||
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
|
||||
SCOPE_EXIT({
|
||||
|
@ -95,7 +95,7 @@ size_t ReadBufferFromFileDescriptor::readImpl(char * to, size_t min_bytes, size_
|
||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
||||
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||
watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
||||
|
||||
|
@ -78,7 +78,7 @@ std::future<IAsynchronousReader::Result> SynchronousReader::submit(Request reque
|
||||
/// It reports real time spent including the time spent while thread was preempted doing nothing.
|
||||
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
|
||||
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it
|
||||
/// (TaskStatsInfoGetter has about 500K RPS).
|
||||
/// (NetlinkMetricsProvider has about 500K RPS).
|
||||
watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user