mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #11226 from ClickHouse/aku/merge-perf-events
Merge Perf events metrics #9545
This commit is contained in:
commit
d17a435fb6
@ -5,6 +5,7 @@
|
||||
<query_profiler_cpu_time_period_ns>0</query_profiler_cpu_time_period_ns>
|
||||
<allow_introspection_functions>1</allow_introspection_functions>
|
||||
<log_queries>1</log_queries>
|
||||
<metrics_perf_events_enabled>1</metrics_perf_events_enabled>
|
||||
</default>
|
||||
</profiles>
|
||||
</yandex>
|
||||
|
@ -131,5 +131,5 @@ done
|
||||
|
||||
dmesg -T > dmesg.log
|
||||
|
||||
7z a /output/output.7z ./*.{log,tsv,html,txt,rep,svg} {right,left}/{performance,db/preprocessed_configs,scripts} report analyze
|
||||
7z a '-x!*/tmp' /output/output.7z ./*.{log,tsv,html,txt,rep,svg,columns} {right,left}/{performance,db/preprocessed_configs,scripts} report analyze
|
||||
cp compare.log /output
|
||||
|
@ -100,11 +100,20 @@ for c in connections:
|
||||
|
||||
report_stage_end('drop1')
|
||||
|
||||
# Apply settings
|
||||
# Apply settings.
|
||||
# If there are errors, report them and continue -- maybe a new test uses a setting
|
||||
# that is not in master, but the queries can still run. If we have multiple
|
||||
# settings and one of them throws an exception, all previous settings for this
|
||||
# connection will be reset, because the driver reconnects on error (not
|
||||
# configurable). So the end result is uncertain, but hopefully we'll be able to
|
||||
# run at least some queries.
|
||||
settings = root.findall('settings/*')
|
||||
for c in connections:
|
||||
for s in settings:
|
||||
c.execute("set {} = '{}'".format(s.tag, s.text))
|
||||
try:
|
||||
c.execute("set {} = '{}'".format(s.tag, s.text))
|
||||
except:
|
||||
print(traceback.format_exc(), file=sys.stderr)
|
||||
|
||||
report_stage_end('settings')
|
||||
|
||||
|
@ -180,6 +180,25 @@
|
||||
M(OSWriteBytes, "Number of bytes written to disks or block devices. Doesn't include bytes that are in page cache dirty pages. May not include data that was written by OS asynchronously.") \
|
||||
M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \
|
||||
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
|
||||
\
|
||||
M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \
|
||||
M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \
|
||||
M(PerfCacheReferences, "Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \
|
||||
M(PerfCacheMisses, "Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction with the PERFCOUNTHWCACHEREFERENCES event to calculate cache miss rates.") \
|
||||
M(PerfBranchInstructions, "Retired branch instructions. Prior to Linux 2.6.35, this used the wrong event on AMD processors.") \
|
||||
M(PerfBranchMisses, "Mispredicted branch instructions.") \
|
||||
M(PerfBusCycles, "Bus cycles, which can be different from total cycles.") \
|
||||
M(PerfStalledCyclesFrontend, "Stalled cycles during issue.") \
|
||||
M(PerfStalledCyclesBackend, "Stalled cycles during retirement.") \
|
||||
M(PerfRefCpuCycles, "Total cycles; not affected by CPU frequency scaling.") \
|
||||
\
|
||||
M(PerfCpuClock, "The CPU clock, a high-resolution per-CPU timer") \
|
||||
M(PerfTaskClock, "A clock count specific to the task that is running") \
|
||||
M(PerfContextSwitches, "Number of context switches") \
|
||||
M(PerfCpuMigrations, "Number of times the process has migrated to a new CPU") \
|
||||
M(PerfAlignmentFaults, "Number of alignment faults. These happen when unaligned memory accesses happen; the kernel can handle these but it reduces performance. This happens only on some architectures (never on x86).") \
|
||||
M(PerfEmulationFaults, "Number of emulation faults. The kernel sometimes traps on unimplemented instructions and emulates them for user space. This can negatively impact performance.") \
|
||||
\
|
||||
M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \
|
||||
\
|
||||
M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \
|
||||
|
@ -4,9 +4,22 @@
|
||||
|
||||
#include "TaskStatsInfoGetter.h"
|
||||
#include "ProcfsMetricsProvider.h"
|
||||
#include "hasLinuxCapability.h"
|
||||
|
||||
#include <filesystem>
|
||||
#include <fstream>
|
||||
#include <optional>
|
||||
#include <sstream>
|
||||
#include <unordered_set>
|
||||
|
||||
#include <fcntl.h>
|
||||
#include <unistd.h>
|
||||
#include <linux/perf_event.h>
|
||||
#include <syscall.h>
|
||||
#include <sys/ioctl.h>
|
||||
#include <cerrno>
|
||||
#include <sys/types.h>
|
||||
#include <dirent.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -104,6 +117,404 @@ void TasksStatsCounters::incrementProfileEvents(const ::taskstats & prev, const
|
||||
profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.read_bytes, curr.read_bytes));
|
||||
profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.write_bytes, curr.write_bytes));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
#if defined(__linux__) && !defined(ARCADIA_BUILD)
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
thread_local PerfEventsCounters current_thread_counters;
|
||||
|
||||
#define SOFTWARE_EVENT(PERF_NAME, LOCAL_NAME) \
|
||||
PerfEventInfo \
|
||||
{ \
|
||||
.event_type = perf_type_id::PERF_TYPE_SOFTWARE, \
|
||||
.event_config = (PERF_NAME), \
|
||||
.profile_event = ProfileEvents::LOCAL_NAME, \
|
||||
.settings_name = #LOCAL_NAME \
|
||||
}
|
||||
|
||||
#define HARDWARE_EVENT(PERF_NAME, LOCAL_NAME) \
|
||||
PerfEventInfo \
|
||||
{ \
|
||||
.event_type = perf_type_id::PERF_TYPE_HARDWARE, \
|
||||
.event_config = (PERF_NAME), \
|
||||
.profile_event = ProfileEvents::LOCAL_NAME, \
|
||||
.settings_name = #LOCAL_NAME \
|
||||
}
|
||||
|
||||
// descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html
|
||||
static const PerfEventInfo raw_events_info[] = {
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CPU_CYCLES, PerfCpuCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_INSTRUCTIONS, PerfInstructions),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CACHE_REFERENCES, PerfCacheReferences),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CACHE_MISSES, PerfCacheMisses),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, PerfBranchInstructions),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_BRANCH_MISSES, PerfBranchMisses),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_BUS_CYCLES, PerfBusCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, PerfStalledCyclesFrontend),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, PerfStalledCyclesBackend),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_REF_CPU_CYCLES, PerfRefCpuCycles),
|
||||
// `cpu-clock` is a bit broken according to this: https://stackoverflow.com/a/56967896
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_CLOCK, PerfCpuClock),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_TASK_CLOCK, PerfTaskClock),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CONTEXT_SWITCHES, PerfContextSwitches),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_MIGRATIONS, PerfCpuMigrations),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_ALIGNMENT_FAULTS, PerfAlignmentFaults),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_EMULATION_FAULTS, PerfEmulationFaults)
|
||||
};
|
||||
|
||||
#undef HARDWARE_EVENT
|
||||
#undef SOFTWARE_EVENT
|
||||
|
||||
// A map of event name -> event index, to parse event list in settings.
|
||||
static std::unordered_map<std::string, size_t> populateEventMap()
|
||||
{
|
||||
std::unordered_map<std::string, size_t> name_to_index;
|
||||
name_to_index.reserve(NUMBER_OF_RAW_EVENTS);
|
||||
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
{
|
||||
name_to_index.emplace(raw_events_info[i].settings_name, i);
|
||||
}
|
||||
|
||||
return name_to_index;
|
||||
}
|
||||
|
||||
static const auto event_name_to_index = populateEventMap();
|
||||
|
||||
static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, UInt64 flags)
|
||||
{
|
||||
return static_cast<int>(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags));
|
||||
}
|
||||
|
||||
static int openPerfEventDisabled(Int32 perf_event_paranoid, bool has_cap_sys_admin, UInt32 perf_event_type, UInt64 perf_event_config)
|
||||
{
|
||||
perf_event_attr pe{};
|
||||
pe.type = perf_event_type;
|
||||
pe.size = sizeof(struct perf_event_attr);
|
||||
pe.config = perf_event_config;
|
||||
// disable by default to add as little extra time as possible
|
||||
pe.disabled = 1;
|
||||
// can record kernel only when `perf_event_paranoid` <= 1 or have CAP_SYS_ADMIN
|
||||
pe.exclude_kernel = perf_event_paranoid >= 2 && !has_cap_sys_admin;
|
||||
pe.read_format = PERF_FORMAT_TOTAL_TIME_ENABLED | PERF_FORMAT_TOTAL_TIME_RUNNING;
|
||||
|
||||
return openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0);
|
||||
}
|
||||
|
||||
static void enablePerfEvent(int event_fd)
|
||||
{
|
||||
if (ioctl(event_fd, PERF_EVENT_IOC_ENABLE, 0))
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Can't enable perf event with file descriptor {}: '{}' ({})",
|
||||
event_fd, strerror(errno), errno);
|
||||
}
|
||||
}
|
||||
|
||||
static void disablePerfEvent(int event_fd)
|
||||
{
|
||||
if (ioctl(event_fd, PERF_EVENT_IOC_DISABLE, 0))
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Can't disable perf event with file descriptor {}: '{}' ({})",
|
||||
event_fd, strerror(errno), errno);
|
||||
}
|
||||
}
|
||||
|
||||
static void releasePerfEvent(int event_fd)
|
||||
{
|
||||
if (close(event_fd))
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Can't close perf event file descriptor {}: {} ({})",
|
||||
event_fd, strerror(errno), errno);
|
||||
}
|
||||
}
|
||||
|
||||
static bool validatePerfEventDescriptor(int & fd)
|
||||
{
|
||||
if (fcntl(fd, F_GETFL) != -1)
|
||||
return true;
|
||||
|
||||
if (errno == EBADF)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Event descriptor {} was closed from the outside; reopening", fd);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Error while checking availability of event descriptor {}: {} ({})",
|
||||
fd, strerror(errno), errno);
|
||||
|
||||
disablePerfEvent(fd);
|
||||
releasePerfEvent(fd);
|
||||
}
|
||||
|
||||
fd = -1;
|
||||
return false;
|
||||
}
|
||||
|
||||
bool PerfEventsCounters::processThreadLocalChanges(const std::string & needed_events_list)
|
||||
{
|
||||
const auto valid_event_indices = eventIndicesFromString(needed_events_list);
|
||||
|
||||
// find state changes (if there are any)
|
||||
bool old_state[NUMBER_OF_RAW_EVENTS];
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
old_state[i] = thread_events_descriptors_holder.descriptors[i] != -1;
|
||||
|
||||
bool new_state[NUMBER_OF_RAW_EVENTS];
|
||||
std::fill_n(new_state, NUMBER_OF_RAW_EVENTS, false);
|
||||
for (size_t opened_index : valid_event_indices)
|
||||
new_state[opened_index] = true;
|
||||
|
||||
std::vector<size_t> events_to_open;
|
||||
std::vector<size_t> events_to_release;
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
{
|
||||
bool old_one = old_state[i];
|
||||
bool new_one = new_state[i];
|
||||
|
||||
if (old_one == new_one)
|
||||
{
|
||||
if (old_one
|
||||
&& !validatePerfEventDescriptor(
|
||||
thread_events_descriptors_holder.descriptors[i]))
|
||||
{
|
||||
events_to_open.push_back(i);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
if (new_one)
|
||||
events_to_open.push_back(i);
|
||||
else
|
||||
events_to_release.push_back(i);
|
||||
}
|
||||
|
||||
// release unused descriptors
|
||||
for (size_t i : events_to_release)
|
||||
{
|
||||
int & fd = thread_events_descriptors_holder.descriptors[i];
|
||||
disablePerfEvent(fd);
|
||||
releasePerfEvent(fd);
|
||||
fd = -1;
|
||||
}
|
||||
|
||||
if (events_to_open.empty())
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
// check permissions
|
||||
// cat /proc/sys/kernel/perf_event_paranoid
|
||||
// -1: Allow use of (almost) all events by all users
|
||||
// >=0: Disallow raw tracepoint access by users without CAP_IOC_LOCK
|
||||
// >=1: Disallow CPU event access by users without CAP_SYS_ADMIN
|
||||
// >=2: Disallow kernel profiling by users without CAP_SYS_ADMIN
|
||||
// >=3: Disallow all event access by users without CAP_SYS_ADMIN
|
||||
Int32 perf_event_paranoid = 0;
|
||||
std::ifstream paranoid_file("/proc/sys/kernel/perf_event_paranoid");
|
||||
paranoid_file >> perf_event_paranoid;
|
||||
|
||||
bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN);
|
||||
if (perf_event_paranoid >= 3 && !has_cap_sys_admin)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Not enough permissions to record perf events: "
|
||||
"perf_event_paranoid = {} and CAP_SYS_ADMIN = 0",
|
||||
perf_event_paranoid);
|
||||
return false;
|
||||
}
|
||||
|
||||
// Open descriptors for new events.
|
||||
// Theoretically, we can run out of file descriptors. Threads go up to 10k,
|
||||
// and there might be a dozen perf events per thread, so we're looking at
|
||||
// 100k open files. In practice, this is not likely -- perf events are
|
||||
// mostly used in performance tests or other kinds of testing, and the
|
||||
// number of threads stays below hundred.
|
||||
// We used to check the number of open files by enumerating /proc/self/fd,
|
||||
// but listing all open files before opening more files is obviously
|
||||
// quadratic, and quadraticity never ends well.
|
||||
for (size_t i : events_to_open)
|
||||
{
|
||||
const PerfEventInfo & event_info = raw_events_info[i];
|
||||
int & fd = thread_events_descriptors_holder.descriptors[i];
|
||||
// disable by default to add as little extra time as possible
|
||||
fd = openPerfEventDisabled(perf_event_paranoid, has_cap_sys_admin, event_info.event_type, event_info.event_config);
|
||||
|
||||
if (fd == -1 && errno != ENOENT)
|
||||
{
|
||||
// ENOENT means that the event is not supported. Don't log it, because
|
||||
// this is called for each thread and would be too verbose. Log other
|
||||
// error codes because they might signify an error.
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Failed to open perf event {} (event_type={}, event_config={}): "
|
||||
"'{}' ({})", event_info.settings_name, event_info.event_type,
|
||||
event_info.event_config, strerror(errno), errno);
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
// Parse comma-separated list of event names. Empty means all available
|
||||
// events.
|
||||
std::vector<size_t> PerfEventsCounters::eventIndicesFromString(const std::string & events_list)
|
||||
{
|
||||
std::vector<size_t> result;
|
||||
result.reserve(NUMBER_OF_RAW_EVENTS);
|
||||
|
||||
if (events_list.empty())
|
||||
{
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
{
|
||||
result.push_back(i);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
std::istringstream iss(events_list);
|
||||
std::string event_name;
|
||||
while (std::getline(iss, event_name, ','))
|
||||
{
|
||||
// Allow spaces at the beginning of the token, so that you can write
|
||||
// 'a, b'.
|
||||
event_name.erase(0, event_name.find_first_not_of(' '));
|
||||
|
||||
auto entry = event_name_to_index.find(event_name);
|
||||
if (entry != event_name_to_index.end())
|
||||
{
|
||||
result.push_back(entry->second);
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_ERROR(&Poco::Logger::get("PerfEvents"),
|
||||
"Unknown perf event name '{}' specified in settings", event_name);
|
||||
}
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void PerfEventsCounters::initializeProfileEvents(const std::string & events_list)
|
||||
{
|
||||
if (!processThreadLocalChanges(events_list))
|
||||
return;
|
||||
|
||||
for (int fd : thread_events_descriptors_holder.descriptors)
|
||||
{
|
||||
if (fd == -1)
|
||||
continue;
|
||||
|
||||
// We don't reset the event, because the time_running and time_enabled
|
||||
// can't be reset anyway and we have to calculate deltas.
|
||||
enablePerfEvent(fd);
|
||||
}
|
||||
}
|
||||
|
||||
void PerfEventsCounters::finalizeProfileEvents(ProfileEvents::Counters & profile_events)
|
||||
{
|
||||
// Disable all perf events.
|
||||
for (auto fd : thread_events_descriptors_holder.descriptors)
|
||||
{
|
||||
if (fd == -1)
|
||||
continue;
|
||||
disablePerfEvent(fd);
|
||||
}
|
||||
|
||||
// Read the counter values.
|
||||
PerfEventValue current_values[NUMBER_OF_RAW_EVENTS];
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
{
|
||||
int fd = thread_events_descriptors_holder.descriptors[i];
|
||||
if (fd == -1)
|
||||
continue;
|
||||
|
||||
constexpr ssize_t bytes_to_read = sizeof(current_values[0]);
|
||||
const int bytes_read = read(fd, ¤t_values[i], bytes_to_read);
|
||||
|
||||
if (bytes_read != bytes_to_read)
|
||||
{
|
||||
LOG_WARNING(&Poco::Logger::get("PerfEvents"),
|
||||
"Can't read event value from file descriptor {}: '{}' ({})",
|
||||
fd, strerror(errno), errno);
|
||||
current_values[i] = {};
|
||||
}
|
||||
}
|
||||
|
||||
// actually process counters' values
|
||||
for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i)
|
||||
{
|
||||
int fd = thread_events_descriptors_holder.descriptors[i];
|
||||
if (fd == -1)
|
||||
continue;
|
||||
|
||||
const PerfEventInfo & info = raw_events_info[i];
|
||||
const PerfEventValue & previous_value = previous_values[i];
|
||||
const PerfEventValue & current_value = current_values[i];
|
||||
|
||||
// Account for counter multiplexing. time_running and time_enabled are
|
||||
// not reset by PERF_EVENT_IOC_RESET, so we don't use it and calculate
|
||||
// deltas from old values.
|
||||
const UInt64 delta = (current_value.value - previous_value.value)
|
||||
* (current_value.time_enabled - previous_value.time_enabled)
|
||||
/ std::max(1.f,
|
||||
float(current_value.time_running - previous_value.time_running));
|
||||
|
||||
profile_events.increment(info.profile_event, delta);
|
||||
}
|
||||
|
||||
// Store current counter values for the next profiling period.
|
||||
memcpy(previous_values, current_values, sizeof(current_values));
|
||||
}
|
||||
|
||||
void PerfEventsCounters::closeEventDescriptors()
|
||||
{
|
||||
thread_events_descriptors_holder.releaseResources();
|
||||
}
|
||||
|
||||
PerfDescriptorsHolder::PerfDescriptorsHolder()
|
||||
{
|
||||
for (int & descriptor : descriptors)
|
||||
descriptor = -1;
|
||||
}
|
||||
|
||||
PerfDescriptorsHolder::~PerfDescriptorsHolder()
|
||||
{
|
||||
releaseResources();
|
||||
}
|
||||
|
||||
void PerfDescriptorsHolder::releaseResources()
|
||||
{
|
||||
for (int & descriptor : descriptors)
|
||||
{
|
||||
if (descriptor == -1)
|
||||
continue;
|
||||
|
||||
disablePerfEvent(descriptor);
|
||||
releasePerfEvent(descriptor);
|
||||
descriptor = -1;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
#else
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
// Not on Linux or in Arcadia: the functionality is disabled.
|
||||
PerfEventsCounters current_thread_counters;
|
||||
|
||||
}
|
||||
|
||||
#endif
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <sys/time.h>
|
||||
#include <sys/resource.h>
|
||||
#include <pthread.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
#if defined(__linux__)
|
||||
@ -34,6 +35,24 @@ namespace ProfileEvents
|
||||
extern const Event OSWriteChars;
|
||||
extern const Event OSReadBytes;
|
||||
extern const Event OSWriteBytes;
|
||||
|
||||
extern const Event PerfCpuCycles;
|
||||
extern const Event PerfInstructions;
|
||||
extern const Event PerfCacheReferences;
|
||||
extern const Event PerfCacheMisses;
|
||||
extern const Event PerfBranchInstructions;
|
||||
extern const Event PerfBranchMisses;
|
||||
extern const Event PerfBusCycles;
|
||||
extern const Event PerfStalledCyclesFrontend;
|
||||
extern const Event PerfStalledCyclesBackend;
|
||||
extern const Event PerfRefCpuCycles;
|
||||
|
||||
extern const Event PerfCpuClock;
|
||||
extern const Event PerfTaskClock;
|
||||
extern const Event PerfContextSwitches;
|
||||
extern const Event PerfCpuMigrations;
|
||||
extern const Event PerfAlignmentFaults;
|
||||
extern const Event PerfEmulationFaults;
|
||||
#endif
|
||||
}
|
||||
|
||||
@ -116,6 +135,78 @@ struct RUsageCounters
|
||||
}
|
||||
};
|
||||
|
||||
// thread_local is disabled in Arcadia, so we have to use a dummy implementation
|
||||
// there.
|
||||
#if defined(__linux__) && !defined(ARCADIA_BUILD)
|
||||
|
||||
struct PerfEventInfo
|
||||
{
|
||||
// see perf_event.h/perf_type_id enum
|
||||
int event_type;
|
||||
// see configs in perf_event.h
|
||||
int event_config;
|
||||
ProfileEvents::Event profile_event;
|
||||
std::string settings_name;
|
||||
};
|
||||
|
||||
struct PerfEventValue
|
||||
{
|
||||
UInt64 value = 0;
|
||||
UInt64 time_enabled = 0;
|
||||
UInt64 time_running = 0;
|
||||
};
|
||||
|
||||
static constexpr size_t NUMBER_OF_RAW_EVENTS = 16;
|
||||
|
||||
struct PerfDescriptorsHolder : boost::noncopyable
|
||||
{
|
||||
int descriptors[NUMBER_OF_RAW_EVENTS]{};
|
||||
|
||||
PerfDescriptorsHolder();
|
||||
|
||||
~PerfDescriptorsHolder();
|
||||
|
||||
void releaseResources();
|
||||
};
|
||||
|
||||
struct PerfEventsCounters
|
||||
{
|
||||
PerfDescriptorsHolder thread_events_descriptors_holder;
|
||||
|
||||
// time_enabled and time_running can't be reset, so we have to store the
|
||||
// data from the previous profiling period and calculate deltas to them,
|
||||
// to be able to properly account for counter multiplexing.
|
||||
PerfEventValue previous_values[NUMBER_OF_RAW_EVENTS]{};
|
||||
|
||||
|
||||
void initializeProfileEvents(const std::string & events_list);
|
||||
void finalizeProfileEvents(ProfileEvents::Counters & profile_events);
|
||||
void closeEventDescriptors();
|
||||
bool processThreadLocalChanges(const std::string & needed_events_list);
|
||||
|
||||
|
||||
static std::vector<size_t> eventIndicesFromString(const std::string & events_list);
|
||||
};
|
||||
|
||||
// Perf event creation is moderately heavy, so we create them once per thread and
|
||||
// then reuse.
|
||||
extern thread_local PerfEventsCounters current_thread_counters;
|
||||
|
||||
#else
|
||||
|
||||
// Not on Linux, or in Arcadia: the functionality is disabled.
|
||||
struct PerfEventsCounters
|
||||
{
|
||||
void initializeProfileEvents(const std::string & /* events_list */) {}
|
||||
void finalizeProfileEvents(ProfileEvents::Counters & /* profile_events */) {}
|
||||
void closeEventDescriptors() {}
|
||||
};
|
||||
|
||||
// thread_local is disabled in Arcadia, so we are going to use a static dummy.
|
||||
extern PerfEventsCounters current_thread_counters;
|
||||
|
||||
#endif
|
||||
|
||||
#if defined(__linux__)
|
||||
|
||||
class TasksStatsCounters
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <sstream>
|
||||
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/ThreadProfileEvents.h>
|
||||
#include <Common/QueryProfiler.h>
|
||||
@ -57,36 +56,6 @@ ThreadStatus::~ThreadStatus()
|
||||
current_thread = nullptr;
|
||||
}
|
||||
|
||||
void ThreadStatus::initPerformanceCounters()
|
||||
{
|
||||
performance_counters_finalized = false;
|
||||
|
||||
/// Clear stats from previous query if a new query is started
|
||||
/// TODO: make separate query_thread_performance_counters and thread_performance_counters
|
||||
performance_counters.resetCounters();
|
||||
memory_tracker.resetCounters();
|
||||
memory_tracker.setDescription("(for thread)");
|
||||
|
||||
query_start_time_nanoseconds = getCurrentTimeNanoseconds();
|
||||
query_start_time = time(nullptr);
|
||||
++queries_started;
|
||||
|
||||
*last_rusage = RUsageCounters::current(query_start_time_nanoseconds);
|
||||
if (!taskstats)
|
||||
{
|
||||
try
|
||||
{
|
||||
taskstats = TasksStatsCounters::create(thread_id);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
if (taskstats)
|
||||
taskstats->reset();
|
||||
}
|
||||
|
||||
void ThreadStatus::updatePerformanceCounters()
|
||||
{
|
||||
try
|
||||
|
@ -33,6 +33,7 @@ class QueryProfilerCpu;
|
||||
class QueryThreadLog;
|
||||
class TasksStatsCounters;
|
||||
struct RUsageCounters;
|
||||
struct PerfEventsCounters;
|
||||
class TaskStatsInfoGetter;
|
||||
class InternalTextLogsQueue;
|
||||
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
|
||||
|
@ -28,6 +28,7 @@ class IColumn;
|
||||
|
||||
|
||||
/** Settings of query execution.
|
||||
* These settings go to users.xml.
|
||||
*/
|
||||
struct Settings : public SettingsCollection<Settings>
|
||||
{
|
||||
@ -276,6 +277,8 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \
|
||||
M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off the real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off the CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \
|
||||
M(SettingBool, metrics_perf_events_enabled, false, "If enabled, some of the perf events will be measured throughout queries' execution.", 0) \
|
||||
M(SettingString, metrics_perf_events_list, "", "Comma separated list of perf metrics that will be measured throughout queries' execution. Empty means all events. See PerfEventInfo in sources for the available events.", 0) \
|
||||
\
|
||||
\
|
||||
/** Limits during query execution are part of the settings. \
|
||||
@ -441,6 +444,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingUInt64, mark_cache_min_lifetime, 0, "Obsolete setting, does nothing. Will be removed after 2020-05-31", 0) \
|
||||
M(SettingBool, partial_merge_join, false, "Obsolete. Use join_algorithm='prefer_partial_merge' instead.", 0) \
|
||||
M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Obsolete. Will be removed after 2020-10-20", 0) \
|
||||
\
|
||||
M(SettingBool, experimental_use_processors, true, "Obsolete setting, does nothing. Will be removed after 2020-11-29.", 0) \
|
||||
|
||||
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)
|
||||
|
@ -134,6 +134,54 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool
|
||||
setupState(thread_group_);
|
||||
}
|
||||
|
||||
void ThreadStatus::initPerformanceCounters()
|
||||
{
|
||||
performance_counters_finalized = false;
|
||||
|
||||
/// Clear stats from previous query if a new query is started
|
||||
/// TODO: make separate query_thread_performance_counters and thread_performance_counters
|
||||
performance_counters.resetCounters();
|
||||
memory_tracker.resetCounters();
|
||||
memory_tracker.setDescription("(for thread)");
|
||||
|
||||
query_start_time_nanoseconds = getCurrentTimeNanoseconds();
|
||||
query_start_time = time(nullptr);
|
||||
++queries_started;
|
||||
|
||||
*last_rusage = RUsageCounters::current(query_start_time_nanoseconds);
|
||||
|
||||
if (query_context)
|
||||
{
|
||||
const Settings & settings = query_context->getSettingsRef();
|
||||
if (settings.metrics_perf_events_enabled)
|
||||
{
|
||||
try
|
||||
{
|
||||
current_thread_counters.initializeProfileEvents(
|
||||
settings.metrics_perf_events_list);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!taskstats)
|
||||
{
|
||||
try
|
||||
{
|
||||
taskstats = TasksStatsCounters::create(thread_id);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
if (taskstats)
|
||||
taskstats->reset();
|
||||
}
|
||||
|
||||
void ThreadStatus::finalizePerformanceCounters()
|
||||
{
|
||||
if (performance_counters_finalized)
|
||||
@ -142,6 +190,21 @@ void ThreadStatus::finalizePerformanceCounters()
|
||||
performance_counters_finalized = true;
|
||||
updatePerformanceCounters();
|
||||
|
||||
bool close_perf_descriptors = true;
|
||||
if (query_context)
|
||||
close_perf_descriptors = !query_context->getSettingsRef().metrics_perf_events_enabled;
|
||||
|
||||
try
|
||||
{
|
||||
current_thread_counters.finalizeProfileEvents(performance_counters);
|
||||
if (close_perf_descriptors)
|
||||
current_thread_counters.closeEventDescriptors();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
if (global_context && query_context)
|
||||
|
Loading…
Reference in New Issue
Block a user