ClickHouse/src/Interpreters/AsynchronousMetrics.h

201 lines
5.9 KiB
C++
Raw Normal View History

#pragma once
#include <Interpreters/Context_fwd.h>
#include <Common/MemoryStatisticsOS.h>
#include <Common/ThreadPool.h>
#include <Common/Stopwatch.h>
2021-07-04 03:03:49 +00:00
#include <IO/ReadBufferFromFile.h>
#include <condition_variable>
2021-07-05 01:18:12 +00:00
#include <map>
#include <mutex>
#include <string>
#include <thread>
2021-07-04 20:49:36 +00:00
#include <vector>
2021-07-04 03:03:49 +00:00
#include <optional>
#include <unordered_map>
2021-07-04 03:03:49 +00:00
namespace Poco
{
class Logger;
}
namespace DB
{
2020-12-17 13:47:03 +00:00
class ProtocolServerAdapter;
2021-07-04 20:49:36 +00:00
class ReadBuffer;
2020-12-17 13:47:03 +00:00
using AsynchronousMetricValue = double;
using AsynchronousMetricValues = std::unordered_map<std::string, AsynchronousMetricValue>;
2020-06-10 19:17:30 +00:00
2020-06-26 00:16:58 +00:00
/** Periodically (by default, each minute, starting at 30 seconds offset)
* calculates and updates some metrics,
* that are not updated automatically (so, need to be asynchronously calculated).
2021-07-05 19:41:50 +00:00
*
* This includes both ClickHouse-related metrics (like memory usage of ClickHouse process)
* and common OS-related metrics (like total memory usage on the server).
*/
class AsynchronousMetrics : WithContext
{
public:
2020-12-21 23:03:08 +00:00
/// The default value of update_period_seconds is for ClickHouse-over-YT
/// in Arcadia -- it uses its own server implementation that also uses these
/// metrics.
2020-12-17 13:47:03 +00:00
AsynchronousMetrics(
ContextPtr global_context_,
2020-12-17 13:47:03 +00:00
int update_period_seconds,
2020-12-21 21:47:10 +00:00
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_to_start_before_tables_,
2021-07-04 03:03:49 +00:00
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_);
~AsynchronousMetrics();
2020-12-17 13:47:03 +00:00
/// Separate method allows to initialize the `servers` variable beforehand.
2021-07-04 03:03:49 +00:00
void start();
/// Returns copy of all values.
2020-06-10 19:17:30 +00:00
AsynchronousMetricValues getValues() const;
2021-07-04 03:03:49 +00:00
#if defined(ARCADIA_BUILD)
/// This constructor needs only to provide backward compatibility with some other projects (hello, Arcadia).
/// Never use this in the ClickHouse codebase.
AsynchronousMetrics(
ContextPtr global_context_,
int update_period_seconds = 60)
: WithContext(global_context_)
, update_period(update_period_seconds)
{
}
#endif
private:
2020-06-26 00:16:58 +00:00
const std::chrono::seconds update_period;
2020-12-21 21:47:10 +00:00
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers_to_start_before_tables{nullptr};
std::shared_ptr<std::vector<ProtocolServerAdapter>> servers{nullptr};
2020-06-10 19:17:30 +00:00
mutable std::mutex mutex;
std::condition_variable wait_cond;
2020-06-10 19:17:30 +00:00
bool quit {false};
AsynchronousMetricValues values;
2021-07-04 20:49:36 +00:00
/// Some values are incremental and we have to calculate the difference.
/// On first run we will only collect the values to subtract later.
bool first_run = true;
2021-07-04 22:33:32 +00:00
std::chrono::system_clock::time_point previous_update_time;
2021-07-04 20:49:36 +00:00
2020-04-21 18:49:19 +00:00
#if defined(OS_LINUX)
2020-04-19 20:49:13 +00:00
MemoryStatisticsOS memory_stat;
2021-07-04 03:03:49 +00:00
std::optional<ReadBufferFromFilePRead> meminfo;
std::optional<ReadBufferFromFilePRead> loadavg;
std::optional<ReadBufferFromFilePRead> proc_stat;
std::optional<ReadBufferFromFilePRead> cpuinfo;
std::optional<ReadBufferFromFilePRead> file_nr;
std::optional<ReadBufferFromFilePRead> uptime;
std::optional<ReadBufferFromFilePRead> net_dev;
2021-07-05 01:18:12 +00:00
std::vector<std::unique_ptr<ReadBufferFromFilePRead>> thermal;
2021-07-04 20:49:36 +00:00
2021-07-05 00:40:28 +00:00
std::unordered_map<String /* device name */,
std::unordered_map<String /* label name */,
std::unique_ptr<ReadBufferFromFilePRead>>> hwmon_devices;
2021-07-05 00:40:28 +00:00
2021-07-05 01:18:12 +00:00
std::vector<std::pair<
std::unique_ptr<ReadBufferFromFilePRead> /* correctable errors */,
std::unique_ptr<ReadBufferFromFilePRead> /* uncorrectable errors */>> edac;
2021-07-05 01:18:12 +00:00
std::unordered_map<String /* device name */, std::unique_ptr<ReadBufferFromFilePRead>> block_devs;
2021-07-05 02:24:36 +00:00
2021-07-05 02:47:33 +00:00
/// TODO: socket statistics.
2021-07-04 21:22:58 +00:00
2021-07-04 20:49:36 +00:00
struct ProcStatValuesCPU
{
uint64_t user;
uint64_t nice;
uint64_t system;
uint64_t idle;
uint64_t iowait;
uint64_t irq;
uint64_t softirq;
uint64_t steal;
uint64_t guest;
uint64_t guest_nice;
void read(ReadBuffer & in);
ProcStatValuesCPU operator-(const ProcStatValuesCPU & other) const;
};
struct ProcStatValuesOther
{
uint64_t interrupts;
uint64_t context_switches;
uint64_t processes_created;
ProcStatValuesOther operator-(const ProcStatValuesOther & other) const;
};
ProcStatValuesCPU proc_stat_values_all_cpus{};
ProcStatValuesOther proc_stat_values_other{};
std::vector<ProcStatValuesCPU> proc_stat_values_per_cpu;
2021-07-05 02:24:36 +00:00
/// https://www.kernel.org/doc/Documentation/block/stat.txt
struct BlockDeviceStatValues
{
uint64_t read_ios;
uint64_t read_merges;
uint64_t read_sectors;
uint64_t read_ticks;
uint64_t write_ios;
uint64_t write_merges;
uint64_t write_sectors;
uint64_t write_ticks;
uint64_t in_flight_ios;
uint64_t io_ticks;
uint64_t time_in_queue;
uint64_t discard_ops;
uint64_t discard_merges;
uint64_t discard_sectors;
uint64_t discard_ticks;
void read(ReadBuffer & in);
BlockDeviceStatValues operator-(const BlockDeviceStatValues & other) const;
};
std::unordered_map<String /* device name */, BlockDeviceStatValues> block_device_stats;
2021-07-05 02:47:33 +00:00
struct NetworkInterfaceStatValues
{
uint64_t recv_bytes;
uint64_t recv_packets;
uint64_t recv_errors;
uint64_t recv_drop;
uint64_t send_bytes;
uint64_t send_packets;
uint64_t send_errors;
uint64_t send_drop;
NetworkInterfaceStatValues operator-(const NetworkInterfaceStatValues & other) const;
};
std::unordered_map<String /* device name */, NetworkInterfaceStatValues> network_interface_stats;
Stopwatch block_devices_rescan_delay;
2021-08-11 07:09:00 +00:00
void openSensors();
void openBlockDevices();
void openSensorsChips();
void openEDAC();
2020-04-21 18:49:19 +00:00
#endif
2020-12-17 13:47:03 +00:00
std::unique_ptr<ThreadFromGlobalPool> thread;
void run();
2021-07-04 22:33:32 +00:00
void update(std::chrono::system_clock::time_point update_time);
Poco::Logger * log;
};
}