Merge pull request #16101 from azat/http-memory-tracking-fixes

Fix memory tracker settings (for HTTP queries)
This commit is contained in:
alexey-milovidov 2020-10-22 00:00:47 +03:00 committed by GitHub
commit 2ae32e3d5c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 76 additions and 31 deletions

View File

@ -164,6 +164,8 @@ public:
void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false);
protected:
void applyQuerySettings();
void initPerformanceCounters();
void initQueryProfiler();

View File

@ -30,11 +30,37 @@ namespace ErrorCodes
extern const int CANNOT_SET_THREAD_PRIORITY;
}
void ThreadStatus::applyQuerySettings()
{
const Settings & settings = query_context->getSettingsRef();
query_id = query_context->getCurrentQueryId();
initQueryProfiler();
untracked_memory_limit = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < UInt64(untracked_memory_limit))
untracked_memory_limit = settings.memory_profiler_step;
#if defined(OS_LINUX)
/// Set "nice" value if required.
Int32 new_os_thread_priority = settings.os_thread_priority;
if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE))
{
LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority);
if (0 != setpriority(PRIO_PROCESS, thread_id, new_os_thread_priority))
throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY);
os_thread_priority = new_os_thread_priority;
}
#endif
}
void ThreadStatus::attachQueryContext(Context & query_context_)
{
query_context = &query_context_;
query_id = query_context->getCurrentQueryId();
if (!global_context)
global_context = &query_context->getGlobalContext();
@ -47,7 +73,7 @@ void ThreadStatus::attachQueryContext(Context & query_context_)
thread_group->global_context = global_context;
}
initQueryProfiler();
applyQuerySettings();
}
void CurrentThread::defaultThreadDeleter()
@ -82,30 +108,7 @@ void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_)
}
if (query_context)
{
query_id = query_context->getCurrentQueryId();
initQueryProfiler();
const Settings & settings = query_context->getSettingsRef();
untracked_memory_limit = settings.max_untracked_memory;
if (settings.memory_profiler_step && settings.memory_profiler_step < UInt64(untracked_memory_limit))
untracked_memory_limit = settings.memory_profiler_step;
#if defined(OS_LINUX)
/// Set "nice" value if required.
Int32 new_os_thread_priority = settings.os_thread_priority;
if (new_os_thread_priority && hasLinuxCapability(CAP_SYS_NICE))
{
LOG_TRACE(log, "Setting nice to {}", new_os_thread_priority);
if (0 != setpriority(PRIO_PROCESS, thread_id, new_os_thread_priority))
throwFromErrno("Cannot 'setpriority'", ErrorCodes::CANNOT_SET_THREAD_PRIORITY);
os_thread_priority = new_os_thread_priority;
}
#endif
}
applyQuerySettings();
initPerformanceCounters();

View File

@ -17,7 +17,6 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <common/getFQDNOrHostName.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <Common/SettingsChanges.h>
#include <Disks/StoragePolicy.h>
@ -236,7 +235,8 @@ void HTTPHandler::processQuery(
Poco::Net::HTTPServerRequest & request,
HTMLForm & params,
Poco::Net::HTTPServerResponse & response,
Output & used_output)
Output & used_output,
std::optional<CurrentThread::QueryScope> & query_scope)
{
LOG_TRACE(log, "Request URI: {}", request.getURI());
@ -595,6 +595,8 @@ void HTTPHandler::processQuery(
customizeContext(request, context);
query_scope.emplace(context);
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context,
[&response] (const String & current_query_id, const String & content_type, const String & format, const String & timezone)
{
@ -694,7 +696,8 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
/// Should be initialized before anything,
/// For correct memory accounting.
Context context = server.context();
CurrentThread::QueryScope query_scope(context);
/// Cannot be set here, since query_id is unknown.
std::optional<CurrentThread::QueryScope> query_scope;
Output used_output;
@ -719,7 +722,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED);
}
processQuery(context, request, params, response, used_output);
processQuery(context, request, params, response, used_output, query_scope);
LOG_DEBUG(log, "Done processing query");
}
catch (...)

View File

@ -4,6 +4,7 @@
#include <Poco/Net/HTTPRequestHandler.h>
#include <Common/CurrentThread.h>
#include <Common/CurrentMetrics.h>
#include <Common/HTMLForm.h>
#include <Core/Names.h>
@ -76,7 +77,8 @@ private:
Poco::Net::HTTPServerRequest & request,
HTMLForm & params,
Poco::Net::HTTPServerResponse & response,
Output & used_output);
Output & used_output,
std::optional<CurrentThread::QueryScope> & query_scope);
void trySendExceptionToClient(
const std::string & s,

View File

@ -0,0 +1,2 @@
1
1

View File

@ -0,0 +1,28 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
query="select randomPrintableASCII(number) from numbers(1000)"
# at least 2, one allocation, one deallocation
# (but actually even more)
min_trace_entries=2
# TCP
# do not use _, they should be escaped for LIKE
query_id_tcp_prefix="01526-tcp-memory-tracking-$RANDOM-$$"
${CLICKHOUSE_CLIENT} --log_queries=1 --max_threads=1 --max_untracked_memory=0 --memory_profiler_sample_probability=1 -q "with '$query_id_tcp_prefix' as __id $query FORMAT Null"
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
query_id_tcp="$(${CLICKHOUSE_CLIENT} -q "SELECT DISTINCT query_id FROM system.query_log WHERE query LIKE '%$query_id_tcp_prefix%'")"
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_tcp' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"
# HTTP
# query_id cannot be longer then 28 bytes
query_id_http="01526_http_${RANDOM}_$$"
echo "$query" | ${CLICKHOUSE_CURL} -sSg -o /dev/null "${CLICKHOUSE_URL}&query_id=$query_id_http&max_untracked_memory=0&memory_profiler_sample_probability=1&max_threads=1" -d @-
${CLICKHOUSE_CLIENT} -q "SYSTEM FLUSH LOGS"
# at least 2, one allocation, one deallocation
# (but actually even more)
${CLICKHOUSE_CLIENT} -q "SELECT count()>=$min_trace_entries FROM system.trace_log WHERE query_id = '$query_id_http' AND abs(size) < 4e6 AND event_time >= now() - interval 1 hour"

View File

@ -153,3 +153,4 @@
01515_force_data_skipping_indices
01526_complex_key_dict_direct_layout
01509_dictionary_preallocate
01526_max_untracked_memory

View File

@ -18,6 +18,7 @@
"00157_cache_dictionary",
"01193_metadata_loading",
"01473_event_time_microseconds",
"01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers
"01474_executable_dictionary" /// informational stderr from sanitizer at start
],
"address-sanitizer": [
@ -27,6 +28,7 @@
"odbc_roundtrip",
"01103_check_cpu_instructions_at_startup",
"01473_event_time_microseconds",
"01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers
"01193_metadata_loading"
],
"ub-sanitizer": [
@ -36,6 +38,7 @@
"01103_check_cpu_instructions_at_startup",
"00900_orc_load",
"01473_event_time_microseconds",
"01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers
"01193_metadata_loading"
],
"memory-sanitizer": [
@ -47,6 +50,7 @@
"00877_memory_limit_for_new_delete", /// memory limits don't work correctly under msan because it replaces malloc/free
"01114_mysql_database_engine_segfault", /// it fails in _nss_files_parse_servent while using NSS from GLibc to authenticate (need to get rid of it)
"01473_event_time_microseconds",
"01526_max_untracked_memory", /// requires TraceCollector, does not available under sanitizers
"01193_metadata_loading"
],
"debug-build": [