Merge pull request #11226 from ClickHouse/aku/merge-perf-events

Merge Perf events metrics #9545
This commit is contained in:
Alexander Kuzmenkov 2020-06-05 23:10:54 +03:00 committed by GitHub
commit d17a435fb6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 602 additions and 34 deletions

View File

@ -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>

View File

@ -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

View File

@ -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')

View File

@ -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 conjunction 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.") \

View File

@ -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, &current_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

View File

@ -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

View File

@ -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

View File

@ -33,6 +33,7 @@ class QueryProfilerCpu;
class QueryThreadLog;
class TasksStatsCounters;
struct RUsageCounters;
struct PerfEventsCounters;
class TaskStatsInfoGetter;
class InternalTextLogsQueue;
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;

View File

@ -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)

View File

@ -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)