Merge pull request #2482 from yandex/CLICKHOUSE-2910

Performance introspection
This commit is contained in:
alexey-milovidov 2018-08-19 06:37:55 +03:00 committed by GitHub
commit 89655b5f2d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
141 changed files with 4233 additions and 711 deletions

View File

@ -218,7 +218,7 @@ else ()
set (CLICKHOUSE_ETC_DIR "${CMAKE_INSTALL_PREFIX}/etc")
endif ()
option (UNBUNDLED "Try find all libraries in system (if fail - use bundled from contrib/)" OFF)
option (UNBUNDLED "Try find all libraries in system. We recommend to avoid this mode for production builds, because we cannot guarantee exact versions and variants of libraries your system has installed. This mode exists for enthusiastic developers who search for trouble. Also it is useful for maintainers of OS packages." OFF)
if (UNBUNDLED)
set(NOT_UNBUNDLED 0)
else ()

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54405 CACHE STRING "")
set(VERSION_REVISION 54406 CACHE STRING "")
set(VERSION_MAJOR 18 CACHE STRING "")
set(VERSION_MINOR 10 CACHE STRING "")
set(VERSION_PATCH 3 CACHE STRING "")
set(VERSION_GITHASH 1fa1b34f1ab01ea2e1a833eebd36a4806e529f52 CACHE STRING "")
set(VERSION_DESCRIBE v18.10.3-testing CACHE STRING "")
set(VERSION_STRING 18.10.3 CACHE STRING "")
set(VERSION_MINOR 11 CACHE STRING "")
set(VERSION_PATCH 0 CACHE STRING "")
set(VERSION_GITHASH 76af46ed5d223b3a7af92e31eae291174da16355 CACHE STRING "")
set(VERSION_DESCRIBE v18.11.0-testing CACHE STRING "")
set(VERSION_STRING 18.11.0 CACHE STRING "")
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -39,6 +39,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/InternalTextLogsRowOutputStream.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTUseQuery.h>
@ -235,6 +236,11 @@ private:
std::optional<WriteBufferFromFile> out_file_buf;
BlockOutputStreamPtr block_out_stream;
/// The user could specify special file for server logs (stderr by default)
std::unique_ptr<WriteBuffer> out_logs_buf;
String server_logs_file;
BlockOutputStreamPtr logs_out_stream;
String home_path;
String current_profile;
@ -469,7 +475,12 @@ private:
format_max_block_size = config().getInt("format_max_block_size", context.getSettingsRef().max_block_size);
insert_format = "Values";
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
/// Setting value from cmd arg overrides one from config
if (context.getSettingsRef().max_insert_block_size.changed)
insert_format_max_block_size = context.getSettingsRef().max_insert_block_size;
else
insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size);
if (!is_interactive)
{
@ -990,7 +1001,7 @@ private:
/// If structure was received (thus, server has not thrown an exception),
/// send our data with that structure.
sendData(sample);
receivePacket();
receiveEndOfQuery();
}
}
@ -1072,6 +1083,11 @@ private:
connection->sendData(block);
processed_rows += block.rows();
/// Check if server send Log packet
auto packet_type = connection->checkPacket();
if (packet_type && *packet_type == Protocol::Server::Log)
receiveAndProcessPacket();
if (!block)
break;
}
@ -1083,18 +1099,28 @@ private:
/// Flush all buffers.
void resetOutput()
{
block_out_stream = nullptr;
block_out_stream.reset();
logs_out_stream.reset();
if (pager_cmd)
{
pager_cmd->in.close();
pager_cmd->wait();
}
pager_cmd = nullptr;
if (out_file_buf)
{
out_file_buf->next();
out_file_buf.reset();
}
if (out_logs_buf)
{
out_logs_buf->next();
out_logs_buf.reset();
}
std_out.next();
}
@ -1127,7 +1153,7 @@ private:
continue; /// If there is no new data, continue checking whether the query was cancelled after a timeout.
}
if (!receivePacket())
if (!receiveAndProcessPacket())
break;
}
@ -1138,7 +1164,7 @@ private:
/// Receive a part of the result, or progress info or an exception and process it.
/// Returns true if one should continue receiving packets.
bool receivePacket()
bool receiveAndProcessPacket()
{
Connection::Packet packet = connection->receivePacket();
@ -1169,6 +1195,10 @@ private:
last_exception = std::move(packet.exception);
return false;
case Protocol::Server::Log:
onLogData(packet.block);
return true;
case Protocol::Server::EndOfStream:
onEndOfStream();
return false;
@ -1182,22 +1212,59 @@ private:
/// Receive the block that serves as an example of the structure of table where data will be inserted.
bool receiveSampleBlock(Block & out)
{
Connection::Packet packet = connection->receivePacket();
switch (packet.type)
while (true)
{
case Protocol::Server::Data:
out = packet.block;
return true;
Connection::Packet packet = connection->receivePacket();
case Protocol::Server::Exception:
onException(*packet.exception);
last_exception = std::move(packet.exception);
return false;
switch (packet.type)
{
case Protocol::Server::Data:
out = packet.block;
return true;
default:
throw NetException("Unexpected packet from server (expected Data, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
case Protocol::Server::Exception:
onException(*packet.exception);
last_exception = std::move(packet.exception);
return false;
case Protocol::Server::Log:
onLogData(packet.block);
break;
default:
throw NetException("Unexpected packet from server (expected Data, Exception or Log, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
}
}
/// Process Log packets, exit when recieve Exception or EndOfStream
bool receiveEndOfQuery()
{
while (true)
{
Connection::Packet packet = connection->receivePacket();
switch (packet.type)
{
case Protocol::Server::EndOfStream:
onEndOfStream();
return true;
case Protocol::Server::Exception:
onException(*packet.exception);
last_exception = std::move(packet.exception);
return false;
case Protocol::Server::Log:
onLogData(packet.block);
break;
default:
throw NetException("Unexpected packet from server (expected Exception, EndOfStream or Log, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
}
}
@ -1253,6 +1320,38 @@ private:
}
void initLogsOutputStream()
{
if (!logs_out_stream)
{
WriteBuffer * wb = out_logs_buf.get();
if (!out_logs_buf)
{
if (server_logs_file.empty())
{
/// Use stderr by default
out_logs_buf = std::make_unique<WriteBufferFromFileDescriptor>(STDERR_FILENO);
wb = out_logs_buf.get();
}
else if (server_logs_file == "-")
{
/// Use stdout if --server_logs_file=- specified
wb = &std_out;
}
else
{
out_logs_buf = std::make_unique<WriteBufferFromFile>(server_logs_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT);
wb = out_logs_buf.get();
}
}
logs_out_stream = std::make_shared<InternalTextLogsRowOutputStream>(*wb);
logs_out_stream->writePrefix();
}
}
void onData(Block & block)
{
if (written_progress_chars)
@ -1276,6 +1375,14 @@ private:
}
void onLogData(Block & block)
{
initLogsOutputStream();
logs_out_stream->write(block);
logs_out_stream->flush();
}
void onTotals(Block & block)
{
initBlockOutputStream(block);
@ -1436,6 +1543,9 @@ private:
if (block_out_stream)
block_out_stream->writeSuffix();
if (logs_out_stream)
logs_out_stream->writeSuffix();
resetOutput();
if (is_interactive && !written_first_block)
@ -1511,7 +1621,9 @@ public:
ioctl(0, TIOCGWINSZ, &terminal_size);
unsigned line_length = boost::program_options::options_description::m_default_line_length;
namespace po = boost::program_options;
unsigned line_length = po::options_description::m_default_line_length;
unsigned min_description_length = line_length / 2;
if (!stdin_is_not_tty)
{
@ -1519,28 +1631,28 @@ public:
min_description_length = std::min(min_description_length, line_length - 2);
}
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value<std::string> (), DESCRIPTION)
#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value<std::string> (), DESCRIPTION)
/// Main commandline options related to client functionality and all parameters from Settings.
boost::program_options::options_description main_description("Main options", line_length, min_description_length);
po::options_description main_description("Main options", line_length, min_description_length);
main_description.add_options()
("help", "produce help message")
("config-file,c", boost::program_options::value<std::string>(), "config-file path")
("host,h", boost::program_options::value<std::string>()->default_value("localhost"), "server host")
("port", boost::program_options::value<int>()->default_value(9000), "server port")
("config-file,c", po::value<std::string>(), "config-file path")
("host,h", po::value<std::string>()->default_value("localhost"), "server host")
("port", po::value<int>()->default_value(9000), "server port")
("secure,s", "secure")
("user,u", boost::program_options::value<std::string>()->default_value("default"), "user")
("password", boost::program_options::value<std::string>(), "password")
("user,u", po::value<std::string>()->default_value("default"), "user")
("password", po::value<std::string>(), "password")
("ask-password", "ask-password")
("query_id", boost::program_options::value<std::string>(), "query_id")
("query,q", boost::program_options::value<std::string>(), "query")
("database,d", boost::program_options::value<std::string>(), "database")
("pager", boost::program_options::value<std::string>(), "pager")
("query_id", po::value<std::string>(), "query_id")
("query,q", po::value<std::string>(), "query")
("database,d", po::value<std::string>(), "database")
("pager", po::value<std::string>(), "pager")
("multiline,m", "multiline")
("multiquery,n", "multiquery")
("format,f", po::value<std::string>(), "default output format")
("testmode,T", "enable test hints in comments")
("ignore-error", "do not stop processing in multiquery mode")
("format,f", boost::program_options::value<std::string>(), "default output format")
("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command")
("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)")
("stacktrace", "print stack traces of exceptions")
@ -1548,27 +1660,29 @@ public:
("version,V", "print version information and exit")
("version-clean", "print version in machine-readable format and exit")
("echo", "in batch mode, print query before execution")
("max_client_network_bandwidth", boost::program_options::value<int>(), "the maximum speed of data exchange over the network for the client in bytes per second.")
("compression", boost::program_options::value<bool>(), "enable or disable compression")
("max_client_network_bandwidth", po::value<int>(), "the maximum speed of data exchange over the network for the client in bytes per second.")
("compression", po::value<bool>(), "enable or disable compression")
("log-level", po::value<std::string>(), "client log level")
("server_logs_file", po::value<std::string>(), "put server logs into specified file")
APPLY_FOR_SETTINGS(DECLARE_SETTING)
;
#undef DECLARE_SETTING
/// Commandline options related to external tables.
boost::program_options::options_description external_description("External tables options");
po::options_description external_description("External tables options");
external_description.add_options()
("file", boost::program_options::value<std::string>(), "data file or - for stdin")
("name", boost::program_options::value<std::string>()->default_value("_data"), "name of the table")
("format", boost::program_options::value<std::string>()->default_value("TabSeparated"), "data format")
("structure", boost::program_options::value<std::string>(), "structure")
("types", boost::program_options::value<std::string>(), "types")
("file", po::value<std::string>(), "data file or - for stdin")
("name", po::value<std::string>()->default_value("_data"), "name of the table")
("format", po::value<std::string>()->default_value("TabSeparated"), "data format")
("structure", po::value<std::string>(), "structure")
("types", po::value<std::string>(), "types")
;
/// Parse main commandline options.
boost::program_options::parsed_options parsed = boost::program_options::command_line_parser(
po::parsed_options parsed = po::command_line_parser(
common_arguments.size(), common_arguments.data()).options(main_description).run();
boost::program_options::variables_map options;
boost::program_options::store(parsed, options);
po::variables_map options;
po::store(parsed, options);
if (options.count("version") || options.count("V"))
{
@ -1591,14 +1705,17 @@ public:
exit(0);
}
if (options.count("log-level"))
Poco::Logger::root().setLevel(options["log-level"].as<std::string>());
size_t number_of_external_tables_with_stdin_source = 0;
for (size_t i = 0; i < external_tables_arguments.size(); ++i)
{
/// Parse commandline options related to external tables.
boost::program_options::parsed_options parsed = boost::program_options::command_line_parser(
po::parsed_options parsed = po::command_line_parser(
external_tables_arguments[i].size(), external_tables_arguments[i].data()).options(external_description).run();
boost::program_options::variables_map external_options;
boost::program_options::store(parsed, external_options);
po::variables_map external_options;
po::store(parsed, external_options);
try
{
@ -1672,6 +1789,8 @@ public:
max_client_network_bandwidth = options["max_client_network_bandwidth"].as<int>();
if (options.count("compression"))
config().setBool("compression", options["compression"].as<bool>());
if (options.count("server_logs_file"))
server_logs_file = options["server_logs_file"].as<std::string>();
}
};
@ -1691,6 +1810,11 @@ int mainEntryClickHouseClient(int argc, char ** argv)
std::cerr << "Bad arguments: " << e.what() << std::endl;
return 1;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << std::endl;
return 1;
}
return client.run();
}

View File

@ -30,6 +30,7 @@
#include <Common/ClickHouseRevision.h>
#include <Common/formatReadable.h>
#include <Common/DNSResolver.h>
#include <Common/CurrentThread.h>
#include <Common/escapeForFileName.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Client/Connection.h>
@ -2143,6 +2144,9 @@ void ClusterCopierApp::mainImpl()
context->addDatabase(default_database, std::make_shared<DatabaseMemory>(default_database));
context->setCurrentDatabase(default_database);
/// Initialize query scope just in case.
CurrentThread::QueryScope query_scope(*context);
auto copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, *context);
copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability);

View File

@ -25,6 +25,7 @@
#include <Parsers/IAST.h>
#include <common/ErrorHandlers.h>
#include <Common/StatusFile.h>
#include <Common/ThreadStatus.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <TableFunctions/registerTableFunctions.h>
@ -270,6 +271,9 @@ void LocalServer::processQueries()
context->setCurrentQueryId("");
applyCmdSettings(*context);
/// Use the same query_id (and thread group) for all queries
CurrentThread::QueryScope query_scope_holder(*context);
bool echo_query = config().hasOption("echo") || config().hasOption("verbose");
std::exception_ptr exception;

View File

@ -13,6 +13,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
#include <Common/getFQDNOrHostName.h>
#include <Common/CurrentThread.h>
#include <IO/ReadBufferFromIStream.h>
#include <IO/ZlibInflatingReadBuffer.h>
#include <IO/ReadBufferFromString.h>
@ -208,6 +209,13 @@ void HTTPHandler::processQuery(
Poco::Net::HTTPServerResponse & response,
Output & used_output)
{
Context context = server.context();
context.setGlobalContext(server.context());
/// It will forcibly detach query even if unexpected error ocurred and detachQuery() was not called
/// Normal detaching is happen in BlockIO callbacks
CurrentThread::QueryScope query_scope_holder(context);
LOG_TRACE(log, "Request URI: " << request.getURI());
std::istream & istr = request.stream();
@ -257,14 +265,9 @@ void HTTPHandler::processQuery(
}
std::string query_id = params.get("query_id", "");
const auto & config = server.config();
Context context = server.context();
context.setGlobalContext(server.context());
context.setUser(user, password, request.clientAddress(), quota_key);
context.setCurrentQueryId(query_id);
CurrentThread::attachQueryContext(context);
/// The user could specify session identifier and session timeout.
/// It allows to modify settings, create temporary tables and reuse them in subsequent requests.
@ -273,6 +276,7 @@ void HTTPHandler::processQuery(
String session_id;
std::chrono::steady_clock::duration session_timeout;
bool session_is_set = params.has("session_id");
const auto & config = server.config();
if (session_is_set)
{
@ -421,17 +425,43 @@ void HTTPHandler::processQuery(
std::unique_ptr<ReadBuffer> in;
// Used in case of POST request with form-data, but it not to be expectd to be deleted after that scope
static const NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace",
"buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"};
Names reserved_param_suffixes;
auto param_could_be_skipped = [&] (const String & name)
{
if (reserved_param_names.count(name))
return true;
for (const String & suffix : reserved_param_suffixes)
{
if (endsWith(name, suffix))
return true;
}
return false;
};
/// Used in case of POST request with form-data, but it not to be expectd to be deleted after that scope
std::string full_query;
/// Support for "external data for query processing".
if (startsWith(request.getContentType().data(), "multipart/form-data"))
{
ExternalTablesHandler handler(context, params);
context.setExternalTablesInitializer([&params, &request, &istr] (Context & context_query)
{
ExternalTablesHandler handler(context_query, params);
params.load(request, istr, handler);
});
/// Skip unneeded parameters to avoid confusing them later with context settings or query parameters.
reserved_param_suffixes.emplace_back("_format");
reserved_param_suffixes.emplace_back("_types");
reserved_param_suffixes.emplace_back("_structure");
/// Params are of both form params POST and uri (GET params)
params.load(request, istr, handler);
for (const auto & it : params)
{
if (it.first == "query")
@ -440,15 +470,6 @@ void HTTPHandler::processQuery(
}
}
in = std::make_unique<ReadBufferFromString>(full_query);
/// Erase unneeded parameters to avoid confusing them later with context settings or query
/// parameters.
for (const auto & it : handler.names)
{
params.erase(it + "_format");
params.erase(it + "_types");
params.erase(it + "_structure");
}
}
else
in = std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed);
@ -475,11 +496,6 @@ void HTTPHandler::processQuery(
auto readonly_before_query = settings.readonly;
NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace",
"buffer_size", "wait_end_of_query",
"session_id", "session_timeout", "session_check"
};
for (auto it = params.begin(); it != params.end(); ++it)
{
if (it->first == "database")
@ -490,7 +506,7 @@ void HTTPHandler::processQuery(
{
context.setDefaultFormat(it->second);
}
else if (reserved_param_names.find(it->first) != reserved_param_names.end())
else if (param_could_be_skipped(it->first))
{
}
else

View File

@ -81,7 +81,7 @@ void MetricsTransmitter::transmit(std::vector<ProfileEvents::Count> & prev_count
{
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
{
const auto counter = ProfileEvents::counters[i].load(std::memory_order_relaxed);
const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed);
const auto counter_increment = counter - prev_counters[i];
prev_counters[i] = counter;

View File

@ -22,6 +22,7 @@
#include <Common/getFQDNOrHostName.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/TaskStatsInfoGetter.h>
#include <IO/HTTPCommon.h>
#include <Interpreters/AsynchronousMetrics.h>
#include <Interpreters/DDLWorker.h>
@ -365,6 +366,13 @@ int Server::main(const std::vector<std::string> & /*args*/)
dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context);
}
if (!TaskStatsInfoGetter::checkProcessHasRequiredPermissions())
{
LOG_INFO(log, "It looks like the process has not CAP_NET_ADMIN capability, some performance statistics will be disabled."
" It could happen due to incorrect clickhouse package installation."
" You could resolve the problem manually calling 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'");
}
{
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);

View File

@ -1,7 +1,11 @@
#include "TCPHandler.h"
#include <iomanip>
#include <ext/scope_guard.h>
#include <Poco/Net/NetException.h>
#include <daemon/OwnSplitChannel.h>
#include <Common/ClickHouseRevision.h>
#include <Common/CurrentThread.h>
#include <Common/Stopwatch.h>
#include <IO/Progress.h>
#include <IO/CompressedReadBuffer.h>
#include <IO/CompressedWriteBuffer.h>
@ -15,6 +19,7 @@
#include <Interpreters/executeQuery.h>
#include <Interpreters/Quota.h>
#include <Interpreters/TablesStatus.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Storages/StorageMemory.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Common/ClickHouseRevision.h>
@ -22,7 +27,9 @@
#include <Common/ExternalTable.h>
#include <Common/NetException.h>
#include <Common/config_version.h>
#include <ext/scope_guard.h>
#include "TCPHandler.h"
namespace DB
{
@ -140,13 +147,29 @@ void TCPHandler::runImpl()
if (!receivePacket())
continue;
/// Get blocks of temporary tables
readData(global_settings);
CurrentThread::initializeQuery();
/// Reset the input stream, as we received an empty block while receiving external table data.
/// So, the stream has been marked as cancelled and we can't read from it anymore.
state.block_in.reset();
state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker.
/// Should we send internal logs to client?
if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS
&& query_context.getSettingsRef().send_logs_level.value != "none")
{
state.logs_queue = std::make_shared<InternalTextLogsQueue>();
state.logs_queue->max_priority = Poco::Logger::parseLevel(query_context.getSettingsRef().send_logs_level.value);
CurrentThread::attachInternalTextLogsQueue(state.logs_queue);
}
query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) {
if (&context != &query_context)
throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR);
/// Get blocks of temporary tables
readData(global_settings);
/// Reset the input stream, as we received an empty block while receiving external table data.
/// So, the stream has been marked as cancelled and we can't read from it anymore.
state.block_in.reset();
state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker.
});
/// Processing Query
state.io = executeQuery(state.query, query_context, false, state.stage);
@ -163,8 +186,11 @@ void TCPHandler::runImpl()
else
processOrdinaryQuery();
sendEndOfStream();
/// Reset BlockIO in advance to log destruction actions
state.io.reset();
sendLogs();
sendEndOfStream();
state.reset();
}
catch (const Exception & e)
@ -209,7 +235,20 @@ void TCPHandler::runImpl()
try
{
if (exception)
{
try
{
/// Try to send logs to client, but it could be risky too
/// Assume that we can't break output here
sendLogs();
}
catch (...)
{
tryLogCurrentException(log, "Can't send logs to client");
}
sendException(*exception);
}
}
catch (...)
{
@ -220,6 +259,9 @@ void TCPHandler::runImpl()
try
{
/// It will forcibly detach query even if unexpected error ocсurred and detachQuery() was not called
CurrentThread::detachQueryIfNotDetached();
state.reset();
}
catch (...)
@ -252,12 +294,14 @@ void TCPHandler::readData(const Settings & global_settings)
constexpr size_t min_poll_interval = 5000; // 5 ms
size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval));
while (1)
sendLogs();
while (true)
{
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
/// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down.
while (1)
while (true)
{
if (static_cast<ReadBufferFromPocoSocket &>(*in).poll(poll_interval))
break;
@ -289,6 +333,8 @@ void TCPHandler::readData(const Settings & global_settings)
/// We accept and process data. And if they are over, then we leave.
if (!receivePacket())
break;
sendLogs();
}
}
@ -346,6 +392,8 @@ void TCPHandler::processOrdinaryQuery()
sendProgress();
}
sendLogs();
if (async_in.poll(query_context.getSettingsRef().interactive_delay / 1000))
{
/// There is the following result block.
@ -368,6 +416,7 @@ void TCPHandler::processOrdinaryQuery()
sendExtremes();
sendProfileInfo();
sendProgress();
sendLogs();
}
sendData(block);
@ -692,11 +741,14 @@ void TCPHandler::initBlockOutput(const Block & block)
{
if (!state.block_out)
{
if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
*out, CompressionSettings(query_context.getSettingsRef()));
else
state.maybe_compressed_out = out;
if (!state.maybe_compressed_out)
{
if (state.compression == Protocol::Compression::Enable)
state.maybe_compressed_out = std::make_shared<CompressedWriteBuffer>(
*out, CompressionSettings(query_context.getSettingsRef()));
else
state.maybe_compressed_out = out;
}
state.block_out = std::make_shared<NativeBlockOutputStream>(
*state.maybe_compressed_out,
@ -705,6 +757,18 @@ void TCPHandler::initBlockOutput(const Block & block)
}
}
void TCPHandler::initLogsBlockOutput(const Block & block)
{
if (!state.logs_block_out)
{
/// Use uncompressed stream since log blocks usually contain only one row
state.logs_block_out = std::make_shared<NativeBlockOutputStream>(
*out,
client_revision,
block.cloneEmpty());
}
}
bool TCPHandler::isQueryCancelled()
{
@ -745,6 +809,7 @@ void TCPHandler::sendData(const Block & block)
initBlockOutput(block);
writeVarUInt(Protocol::Server::Data, *out);
/// Send external table name (empty name is the main table)
writeStringBinary("", *out);
state.block_out->write(block);
@ -753,6 +818,19 @@ void TCPHandler::sendData(const Block & block)
}
void TCPHandler::sendLogData(const Block & block)
{
initLogsBlockOutput(block);
writeVarUInt(Protocol::Server::Log, *out);
/// Send log tag (empty tag is the default tag)
writeStringBinary("", *out);
state.logs_block_out->write(block);
out->next();
}
void TCPHandler::sendException(const Exception & e)
{
writeVarUInt(Protocol::Server::Exception, *out);
@ -784,6 +862,37 @@ void TCPHandler::sendProgress()
}
void TCPHandler::sendLogs()
{
if (!state.logs_queue)
return;
MutableColumns logs_columns;
MutableColumns curr_logs_columns;
size_t rows = 0;
for (; state.logs_queue->tryPop(curr_logs_columns); ++rows)
{
if (rows == 0)
{
logs_columns = std::move(curr_logs_columns);
}
else
{
for (size_t j = 0; j < logs_columns.size(); ++j)
logs_columns[j]->insertRangeFrom(*curr_logs_columns[j], 0, curr_logs_columns[j]->size());
}
}
if (rows > 0)
{
Block block = InternalTextLogsQueue::getSampleBlock();
block.setColumns(std::move(logs_columns));
sendLogData(block);
}
}
void TCPHandler::run()
{
try

View File

@ -5,16 +5,18 @@
#include <Common/getFQDNOrHostName.h>
#include <Common/CurrentMetrics.h>
#include <Common/Stopwatch.h>
#include <IO/Progress.h>
#include <Core/Protocol.h>
#include <Core/QueryProcessingStage.h>
#include <DataStreams/BlockIO.h>
#include <IO/Progress.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/BlockIO.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Client/TimeoutSetter.h>
#include "IServer.h"
namespace CurrentMetrics
{
extern const Metric TCPConnection;
@ -63,6 +65,9 @@ struct QueryState
/// Timeouts setter for current query
std::unique_ptr<TimeoutSetter> timeout_setter;
/// A queue with internal logs that will be passed to client
InternalTextLogsQueuePtr logs_queue;
BlockOutputStreamPtr logs_block_out;
void reset()
{
@ -140,8 +145,10 @@ private:
void sendHello();
void sendData(const Block & block); /// Write a block to the network.
void sendLogData(const Block & block);
void sendException(const Exception & e);
void sendProgress();
void sendLogs();
void sendEndOfStream();
void sendProfileInfo();
void sendTotals();
@ -150,6 +157,7 @@ private:
/// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled.
void initBlockInput();
void initBlockOutput(const Block & block);
void initLogsBlockOutput(const Block & block);
bool isQueryCancelled();

View File

@ -114,6 +114,7 @@ void Connection::disconnect()
//LOG_TRACE(log_wrapper.get(), "Disconnecting");
in = nullptr;
last_input_packet_type.reset();
out = nullptr; // can write to socket
if (socket)
socket->close();
@ -379,6 +380,7 @@ void Connection::sendQuery(
maybe_compressed_in.reset();
maybe_compressed_out.reset();
block_in.reset();
block_logs_in.reset();
block_out.reset();
/// Send empty block which means end of data.
@ -506,20 +508,50 @@ bool Connection::poll(size_t timeout_microseconds)
}
bool Connection::hasReadBufferPendingData() const
bool Connection::hasReadPendingData() const
{
return static_cast<const ReadBufferFromPocoSocket &>(*in).hasPendingData();
return last_input_packet_type.has_value() || static_cast<const ReadBufferFromPocoSocket &>(*in).hasPendingData();
}
std::optional<UInt64> Connection::checkPacket(size_t timeout_microseconds)
{
if (last_input_packet_type.has_value())
return last_input_packet_type;
if (hasReadPendingData() || poll(timeout_microseconds))
{
// LOG_TRACE(log_wrapper.get(), "Receiving packet type");
UInt64 packet_type;
readVarUInt(packet_type, *in);
last_input_packet_type.emplace(packet_type);
return last_input_packet_type;
}
return {};
}
Connection::Packet Connection::receivePacket()
{
//LOG_TRACE(log_wrapper.get(), "Receiving packet");
try
{
Packet res;
readVarUInt(res.type, *in);
/// Have we already read packet type?
if (last_input_packet_type)
{
res.type = *last_input_packet_type;
last_input_packet_type.reset();
}
else
{
//LOG_TRACE(log_wrapper.get(), "Receiving packet type");
readVarUInt(res.type, *in);
}
//LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type));
switch (res.type)
{
@ -549,6 +581,10 @@ Connection::Packet Connection::receivePacket()
res.block = receiveData();
return res;
case Protocol::Server::Log:
res.block = receiveLogData();
return res;
case Protocol::Server::EndOfStream:
return res;
@ -576,14 +612,26 @@ Block Connection::receiveData()
//LOG_TRACE(log_wrapper.get(), "Receiving data");
initBlockInput();
return receiveDataImpl(block_in);
}
Block Connection::receiveLogData()
{
initBlockLogsInput();
return receiveDataImpl(block_logs_in);
}
Block Connection::receiveDataImpl(BlockInputStreamPtr & stream)
{
String external_table_name;
readStringBinary(external_table_name, *in);
size_t prev_bytes = in->count();
/// Read one block from network.
Block res = block_in->read();
Block res = stream->read();
if (throttler)
throttler->add(in->count() - prev_bytes);
@ -592,20 +640,39 @@ Block Connection::receiveData()
}
void Connection::initInputBuffers()
{
}
void Connection::initBlockInput()
{
if (!block_in)
{
if (compression == Protocol::Compression::Enable)
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
else
maybe_compressed_in = in;
if (!maybe_compressed_in)
{
if (compression == Protocol::Compression::Enable)
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
else
maybe_compressed_in = in;
}
block_in = std::make_shared<NativeBlockInputStream>(*maybe_compressed_in, server_revision);
}
}
void Connection::initBlockLogsInput()
{
if (!block_logs_in)
{
/// Have to return superset of SystemLogsQueue::getSampleBlock() columns
block_logs_in = std::make_shared<NativeBlockInputStream>(*in, server_revision);
}
}
void Connection::setDescription()
{
auto resolved_address = getResolvedAddress();

View File

@ -23,6 +23,7 @@
#include <Interpreters/TablesStatus.h>
#include <atomic>
#include <optional>
namespace DB
@ -138,7 +139,10 @@ public:
bool poll(size_t timeout_microseconds = 0);
/// Check, if has data in read buffer.
bool hasReadBufferPendingData() const;
bool hasReadPendingData() const;
/// Checks if there is input data in connection and reads packet ID.
std::optional<UInt64> checkPacket(size_t timeout_microseconds = 0);
/// Receive packet from server.
Packet receivePacket();
@ -195,6 +199,7 @@ private:
std::unique_ptr<Poco::Net::StreamSocket> socket;
std::shared_ptr<ReadBuffer> in;
std::shared_ptr<WriteBuffer> out;
std::optional<UInt64> last_input_packet_type;
String query_id;
Protocol::Compression compression; /// Enable data compression for communication.
@ -214,6 +219,7 @@ private:
/// From where to read query execution result.
std::shared_ptr<ReadBuffer> maybe_compressed_in;
BlockInputStreamPtr block_in;
BlockInputStreamPtr block_logs_in;
/// Where to write data for INSERT.
std::shared_ptr<WriteBuffer> maybe_compressed_out;
@ -249,11 +255,16 @@ private:
bool ping();
Block receiveData();
Block receiveLogData();
Block receiveDataImpl(BlockInputStreamPtr & stream);
std::unique_ptr<Exception> receiveException();
Progress receiveProgress();
BlockStreamProfileInfo receiveProfileInfo();
void initInputBuffers();
void initBlockInput();
void initBlockLogsInput();
void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const;
};

View File

@ -247,6 +247,7 @@ Connection::Packet MultiplexedConnections::receivePacketUnlocked()
case Protocol::Server::ProfileInfo:
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::Log:
break;
case Protocol::Server::EndOfStream:
@ -276,7 +277,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
for (const ReplicaState & state : replica_states)
{
Connection * connection = state.connection;
if ((connection != nullptr) && connection->hasReadBufferPendingData())
if ((connection != nullptr) && connection->hasReadPendingData())
read_list.push_back(*connection->socket);
}

View File

@ -1,14 +1,15 @@
#pragma once
#include <atomic>
#include <memory>
#include <Common/ActionLock.h>
namespace DB
{
/// An atomic variable that is used to block and interrupt certain actions
/// If it is not zero then actions related with it should be considered as interrupted
/// An atomic variable that is used to block and interrupt certain actions.
/// If it is not zero then actions related with it should be considered as interrupted.
/// Uses shared_ptr and the lock uses weak_ptr to be able to "hold" a lock when an object with blocker has already died.
class ActionBlocker
{
public:
@ -33,4 +34,5 @@ private:
CounterPtr counter;
};
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <memory>
#include <atomic>
#include <Core/Types.h>
namespace DB
{

View File

@ -4,8 +4,11 @@
#include <Common/Exception.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/CurrentThread.h>
#include <common/logger_useful.h>
#include <chrono>
#include <ext/scope_guard.h>
namespace CurrentMetrics
{
@ -140,6 +143,12 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size)
{
LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Create BackgroundSchedulePool with " << size << " threads");
/// Put all threads of both thread pools to one thread group
/// The master thread exits immediately
CurrentThread::initializeQuery();
thread_group = CurrentThread::getGroup();
CurrentThread::detachQuery();
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
@ -212,9 +221,11 @@ void BackgroundSchedulePool::threadFunction()
{
setThreadName("BackgrSchedPool");
MemoryTracker memory_tracker;
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
current_memory_tracker = &memory_tracker;
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
while (!shutdown)
{
@ -224,8 +235,6 @@ void BackgroundSchedulePool::threadFunction()
task_notification.execute();
}
}
current_memory_tracker = nullptr;
}
@ -233,6 +242,10 @@ void BackgroundSchedulePool::delayExecutionThreadFunction()
{
setThreadName("BckSchPoolDelay");
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
while (!shutdown)
{
TaskInfoPtr task;

View File

@ -12,6 +12,7 @@
#include <functional>
#include <boost/noncopyable.hpp>
#include <Common/ZooKeeper/Types.h>
#include <Common/CurrentThread.h>
namespace DB
{
@ -138,6 +139,9 @@ private:
std::thread delayed_thread;
/// Tasks ordered by scheduled time.
DelayedTasks delayed_tasks;
/// Thread group used for profiling purposes
ThreadGroupStatusPtr thread_group;
};
using BackgroundSchedulePoolPtr = std::shared_ptr<BackgroundSchedulePool>;

View File

@ -0,0 +1,80 @@
#include "CurrentThread.h"
#include <common/logger_useful.h>
#include <Common/ThreadStatus.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/Context.h>
#include <Poco/Ext/ThreadNumber.h>
#include <Poco/Logger.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void CurrentThread::updatePerformanceCounters()
{
get()->updatePerformanceCounters();
}
ThreadStatusPtr CurrentThread::get()
{
#ifndef NDEBUG
if (!current_thread || current_thread.use_count() <= 0)
throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR);
if (Poco::ThreadNumber::get() != current_thread->thread_number)
throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR);
#endif
return current_thread;
}
ProfileEvents::Counters & CurrentThread::getProfileEvents()
{
return current_thread->performance_counters;
}
MemoryTracker & CurrentThread::getMemoryTracker()
{
return current_thread->memory_tracker;
}
void CurrentThread::updateProgressIn(const Progress & value)
{
current_thread->progress_in.incrementPiecewiseAtomically(value);
}
void CurrentThread::updateProgressOut(const Progress & value)
{
current_thread->progress_out.incrementPiecewiseAtomically(value);
}
void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr<InternalTextLogsQueue> & logs_queue)
{
get()->attachInternalTextLogsQueue(logs_queue);
}
std::shared_ptr<InternalTextLogsQueue> CurrentThread::getInternalTextLogsQueue()
{
/// NOTE: this method could be called at early server startup stage
/// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check just in case
if (!current_thread || current_thread.use_count() <= 0)
return nullptr;
if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died)
return nullptr;
return current_thread->getInternalTextLogsQueue();
}
ThreadGroupStatusPtr CurrentThread::getGroup()
{
return get()->getThreadGroup();
}
}

View File

@ -0,0 +1,83 @@
#pragma once
#include <memory>
#include <string>
namespace ProfileEvents
{
class Counters;
}
class MemoryTracker;
namespace DB
{
class Context;
class QueryStatus;
class ThreadStatus;
struct Progress;
using ThreadStatusPtr = std::shared_ptr<ThreadStatus>;
class InternalTextLogsQueue;
class ThreadGroupStatus;
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
class CurrentThread
{
public:
/// Handler to current thread
static ThreadStatusPtr get();
/// Group to which belongs current thread
static ThreadGroupStatusPtr getGroup();
/// A logs queue used by TCPHandler to pass logs to a client
static void attachInternalTextLogsQueue(const std::shared_ptr<InternalTextLogsQueue> & logs_queue);
static std::shared_ptr<InternalTextLogsQueue> getInternalTextLogsQueue();
/// Makes system calls to update ProfileEvents that contain info from rusage and taskstats
static void updatePerformanceCounters();
static ProfileEvents::Counters & getProfileEvents();
static MemoryTracker & getMemoryTracker();
/// Update read and write rows (bytes) statistics (used in system.query_thread_log)
static void updateProgressIn(const Progress & value);
static void updateProgressOut(const Progress & value);
/// Query management:
/// Call from master thread as soon as possible (e.g. when thread accepted connection)
static void initializeQuery();
/// Sets query_context for current thread group
static void attachQueryContext(Context & query_context);
/// You must call one of these methods when create a query child thread:
/// Add current thread to a group associated with the thread group
static void attachTo(const ThreadGroupStatusPtr & thread_group);
/// Is useful for a ThreadPool tasks
static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group);
/// Update ProfileEvents and dumps info to system.query_thread_log
static void finalizePerformanceCounters();
/// Returns a non-empty string if the thread is attached to a query
static std::string getCurrentQueryID();
/// Non-master threads call this method in destructor automatically
static void detachQuery();
static void detachQueryIfNotDetached();
/// Initializes query with current thread as master thread in constructor, and detaches it in desstructor
struct QueryScope
{
explicit QueryScope(Context & query_context);
~QueryScope();
};
};
}

View File

@ -7,6 +7,7 @@
#include <Poco/NumberParser.h>
#include <arpa/inet.h>
#include <atomic>
#include <optional>
namespace DB
@ -79,6 +80,10 @@ struct DNSResolver::Impl
{
SimpleCache<decltype(resolveIPAddressImpl), &resolveIPAddressImpl> cache_host;
/// Cached server host name
std::mutex mutex;
std::optional<String> host_name;
/// If disabled, will not make cache lookups, will resolve addresses manually on each call
std::atomic<bool> disable_cache{false};
};
@ -108,6 +113,9 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U
void DNSResolver::dropCache()
{
impl->cache_host.drop();
std::unique_lock lock(impl->mutex);
impl->host_name.reset();
}
void DNSResolver::setDisableCacheFlag(bool is_disabled)
@ -115,6 +123,19 @@ void DNSResolver::setDisableCacheFlag(bool is_disabled)
impl->disable_cache = is_disabled;
}
String DNSResolver::getHostName()
{
if (impl->disable_cache)
return Poco::Net::DNS::hostName();
std::unique_lock lock(impl->mutex);
if (!impl->host_name.has_value())
impl->host_name.emplace(Poco::Net::DNS::hostName());
return *impl->host_name;
}
DNSResolver::~DNSResolver() = default;

View File

@ -25,6 +25,9 @@ public:
Poco::Net::SocketAddress resolveAddress(const std::string & host, UInt16 port);
/// Get this server host name
String getHostName();
/// Disables caching
void setDisableCacheFlag(bool is_disabled = true);

View File

@ -385,6 +385,8 @@ namespace ErrorCodes
extern const int BAD_REQUEST_PARAMETER = 408;
extern const int EXTERNAL_EXECUTABLE_NOT_FOUND = 409;
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING = 410;
extern const int PTHREAD_ERROR = 411;
extern const int NETLINK_ERROR = 412;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -50,9 +50,8 @@ public:
{
initReadBuffer();
initSampleBlock();
ExternalTableData res = std::make_pair(std::make_shared<AsynchronousBlockInputStream>(context.getInputFormat(
format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE)), name);
return res;
auto input = context.getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE);
return std::make_pair(std::make_shared<AsynchronousBlockInputStream>(input), name);
}
protected:
@ -169,9 +168,8 @@ public:
class ExternalTablesHandler : public Poco::Net::PartHandler, BaseExternalTable
{
public:
std::vector<std::string> names;
ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { }
ExternalTablesHandler(Context & context_, const Poco::Net::NameValueCollection & params_) : context(context_), params(params_) { }
void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream)
{
@ -211,14 +209,13 @@ public:
data.first->readSuffix();
output->writeSuffix();
names.push_back(name);
/// We are ready to receive the next file, for this we clear all the information received
clean();
}
private:
Context & context;
Poco::Net::NameValueCollection params;
const Poco::Net::NameValueCollection & params;
};

View File

@ -1,11 +1,10 @@
#include "MemoryTracker.h"
#include <common/likely.h>
#include <common/logger_useful.h>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <iomanip>
#include <Common/MemoryTracker.h>
namespace DB
@ -19,7 +18,7 @@ namespace DB
MemoryTracker::~MemoryTracker()
{
if (peak)
if (static_cast<int>(level) < static_cast<int>(VariableContext::Process) && peak)
{
try
{
@ -56,13 +55,16 @@ void MemoryTracker::logPeakMemoryUsage() const
void MemoryTracker::alloc(Int64 size)
{
if (blocker.isCancelled())
return;
/** Using memory_order_relaxed means that if allocations are done simultaneously,
* we allow exception about memory limit exceeded to be thrown only on next allocation.
* So, we allow over-allocations.
*/
Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed);
if (!next.load(std::memory_order_relaxed))
if (!parent.load(std::memory_order_relaxed))
CurrentMetrics::add(metric, size);
Int64 current_limit = limit.load(std::memory_order_relaxed);
@ -102,45 +104,62 @@ void MemoryTracker::alloc(Int64 size)
if (will_be > peak.load(std::memory_order_relaxed)) /// Races doesn't matter. Could rewrite with CAS, but not worth.
peak.store(will_be, std::memory_order_relaxed);
if (auto loaded_next = next.load(std::memory_order_relaxed))
if (auto loaded_next = parent.load(std::memory_order_relaxed))
loaded_next->alloc(size);
}
void MemoryTracker::free(Int64 size)
{
Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size;
if (blocker.isCancelled())
return;
/** Sometimes, query could free some data, that was allocated outside of query context.
* Example: cache eviction.
* To avoid negative memory usage, we "saturate" amount.
* Memory usage will be calculated with some error.
* NOTE The code is not atomic. Not worth to fix.
*/
if (new_amount < 0)
if (level == VariableContext::Thread)
{
amount.fetch_sub(new_amount);
size += new_amount;
/// Could become negative if memory allocated in this thread is freed in another one
amount.fetch_sub(size, std::memory_order_relaxed);
}
else
{
Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size;
/** Sometimes, query could free some data, that was allocated outside of query context.
* Example: cache eviction.
* To avoid negative memory usage, we "saturate" amount.
* Memory usage will be calculated with some error.
* NOTE: The code is not atomic. Not worth to fix.
*/
if (unlikely(new_amount < 0))
{
amount.fetch_sub(new_amount);
size += new_amount;
}
}
if (auto loaded_next = next.load(std::memory_order_relaxed))
if (auto loaded_next = parent.load(std::memory_order_relaxed))
loaded_next->free(size);
else
CurrentMetrics::sub(metric, size);
}
void MemoryTracker::reset()
void MemoryTracker::resetCounters()
{
if (!next.load(std::memory_order_relaxed))
CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed));
amount.store(0, std::memory_order_relaxed);
peak.store(0, std::memory_order_relaxed);
limit.store(0, std::memory_order_relaxed);
}
void MemoryTracker::reset()
{
if (!parent.load(std::memory_order_relaxed))
CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed));
resetCounters();
}
void MemoryTracker::setOrRaiseLimit(Int64 value)
{
/// This is just atomic set to maximum.
@ -149,29 +168,26 @@ void MemoryTracker::setOrRaiseLimit(Int64 value)
;
}
#if __APPLE__ && __clang__
__thread MemoryTracker * current_memory_tracker = nullptr;
#else
thread_local MemoryTracker * current_memory_tracker = nullptr;
#endif
namespace CurrentMemoryTracker
{
void alloc(Int64 size)
{
if (current_memory_tracker)
current_memory_tracker->alloc(size);
DB::CurrentThread::getMemoryTracker().alloc(size);
}
void realloc(Int64 old_size, Int64 new_size)
{
if (current_memory_tracker)
current_memory_tracker->alloc(new_size - old_size);
DB::CurrentThread::getMemoryTracker().alloc(new_size - old_size);
}
void free(Int64 size)
{
if (current_memory_tracker)
current_memory_tracker->free(size);
DB::CurrentThread::getMemoryTracker().free(size);
}
}
DB::SimpleActionLock getCurrentMemoryTrackerActionLock()
{
return DB::CurrentThread::getMemoryTracker().blocker.cancel();
}

View File

@ -3,6 +3,8 @@
#include <atomic>
#include <common/Types.h>
#include <Common/CurrentMetrics.h>
#include <Common/SimpleActionBlocker.h>
#include <Common/VariableContext.h>
namespace CurrentMetrics
@ -26,7 +28,7 @@ class MemoryTracker
/// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy).
/// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker.
std::atomic<MemoryTracker *> next {};
std::atomic<MemoryTracker *> parent {};
/// You could specify custom metric to track memory usage.
CurrentMetrics::Metric metric = CurrentMetrics::MemoryTracking;
@ -35,11 +37,14 @@ class MemoryTracker
const char * description = nullptr;
public:
MemoryTracker() {}
MemoryTracker(Int64 limit_) : limit(limit_) {}
MemoryTracker(VariableContext level = VariableContext::Thread) : level(level) {}
MemoryTracker(Int64 limit_, VariableContext level = VariableContext::Thread) : limit(limit_), level(level) {}
MemoryTracker(MemoryTracker * parent_, VariableContext level = VariableContext::Thread) : parent(parent_), level(level) {}
~MemoryTracker();
VariableContext level;
/** Call the following functions before calling of corresponding operations with memory allocators.
*/
void alloc(Int64 size);
@ -79,9 +84,15 @@ public:
}
/// next should be changed only once: from nullptr to some value.
void setNext(MemoryTracker * elem)
/// NOTE: It is not true in MergeListElement
void setParent(MemoryTracker * elem)
{
next.store(elem, std::memory_order_relaxed);
parent.store(elem, std::memory_order_relaxed);
}
MemoryTracker * getParent()
{
return parent.load(std::memory_order_relaxed);
}
/// The memory consumption could be shown in realtime via CurrentMetrics counter
@ -95,26 +106,21 @@ public:
description = description_;
}
/// Reset the accumulated data.
/// Reset the accumulated data
void resetCounters();
/// Reset the accumulated data and the parent.
void reset();
/// Prints info about peak memory consumption into log.
void logPeakMemoryUsage() const;
/// To be able to temporarily stop memory tracker
DB::SimpleActionBlocker blocker;
};
/** The MemoryTracker object is quite difficult to pass to all places where significant amounts of memory are allocated.
* Therefore, a thread-local pointer to used MemoryTracker is set, or nullptr if MemoryTracker does not need to be used.
* This pointer is set when memory consumption is monitored in current thread.
* So, you just need to pass it to all the threads that handle one request.
*/
#if defined(__APPLE__) && defined(__clang__)
extern __thread MemoryTracker * current_memory_tracker;
#else
extern thread_local MemoryTracker * current_memory_tracker;
#endif
/// Convenience methods, that use current_memory_tracker if it is available.
/// Convenience methods, that use current thread's memory_tracker if it is available.
namespace CurrentMemoryTracker
{
void alloc(Int64 size);
@ -123,20 +129,4 @@ namespace CurrentMemoryTracker
}
#include <boost/noncopyable.hpp>
struct TemporarilyDisableMemoryTracker : private boost::noncopyable
{
MemoryTracker * memory_tracker;
TemporarilyDisableMemoryTracker()
{
memory_tracker = current_memory_tracker;
current_memory_tracker = nullptr;
}
~TemporarilyDisableMemoryTracker()
{
current_memory_tracker = memory_tracker;
}
};
DB::SimpleActionLock getCurrentMemoryTrackerActionLock();

View File

@ -1,5 +1,7 @@
#include <Common/ProfileEvents.h>
#include <Common/CurrentThread.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnArray.h>
/// Available events. Add something here as you wish.
#define APPLY_FOR_EVENTS(M) \
@ -37,6 +39,11 @@
M(CreatedReadBufferAIO) \
M(CreatedWriteBufferOrdinary) \
M(CreatedWriteBufferAIO) \
M(DiskReadElapsedMicroseconds) \
M(DiskWriteElapsedMicroseconds) \
M(NetworkReceiveElapsedMicroseconds) \
M(NetworkSendElapsedMicroseconds) \
M(ThrottlerSleepMicroseconds) \
\
M(ReplicatedPartFetches) \
M(ReplicatedPartFailedFetches) \
@ -143,31 +150,93 @@
M(RWLockAcquiredWriteLocks) \
M(RWLockReadersWaitMilliseconds) \
M(RWLockWritersWaitMilliseconds) \
M(NetworkErrors) \
\
M(NetworkErrors)
M(RealTimeMicroseconds) \
M(UserTimeMicroseconds) \
M(SystemTimeMicroseconds) \
M(SoftPageFaults) \
M(HardPageFaults) \
M(VoluntaryContextSwitches) \
M(InvoluntaryContextSwitches) \
\
M(OSIOWaitMicroseconds) \
M(OSCPUWaitMicroseconds) \
M(OSCPUVirtualTimeMicroseconds) \
M(OSReadBytes) \
M(OSWriteBytes) \
M(OSReadChars) \
M(OSWriteChars) \
namespace ProfileEvents
{
#define M(NAME) extern const Event NAME = __COUNTER__;
#define M(NAME) extern const Event NAME = __COUNTER__;
APPLY_FOR_EVENTS(M)
#undef M
constexpr Event END = __COUNTER__;
/// Global variable, initialized by zeros.
Counter global_counters_array[END] {};
/// Initialize global counters statically
Counters global_counters(global_counters_array);
const Event Counters::num_counters = END;
Counters::Counters(VariableContext level, Counters * parent)
: counters_holder(new Counter[num_counters] {}),
parent(parent),
level(level)
{
counters = counters_holder.get();
}
void Counters::resetCounters()
{
if (counters)
{
for (Event i = 0; i < num_counters; ++i)
counters[i].store(0, std::memory_order_relaxed);
}
}
void Counters::reset()
{
parent = nullptr;
resetCounters();
}
Counters Counters::getPartiallyAtomicSnapshot() const
{
Counters res(VariableContext::Snapshot, nullptr);
for (Event i = 0; i < num_counters; ++i)
res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed);
return res;
}
const char * getDescription(Event event)
{
static const char * descriptions[] =
{
#define M(NAME) #NAME,
APPLY_FOR_EVENTS(M)
#undef M
constexpr Event END = __COUNTER__;
};
std::atomic<Count> counters[END] {}; /// Global variable, initialized by zeros.
return descriptions[event];
}
const char * getDescription(Event event)
{
static const char * descriptions[] =
{
#define M(NAME) #NAME,
APPLY_FOR_EVENTS(M)
#undef M
};
return descriptions[event];
}
Event end() { return END; }
void increment(Event event, Count amount)
{
DB::CurrentThread::getProfileEvents().increment(event, amount);
}
Event end() { return END; }
}
#undef APPLY_FOR_EVENTS

View File

@ -1,8 +1,9 @@
#pragma once
#include <stddef.h>
#include <Common/VariableContext.h>
#include <atomic>
#include <memory>
#include <stddef.h>
/** Implements global counters for various events happening in the application
* - for high level profiling.
@ -14,19 +15,80 @@ namespace ProfileEvents
/// Event identifier (index in array).
using Event = size_t;
using Count = size_t;
using Counter = std::atomic<Count>;
class Counters;
/// Counters - how many times each event happened
extern Counters global_counters;
class Counters
{
Counter * counters = nullptr;
std::unique_ptr<Counter[]> counters_holder;
/// Used to propagate increments
Counters * parent = nullptr;
public:
VariableContext level = VariableContext::Thread;
/// By default, any instance have to increment global counters
Counters(VariableContext level = VariableContext::Thread, Counters * parent = &global_counters);
/// Global level static initializer
Counters(Counter * allocated_counters)
: counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {}
Counter & operator[] (Event event)
{
return counters[event];
}
const Counter & operator[] (Event event) const
{
return counters[event];
}
inline void increment(Event event, Count amount = 1)
{
Counters * current = this;
do
{
current->counters[event].fetch_add(amount, std::memory_order_relaxed);
current = current->parent;
} while (current != nullptr);
}
/// Every single value is fetched atomically, but not all values as a whole.
Counters getPartiallyAtomicSnapshot() const;
/// Reset all counters to zero and reset parent.
void reset();
/// Get parent (thread unsafe)
Counters * getParent()
{
return parent;
}
/// Set parent (thread unsafe)
void setParent(Counters * parent_)
{
parent = parent_;
}
/// Set all counters to zero
void resetCounters();
static const Event num_counters;
};
/// Increment a counter for event. Thread-safe.
void increment(Event event, Count amount = 1);
/// Get text description of event by identifier. Returns statically allocated string.
const char * getDescription(Event event);
/// Counters - how many times each event happened.
extern std::atomic<Count> counters[];
/// Increment a counter for event. Thread-safe.
inline void increment(Event event, Count amount = 1)
{
counters[event].fetch_add(amount, std::memory_order_relaxed);
}
/// Get index just after last event identifier.
Event end();
}

View File

@ -86,7 +86,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C
handler_ptr->it_client->info += "; " + client.info;
return handler_ptr;
return handler_ptr;
}
if (type == Type::Write || queue.empty() || queue.back().type == Type::Write)

View File

@ -0,0 +1,79 @@
#pragma once
#include <atomic>
namespace DB
{
class SimpleActionLock;
/// Similar to ActionBlocker, but without weak_ptr magic
class SimpleActionBlocker
{
using Counter = std::atomic<int>;
Counter counter = 0;
public:
SimpleActionBlocker() = default;
bool isCancelled() const { return counter > 0; }
/// Temporarily blocks corresponding actions (while the returned object is alive)
friend class SimpleActionLock;
inline SimpleActionLock cancel();
/// Cancel the actions forever.
void cancelForever() { ++counter; }
};
/// Blocks related action while a SimpleActionLock instance exists
class SimpleActionLock
{
SimpleActionBlocker * block = nullptr;
public:
SimpleActionLock() = default;
explicit SimpleActionLock(SimpleActionBlocker & block_) : block(&block_)
{
++block->counter;
}
SimpleActionLock(const SimpleActionLock &) = delete;
SimpleActionLock(SimpleActionLock && rhs) noexcept
{
*this = std::move(rhs);
}
SimpleActionLock & operator=(const SimpleActionLock &) = delete;
SimpleActionLock & operator=(SimpleActionLock && rhs) noexcept
{
if (block)
--block->counter;
block = rhs.block;
rhs.block = nullptr;
return *this;
}
~SimpleActionLock()
{
if (block)
--block->counter;
}
};
SimpleActionLock SimpleActionBlocker::cancel()
{
return SimpleActionLock(*this);
}
}

View File

@ -0,0 +1,14 @@
#include <sys/resource.h>
#include "Stopwatch.h"
StopWatchRUsage::Timestamp StopWatchRUsage::Timestamp::current()
{
StopWatchRUsage::Timestamp res;
::rusage rusage;
::getrusage(RUSAGE_THREAD, &rusage);
res.user_ns = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec;
res.sys_ns = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec;
return res;
}

View File

@ -32,9 +32,11 @@ public:
void stop() { stop_ns = nanoseconds(); is_running = false; }
void reset() { start_ns = 0; stop_ns = 0; is_running = false; }
void restart() { start(); }
UInt64 elapsed() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; }
UInt64 elapsedMilliseconds() const { return elapsed() / 1000000UL; }
double elapsedSeconds() const { return static_cast<double>(elapsed()) / 1000000000ULL; }
UInt64 elapsed() const { return elapsedNanoseconds(); }
UInt64 elapsedNanoseconds() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; }
UInt64 elapsedMicroseconds() const { return elapsedNanoseconds() / 1000U; }
UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; }
double elapsedSeconds() const { return static_cast<double>(elapsedNanoseconds()) / 1000000000ULL; }
private:
UInt64 start_ns = 0;
@ -131,3 +133,59 @@ private:
/// Most significant bit is a lock. When it is set, compareAndRestartDeferred method will return false.
UInt64 nanoseconds() const { return StopWatchDetail::nanoseconds(clock_type) & 0x7FFFFFFFFFFFFFFFULL; }
};
/// Like ordinary StopWatch, but uses getrusage() system call
struct StopWatchRUsage
{
StopWatchRUsage() = default;
void start() { start_ts = Timestamp::current(); is_running = true; }
void stop() { stop_ts = Timestamp::current(); is_running = false; }
void reset() { start_ts = Timestamp(); stop_ts = Timestamp(); is_running = false; }
void restart() { start(); }
UInt64 elapsed(bool count_user = true, bool count_sys = true) const
{
return elapsedNanoseconds(count_user, count_sys);
}
UInt64 elapsedNanoseconds(bool count_user = true, bool count_sys = true) const
{
return (is_running ? Timestamp::current() : stop_ts).nanoseconds(count_user, count_sys) - start_ts.nanoseconds(count_user, count_sys);
}
UInt64 elapsedMicroseconds(bool count_user = true, bool count_sys = true) const
{
return elapsedNanoseconds(count_user, count_sys) / 1000UL;
}
UInt64 elapsedMilliseconds(bool count_user = true, bool count_sys = true) const
{
return elapsedNanoseconds(count_user, count_sys) / 1000000UL;
}
double elapsedSeconds(bool count_user = true, bool count_sys = true) const
{
return static_cast<double>(elapsedNanoseconds(count_user, count_sys)) / 1000000000.0;
}
private:
struct Timestamp
{
UInt64 user_ns = 0;
UInt64 sys_ns = 0;
static Timestamp current();
UInt64 nanoseconds(bool count_user = true, bool count_sys = true) const
{
return (count_user ? user_ns : 0) + (count_sys ? sys_ns : 0);
}
};
Timestamp start_ts;
Timestamp stop_ts;
bool is_running = false;
};

View File

@ -0,0 +1,262 @@
#include "TaskStatsInfoGetter.h"
#include <Common/Exception.h>
#include <asm/types.h>
#include <errno.h>
#include <linux/genetlink.h>
#include <linux/netlink.h>
#include <linux/taskstats.h>
#include <linux/unistd.h>
#include <pwd.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/socket.h>
#include <sys/types.h>
#include <unistd.h>
#include <syscall.h>
/// Based on: https://github.com/Tomas-M/iotop/tree/master/src
/// More info: https://www.kernel.org/doc/Documentation/accounting/taskstats.txt
/*
* Generic macros for dealing with netlink sockets. Might be duplicated
* elsewhere. It is recommended that commercial grade applications use
* libnl or libnetlink and use the interfaces provided by the library
*/
#define GENLMSG_DATA(glh) ((void *)((char*)NLMSG_DATA(glh) + GENL_HDRLEN))
#define GENLMSG_PAYLOAD(glh) (NLMSG_PAYLOAD(glh, 0) - GENL_HDRLEN)
#define NLA_DATA(na) ((void *)((char*)(na) + NLA_HDRLEN))
#define NLA_PAYLOAD(len) (len - NLA_HDRLEN)
namespace DB
{
namespace ErrorCodes
{
extern const int NETLINK_ERROR;
}
namespace
{
static size_t constexpr MAX_MSG_SIZE = 1024;
struct NetlinkMessage
{
::nlmsghdr n;
::genlmsghdr g;
char buf[MAX_MSG_SIZE];
};
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
int send_cmd(int sock_fd, __u16 nlmsg_type, __u32 nlmsg_pid,
__u8 genl_cmd, __u16 nla_type,
void * nla_data, int nla_len) noexcept
{
NetlinkMessage msg;
memset(&msg, 0, sizeof(msg));
msg.n.nlmsg_len = NLMSG_LENGTH(GENL_HDRLEN);
msg.n.nlmsg_type = nlmsg_type;
msg.n.nlmsg_flags = NLM_F_REQUEST;
msg.n.nlmsg_seq = 0;
msg.n.nlmsg_pid = nlmsg_pid;
msg.g.cmd = genl_cmd;
msg.g.version = 0x1;
::nlattr * na = static_cast<::nlattr *>(GENLMSG_DATA(&msg));
na->nla_type = nla_type;
na->nla_len = nla_len + 1 + NLA_HDRLEN;
memcpy(NLA_DATA(na), nla_data, nla_len);
msg.n.nlmsg_len += NLMSG_ALIGN(na->nla_len);
char * buf = reinterpret_cast<char *>(&msg);
ssize_t buflen = msg.n.nlmsg_len;
::sockaddr_nl nladdr;
memset(&nladdr, 0, sizeof(nladdr));
nladdr.nl_family = AF_NETLINK;
while (true)
{
ssize_t r = ::sendto(sock_fd, buf, buflen, 0, reinterpret_cast<::sockaddr *>(&nladdr), sizeof(nladdr));
if (r >= buflen)
break;
if (r > 0)
{
buf += r;
buflen -= r;
}
else if (errno != EAGAIN)
return -1;
}
return 0;
}
UInt16 get_family_id(int nl_sock_fd) noexcept
{
struct
{
::nlmsghdr n;
::genlmsghdr g;
char buf[256];
} ans;
static char name[] = TASKSTATS_GENL_NAME;
if (send_cmd(nl_sock_fd, GENL_ID_CTRL, getpid(), CTRL_CMD_GETFAMILY,
CTRL_ATTR_FAMILY_NAME, (void *) name,
strlen(TASKSTATS_GENL_NAME) + 1))
return 0;
UInt16 id = 0;
ssize_t rep_len = ::recv(nl_sock_fd, &ans, sizeof(ans), 0);
if (ans.n.nlmsg_type == NLMSG_ERROR || (rep_len < 0) || !NLMSG_OK((&ans.n), rep_len))
return 0;
::nlattr * na;
na = static_cast<::nlattr *>(GENLMSG_DATA(&ans));
na = reinterpret_cast<::nlattr *>((char *) na + NLA_ALIGN(na->nla_len));
if (na->nla_type == CTRL_ATTR_FAMILY_ID)
id = *static_cast<__u16 *>(NLA_DATA(na));
return id;
}
#pragma GCC diagnostic pop
}
TaskStatsInfoGetter::TaskStatsInfoGetter() = default;
void TaskStatsInfoGetter::init()
{
if (netlink_socket_fd >= 0)
return;
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
if (netlink_socket_fd < 0)
throwFromErrno("Can't create PF_NETLINK socket");
::sockaddr_nl addr;
memset(&addr, 0, sizeof(addr));
addr.nl_family = AF_NETLINK;
if (::bind(netlink_socket_fd, reinterpret_cast<::sockaddr *>(&addr), sizeof(addr)) < 0)
throwFromErrno("Can't bind PF_NETLINK socket");
netlink_family_id = get_family_id(netlink_socket_fd);
}
bool TaskStatsInfoGetter::getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error)
{
init();
if (send_cmd(netlink_socket_fd, netlink_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(pid_t)))
throwFromErrno("Can't send a Netlink command");
NetlinkMessage msg;
ssize_t rv = ::recv(netlink_socket_fd, &msg, sizeof(msg), 0);
if (msg.n.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&msg.n), rv))
{
::nlmsgerr * err = static_cast<::nlmsgerr *>(NLMSG_DATA(&msg));
if (throw_on_error)
throw Exception("Can't get Netlink response, error: " + std::to_string(err->error), ErrorCodes::NETLINK_ERROR);
else
return false;
}
rv = GENLMSG_PAYLOAD(&msg.n);
::nlattr * na = static_cast<::nlattr *>(GENLMSG_DATA(&msg));
ssize_t len = 0;
while (len < rv)
{
len += NLA_ALIGN(na->nla_len);
if (na->nla_type == TASKSTATS_TYPE_AGGR_TGID || na->nla_type == TASKSTATS_TYPE_AGGR_PID)
{
int aggr_len = NLA_PAYLOAD(na->nla_len);
int len2 = 0;
na = static_cast<::nlattr *>(NLA_DATA(na));
while (len2 < aggr_len)
{
if (na->nla_type == TASKSTATS_TYPE_STATS)
{
::taskstats * ts = static_cast<::taskstats *>(NLA_DATA(na));
out_stats = *ts;
}
len2 += NLA_ALIGN(na->nla_len);
na = reinterpret_cast<::nlattr *>(reinterpret_cast<char *>(na) + len2);
}
}
na = reinterpret_cast<::nlattr *>(reinterpret_cast<char *>(GENLMSG_DATA(&msg)) + len);
}
return true;
}
void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid)
{
tid = tid < 0 ? getDefaultTID() : tid;
getStatImpl(tid, stat, true);
}
bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid)
{
tid = tid < 0 ? getDefaultTID() : tid;
return getStatImpl(tid, stat, false);
}
TaskStatsInfoGetter::~TaskStatsInfoGetter()
{
if (netlink_socket_fd >= 0)
close(netlink_socket_fd);
}
int TaskStatsInfoGetter::getCurrentTID()
{
/// This call is always successful. - man gettid
return static_cast<int>(syscall(SYS_gettid));
}
int TaskStatsInfoGetter::getDefaultTID()
{
if (default_tid < 0)
default_tid = getCurrentTID();
return default_tid;
}
static bool tryGetTaskStats()
{
TaskStatsInfoGetter getter;
::taskstats stat;
return getter.tryGetStat(stat);
}
bool TaskStatsInfoGetter::checkProcessHasRequiredPermissions()
{
/// It is thread- and exception- safe since C++11
static bool res = tryGetTaskStats();
return res;
}
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Core/Types.h>
struct taskstats;
namespace DB
{
class Exception;
/// Get taskstat info from OS kernel via Netlink protocol.
class TaskStatsInfoGetter
{
public:
TaskStatsInfoGetter();
TaskStatsInfoGetter(const TaskStatsInfoGetter &) = delete;
void getStat(::taskstats & stat, int tid = -1);
bool tryGetStat(::taskstats & stat, int tid = -1);
~TaskStatsInfoGetter();
/// Make a syscall and returns Linux thread id
static int getCurrentTID();
/// Whether the current process has permissions (sudo or cap_net_admin capabilties) to get taskstats info
static bool checkProcessHasRequiredPermissions();
private:
/// Caches current thread tid to avoid extra sys calls
int getDefaultTID();
int default_tid = -1;
bool getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error = false);
void init();
int netlink_socket_fd = -1;
UInt16 netlink_family_id = 0;
};
}

View File

@ -0,0 +1,144 @@
#pragma once
#include <Common/TaskStatsInfoGetter.h>
#include <Common/ProfileEvents.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <pthread.h>
#include <linux/taskstats.h>
namespace ProfileEvents
{
extern const Event RealTimeMicroseconds;
extern const Event UserTimeMicroseconds;
extern const Event SystemTimeMicroseconds;
extern const Event SoftPageFaults;
extern const Event HardPageFaults;
extern const Event VoluntaryContextSwitches;
extern const Event InvoluntaryContextSwitches;
extern const Event OSIOWaitMicroseconds;
extern const Event OSCPUWaitMicroseconds;
extern const Event OSCPUVirtualTimeMicroseconds;
extern const Event OSReadChars;
extern const Event OSWriteChars;
extern const Event OSReadBytes;
extern const Event OSWriteBytes;
}
namespace DB
{
/// Handles overflow
template <typename TUInt>
inline TUInt safeDiff(TUInt prev, TUInt curr)
{
return curr >= prev ? curr - prev : 0;
}
inline UInt64 getCurrentTimeNanoseconds(clockid_t clock_type = CLOCK_MONOTONIC)
{
struct timespec ts;
clock_gettime(clock_type, &ts);
return ts.tv_sec * 1000000000ULL + ts.tv_nsec;
}
struct RUsageCounters
{
/// In nanoseconds
UInt64 real_time = 0;
UInt64 user_time = 0;
UInt64 sys_time = 0;
UInt64 soft_page_faults = 0;
UInt64 hard_page_faults = 0;
RUsageCounters() = default;
RUsageCounters(const ::rusage & rusage_, UInt64 real_time_)
{
set(rusage_, real_time_);
}
void set(const ::rusage & rusage, UInt64 real_time_)
{
real_time = real_time_;
user_time = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec;
sys_time = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec;
soft_page_faults = static_cast<UInt64>(rusage.ru_minflt);
hard_page_faults = static_cast<UInt64>(rusage.ru_majflt);
}
static RUsageCounters zeros(UInt64 real_time_ = getCurrentTimeNanoseconds())
{
RUsageCounters res;
res.real_time = real_time_;
return res;
}
static RUsageCounters current(UInt64 real_time_ = getCurrentTimeNanoseconds())
{
::rusage rusage;
::getrusage(RUSAGE_THREAD, &rusage);
return RUsageCounters(rusage, real_time_);
}
static void incrementProfileEvents(const RUsageCounters & prev, const RUsageCounters & curr, ProfileEvents::Counters & profile_events)
{
profile_events.increment(ProfileEvents::RealTimeMicroseconds, (curr.real_time - prev.real_time) / 1000U);
profile_events.increment(ProfileEvents::UserTimeMicroseconds, (curr.user_time - prev.user_time) / 1000U);
profile_events.increment(ProfileEvents::SystemTimeMicroseconds, (curr.sys_time - prev.sys_time) / 1000U);
profile_events.increment(ProfileEvents::SoftPageFaults, curr.soft_page_faults - prev.soft_page_faults);
profile_events.increment(ProfileEvents::HardPageFaults, curr.hard_page_faults - prev.hard_page_faults);
}
static void updateProfileEvents(RUsageCounters & last_counters, ProfileEvents::Counters & profile_events)
{
auto current_counters = current();
incrementProfileEvents(last_counters, current_counters, profile_events);
last_counters = current_counters;
}
};
struct TasksStatsCounters
{
::taskstats stat;
TasksStatsCounters() = default;
static TasksStatsCounters current();
static void incrementProfileEvents(const TasksStatsCounters & prev, const TasksStatsCounters & curr, ProfileEvents::Counters & profile_events)
{
profile_events.increment(ProfileEvents::OSCPUWaitMicroseconds,
safeDiff(prev.stat.cpu_delay_total, curr.stat.cpu_delay_total) / 1000U);
profile_events.increment(ProfileEvents::OSIOWaitMicroseconds,
safeDiff(prev.stat.blkio_delay_total, curr.stat.blkio_delay_total) / 1000U);
profile_events.increment(ProfileEvents::OSCPUVirtualTimeMicroseconds,
safeDiff(prev.stat.cpu_run_virtual_total, curr.stat.cpu_run_virtual_total) / 1000U);
/// Too old struct version, do not read new fields
if (curr.stat.version < TASKSTATS_VERSION)
return;
profile_events.increment(ProfileEvents::OSReadChars, safeDiff(prev.stat.read_char, curr.stat.read_char));
profile_events.increment(ProfileEvents::OSWriteChars, safeDiff(prev.stat.write_char, curr.stat.write_char));
profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.stat.read_bytes, curr.stat.read_bytes));
profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.stat.write_bytes, curr.stat.write_bytes));
}
static void updateProfileEvents(TasksStatsCounters & last_counters, ProfileEvents::Counters & profile_events)
{
auto current_counters = current();
incrementProfileEvents(last_counters, current_counters, profile_events);
last_counters = current_counters;
}
};
}

View File

@ -0,0 +1,118 @@
#include "ThreadStatus.h"
#include <common/logger_useful.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/ThreadProfileEvents.h>
#include <Poco/Thread.h>
#include <Poco/Ext/ThreadNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int PTHREAD_ERROR;
}
/// Order of current_thread and current_thread_scope matters
thread_local ThreadStatusPtr current_thread = ThreadStatus::create();
thread_local ThreadStatus::CurrentThreadScope current_thread_scope;
TasksStatsCounters TasksStatsCounters::current()
{
TasksStatsCounters res;
current_thread->taskstats_getter->getStat(res.stat, current_thread->os_thread_id);
return res;
}
ThreadStatus::ThreadStatus()
{
thread_number = Poco::ThreadNumber::get();
os_thread_id = TaskStatsInfoGetter::getCurrentTID();
last_rusage = std::make_unique<RUsageCounters>();
last_taskstats = std::make_unique<TasksStatsCounters>();
taskstats_getter = std::make_unique<TaskStatsInfoGetter>();
memory_tracker.setDescription("(for thread)");
log = &Poco::Logger::get("ThreadStatus");
/// NOTE: It is important not to do any non-trivial actions (like updating ProfileEvents or logging) before ThreadStatus is created
/// Otherwise it could lead to SIGSEGV due to current_thread dereferencing
}
ThreadStatusPtr ThreadStatus::create()
{
return ThreadStatusPtr(new ThreadStatus);
}
ThreadStatus::~ThreadStatus() = default;
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);
has_permissions_for_taskstats = TaskStatsInfoGetter::checkProcessHasRequiredPermissions();
if (has_permissions_for_taskstats)
*last_taskstats = TasksStatsCounters::current();
}
void ThreadStatus::updatePerformanceCounters()
{
try
{
RUsageCounters::updateProfileEvents(*last_rusage, performance_counters);
if (has_permissions_for_taskstats)
TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters);
}
catch (...)
{
tryLogCurrentException(log);
}
}
void ThreadStatus::assertState(const std::initializer_list<int> & permitted_states, const char * description)
{
for (auto permitted_state : permitted_states)
{
if (getCurrentState() == permitted_state)
return;
}
std::stringstream ss;
ss << "Unexpected thread state " << getCurrentState();
if (description)
ss << ": " << description;
throw Exception(ss.str(), ErrorCodes::LOGICAL_ERROR);
}
void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue)
{
logs_queue_ptr = logs_queue;
if (!thread_group)
return;
std::unique_lock lock(thread_group->mutex);
thread_group->logs_queue_ptr = logs_queue;
}
}

View File

@ -0,0 +1,197 @@
#pragma once
#include <Common/ProfileEvents.h>
#include <Common/MemoryTracker.h>
#include <IO/Progress.h>
#include <memory>
#include <map>
#include <mutex>
#include <shared_mutex>
namespace Poco
{
class Logger;
}
namespace DB
{
class Context;
class QueryStatus;
class ThreadStatus;
using ThreadStatusPtr = std::shared_ptr<ThreadStatus>;
class QueryThreadLog;
struct TasksStatsCounters;
struct RUsageCounters;
class TaskStatsInfoGetter;
class InternalTextLogsQueue;
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
using InternalTextLogsQueueWeakPtr = std::weak_ptr<InternalTextLogsQueue>;
class ThreadGroupStatus
{
public:
mutable std::shared_mutex mutex;
ProfileEvents::Counters performance_counters{VariableContext::Process};
MemoryTracker memory_tracker{VariableContext::Process};
Context * query_context = nullptr;
Context * global_context = nullptr;
InternalTextLogsQueueWeakPtr logs_queue_ptr;
/// Key is Poco's thread_id
using QueryThreadStatuses = std::map<UInt32, ThreadStatusPtr>;
QueryThreadStatuses thread_statuses;
/// The first thread created this thread group
ThreadStatusPtr master_thread;
String query;
};
using ThreadGroupStatusPtr = std::shared_ptr<ThreadGroupStatus>;
class ThreadStatus : public std::enable_shared_from_this<ThreadStatus>
{
public:
/// Poco's thread number (the same number is used in logs)
UInt32 thread_number = 0;
/// Linux's PID (or TGID) (the same id is shown by ps util)
Int32 os_thread_id = -1;
/// TODO: merge them into common entity
ProfileEvents::Counters performance_counters{VariableContext::Thread};
MemoryTracker memory_tracker{VariableContext::Thread};
/// Statistics of read and write rows/bytes
Progress progress_in;
Progress progress_out;
public:
static ThreadStatusPtr create();
ThreadGroupStatusPtr getThreadGroup() const
{
return thread_group;
}
enum ThreadState
{
DetachedFromQuery = 0, /// We just created thread or it is a background thread
AttachedToQuery, /// Thread executes enqueued query
Died, /// Thread does not exist
};
int getCurrentState() const
{
return thread_state.load(std::memory_order_relaxed);
}
String getQueryID();
/// Starts new query and create new thread group for it, current thread becomes master thread of the query
void initializeQuery();
/// Attaches slave thread to existing thread group
void attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached = true);
InternalTextLogsQueuePtr getInternalTextLogsQueue() const
{
return thread_state == Died ? nullptr : logs_queue_ptr.lock();
}
void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue);
/// Sets query context for current thread and its thread group
/// NOTE: query_context have to be alive until detachQuery() is called
void attachQueryContext(Context & query_context);
/// Update several ProfileEvents counters
void updatePerformanceCounters();
/// Update ProfileEvents and dumps info to system.query_thread_log
void finalizePerformanceCounters();
/// Detaches thread from the thread group and the query, dumps performance counters if they have not been dumped
void detachQuery(bool exit_if_already_detached = false, bool thread_exits = false);
~ThreadStatus();
protected:
ThreadStatus();
void initPerformanceCounters();
void logToQueryThreadLog(QueryThreadLog & thread_log);
void assertState(const std::initializer_list<int> & permitted_states, const char * description = nullptr);
ThreadGroupStatusPtr thread_group;
std::atomic<int> thread_state{ThreadState::DetachedFromQuery};
/// Is set once
Context * global_context = nullptr;
/// Use it only from current thread
Context * query_context = nullptr;
/// A logs queue used by TCPHandler to pass logs to a client
InternalTextLogsQueueWeakPtr logs_queue_ptr;
bool performance_counters_finalized = false;
UInt64 query_start_time_nanoseconds = 0;
time_t query_start_time = 0;
size_t queries_started = 0;
Poco::Logger * log = nullptr;
friend class CurrentThread;
friend struct TasksStatsCounters;
/// Use ptr not to add extra dependencies in the header
std::unique_ptr<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> last_taskstats;
std::unique_ptr<TaskStatsInfoGetter> taskstats_getter;
bool has_permissions_for_taskstats = false;
public:
/// Implicitly finalizes current thread in the destructor
class CurrentThreadScope
{
public:
void (*deleter)() = nullptr;
CurrentThreadScope() = default;
~CurrentThreadScope()
{
try
{
if (deleter)
deleter();
}
catch (...)
{
std::terminate();
}
}
};
private:
static void defaultThreadDeleter();
};
extern thread_local ThreadStatusPtr current_thread;
extern thread_local ThreadStatus::CurrentThreadScope current_thread_scope;
}

View File

@ -5,10 +5,17 @@
#include <memory>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <IO/WriteHelpers.h>
#include <port/clock.h>
namespace ProfileEvents
{
extern const Event ThrottlerSleepMicroseconds;
}
namespace DB
{
@ -69,10 +76,14 @@ public:
if (desired_ns > elapsed_ns)
{
UInt64 sleep_ns = desired_ns - elapsed_ns;
timespec sleep_ts;
::timespec sleep_ts;
sleep_ts.tv_sec = sleep_ns / 1000000000;
sleep_ts.tv_nsec = sleep_ns % 1000000000;
nanosleep(&sleep_ts, nullptr); /// NOTE Returns early in case of a signal. This is considered normal.
/// NOTE: Returns early in case of a signal. This is considered normal.
::nanosleep(&sleep_ts, nullptr);
ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL);
}
}

View File

@ -0,0 +1,12 @@
#pragma once
/// Used in ProfileEvents and MemoryTracker to determine their hierarchy level
/// The less value the higher level (zero level is the root)
enum class VariableContext
{
Global = 0,
User, /// Group of processes
Process, /// For example, a query or a merge
Thread, /// A thread of a process
Snapshot /// Does not belong to anybody
};

View File

@ -6,10 +6,20 @@
#else
#include <sys/prctl.h>
#endif
#include <pthread.h>
#include <cstring>
#include <Common/Exception.h>
#include <Common/setThreadName.h>
namespace DB
{
namespace ErrorCodes
{
extern const int PTHREAD_ERROR;
}
}
void setThreadName(const char * name)
{
@ -22,5 +32,21 @@ void setThreadName(const char * name)
#else
if (0 != prctl(PR_SET_NAME, name, 0, 0, 0))
#endif
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME...)");
DB::throwFromErrno("Cannot set thread name with prctl(PR_SET_NAME, ...)");
}
std::string getThreadName()
{
std::string name(16, '\0');
#if defined(__FreeBSD__) || defined(__APPLE__)
if (pthread_get_name_np(pthread_self(), name.data(), name.size());
throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR);
#else
if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0))
#endif
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)");
name.resize(std::strlen(name.data()));
return name;
}

View File

@ -1,7 +1,10 @@
#pragma once
#include <string>
/** Sets the thread name (maximum length is 15 bytes),
* which will be visible in ps, gdb, /proc,
* for convenience of observation and debugging.
*/
void setThreadName(const char * name);
std::string getThreadName();

View File

@ -32,7 +32,7 @@ TEST(Common, RWLockFIFO_1)
auto func = [&] (size_t threads, int round)
{
for (int i = 0; i < cycles; ++i)
for (int i = 0; i < cycles; ++i)
{
auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write;
auto sleep_for = std::chrono::duration<int, std::micro>(std::uniform_int_distribution<>(1, 100)(gen));

View File

@ -46,6 +46,7 @@
#define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337
#define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372
#define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401
#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406
/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change.
#define DBMS_TCP_PROTOCOL_VERSION 54226

View File

@ -69,6 +69,7 @@ namespace Protocol
Totals = 7, /// A block with totals (compressed or not).
Extremes = 8, /// A block with minimums and maximums (compressed or not).
TablesStatusResponse = 9, /// A response to TablesStatus request.
Log = 10 /// System logs of the query execution
};
/// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10
@ -77,8 +78,8 @@ namespace Protocol
/// See https://www.securecoding.cert.org/confluence/display/cplusplus/INT36-CPP.+Do+not+use+out-of-range+enumeration+values
inline const char * toString(UInt64 packet)
{
static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse" };
return packet < 10
static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse", "Log" };
return packet < 11
? data[packet]
: "Unknown packet";
}
@ -97,6 +98,7 @@ namespace Protocol
Cancel = 3, /// Cancel the query execution.
Ping = 4, /// Check that connection to the server is alive.
TablesStatusRequest = 5, /// Check status of tables on the server.
KeepAlive = 6 /// Keep the connection alive
};
inline const char * toString(UInt64 packet)

View File

@ -0,0 +1,84 @@
#include "AsynchronousBlockInputStream.h"
#include <Common/CurrentThread.h>
namespace DB
{
Block AsynchronousBlockInputStream::readImpl()
{
/// If there were no calculations yet, calculate the first block synchronously
if (!started)
{
calculate();
started = true;
}
else /// If the calculations are already in progress - wait for the result
pool.wait();
if (exception)
std::rethrow_exception(exception);
Block res = block;
if (!res)
return res;
/// Start the next block calculation
block.clear();
next();
return res;
}
void AsynchronousBlockInputStream::next()
{
ready.reset();
pool.schedule([this, thread_group=CurrentThread::getGroup()] ()
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
try
{
if (first)
setThreadName("AsyncBlockInput");
/// AsynchronousBlockInputStream is used in Client which does not create queries and thread groups
if (thread_group)
CurrentThread::attachToIfDetached(thread_group);
}
catch (...)
{
exception = std::current_exception();
ready.set();
return;
}
calculate();
});
}
void AsynchronousBlockInputStream::calculate()
{
try
{
if (first)
{
first = false;
children.back()->readPrefix();
}
block = children.back()->read();
}
catch (...)
{
exception = std::current_exception();
}
ready.set();
}
}

View File

@ -7,6 +7,7 @@
#include <Common/CurrentMetrics.h>
#include <common/ThreadPool.h>
#include <Common/MemoryTracker.h>
#include <Poco/Ext/ThreadNumber.h>
namespace CurrentMetrics
@ -91,64 +92,12 @@ protected:
Block block;
std::exception_ptr exception;
Block readImpl() override;
Block readImpl() override
{
/// If there were no calculations yet, calculate the first block synchronously
if (!started)
{
calculate(current_memory_tracker);
started = true;
}
else /// If the calculations are already in progress - wait for the result
pool.wait();
if (exception)
std::rethrow_exception(exception);
Block res = block;
if (!res)
return res;
/// Start the next block calculation
block.clear();
next();
return res;
}
void next()
{
ready.reset();
pool.schedule(std::bind(&AsynchronousBlockInputStream::calculate, this, current_memory_tracker));
}
void next();
/// Calculations that can be performed in a separate thread
void calculate(MemoryTracker * memory_tracker)
{
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
try
{
if (first)
{
first = false;
setThreadName("AsyncBlockInput");
current_memory_tracker = memory_tracker;
children.back()->readPrefix();
}
block = children.back()->read();
}
catch (...)
{
exception = std::current_exception();
}
ready.set();
}
void calculate();
};
}

View File

@ -12,8 +12,7 @@ class ProcessListEntry;
struct BlockIO
{
/** process_list_entry should be destroyed after in and after out,
* since in and out contain pointer to an object inside process_list_entry
* (MemoryTracker * current_memory_tracker),
* since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example),
* which could be used before destroying of in and out.
*/
std::shared_ptr<ProcessListEntry> process_list_entry;
@ -38,12 +37,17 @@ struct BlockIO
exception_callback();
}
/// We provide the correct order of destruction.
void reset()
{
out.reset();
in.reset();
process_list_entry.reset();
}
BlockIO & operator= (const BlockIO & rhs)
{
/// We provide the correct order of destruction.
out = nullptr;
in = nullptr;
process_list_entry = nullptr;
reset();
process_list_entry = rhs.process_list_entry;
in = rhs.in;

View File

@ -20,7 +20,7 @@ public:
progress_callback = callback;
}
void setProcessListElement(ProcessListElement * elem)
void setProcessListElement(QueryStatus * elem)
{
process_elem = elem;
}
@ -43,7 +43,7 @@ protected:
BlockOutputStreamPtr stream;
Progress progress;
ProgressCallback progress_callback;
ProcessListElement * process_elem = nullptr;
QueryStatus * process_elem = nullptr;
};
}

View File

@ -1,6 +1,13 @@
#include <Interpreters/Quota.h>
#include <Interpreters/ProcessList.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/CurrentThread.h>
namespace ProfileEvents
{
extern const Event ThrottlerSleepMicroseconds;
}
namespace DB
@ -286,21 +293,34 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
size_t total_rows = progress.total_rows;
if (limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0))
{
double total_elapsed = info.total_stopwatch.elapsedSeconds();
constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds
UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds();
if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0)
if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds)
{
CurrentThread::updatePerformanceCounters();
last_profile_events_update_time = total_elapsed_microseconds;
}
if ((limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0))
&& (static_cast<Int64>(total_elapsed_microseconds) > limits.timeout_before_checking_execution_speed.totalMicroseconds()))
{
/// Do not count sleeps in throttlers
UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds];
double elapsed_seconds = (throttler_sleep_microseconds > total_elapsed_microseconds)
? 0.0 : (total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0;
if (elapsed_seconds > 0)
{
if (limits.min_execution_speed && progress.rows / total_elapsed < limits.min_execution_speed)
throw Exception("Query is executing too slow: " + toString(progress.rows / total_elapsed)
if (limits.min_execution_speed && progress.rows / elapsed_seconds < limits.min_execution_speed)
throw Exception("Query is executing too slow: " + toString(progress.rows / elapsed_seconds)
+ " rows/sec., minimum: " + toString(limits.min_execution_speed),
ErrorCodes::TOO_SLOW);
/// If the predicted execution time is longer than `max_execution_time`.
if (limits.max_execution_time != 0 && total_rows)
{
double estimated_execution_time_seconds = total_elapsed * (static_cast<double>(total_rows) / progress.rows);
double estimated_execution_time_seconds = elapsed_seconds * (static_cast<double>(total_rows) / progress.rows);
if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds())
throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)"
@ -363,7 +383,7 @@ void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & ca
}
void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem)
void IProfilingBlockInputStream::setProcessListElement(QueryStatus * elem)
{
process_list_elem = elem;

View File

@ -20,6 +20,7 @@ namespace ErrorCodes
}
class QuotaForIntervals;
class QueryStatus;
class ProcessListElement;
class IProfilingBlockInputStream;
@ -103,7 +104,7 @@ public:
* Based on this information, the quota and some restrictions will be checked.
* This information will also be available in the SHOW PROCESSLIST request.
*/
void setProcessListElement(ProcessListElement * elem);
void setProcessListElement(QueryStatus * elem);
/** Set the approximate total number of rows to read.
*/
@ -178,7 +179,9 @@ protected:
std::atomic<bool> is_cancelled{false};
std::atomic<bool> is_killed{false};
ProgressCallback progress_callback;
ProcessListElement * process_list_elem = nullptr;
QueryStatus * process_list_elem = nullptr;
/// According to total_stopwatch in microseconds
UInt64 last_profile_events_update_time = 0;
/// Additional information that can be generated during the work process.

View File

@ -0,0 +1,82 @@
#include "InternalTextLogsRowOutputStream.h"
#include <Core/Block.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Common/typeid_cast.h>
#include <DataTypes/IDataType.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <IO/WriteHelpers.h>
namespace DB
{
Block InternalTextLogsRowOutputStream::getHeader() const
{
return InternalTextLogsQueue::getSampleBlock();
}
void InternalTextLogsRowOutputStream::write(const Block & block)
{
auto & array_event_time = typeid_cast<const ColumnUInt32 &>(*block.getByName("event_time").column).getData();
auto & array_microseconds = typeid_cast<const ColumnUInt32 &>(*block.getByName("event_time_microseconds").column).getData();
auto & column_host_name = typeid_cast<const ColumnString &>(*block.getByName("host_name").column);
auto & column_query_id = typeid_cast<const ColumnString &>(*block.getByName("query_id").column);
auto & array_thread_number = typeid_cast<const ColumnUInt32 &>(*block.getByName("thread_number").column).getData();
auto & array_priority = typeid_cast<const ColumnInt8 &>(*block.getByName("priority").column).getData();
auto & column_source = typeid_cast<const ColumnString &>(*block.getByName("source").column);
auto & column_text = typeid_cast<const ColumnString &>(*block.getByName("text").column);
for (size_t row_num = 0; row_num < block.rows(); ++row_num)
{
auto host_name = column_host_name.getDataAt(row_num);
if (host_name.size)
{
writeCString("[", wb);
writeString(host_name, wb);
writeCString("] ", wb);
}
auto event_time = array_event_time[row_num];
writeDateTimeText<'.', ':'>(event_time, wb);
auto microseconds = array_microseconds[row_num];
writeChar('.', wb);
writeChar('0' + ((microseconds / 100000) % 10), wb);
writeChar('0' + ((microseconds / 10000) % 10), wb);
writeChar('0' + ((microseconds / 1000) % 10), wb);
writeChar('0' + ((microseconds / 100) % 10), wb);
writeChar('0' + ((microseconds / 10) % 10), wb);
writeChar('0' + ((microseconds / 1) % 10), wb);
auto query_id = column_query_id.getDataAt(row_num);
if (query_id.size)
{
writeCString(" {", wb);
writeString(query_id, wb);
writeCString("}", wb);
}
UInt32 thread_number = array_thread_number[row_num];
writeCString(" [ ", wb);
writeIntText(thread_number, wb);
writeCString(" ] <", wb);
Int8 priority = array_priority[row_num];
writeString(InternalTextLogsQueue::getPriorityName(priority), wb);
writeCString("> ", wb);
auto source = column_source.getDataAt(row_num);
writeString(source, wb);
writeCString(": ", wb);
auto text = column_text.getDataAt(row_num);
writeString(text, wb);
writeChar('\n', wb);
}
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <DataStreams/IBlockOutputStream.h>
#include <IO/WriteBuffer.h>
namespace DB
{
/// Prints internal server logs
/// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock()
/// NOTE: IRowOutputStream does not suite well for this case
class InternalTextLogsRowOutputStream : public IBlockOutputStream
{
public:
InternalTextLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {}
Block getHeader() const override;
void write(const Block & block) override;
void flush() override
{
wb.next();
}
private:
WriteBuffer & wb;
};
}

View File

@ -3,6 +3,7 @@
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
#include <DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h>
#include <Common/CurrentThread.h>
namespace CurrentMetrics
@ -175,10 +176,10 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start()
{
auto & child = children[i];
auto memory_tracker = current_memory_tracker;
reading_pool->schedule([&child, memory_tracker]
auto thread_group = CurrentThread::getGroup();
reading_pool->schedule([&child, thread_group]
{
current_memory_tracker = memory_tracker;
CurrentThread::attachToIfDetached(thread_group);
setThreadName("MergeAggReadThr");
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
child->readPrefix();
@ -196,8 +197,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start()
*/
for (size_t i = 0; i < merging_threads; ++i)
pool.schedule(std::bind(&MergingAggregatedMemoryEfficientBlockInputStream::mergeThread,
this, current_memory_tracker));
pool.schedule([this, thread_group=CurrentThread::getGroup()] () { mergeThread(thread_group); } );
}
}
@ -293,14 +293,16 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize()
}
void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(MemoryTracker * memory_tracker)
void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupStatusPtr thread_group)
{
setThreadName("MergeAggMergThr");
current_memory_tracker = memory_tracker;
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
try
{
if (thread_group)
CurrentThread::attachTo(thread_group);
setThreadName("MergeAggMergThr");
while (!parallel_merge_data->finish)
{
/** Receiving next blocks is processing by one thread pool, and merge is in another.
@ -480,10 +482,10 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate
{
if (need_that_input(input))
{
auto memory_tracker = current_memory_tracker;
reading_pool->schedule([&input, &read_from_input, memory_tracker]
auto thread_group = CurrentThread::getGroup();
reading_pool->schedule([&input, &read_from_input, thread_group]
{
current_memory_tracker = memory_tracker;
CurrentThread::attachToIfDetached(thread_group);
setThreadName("MergeAggReadThr");
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
read_from_input(input);

View File

@ -3,6 +3,7 @@
#include <Interpreters/Aggregator.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentThread.h>
#include <common/ThreadPool.h>
#include <condition_variable>
@ -151,7 +152,7 @@ private:
std::unique_ptr<ParallelMergeData> parallel_merge_data;
void mergeThread(MemoryTracker * memory_tracker);
void mergeThread(ThreadGroupStatusPtr main_thread);
void finalize();
};

View File

@ -12,6 +12,7 @@
#include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
#include <Common/CurrentThread.h>
/** Allows to process multiple block input streams (sources) in parallel, using specified number of threads.
@ -105,8 +106,9 @@ public:
{
active_threads = max_threads;
threads.reserve(max_threads);
auto thread_group = CurrentThread::getGroup();
for (size_t i = 0; i < max_threads; ++i)
threads.emplace_back(std::bind(&ParallelInputsProcessor::thread, this, current_memory_tracker, i));
threads.emplace_back([=] () { thread(thread_group, i); } );
}
/// Ask all sources to stop earlier than they run out.
@ -174,16 +176,16 @@ private:
}
}
void thread(MemoryTracker * memory_tracker, size_t thread_num)
void thread(ThreadGroupStatusPtr thread_group, size_t thread_num)
{
current_memory_tracker = memory_tracker;
std::exception_ptr exception;
setThreadName("ParalInputsProc");
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
try
{
setThreadName("ParalInputsProc");
CurrentThread::attachTo(thread_group);
while (!finish)
{
InputData unprepared_input;

View File

@ -1,8 +1,10 @@
#include <DataStreams/RemoteBlockInputStream.h>
#include <DataStreams/OneBlockInputStream.h>
#include <Common/NetException.h>
#include <Common/CurrentThread.h>
#include <Interpreters/Context.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/InternalTextLogsQueue.h>
#include <Storages/IStorage.h>
@ -232,6 +234,12 @@ Block RemoteBlockInputStream::readImpl()
extremes = packet.block;
break;
case Protocol::Server::Log:
/// Pass logs from remote server to client
if (auto log_queue = CurrentThread::getInternalTextLogsQueue())
log_queue->pushBlock(std::move(packet.block));
break;
default:
got_unknown_packet_from_replica = true;
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);

View File

@ -4,6 +4,8 @@
#include <common/logger_useful.h>
#include <Common/NetException.h>
#include <Common/CurrentThread.h>
#include <Interpreters/InternalTextLogsQueue.h>
namespace DB
@ -24,23 +26,33 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const
*/
connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr);
Connection::Packet packet = connection.receivePacket();
if (Protocol::Server::Data == packet.type)
while (true)
{
header = packet.block;
Connection::Packet packet = connection.receivePacket();
if (!header)
throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR);
if (Protocol::Server::Data == packet.type)
{
header = packet.block;
if (!header)
throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR);
break;
}
else if (Protocol::Server::Exception == packet.type)
{
packet.exception->rethrow();
break;
}
else if (Protocol::Server::Log == packet.type)
{
/// Pass logs from remote server to client
if (auto log_queue = CurrentThread::getInternalTextLogsQueue())
log_queue->pushBlock(std::move(packet.block));
}
else
throw NetException("Unexpected packet from server (expected Data or Exception, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
else if (Protocol::Server::Exception == packet.type)
{
packet.exception->rethrow();
return;
}
else
throw NetException("Unexpected packet from server (expected Data or Exception, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
@ -55,15 +67,11 @@ void RemoteBlockOutputStream::write(const Block & block)
catch (const NetException &)
{
/// Try to get more detailed exception from server
if (connection.poll(0))
auto packet_type = connection.checkPacket();
if (packet_type && *packet_type == Protocol::Server::Exception)
{
Connection::Packet packet = connection.receivePacket();
if (Protocol::Server::Exception == packet.type)
{
packet.exception->rethrow();
return;
}
packet.exception->rethrow();
}
throw;
@ -83,18 +91,23 @@ void RemoteBlockOutputStream::writeSuffix()
/// Empty block means end of data.
connection.sendData(Block());
/// Receive EndOfStream packet.
Connection::Packet packet = connection.receivePacket();
if (Protocol::Server::EndOfStream == packet.type)
/// Wait for EndOfStream or Exception packet, skip Log packets.
while (true)
{
/// Do nothing.
Connection::Packet packet = connection.receivePacket();
if (Protocol::Server::EndOfStream == packet.type)
break;
else if (Protocol::Server::Exception == packet.type)
packet.exception->rethrow();
else if (Protocol::Server::Log == packet.type)
{
// Do nothing
}
else
throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
}
else if (Protocol::Server::Exception == packet.type)
packet.exception->rethrow();
else
throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got "
+ String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER);
finished = true;
}

View File

@ -26,6 +26,7 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NumberTraits.h>
#include <Formats/FormatSettings.h>
#include <Functions/FunctionFactory.h>
@ -36,7 +37,7 @@
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include <Storages/getStructureOfRemoteTable.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/DNSResolver.h>
namespace DB
@ -160,7 +161,7 @@ public:
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
{
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(
input_rows_count, Poco::Net::DNS::hostName())->convertToFullColumnIfConst();
input_rows_count, DNSResolver::instance().getHostName())->convertToFullColumnIfConst();
}
};

View File

@ -14,6 +14,7 @@ namespace ProfileEvents
extern const Event ReadBufferFromFileDescriptorRead;
extern const Event ReadBufferFromFileDescriptorReadFailed;
extern const Event ReadBufferFromFileDescriptorReadBytes;
extern const Event DiskReadElapsedMicroseconds;
extern const Event Seek;
}
@ -47,9 +48,7 @@ bool ReadBufferFromFileDescriptor::nextImpl()
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorRead);
std::optional<Stopwatch> watch;
if (profile_callback)
watch.emplace(clock_type);
Stopwatch watch(profile_callback ? clock_type : CLOCK_MONOTONIC);
ssize_t res = 0;
{
@ -68,12 +67,17 @@ bool ReadBufferFromFileDescriptor::nextImpl()
if (res > 0)
bytes_read += res;
/// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one and we will count cpu time of other thread
/// It is better to use taskstats::blkio_delay_total, but it is quite expensive to get it (TaskStatsInfoGetter has about 500K RPS)
watch.stop();
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());
if (profile_callback)
{
ProfileInfo info;
info.bytes_requested = internal_buffer.size();
info.bytes_read = res;
info.nanoseconds = watch->elapsed();
info.nanoseconds = watch.elapsed();
profile_callback(info);
}
}
@ -114,12 +118,16 @@ off_t ReadBufferFromFileDescriptor::doSeek(off_t offset, int whence)
else
{
ProfileEvents::increment(ProfileEvents::Seek);
StopWatchRUsage watch_ru;
pos = working_buffer.end();
off_t res = lseek(fd, new_pos, SEEK_SET);
off_t res = ::lseek(fd, new_pos, SEEK_SET);
if (-1 == res)
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
pos_in_file = new_pos;
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch_ru.elapsedMicroseconds());
return res;
}
}

View File

@ -4,6 +4,13 @@
#include <IO/ReadBufferFromPocoSocket.h>
#include <Common/NetException.h>
#include <Common/Stopwatch.h>
namespace ProfileEvents
{
extern const Event NetworkReceiveElapsedMicroseconds;
}
namespace DB
@ -20,6 +27,7 @@ namespace ErrorCodes
bool ReadBufferFromPocoSocket::nextImpl()
{
ssize_t bytes_read = 0;
Stopwatch watch;
/// Add more details to exceptions.
try
@ -42,6 +50,9 @@ bool ReadBufferFromPocoSocket::nextImpl()
if (bytes_read < 0)
throw NetException("Cannot read from socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_READ_FROM_SOCKET);
/// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one
ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds());
if (bytes_read)
working_buffer.resize(bytes_read);
else

View File

@ -7,6 +7,7 @@
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>
#include <Common/Stopwatch.h>
namespace ProfileEvents
@ -14,6 +15,7 @@ namespace ProfileEvents
extern const Event WriteBufferFromFileDescriptorWrite;
extern const Event WriteBufferFromFileDescriptorWriteFailed;
extern const Event WriteBufferFromFileDescriptorWriteBytes;
extern const Event DiskWriteElapsedMicroseconds;
}
namespace CurrentMetrics
@ -38,6 +40,8 @@ void WriteBufferFromFileDescriptor::nextImpl()
if (!offset())
return;
Stopwatch watch;
size_t bytes_written = 0;
while (bytes_written != offset())
{
@ -59,6 +63,7 @@ void WriteBufferFromFileDescriptor::nextImpl()
bytes_written += res;
}
ProfileEvents::increment(ProfileEvents::DiskWriteElapsedMicroseconds, watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written);
}

View File

@ -4,6 +4,13 @@
#include <IO/WriteBufferFromPocoSocket.h>
#include <Common/NetException.h>
#include <Common/Stopwatch.h>
namespace ProfileEvents
{
extern const Event NetworkSendElapsedMicroseconds;
}
namespace DB
@ -22,6 +29,8 @@ void WriteBufferFromPocoSocket::nextImpl()
if (!offset())
return;
Stopwatch watch;
size_t bytes_written = 0;
while (bytes_written < offset())
{
@ -47,8 +56,11 @@ void WriteBufferFromPocoSocket::nextImpl()
if (res < 0)
throw NetException("Cannot write to socket (" + peer_address.toString() + ")", ErrorCodes::CANNOT_WRITE_TO_SOCKET);
bytes_written += res;
}
ProfileEvents::increment(ProfileEvents::NetworkSendElapsedMicroseconds, watch.elapsedMicroseconds());
}
WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size)

View File

@ -19,6 +19,7 @@
#include <Interpreters/Aggregator.h>
#include <Common/ClickHouseRevision.h>
#include <Common/MemoryTracker.h>
#include <Common/CurrentThread.h>
#include <Common/typeid_cast.h>
#include <common/demangle.h>
#if __has_include(<Interpreters/config_compile.h>)
@ -139,8 +140,9 @@ Aggregator::Aggregator(const Params & params_)
: params(params_),
isCancelled([]() { return false; })
{
if (current_memory_tracker)
memory_usage_before_aggregation = current_memory_tracker->get();
/// Use query-level memory tracker
if (auto memory_tracker = CurrentThread::getMemoryTracker().getParent())
memory_usage_before_aggregation = memory_tracker->get();
aggregate_functions.resize(params.aggregates_size);
for (size_t i = 0; i < params.aggregates_size; ++i)
@ -804,8 +806,8 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
size_t result_size = result.sizeWithoutOverflowRow();
Int64 current_memory_usage = 0;
if (current_memory_tracker)
current_memory_usage = current_memory_tracker->get();
if (auto memory_tracker = CurrentThread::getMemoryTracker().getParent())
current_memory_usage = memory_tracker->get();
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads.
@ -1278,9 +1280,9 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
bool final,
ThreadPool * thread_pool) const
{
auto converter = [&](size_t bucket, MemoryTracker * memory_tracker)
auto converter = [&](size_t bucket, ThreadGroupStatusPtr thread_group)
{
current_memory_tracker = memory_tracker;
CurrentThread::attachToIfDetached(thread_group);
return convertOneBucketToBlock(data_variants, method, final, bucket);
};
@ -1295,7 +1297,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
if (method.data.impls[bucket].empty())
continue;
tasks[bucket] = std::packaged_task<Block()>(std::bind(converter, bucket, current_memory_tracker));
tasks[bucket] = std::packaged_task<Block()>(std::bind(converter, bucket, CurrentThread::getGroup()));
if (thread_pool)
thread_pool->schedule([bucket, &tasks] { tasks[bucket](); });
@ -1725,17 +1727,17 @@ private:
return;
parallel_merge_data->pool.schedule(std::bind(&MergingAndConvertingBlockInputStream::thread, this,
max_scheduled_bucket_num, current_memory_tracker));
max_scheduled_bucket_num, CurrentThread::getGroup()));
}
void thread(Int32 bucket_num, MemoryTracker * memory_tracker)
void thread(Int32 bucket_num, ThreadGroupStatusPtr thread_group)
{
current_memory_tracker = memory_tracker;
setThreadName("MergingAggregtd");
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
try
{
setThreadName("MergingAggregtd");
CurrentThread::attachToIfDetached(thread_group);
CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread};
/// TODO: add no_more_keys support maybe
auto & merged_data = *data[0];
@ -2035,9 +2037,9 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
LOG_TRACE(log, "Merging partially aggregated two-level data.");
auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, MemoryTracker * memory_tracker)
auto merge_bucket = [&bucket_to_blocks, &result, this](Int32 bucket, Arena * aggregates_pool, ThreadGroupStatusPtr thread_group)
{
current_memory_tracker = memory_tracker;
CurrentThread::attachToIfDetached(thread_group);
for (Block & block : bucket_to_blocks[bucket])
{
@ -2070,7 +2072,7 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
result.aggregates_pools.push_back(std::make_shared<Arena>());
Arena * aggregates_pool = result.aggregates_pools.back().get();
auto task = std::bind(merge_bucket, bucket, aggregates_pool, current_memory_tracker);
auto task = std::bind(merge_bucket, bucket, aggregates_pool, CurrentThread::getGroup());
if (thread_pool)
thread_pool->schedule(task);

View File

@ -39,6 +39,7 @@
#include <Interpreters/Compiler.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/Context.h>
#include <Common/DNSResolver.h>
@ -1529,7 +1530,7 @@ void Context::initializeSystemLogs()
}
QueryLog * Context::getQueryLog()
QueryLog * Context::getQueryLog(bool create_if_not_exists)
{
auto lock = getLock();
@ -1538,29 +1539,49 @@ QueryLog * Context::getQueryLog()
if (!system_logs->query_log)
{
if (!create_if_not_exists)
return nullptr;
if (shared->shutdown_called)
throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
if (!global_context)
throw Exception("Logical error: no global context for query log", ErrorCodes::LOGICAL_ERROR);
auto & config = getConfigRef();
String database = config.getString("query_log.database", "system");
String table = config.getString("query_log.table", "query_log");
String partition_by = config.getString("query_log.partition_by", "toYYYYMM(event_date)");
size_t flush_interval_milliseconds = config.getUInt64("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024";
system_logs->query_log = std::make_unique<QueryLog>(*global_context, database, table, engine, flush_interval_milliseconds);
system_logs->query_log = createDefaultSystemLog<QueryLog>(*global_context, "system", "query_log", getConfigRef(), "query_log");
}
return system_logs->query_log.get();
}
PartLog * Context::getPartLog(const String & part_database)
QueryThreadLog * Context::getQueryThreadLog(bool create_if_not_exists)
{
auto lock = getLock();
if (!system_logs)
return nullptr;
if (!system_logs->query_thread_log)
{
if (!create_if_not_exists)
return nullptr;
if (shared->shutdown_called)
throw Exception("Logical error: query log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
if (!global_context)
throw Exception("Logical error: no global context for query thread log", ErrorCodes::LOGICAL_ERROR);
system_logs->query_thread_log = createDefaultSystemLog<QueryThreadLog>(
*global_context, "system", "query_thread_log", getConfigRef(), "query_thread_log");
}
return system_logs->query_thread_log.get();
}
PartLog * Context::getPartLog(const String & part_database, bool create_if_not_exists)
{
auto lock = getLock();
@ -1577,24 +1598,21 @@ PartLog * Context::getPartLog(const String & part_database)
/// Will not log operations on system tables (including part_log itself).
/// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing,
/// and also make troubles on startup.
if (part_database == database)
if (!part_database.empty() && part_database == database)
return nullptr;
if (!system_logs->part_log)
{
if (!create_if_not_exists)
return nullptr;
if (shared->shutdown_called)
throw Exception("Logical error: part log should be destroyed before tables shutdown", ErrorCodes::LOGICAL_ERROR);
if (!global_context)
throw Exception("Logical error: no global context for part log", ErrorCodes::LOGICAL_ERROR);
String table = config.getString("part_log.table", "part_log");
String partition_by = config.getString("query_log.partition_by", "toYYYYMM(event_date)");
size_t flush_interval_milliseconds = config.getUInt64("part_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024";
system_logs->part_log = std::make_unique<PartLog>(*global_context, database, table, engine, flush_interval_milliseconds);
system_logs->part_log = createDefaultSystemLog<PartLog>(*global_context, "system", "part_log", getConfigRef(), "part_log");
}
return system_logs->part_log.get();
@ -1801,6 +1819,25 @@ std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
}
void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer)
{
if (external_tables_initializer_callback)
throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR);
external_tables_initializer_callback = std::move(initializer);
}
void Context::initializeExternalTablesIfSet()
{
if (external_tables_initializer_callback)
{
external_tables_initializer_callback(*this);
/// Reset callback
external_tables_initializer_callback = {};
}
}
SessionCleaner::~SessionCleaner()
{
try

View File

@ -35,6 +35,7 @@ namespace DB
{
struct ContextShared;
class Context;
class IRuntimeComponentsFactory;
class QuotaForIntervals;
class EmbeddedDictionaries;
@ -49,11 +50,12 @@ class Compiler;
class MarkCache;
class UncompressedCache;
class ProcessList;
class ProcessListElement;
class QueryStatus;
class Macros;
struct Progress;
class Clusters;
class QueryLog;
class QueryThreadLog;
class PartLog;
struct MergeTreeSettings;
class IDatabase;
@ -86,6 +88,9 @@ using Dependencies = std::vector<DatabaseAndTableName>;
using TableAndCreateAST = std::pair<StoragePtr, ASTPtr>;
using TableAndCreateASTs = std::map<String, TableAndCreateAST>;
/// Callback for external tables initializer
using ExternalTablesInitializer = std::function<void(Context &)>;
/** A set of known objects that can be used in the query.
* Consists of a shared part (always common to all sessions and queries)
* and copied part (which can be its own for each session or query).
@ -101,13 +106,14 @@ private:
std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory;
ClientInfo client_info;
ExternalTablesInitializer external_tables_initializer_callback;
std::shared_ptr<QuotaForIntervals> quota; /// Current quota. By default - empty quota, that have no limits.
String current_database;
Settings settings; /// Setting for query execution.
using ProgressCallback = std::function<void(const Progress & progress)>;
ProgressCallback progress_callback; /// Callback for tracking progress of query execution.
ProcessListElement * process_list_elem = nullptr; /// For tracking total resource usage for query.
QueryStatus * process_list_elem = nullptr; /// For tracking total resource usage for query.
String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification.
/// Thus, used in HTTP interface. If not specified - then some globally default format is used.
@ -166,6 +172,11 @@ public:
/// Compute and set actual user settings, client_info.current_user should be set
void calculateUserSettings();
/// We have to copy external tables inside executeQuery() to track limits. Therefore, set callback for it. Must set once.
void setExternalTablesInitializer(ExternalTablesInitializer && initializer);
/// This method is called in executeQuery() and will call the external tables initializer.
void initializeExternalTablesIfSet();
ClientInfo & getClientInfo() { return client_info; }
const ClientInfo & getClientInfo() const { return client_info; }
@ -311,9 +322,9 @@ public:
/** Set in executeQuery and InterpreterSelectQuery. Then it is used in IProfilingBlockInputStream,
* to update and monitor information about the total number of resources spent for the query.
*/
void setProcessListElement(ProcessListElement * elem);
void setProcessListElement(QueryStatus * elem);
/// Can return nullptr if the query was not inserted into the ProcessList.
ProcessListElement * getProcessListElement() const;
QueryStatus * getProcessListElement() const;
/// List all queries.
ProcessList & getProcessList();
@ -366,11 +377,12 @@ public:
void initializeSystemLogs();
/// Nullptr if the query log is not ready for this moment.
QueryLog * getQueryLog();
QueryLog * getQueryLog(bool create_if_not_exists = true);
QueryThreadLog * getQueryThreadLog(bool create_if_not_exists = true);
/// Returns an object used to log opertaions with parts if it possible.
/// Provide table name to make required cheks.
PartLog * getPartLog(const String & part_database);
PartLog * getPartLog(const String & part_database, bool create_if_not_exists = true);
const MergeTreeSettings & getMergeTreeSettings() const;

View File

@ -64,7 +64,7 @@ DNSCacheUpdater::DNSCacheUpdater(Context & context_)
bool DNSCacheUpdater::run()
{
/// TODO: Ensusre that we get global counter (not thread local)
auto num_current_network_exceptions = ProfileEvents::counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed);
auto num_current_network_exceptions = ProfileEvents::global_counters[ProfileEvents::NetworkErrors].load(std::memory_order_relaxed);
if (num_current_network_exceptions >= last_num_network_erros + min_errors_to_update_cache
&& time(nullptr) > last_update_time + min_update_period_seconds)

View File

@ -66,7 +66,7 @@ void ExternalLoader::init(bool throw_on_error)
{
/// During synchronous loading of external dictionaries at moment of query execution,
/// we should not use per query memory limit.
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
reloadAndUpdate(throw_on_error);
}

View File

@ -0,0 +1,68 @@
#include "InternalTextLogsQueue.h"
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypesNumber.h>
#include <common/logger_useful.h>
#include <Poco/Message.h>
namespace DB
{
InternalTextLogsQueue::InternalTextLogsQueue()
: ConcurrentBoundedQueue<MutableColumns>(std::numeric_limits<int>::max()),
max_priority(Poco::Message::Priority::PRIO_INFORMATION) {}
Block InternalTextLogsQueue::getSampleBlock()
{
return Block {
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeUInt32>(), "event_time_microseconds"},
{std::make_shared<DataTypeString>(), "host_name"},
{std::make_shared<DataTypeString>(), "query_id"},
{std::make_shared<DataTypeUInt32>(), "thread_number"},
{std::make_shared<DataTypeInt8>(), "priority"},
{std::make_shared<DataTypeString>(), "source"},
{std::make_shared<DataTypeString>(), "text"}
};
}
MutableColumns InternalTextLogsQueue::getSampleColumns()
{
static Block sample_block = getSampleBlock();
return sample_block.cloneEmptyColumns();
}
void InternalTextLogsQueue::pushBlock(Block && log_block)
{
static Block sample_block = getSampleBlock();
if (blocksHaveEqualStructure(sample_block, log_block))
emplace(log_block.mutateColumns());
else
LOG_WARNING(&Poco::Logger::get("InternalTextLogsQueue"), "Log block have different structure");
}
const char * InternalTextLogsQueue::getPriorityName(int priority)
{
/// See Poco::Message::Priority
static const char * PRIORITIES [] = {
"Unknown",
"Fatal",
"Critical",
"Error",
"Warning",
"Notice",
"Information",
"Debug",
"Trace"
};
return (priority >= 1 && priority <= 8) ? PRIORITIES[priority] : PRIORITIES[0];
}
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Common/ConcurrentBoundedQueue.h>
#include <Core/Block.h>
namespace DB
{
class InternalTextLogsQueue : public ConcurrentBoundedQueue<MutableColumns>
{
public:
/// You should not push logs in the queue if their priority greater max_priority
int max_priority;
InternalTextLogsQueue();
static Block getSampleBlock();
static MutableColumns getSampleColumns();
/// Is used to pass block from remote server to the client
void pushBlock(Block && log_block);
/// Converts priority from Poco::Message::Priority to a string
static const char * getPriorityName(int priority);
};
using InternalTextLogsQueuePtr = std::shared_ptr<InternalTextLogsQueue>;
}

View File

@ -10,6 +10,9 @@
#include <Interpreters/ActionLocksManager.h>
#include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/InterpreterCreateQuery.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Databases/IDatabase.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/StorageFactory.h>
@ -48,7 +51,7 @@ ExecutionStatus getOverallExecutionStatusOfCommands()
return ExecutionStatus(0);
}
/// Consequently execute all commands and genreates final exception message for failed commands
/// Consequently tries to execute all commands and genreates final exception message for failed commands
template <typename Callable, typename ... Callables>
ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callables && ... commands)
{
@ -70,6 +73,16 @@ ExecutionStatus getOverallExecutionStatusOfCommands(Callable && command, Callabl
return ExecutionStatus(res_status, res_message);
}
/// Consequently tries to execute all commands and throws exception with info about failed commands
template <typename ... Callables>
void executeCommandsAndThrowIfError(Callables && ... commands)
{
auto status = getOverallExecutionStatusOfCommands(std::forward<Callables>(commands)...);
if (status.code != 0)
throw Exception(status.message, status.code);
}
/// Implements SYSTEM [START|STOP] <something action from ActionLocks>
void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlockType action_type, bool start)
{
@ -139,15 +152,11 @@ BlockIO InterpreterSystemQuery::execute()
system_context.getExternalDictionaries().reloadDictionary(query.target_dictionary);
break;
case Type::RELOAD_DICTIONARIES:
{
auto status = getOverallExecutionStatusOfCommands(
[&] { system_context.getExternalDictionaries().reload(); },
[&] { system_context.getEmbeddedDictionaries().reload(); }
executeCommandsAndThrowIfError(
[&] () { system_context.getExternalDictionaries().reload(); },
[&] () { system_context.getEmbeddedDictionaries().reload(); }
);
if (status.code != 0)
throw Exception(status.message, status.code);
break;
}
case Type::RELOAD_EMBEDDED_DICTIONARIES:
system_context.getEmbeddedDictionaries().reload();
break;
@ -189,6 +198,13 @@ BlockIO InterpreterSystemQuery::execute()
throw Exception("There is no " + query.target_database + "." + query.target_table + " replicated table",
ErrorCodes::BAD_ARGUMENTS);
break;
case Type::FLUSH_SYSTEM_TABLES:
executeCommandsAndThrowIfError(
[&] () { if (auto query_log = context.getQueryLog(false)) query_log->flush(); },
[&] () { if (auto part_log = context.getPartLog("", false)) part_log->flush(); },
[&] () { if (auto query_thread_log = context.getQueryThreadLog(false)) query_thread_log->flush(); }
);
break;
case Type::STOP_LISTEN_QUERIES:
case Type::START_LISTEN_QUERIES:
throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED);

View File

@ -1,18 +1,20 @@
#include <Interpreters/ProcessList.h>
#include <Interpreters/Settings.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTKillQueryQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Common/Exception.h>
#include <Common/CurrentThread.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/typeid_cast.h>
#include <common/logger_useful.h>
#include <chrono>
namespace DB
{
@ -72,11 +74,13 @@ static bool isUnlimitedQuery(const IAST * ast)
}
ProcessList::EntryPtr ProcessList::insert(
const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings)
ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * ast, Context & query_context)
{
EntryPtr res;
const ClientInfo & client_info = query_context.getClientInfo();
const Settings & settings = query_context.getSettingsRef();
if (client_info.current_query_id.empty())
throw Exception("Query id cannot be empty", ErrorCodes::LOGICAL_ERROR);
@ -85,11 +89,11 @@ ProcessList::EntryPtr ProcessList::insert(
{
std::unique_lock lock(mutex);
if (!is_unlimited_query && max_size && cur_size >= max_size)
if (!is_unlimited_query && max_size && processes.size() >= max_size)
{
auto max_wait_ms = settings.queue_max_wait_ms.totalMilliseconds();
if (!max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(max_wait_ms), [&]{ return cur_size < max_size; }))
if (!max_wait_ms || !have_space.wait_for(lock, std::chrono::milliseconds(max_wait_ms), [&]{ return processes.size() < max_size; }))
throw Exception("Too many simultaneous queries. Maximum: " + toString(max_size), ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES);
}
@ -129,50 +133,69 @@ ProcessList::EntryPtr ProcessList::insert(
}
}
++cur_size;
auto process_it = processes.emplace(processes.end(),
query_, client_info, settings.max_memory_usage, settings.memory_tracker_fault_probability, priorities.insert(settings.priority));
res = std::make_shared<Entry>(*this, cont.emplace(cont.end(),
query_, client_info,
settings.max_memory_usage, settings.memory_tracker_fault_probability,
priorities.insert(settings.priority)));
res = std::make_shared<Entry>(*this, process_it);
ProcessListForUser & user_process_list = user_to_queries[client_info.current_user];
user_process_list.queries.emplace(client_info.current_query_id, &res->get());
process_it->query_context = &query_context;
if (current_memory_tracker)
if (!client_info.current_query_id.empty())
{
ProcessListForUser & user_process_list = user_to_queries[client_info.current_user];
user_process_list.queries.emplace(client_info.current_query_id, &res->get());
process_it->setUserProcessList(&user_process_list);
/// Limits are only raised (to be more relaxed) or set to something instead of zero,
/// because settings for different queries will interfere each other:
/// setting from one query effectively sets values for all other queries.
/// Track memory usage for all simultaneously running queries from single user.
user_process_list.user_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
current_memory_tracker->setNext(&user_process_list.user_memory_tracker);
/// Track memory usage for all simultaneously running queries.
/// You should specify this value in configuration for default profile,
/// not for specific users, sessions or queries,
/// because this setting is effectively global.
total_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_all_queries);
total_memory_tracker.setDescription("(total)");
user_process_list.user_memory_tracker.setNext(&total_memory_tracker);
/// Track memory usage for all simultaneously running queries from single user.
user_process_list.user_memory_tracker.setParent(&total_memory_tracker);
user_process_list.user_memory_tracker.setOrRaiseLimit(settings.max_memory_usage_for_user);
user_process_list.user_memory_tracker.setDescription("(for user)");
/// Actualize thread group info
if (auto thread_group = CurrentThread::getGroup())
{
std::unique_lock lock_thread_group(thread_group->mutex);
thread_group->performance_counters.setParent(&user_process_list.user_performance_counters);
thread_group->memory_tracker.setParent(&user_process_list.user_memory_tracker);
thread_group->query = process_it->query;
/// Set query-level memory trackers
thread_group->memory_tracker.setOrRaiseLimit(process_it->max_memory_usage);
thread_group->memory_tracker.setDescription("(for query)");
if (process_it->memory_tracker_fault_probability)
thread_group->memory_tracker.setFaultProbability(process_it->memory_tracker_fault_probability);
/// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values
/// since allocation and deallocation could happen in different threads
process_it->thread_group = std::move(thread_group);
}
if (!user_process_list.user_throttler)
{
if (settings.max_network_bandwidth_for_user)
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, total_network_throttler);
else if (settings.max_network_bandwidth_for_all_users)
user_process_list.user_throttler = total_network_throttler;
}
}
if (!total_network_throttler && settings.max_network_bandwidth_for_all_users)
{
total_network_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_all_users);
}
if (!user_process_list.user_throttler)
{
if (settings.max_network_bandwidth_for_user)
user_process_list.user_throttler = std::make_shared<Throttler>(settings.max_network_bandwidth_for_user, total_network_throttler);
else if (settings.max_network_bandwidth_for_all_users)
user_process_list.user_throttler = total_network_throttler;
}
res->get().user_process_list = &user_process_list;
}
return res;
@ -189,10 +212,10 @@ ProcessListEntry::~ProcessListEntry()
String user = it->getClientInfo().current_user;
String query_id = it->getClientInfo().current_query_id;
const ProcessListElement * process_list_element_ptr = &*it;
const QueryStatus * process_list_element_ptr = &*it;
/// This removes the memory_tracker of one query.
parent.cont.erase(it);
/// This removes the memory_tracker of one request.
parent.processes.erase(it);
auto user_process_list_it = parent.user_to_queries.find(user);
if (user_process_list_it == parent.user_to_queries.end())
@ -225,15 +248,14 @@ ProcessListEntry::~ProcessListEntry()
std::terminate();
}
/// If there are no more queries for the user, then we will reset memory tracker and network throttler.
if (user_process_list.queries.empty())
user_process_list.reset();
--parent.cur_size;
parent.have_space.notify_one();
/// If there are no more queries for the user, then we will reset memory tracker and network throttler.
if (user_process_list.queries.empty())
user_process_list.resetTrackers();
/// This removes memory_tracker for all requests. At this time, no other memory_trackers live.
if (parent.cur_size == 0)
if (parent.processes.size() == 0)
{
/// Reset MemoryTracker, similarly (see above).
parent.total_memory_tracker.logPeakMemoryUsage();
@ -243,7 +265,25 @@ ProcessListEntry::~ProcessListEntry()
}
void ProcessListElement::setQueryStreams(const BlockIO & io)
QueryStatus::QueryStatus(
const String & query_,
const ClientInfo & client_info_,
size_t max_memory_usage_,
double memory_tracker_fault_probability_,
QueryPriorities::Handle && priority_handle_)
:
query(query_),
client_info(client_info_),
priority_handle(std::move(priority_handle_)),
num_queries_increment{CurrentMetrics::Query},
max_memory_usage(max_memory_usage_),
memory_tracker_fault_probability(memory_tracker_fault_probability_)
{
}
QueryStatus::~QueryStatus() = default;
void QueryStatus::setQueryStreams(const BlockIO & io)
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
@ -252,7 +292,7 @@ void ProcessListElement::setQueryStreams(const BlockIO & io)
query_streams_status = QueryStreamsStatus::Initialized;
}
void ProcessListElement::releaseQueryStreams()
void QueryStatus::releaseQueryStreams()
{
BlockInputStreamPtr in;
BlockOutputStreamPtr out;
@ -268,14 +308,14 @@ void ProcessListElement::releaseQueryStreams()
/// Destroy streams outside the mutex lock
}
bool ProcessListElement::streamsAreReleased()
bool QueryStatus::streamsAreReleased()
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
return query_streams_status == QueryStreamsStatus::Released;
}
bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const
bool QueryStatus::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutputStreamPtr & out) const
{
std::lock_guard<std::mutex> lock(query_streams_mutex);
@ -288,7 +328,13 @@ bool ProcessListElement::tryGetQueryStreams(BlockInputStreamPtr & in, BlockOutpu
}
ThrottlerPtr ProcessListElement::getUserNetworkThrottler()
void QueryStatus::setUserProcessList(ProcessListForUser * user_process_list_)
{
user_process_list = user_process_list_;
}
ThrottlerPtr QueryStatus::getUserNetworkThrottler()
{
if (!user_process_list)
return {};
@ -296,7 +342,7 @@ ThrottlerPtr ProcessListElement::getUserNetworkThrottler()
}
ProcessListElement * ProcessList::tryGetProcessListElement(const String & current_query_id, const String & current_user)
QueryStatus * ProcessList::tryGetProcessListElement(const String & current_query_id, const String & current_user)
{
auto user_it = user_to_queries.find(current_user);
if (user_it != user_to_queries.end())
@ -316,7 +362,7 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr
{
std::lock_guard<std::mutex> lock(mutex);
ProcessListElement * elem = tryGetProcessListElement(current_query_id, current_user);
QueryStatus * elem = tryGetProcessListElement(current_query_id, current_user);
if (!elem)
return CancellationCode::NotFound;
@ -342,4 +388,61 @@ ProcessList::CancellationCode ProcessList::sendCancelToQuery(const String & curr
return CancellationCode::QueryIsNotInitializedYet;
}
QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
QueryStatusInfo res;
res.query = query;
res.client_info = client_info;
res.elapsed_seconds = watch.elapsedSeconds();
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
res.read_rows = progress_in.rows;
res.read_bytes = progress_in.bytes;
res.total_rows = progress_in.total_rows;
res.written_rows = progress_out.rows;
res.written_bytes = progress_out.bytes;
if (thread_group)
{
res.memory_usage = thread_group->memory_tracker.get();
res.peak_memory_usage = thread_group->memory_tracker.getPeak();
if (get_thread_list)
{
std::shared_lock lock(thread_group->mutex);
res.thread_numbers.reserve(thread_group->thread_statuses.size());
for (auto & thread_status_elem : thread_group->thread_statuses)
res.thread_numbers.emplace_back(thread_status_elem.second->thread_number);
}
if (get_profile_events)
res.profile_counters = std::make_shared<ProfileEvents::Counters>(thread_group->performance_counters.getPartiallyAtomicSnapshot());
}
if (get_settings && query_context)
res.query_settings = std::make_shared<Settings>(query_context->getSettingsRef());
return res;
}
ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
Info per_query_infos;
std::lock_guard<std::mutex> lock(mutex);
per_query_infos.reserve(processes.size());
for (const auto & process : processes)
per_query_infos.emplace_back(process.getInfo(get_thread_list, get_profile_events, get_settings));
return per_query_infos;
}
ProcessListForUser::ProcessListForUser() = default;
}

View File

@ -6,15 +6,21 @@
#include <mutex>
#include <condition_variable>
#include <unordered_map>
#include <shared_mutex>
#include <Poco/Condition.h>
#include <Common/Stopwatch.h>
#include <Core/Defines.h>
#include <IO/Progress.h>
#include <Common/Stopwatch.h>
#include <Common/MemoryTracker.h>
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
#include <Common/Throttler.h>
#include <Common/CurrentThread.h>
#include <Interpreters/QueryPriorities.h>
#include <Interpreters/ClientInfo.h>
#include <Common/CurrentMetrics.h>
#include <Common/ThreadStatus.h>
#include <DataStreams/BlockIO.h>
#include <Common/Throttler.h>
namespace CurrentMetrics
@ -28,9 +34,14 @@ namespace DB
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
using Tables = std::map<String, StoragePtr>;
class Context;
struct Settings;
class IAST;
struct ProcessListForUser;
class QueryStatus;
class ThreadStatus;
class ProcessListEntry;
/** List of currently executing queries.
@ -40,7 +51,7 @@ struct ProcessListForUser;
/** Information of process list element.
* To output in SHOW PROCESSLIST query. Does not contain any complex objects, that do something on copy or destructor.
*/
struct ProcessInfo
struct QueryStatusInfo
{
String query;
double elapsed_seconds;
@ -53,18 +64,32 @@ struct ProcessInfo
Int64 peak_memory_usage;
ClientInfo client_info;
bool is_cancelled;
/// Optional fields, filled by request
std::vector<UInt32> thread_numbers;
std::shared_ptr<ProfileEvents::Counters> profile_counters;
std::shared_ptr<Settings> query_settings;
};
/// Query and information about its execution.
class ProcessListElement
class QueryStatus
{
protected:
friend class ProcessList;
friend class ThreadStatus;
friend class CurrentThread;
friend class ProcessListEntry;
private:
String query;
ClientInfo client_info;
/// Is set once when init
Context * query_context = nullptr;
/// Info about all threads involved in query execution
ThreadGroupStatusPtr thread_group;
Stopwatch watch;
/// Progress of input stream
@ -72,16 +97,18 @@ private:
/// Progress of output stream
Progress progress_out;
MemoryTracker memory_tracker;
QueryPriorities::Handle priority_handle;
CurrentMetrics::Increment num_queries {CurrentMetrics::Query};
CurrentMetrics::Increment num_queries_increment{CurrentMetrics::Query};
size_t max_memory_usage = 0;
double memory_tracker_fault_probability = 0.0;
std::atomic<bool> is_killed { false };
/// Be careful using it. For example, queries field could be modified concurrently.
const ProcessListForUser * user_process_list = nullptr;
void setUserProcessList(ProcessListForUser * user_process_list_);
/// Be careful using it. For example, queries field of ProcessListForUser could be modified concurrently.
const ProcessListForUser * getUserProcessList() const { return user_process_list; }
mutable std::mutex query_streams_mutex;
@ -100,27 +127,18 @@ private:
QueryStreamsStatus query_streams_status{NotInitialized};
ProcessListForUser * user_process_list = nullptr;
public:
ProcessListElement(
QueryStatus(
const String & query_,
const ClientInfo & client_info_,
size_t max_memory_usage,
double memory_tracker_fault_probability,
QueryPriorities::Handle && priority_handle_)
: query(query_), client_info(client_info_), memory_tracker(max_memory_usage),
priority_handle(std::move(priority_handle_))
{
memory_tracker.setDescription("(for query)");
current_memory_tracker = &memory_tracker;
QueryPriorities::Handle && priority_handle_);
if (memory_tracker_fault_probability)
memory_tracker.setFaultProbability(memory_tracker_fault_probability);
}
~ProcessListElement()
{
current_memory_tracker = nullptr;
}
~QueryStatus();
const ClientInfo & getClientInfo() const
{
@ -141,6 +159,7 @@ public:
bool updateProgressIn(const Progress & value)
{
CurrentThread::updateProgressIn(value);
progress_in.incrementPiecewiseAtomically(value);
if (priority_handle)
@ -151,29 +170,16 @@ public:
bool updateProgressOut(const Progress & value)
{
CurrentThread::updateProgressOut(value);
progress_out.incrementPiecewiseAtomically(value);
return !is_killed.load(std::memory_order_relaxed);
}
QueryStatusInfo getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
ProcessInfo getInfo() const
{
ProcessInfo res;
res.query = query;
res.client_info = client_info;
res.elapsed_seconds = watch.elapsedSeconds();
res.is_cancelled = is_killed.load(std::memory_order_relaxed);
res.read_rows = progress_in.rows;
res.read_bytes = progress_in.bytes;
res.total_rows = progress_in.total_rows;
res.written_rows = progress_out.rows;
res.written_bytes = progress_out.bytes;
res.memory_usage = memory_tracker.get();
res.peak_memory_usage = memory_tracker.getPeak();
return res;
}
Context * tryGetQueryContext() { return query_context; }
const Context * tryGetQueryContext() const { return query_context; }
/// Copies pointers to in/out streams
void setQueryStreams(const BlockIO & io);
@ -192,12 +198,15 @@ public:
/// Data about queries for one user.
struct ProcessListForUser
{
ProcessListForUser();
/// query_id -> ProcessListElement(s). There can be multiple queries with the same query_id as long as all queries except one are cancelled.
using QueryToElement = std::unordered_multimap<String, ProcessListElement *>;
using QueryToElement = std::unordered_multimap<String, QueryStatus *>;
QueryToElement queries;
ProfileEvents::Counters user_performance_counters{VariableContext::User, &ProfileEvents::global_counters};
/// Limit and counter for memory of all simultaneously running queries of single user.
MemoryTracker user_memory_tracker;
MemoryTracker user_memory_tracker{VariableContext::User};
/// Count network usage for all simultaneously running queries of single user.
ThrottlerPtr user_throttler;
@ -206,7 +215,7 @@ struct ProcessListForUser
/// Sometimes it is important to reset the MemoryTracker, because it may accumulate skew
/// due to the fact that there are cases when memory can be allocated while processing the query, but released later.
/// Clears network bandwidth Throttler, so it will not count periods of inactivity.
void reset()
void resetTrackers()
{
user_memory_tracker.reset();
if (user_throttler)
@ -222,44 +231,45 @@ class ProcessList;
class ProcessListEntry
{
private:
using Container = std::list<ProcessListElement>;
using Container = std::list<QueryStatus>;
ProcessList & parent;
Container::iterator it;
public:
ProcessListEntry(ProcessList & parent_, Container::iterator it_)
: parent(parent_), it(it_) {}
~ProcessListEntry();
ProcessListElement * operator->() { return &*it; }
const ProcessListElement * operator->() const { return &*it; }
QueryStatus * operator->() { return &*it; }
const QueryStatus * operator->() const { return &*it; }
ProcessListElement & get() { return *it; }
const ProcessListElement & get() const { return *it; }
QueryStatus & get() { return *it; }
const QueryStatus & get() const { return *it; }
};
class ProcessList
{
friend class ProcessListEntry;
public:
using Element = ProcessListElement;
using Element = QueryStatus;
using Entry = ProcessListEntry;
/// list, for iterators not to invalidate. NOTE: could replace with cyclic buffer, but not worth.
using Container = std::list<Element>;
using Info = std::vector<ProcessInfo>;
using Info = std::vector<QueryStatusInfo>;
/// User -> queries
using UserToQueries = std::unordered_map<String, ProcessListForUser>;
private:
protected:
friend class ProcessListEntry;
mutable std::mutex mutex;
mutable std::condition_variable have_space; /// Number of currently running queries has become less than maximum.
/// List of queries
Container cont;
size_t cur_size; /// In C++03 or C++11 and old ABI, std::list::size is not O(1).
Container processes;
size_t max_size; /// 0 means no limit. Otherwise, when limit exceeded, an exception is thrown.
/// Stores per-user info: queries, statistics and limits
@ -269,16 +279,16 @@ private:
QueryPriorities priorities;
/// Limit and counter for memory of all simultaneously running queries.
MemoryTracker total_memory_tracker;
MemoryTracker total_memory_tracker{VariableContext::Global};
/// Limit network bandwidth for all users
ThrottlerPtr total_network_throttler;
/// Call under lock. Finds process with specified current_user and current_query_id.
ProcessListElement * tryGetProcessListElement(const String & current_query_id, const String & current_user);
QueryStatus * tryGetProcessListElement(const String & current_query_id, const String & current_user);
public:
ProcessList(size_t max_size_ = 0) : cur_size(0), max_size(max_size_) {}
ProcessList(size_t max_size_ = 0) : max_size(max_size_) {}
using EntryPtr = std::shared_ptr<ProcessListEntry>;
@ -287,23 +297,13 @@ public:
* If timeout is passed - throw an exception.
* Don't count KILL QUERY queries.
*/
EntryPtr insert(const String & query_, const IAST * ast, const ClientInfo & client_info, const Settings & settings);
EntryPtr insert(const String & query_, const IAST * ast, Context & query_context);
/// Number of currently executing queries.
size_t size() const { return cur_size; }
size_t size() const { return processes.size(); }
/// Get current state of process list.
Info getInfo() const
{
std::lock_guard<std::mutex> lock(mutex);
Info res;
res.reserve(cur_size);
for (const auto & elem : cont)
res.emplace_back(elem.getInfo());
return res;
}
Info getInfo(bool get_thread_list = false, bool get_profile_events = false, bool get_settings = false) const;
void setMaxSize(size_t max_size_)
{

View File

@ -0,0 +1,59 @@
#include "ProfileEventsExt.h"
#include <Common/typeid_cast.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
namespace ProfileEvents
{
/// Put implementation here to avoid extra linking dependencies for clickhouse_common_io
void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names_, DB::IColumn * column_values_, bool nonzero_only)
{
/// Convert ptr and make simple check
auto column_names = (column_names_) ? &typeid_cast<DB::ColumnArray &>(*column_names_) : nullptr;
auto column_values = (column_values_) ? &typeid_cast<DB::ColumnArray &>(*column_values_) : nullptr;
size_t size = 0;
for (Event event = 0; event < Counters::num_counters; ++event)
{
UInt64 value = counters[event].load(std::memory_order_relaxed);
if (nonzero_only && 0 == value)
continue;
++size;
if (column_names)
{
const char * desc = ProfileEvents::getDescription(event);
column_names->getData().insertData(desc, strlen(desc));
}
if (column_values)
column_values->getData().insert(value);
}
if (column_names)
{
auto & offsets = column_names->getOffsets();
offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size);
}
if (column_values)
{
/// Nested columns case
bool the_same_offsets = column_names && column_names->getOffsetsPtr().get() == column_values->getOffsetsPtr().get();
if (!the_same_offsets)
{
auto & offsets = column_values->getOffsets();
offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size);
}
}
}
}

View File

@ -0,0 +1,12 @@
#pragma once
#include <Common/ProfileEvents.h>
#include <Columns/IColumn.h>
namespace ProfileEvents
{
/// Dumps profile events to two column Array(String) and Array(UInt64)
void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true);
}

View File

@ -1,12 +1,16 @@
#include <Common/ProfileEvents.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeArray.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Common/ClickHouseRevision.h>
#include <Poco/Net/IPAddress.h>
#include <array>
@ -20,55 +24,52 @@ Block QueryLogElement::createBlock()
{
return
{
{ColumnUInt8::create(), std::make_shared<DataTypeUInt8>(), "type"},
{ColumnUInt16::create(), std::make_shared<DataTypeDate>(), "event_date"},
{ColumnUInt32::create(), std::make_shared<DataTypeDateTime>(), "event_time"},
{ColumnUInt32::create(), std::make_shared<DataTypeDateTime>(), "query_start_time"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
{std::make_shared<DataTypeUInt8>(), "type"},
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_rows"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "read_bytes"},
{std::make_shared<DataTypeUInt64>(), "read_rows"},
{std::make_shared<DataTypeUInt64>(), "read_bytes"},
{std::make_shared<DataTypeUInt64>(), "written_rows"},
{std::make_shared<DataTypeUInt64>(), "written_bytes"},
{std::make_shared<DataTypeUInt64>(), "result_rows"},
{std::make_shared<DataTypeUInt64>(), "result_bytes"},
{std::make_shared<DataTypeUInt64>(), "memory_usage"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "written_rows"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "written_bytes"},
{std::make_shared<DataTypeString>(), "query"},
{std::make_shared<DataTypeString>(), "exception"},
{std::make_shared<DataTypeString>(), "stack_trace"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "result_rows"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "result_bytes"},
{std::make_shared<DataTypeUInt8>(), "is_initial_query"},
{std::make_shared<DataTypeString>(), "user"},
{std::make_shared<DataTypeString>(), "query_id"},
{std::make_shared<DataTypeFixedString>(16), "address"},
{std::make_shared<DataTypeUInt16>(), "port"},
{std::make_shared<DataTypeString>(), "initial_user"},
{std::make_shared<DataTypeString>(), "initial_query_id"},
{std::make_shared<DataTypeFixedString>(16), "initial_address"},
{std::make_shared<DataTypeUInt16>(), "initial_port"},
{std::make_shared<DataTypeUInt8>(), "interface"},
{std::make_shared<DataTypeString>(), "os_user"},
{std::make_shared<DataTypeString>(), "client_hostname"},
{std::make_shared<DataTypeString>(), "client_name"},
{std::make_shared<DataTypeUInt32>(), "client_revision"},
{std::make_shared<DataTypeUInt32>(), "client_version_major"},
{std::make_shared<DataTypeUInt32>(), "client_version_minor"},
{std::make_shared<DataTypeUInt32>(), "client_version_patch"},
{std::make_shared<DataTypeUInt8>(), "http_method"},
{std::make_shared<DataTypeString>(), "http_user_agent"},
{std::make_shared<DataTypeString>(), "quota_key"},
{ColumnUInt64::create(), std::make_shared<DataTypeUInt64>(), "memory_usage"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "query"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "exception"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "stack_trace"},
{ColumnUInt8::create(), std::make_shared<DataTypeUInt8>(), "is_initial_query"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "user"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "query_id"},
{ColumnFixedString::create(16), std::make_shared<DataTypeFixedString>(16), "address"},
{ColumnUInt16::create(), std::make_shared<DataTypeUInt16>(), "port"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "initial_user"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "initial_query_id"},
{ColumnFixedString::create(16), std::make_shared<DataTypeFixedString>(16), "initial_address"},
{ColumnUInt16::create(), std::make_shared<DataTypeUInt16>(), "initial_port"},
{ColumnUInt8::create(), std::make_shared<DataTypeUInt8>(), "interface"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "os_user"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "client_hostname"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "client_name"},
{ColumnUInt32::create(), std::make_shared<DataTypeUInt32>(), "client_revision"},
{ColumnUInt32::create(), std::make_shared<DataTypeUInt32>(), "client_version_major"},
{ColumnUInt32::create(), std::make_shared<DataTypeUInt32>(), "client_version_minor"},
{ColumnUInt32::create(), std::make_shared<DataTypeUInt32>(), "client_version_patch"},
{ColumnUInt8::create(), std::make_shared<DataTypeUInt8>(), "http_method"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "http_user_agent"},
{ColumnString::create(), std::make_shared<DataTypeString>(), "quota_key"},
{ColumnUInt32::create(), std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>()), "thread_numbers"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "ProfileEvents.Names"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "ProfileEvents.Values"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Settings.Names"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Settings.Values"}
};
}
@ -110,10 +111,8 @@ void QueryLogElement::appendToBlock(Block & block) const
columns[i++]->insert(UInt64(read_rows));
columns[i++]->insert(UInt64(read_bytes));
columns[i++]->insert(UInt64(written_rows));
columns[i++]->insert(UInt64(written_bytes));
columns[i++]->insert(UInt64(result_rows));
columns[i++]->insert(UInt64(result_bytes));
@ -123,6 +122,47 @@ void QueryLogElement::appendToBlock(Block & block) const
columns[i++]->insertData(exception.data(), exception.size());
columns[i++]->insertData(stack_trace.data(), stack_trace.size());
appendClientInfo(client_info, columns, i);
columns[i++]->insert(UInt64(ClickHouseRevision::get()));
{
Array threads_array;
threads_array.reserve(thread_numbers.size());
for (const UInt32 thread_number : thread_numbers)
threads_array.emplace_back(UInt64(thread_number));
columns[i++]->insert(threads_array);
}
if (profile_counters)
{
auto column_names = columns[i++].get();
auto column_values = columns[i++].get();
ProfileEvents::dumpToArrayColumns(*profile_counters, column_names, column_values, true);
}
else
{
columns[i++]->insertDefault();
columns[i++]->insertDefault();
}
if (query_settings)
{
auto column_names = columns[i++].get();
auto column_values = columns[i++].get();
query_settings->dumpToArrayColumns(column_names, column_values, true);
}
else
{
columns[i++]->insertDefault();
columns[i++]->insertDefault();
}
block.setColumns(std::move(columns));
}
void QueryLogElement::appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i)
{
columns[i++]->insert(UInt64(client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY));
columns[i++]->insert(client_info.current_user);
@ -149,10 +189,5 @@ void QueryLogElement::appendToBlock(Block & block) const
columns[i++]->insert(client_info.http_user_agent);
columns[i++]->insert(client_info.quota_key);
columns[i++]->insert(UInt64(ClickHouseRevision::get()));
block.setColumns(std::move(columns));
}
}

View File

@ -3,6 +3,12 @@
#include <Interpreters/SystemLog.h>
namespace ProfileEvents
{
class Counters;
}
namespace DB
{
@ -53,10 +59,16 @@ struct QueryLogElement
ClientInfo client_info;
std::vector<UInt32> thread_numbers;
std::shared_ptr<ProfileEvents::Counters> profile_counters;
std::shared_ptr<Settings> query_settings;
static std::string name() { return "QueryLog"; }
static Block createBlock();
void appendToBlock(Block & block) const;
static void appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i);
};

View File

@ -0,0 +1,116 @@
#include "QueryThreadLog.h"
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeArray.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Common/ClickHouseRevision.h>
#include <Poco/Net/IPAddress.h>
#include <array>
namespace DB
{
Block QueryThreadLogElement::createBlock()
{
return
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeDateTime>(), "query_start_time"},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"},
{std::make_shared<DataTypeUInt64>(), "read_rows"},
{std::make_shared<DataTypeUInt64>(), "read_bytes"},
{std::make_shared<DataTypeUInt64>(), "written_rows"},
{std::make_shared<DataTypeUInt64>(), "written_bytes"},
{std::make_shared<DataTypeInt64>(), "memory_usage"},
{std::make_shared<DataTypeInt64>(), "peak_memory_usage"},
{std::make_shared<DataTypeString>(), "thread_name"},
{std::make_shared<DataTypeUInt32>(), "thread_number"},
{std::make_shared<DataTypeInt32>(), "os_thread_id"},
{std::make_shared<DataTypeUInt32>(), "master_thread_number"},
{std::make_shared<DataTypeInt32>(), "master_os_thread_id"},
{std::make_shared<DataTypeString>(), "query"},
{std::make_shared<DataTypeUInt8>(), "is_initial_query"},
{std::make_shared<DataTypeString>(), "user"},
{std::make_shared<DataTypeString>(), "query_id"},
{std::make_shared<DataTypeFixedString>(16), "address"},
{std::make_shared<DataTypeUInt16>(), "port"},
{std::make_shared<DataTypeString>(), "initial_user"},
{std::make_shared<DataTypeString>(), "initial_query_id"},
{std::make_shared<DataTypeFixedString>(16), "initial_address"},
{std::make_shared<DataTypeUInt16>(), "initial_port"},
{std::make_shared<DataTypeUInt8>(), "interface"},
{std::make_shared<DataTypeString>(), "os_user"},
{std::make_shared<DataTypeString>(), "client_hostname"},
{std::make_shared<DataTypeString>(), "client_name"},
{std::make_shared<DataTypeUInt32>(), "client_revision"},
{std::make_shared<DataTypeUInt32>(), "client_version_major"},
{std::make_shared<DataTypeUInt32>(), "client_version_minor"},
{std::make_shared<DataTypeUInt32>(), "client_version_patch"},
{std::make_shared<DataTypeUInt8>(), "http_method"},
{std::make_shared<DataTypeString>(), "http_user_agent"},
{std::make_shared<DataTypeString>(), "quota_key"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "ProfileEvents.Names"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "ProfileEvents.Values"}
};
}
void QueryThreadLogElement::appendToBlock(Block & block) const
{
MutableColumns columns = block.mutateColumns();
size_t i = 0;
columns[i++]->insert(UInt64(DateLUT::instance().toDayNum(event_time)));
columns[i++]->insert(UInt64(event_time));
columns[i++]->insert(UInt64(query_start_time));
columns[i++]->insert(UInt64(query_duration_ms));
columns[i++]->insert(UInt64(read_rows));
columns[i++]->insert(UInt64(read_bytes));
columns[i++]->insert(UInt64(written_rows));
columns[i++]->insert(UInt64(written_bytes));
columns[i++]->insert(Int64(memory_usage));
columns[i++]->insert(Int64(peak_memory_usage));
columns[i++]->insertData(thread_name.data(), thread_name.size());
columns[i++]->insert(UInt64(thread_number));
columns[i++]->insert(Int64(os_thread_id));
columns[i++]->insert(UInt64(master_thread_number));
columns[i++]->insert(Int64(master_os_thread_id));
columns[i++]->insertData(query.data(), query.size());
QueryLogElement::appendClientInfo(client_info, columns, i);
columns[i++]->insert(UInt64(ClickHouseRevision::get()));
if (profile_counters)
{
auto column_names = columns[i++].get();
auto column_values = columns[i++].get();
dumpToArrayColumns(*profile_counters, column_names, column_values, true);
}
else
{
columns[i++]->insertDefault();
columns[i++]->insertDefault();
}
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Interpreters/SystemLog.h>
namespace ProfileEvents
{
class Counters;
}
namespace DB
{
struct QueryThreadLogElement
{
time_t event_time{};
/// When query was attached to current thread
time_t query_start_time{};
/// Real time spent by the thread to execute the query
UInt64 query_duration_ms{};
/// The data fetched from DB in current thread to execute the query
UInt64 read_rows{};
UInt64 read_bytes{};
/// The data written to DB
UInt64 written_rows{};
UInt64 written_bytes{};
Int64 memory_usage{};
Int64 peak_memory_usage{};
String thread_name;
UInt32 thread_number{};
Int32 os_thread_id{};
UInt32 master_thread_number{};
Int32 master_os_thread_id{};
String query;
ClientInfo client_info;
std::shared_ptr<ProfileEvents::Counters> profile_counters;
static std::string name() { return "QueryThreadLog"; }
static Block createBlock();
void appendToBlock(Block & block) const;
};
class QueryThreadLog : public SystemLog<QueryThreadLogElement>
{
using SystemLog<QueryThreadLogElement>::SystemLog;
};
}

View File

@ -3,6 +3,9 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Settings.h>
#include <Columns/ColumnArray.h>
#include <Common/typeid_cast.h>
#include <string.h>
namespace DB
@ -176,4 +179,40 @@ void Settings::serialize(WriteBuffer & buf) const
#undef WRITE
}
void Settings::dumpToArrayColumns(IColumn * column_names_, IColumn * column_values_, bool changed_only)
{
/// Convert ptr and make simple check
auto column_names = (column_names_) ? &typeid_cast<ColumnArray &>(*column_names_) : nullptr;
auto column_values = (column_values_) ? &typeid_cast<ColumnArray &>(*column_values_) : nullptr;
size_t size = 0;
#define ADD_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) \
if (!changed_only || NAME.changed) \
{ \
if (column_names) \
column_names->getData().insertData(#NAME, strlen(#NAME)); \
if (column_values) \
column_values->getData().insert(NAME.toString()); \
++size; \
}
APPLY_FOR_SETTINGS(ADD_SETTING)
#undef ADD_SETTING
if (column_names)
{
auto & offsets = column_names->getOffsets();
offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size);
}
/// Nested columns case
bool the_same_offsets = column_names && column_values && column_names->getOffsetsPtr() == column_values->getOffsetsPtr();
if (column_values && !the_same_offsets)
{
auto & offsets = column_values->getOffsets();
offsets.push_back((offsets.empty() ? 0 : offsets.back()) + size);
}
}
}

View File

@ -15,6 +15,7 @@ namespace Poco
namespace DB
{
class IColumn;
class Field;
/** Settings of query execution.
@ -266,8 +267,11 @@ struct Settings
M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \
\
M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \
\
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \
M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table.") \
M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'info', 'warning', 'error', 'none'") \
M(SettingBool, enable_optimize_predicate_expression, 0, "If it is set to true, optimize predicates to subqueries.") \
\
M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \
@ -318,6 +322,9 @@ struct Settings
/// Write changed settings to buffer. (For example, to be sent to remote server.)
void serialize(WriteBuffer & buf) const;
/// Dumps profile events to two column Array(String) and Array(UInt64)
void dumpToArrayColumns(IColumn * column_names, IColumn * column_values, bool changed_only = true);
};

View File

@ -1,5 +1,6 @@
#include <Interpreters/SystemLog.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/PartLog.h>

View File

@ -20,6 +20,7 @@
#include <Common/setThreadName.h>
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
@ -54,17 +55,19 @@ namespace DB
class Context;
class QueryLog;
class QueryThreadLog;
class PartLog;
/// System logs should be destroyed in destructor of last Context and before tables,
/// System logs should be destroyed in destructor of the last Context and before tables,
/// because SystemLog destruction makes insert query while flushing data into underlying tables
struct SystemLogs
{
~SystemLogs();
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
std::unique_ptr<PartLog> part_log; /// Used to log operations with parts
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
std::unique_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
std::unique_ptr<PartLog> part_log; /// Used to log operations with parts
};
@ -73,6 +76,8 @@ class SystemLog : private boost::noncopyable
{
public:
using Self = SystemLog<SystemLog>;
/** Parameter: table name where to write log.
* If table is not exists, then it get created with specified engine.
* If it already exists, then its structure is checked to be compatible with structure of log record.
@ -100,6 +105,9 @@ public:
LOG_ERROR(log, "SystemLog queue is full");
}
/// Flush data in the buffer to disk
void flush(bool quiet = false);
protected:
Context & context;
const String database_name;
@ -118,6 +126,7 @@ protected:
* than accumulation of large amount of log records (for example, for query log - processing of large amount of queries).
*/
std::vector<LogElement> data;
std::mutex data_mutex;
Logger * log;
@ -126,7 +135,6 @@ protected:
std::thread saving_thread;
void threadFunction();
void flush();
/** Creates new table if it does not exist.
* Renames old table if its structure is not suitable.
@ -184,7 +192,16 @@ void SystemLog<LogElement>::threadFunction()
QueueItem element;
bool has_element = false;
if (data.empty())
bool is_empty;
{
std::unique_lock lock(data_mutex);
is_empty = data.empty();
}
/// data.size() is increased only in this function
/// TODO: get rid of data and queue duality
if (is_empty)
{
queue.pop(element);
has_element = true;
@ -206,14 +223,17 @@ void SystemLog<LogElement>::threadFunction()
break;
}
else
{
std::unique_lock lock(data_mutex);
data.push_back(element.second);
}
}
size_t milliseconds_elapsed = time_after_last_write.elapsed() / 1000000;
if (milliseconds_elapsed >= flush_interval_milliseconds)
{
/// Write data to a table.
flush();
flush(true);
time_after_last_write.restart();
}
}
@ -228,10 +248,15 @@ void SystemLog<LogElement>::threadFunction()
template <typename LogElement>
void SystemLog<LogElement>::flush()
void SystemLog<LogElement>::flush(bool quiet)
{
std::unique_lock lock(data_mutex);
try
{
if (quiet && data.empty())
return;
LOG_TRACE(log, "Flushing system log");
/// We check for existence of the table and create it as needed at every flush.
@ -347,5 +372,24 @@ void SystemLog<LogElement>::prepareTable()
is_prepared = true;
}
/// Creates a system log with MergeTree engines using parameters from config
template<typename TSystemLog>
std::unique_ptr<TSystemLog> createDefaultSystemLog(
Context & context_,
const String & default_database_name,
const String & default_table_name,
Poco::Util::AbstractConfiguration & config,
const String & config_prefix)
{
String database = config.getString(config_prefix + ".database", default_database_name);
String table = config.getString(config_prefix + ".table", default_table_name);
String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)");
String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024";
size_t flush_interval_milliseconds = config.getUInt64("query_log.flush_interval_milliseconds", DEFAULT_QUERY_LOG_FLUSH_INTERVAL_MILLISECONDS);
return std::make_unique<TSystemLog>(context_, database, table, engine, flush_interval_milliseconds);
}
}

View File

@ -0,0 +1,256 @@
#include <Common/ThreadStatus.h>
#include <Common/CurrentThread.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/Exception.h>
#include <Interpreters/Context.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/ProcessList.h>
/// Implement some methods of ThreadStatus and CurrentThread here to avoid extra linking dependencies in clickhouse_common_io
namespace DB
{
void ThreadStatus::attachQueryContext(Context & query_context_)
{
query_context = &query_context_;
if (!global_context)
global_context = &query_context->getGlobalContext();
if (!thread_group)
return;
std::unique_lock lock(thread_group->mutex);
thread_group->query_context = query_context;
if (!thread_group->global_context)
thread_group->global_context = global_context;
}
String ThreadStatus::getQueryID()
{
if (query_context)
return query_context->getClientInfo().current_query_id;
return {};
}
void ThreadStatus::defaultThreadDeleter()
{
ThreadStatus & thread = *CurrentThread::get();
LOG_TRACE(thread.log, "Thread " << thread.thread_number << " exited");
thread.detachQuery(true, true);
}
void ThreadStatus::initializeQuery()
{
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
thread_group = std::make_shared<ThreadGroupStatus>();
performance_counters.setParent(&thread_group->performance_counters);
memory_tracker.setParent(&thread_group->memory_tracker);
thread_group->memory_tracker.setDescription("(for query)");
thread_group->master_thread = shared_from_this();
thread_group->thread_statuses.emplace(thread_number, shared_from_this());
initPerformanceCounters();
thread_state = ThreadState::AttachedToQuery;
current_thread_scope.deleter = ThreadStatus::defaultThreadDeleter;
}
void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool check_detached)
{
if (thread_state == ThreadState::AttachedToQuery)
{
if (check_detached)
throw Exception("Can't attach query to the thread, it is already attached", ErrorCodes::LOGICAL_ERROR);
return;
}
assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__);
if (!thread_group_)
throw Exception("Attempt to attach to nullptr thread group", ErrorCodes::LOGICAL_ERROR);
/// Attach current thread to thread group and copy useful information from it
thread_group = thread_group_;
performance_counters.setParent(&thread_group->performance_counters);
memory_tracker.setParent(&thread_group->memory_tracker);
{
std::unique_lock lock(thread_group->mutex);
logs_queue_ptr = thread_group->logs_queue_ptr;
query_context = thread_group->query_context;
if (!global_context)
global_context = thread_group->global_context;
if (!thread_group->thread_statuses.emplace(thread_number, shared_from_this()).second)
throw Exception("Thread " + std::to_string(thread_number) + " is attached twice", ErrorCodes::LOGICAL_ERROR);
}
initPerformanceCounters();
thread_state = ThreadState::AttachedToQuery;
current_thread_scope.deleter = ThreadStatus::defaultThreadDeleter;
}
void ThreadStatus::finalizePerformanceCounters()
{
if (performance_counters_finalized)
return;
performance_counters_finalized = true;
updatePerformanceCounters();
try
{
bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0;
if (log_to_query_thread_log)
if (auto thread_log = global_context->getQueryThreadLog())
logToQueryThreadLog(*thread_log);
}
catch (...)
{
tryLogCurrentException(log);
}
}
void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits)
{
if (exit_if_already_detached && thread_state == ThreadState::DetachedFromQuery)
{
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
return;
}
assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__);
finalizePerformanceCounters();
/// For better logging ({query_id} will be shown here)
if (thread_group && thread_group.use_count() == 1)
thread_group->memory_tracker.logPeakMemoryUsage();
/// Detach from thread group
performance_counters.setParent(&ProfileEvents::global_counters);
memory_tracker.setParent(nullptr);
query_context = nullptr;
thread_group.reset();
thread_state = thread_exits ? ThreadState::Died : ThreadState::DetachedFromQuery;
}
void ThreadStatus::logToQueryThreadLog(QueryThreadLog & thread_log)
{
QueryThreadLogElement elem;
elem.event_time = time(nullptr);
elem.query_start_time = query_start_time;
elem.query_duration_ms = (getCurrentTimeNanoseconds() - query_start_time_nanoseconds) / 1000000U;
elem.read_rows = progress_in.rows.load(std::memory_order_relaxed);
elem.read_bytes = progress_in.bytes.load(std::memory_order_relaxed);
elem.written_rows = progress_out.rows.load(std::memory_order_relaxed);
elem.written_bytes = progress_out.bytes.load(std::memory_order_relaxed);
elem.memory_usage = memory_tracker.get();
elem.peak_memory_usage = memory_tracker.getPeak();
elem.thread_name = getThreadName();
elem.thread_number = thread_number;
elem.os_thread_id = os_thread_id;
if (thread_group)
{
{
std::shared_lock lock(thread_group->mutex);
if (thread_group->master_thread)
{
elem.master_thread_number = thread_group->master_thread->thread_number;
elem.master_os_thread_id = thread_group->master_thread->os_thread_id;
}
elem.query = thread_group->query;
}
}
if (query_context)
{
elem.client_info = query_context->getClientInfo();
if (query_context->getSettingsRef().log_profile_events.value != 0)
{
/// NOTE: Here we are in the same thread, so we can make memcpy()
elem.profile_counters = std::make_shared<ProfileEvents::Counters>(performance_counters.getPartiallyAtomicSnapshot());
}
}
thread_log.add(elem);
}
void CurrentThread::initializeQuery()
{
get()->initializeQuery();
}
void CurrentThread::attachTo(const ThreadGroupStatusPtr & thread_group)
{
get()->attachQuery(thread_group, true);
}
void CurrentThread::attachToIfDetached(const ThreadGroupStatusPtr & thread_group)
{
get()->attachQuery(thread_group, false);
}
std::string CurrentThread::getCurrentQueryID()
{
if (!current_thread || current_thread.use_count() <= 0)
return {};
return current_thread->getQueryID();
}
void CurrentThread::attachQueryContext(Context & query_context)
{
return get()->attachQueryContext(query_context);
}
void CurrentThread::finalizePerformanceCounters()
{
get()->finalizePerformanceCounters();
}
void CurrentThread::detachQuery()
{
get()->detachQuery(false);
}
void CurrentThread::detachQueryIfNotDetached()
{
get()->detachQuery(true);
}
CurrentThread::QueryScope::QueryScope(Context & query_context)
{
CurrentThread::initializeQuery();
CurrentThread::attachQueryContext(query_context);
}
CurrentThread::QueryScope::~QueryScope()
{
try
{
CurrentThread::detachQueryIfNotDetached();
}
catch (...)
{
tryLogCurrentException("CurrentThread", __PRETTY_FUNCTION__);
}
}
}

View File

@ -62,7 +62,6 @@ static void logQuery(const String & query, const Context & context)
LOG_DEBUG(&Logger::get("executeQuery"), "(from " << context.getClientInfo().current_address.toString()
<< (current_user != "default" ? ", user: " + context.getClientInfo().current_user : "")
<< ", query_id: " << current_query_id
<< (!initial_query_id.empty() && current_query_id != initial_query_id ? ", initial_query_id: " + initial_query_id : std::string())
<< ") "
<< joinLines(query)
@ -118,6 +117,9 @@ static void onExceptionBeforeStart(const String & query, Context & context, time
setExceptionStackTrace(elem);
logException(context, elem);
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
if (log_queries)
if (auto query_log = context.getQueryLog())
query_log->add(elem);
@ -134,6 +136,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
time_t current_time = time(nullptr);
context.setQueryContext(context);
CurrentThread::attachQueryContext(context);
const Settings & settings = context.getSettingsRef();
@ -189,15 +192,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ProcessList::EntryPtr process_list_entry;
if (!internal && nullptr == typeid_cast<const ASTShowProcesslistQuery *>(&*ast))
{
process_list_entry = context.getProcessList().insert(
query,
ast.get(),
context.getClientInfo(),
settings);
process_list_entry = context.getProcessList().insert(query, ast.get(), context);
context.setProcessListElement(&process_list_entry->get());
}
/// Load external tables if they were provided
context.initializeExternalTablesIfSet();
auto interpreter = InterpreterFactory::get(ast, context, stage);
res = interpreter->execute();
@ -255,6 +256,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Log into system table start of query execution, if need.
if (log_queries)
{
if (settings.log_query_settings)
elem.query_settings = std::make_shared<Settings>(context.getSettingsRef());
if (auto query_log = context.getQueryLog())
query_log->add(elem);
}
@ -262,12 +266,16 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Also make possible for caller to log successful query finish and exception during execution.
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
{
ProcessListElement * process_list_elem = context.getProcessListElement();
QueryStatus * process_list_elem = context.getProcessListElement();
const Settings & settings = context.getSettingsRef();
if (!process_list_elem)
return;
ProcessInfo info = process_list_elem->getInfo();
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events);
double elapsed_seconds = info.elapsed_seconds;
@ -288,18 +296,18 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (auto profiling_stream = dynamic_cast<const IProfilingBlockInputStream *>(stream_in))
{
const BlockStreamProfileInfo & info = profiling_stream->getProfileInfo();
const BlockStreamProfileInfo & stream_in_info = profiling_stream->getProfileInfo();
/// NOTE: INSERT SELECT query contains zero metrics
elem.result_rows = info.rows;
elem.result_bytes = info.bytes;
elem.result_rows = stream_in_info.rows;
elem.result_bytes = stream_in_info.bytes;
}
}
else if (stream_out) /// will be used only for ordinary INSERT queries
{
if (auto counting_stream = dynamic_cast<const CountingBlockOutputStream *>(stream_out))
{
/// NOTE: Redundancy. The same values could be extracted from process_list_elem->progress_out.
/// NOTE: Redundancy. The same values coulld be extracted from process_list_elem->progress_out.query_settings = process_list_elem->progress_in
elem.result_rows = counting_stream->getProgress().rows;
elem.result_bytes = counting_stream->getProgress().bytes;
}
@ -314,6 +322,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
<< formatReadableSizeWithBinarySuffix(elem.read_bytes / elapsed_seconds) << "/sec.");
}
elem.thread_numbers = std::move(info.thread_numbers);
elem.profile_counters = std::move(info.profile_counters);
if (log_queries)
{
if (auto query_log = context.getQueryLog())
@ -331,11 +342,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.query_duration_ms = 1000 * (elem.event_time - elem.query_start_time);
elem.exception = getCurrentExceptionMessage(false);
ProcessListElement * process_list_elem = context.getProcessListElement();
QueryStatus * process_list_elem = context.getProcessListElement();
const Settings & settings = context.getSettingsRef();
/// Update performance counters before logging to query_log
CurrentThread::finalizePerformanceCounters();
if (process_list_elem)
{
ProcessInfo info = process_list_elem->getInfo();
QueryStatusInfo info = process_list_elem->getInfo(true, settings.log_profile_events, false);
elem.query_duration_ms = info.elapsed_seconds * 1000;
@ -343,11 +358,15 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.read_bytes = info.read_bytes;
elem.memory_usage = info.peak_memory_usage > 0 ? info.peak_memory_usage : 0;
elem.thread_numbers = std::move(info.thread_numbers);
elem.profile_counters = std::move(info.profile_counters);
}
setExceptionStackTrace(elem);
logException(context, elem);
/// In case of exception we log internal queries also
if (log_queries)
{
if (auto query_log = context.getQueryLog())

View File

@ -46,4 +46,7 @@ target_link_libraries (in_join_subqueries_preprocessor dbms)
add_check(in_join_subqueries_preprocessor)
add_executable (users users.cpp)
target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY} dbms)
target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY})
add_executable (internal_iotop internal_iotop.cpp)
target_link_libraries (internal_iotop dbms)

View File

@ -0,0 +1,160 @@
#include <common/ThreadPool.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/copyData.h>
#include <Common/TaskStatsInfoGetter.h>
#include <Poco/File.h>
#include <Common/Stopwatch.h>
#include <IO/WriteBufferFromString.h>
#include <linux/taskstats.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <pthread.h>
std::mutex mutex;
std::ostream & operator << (std::ostream & stream, const ::taskstats & stat)
{
#define PRINT(field) (stream << #field << " " << stat.field)
PRINT(ac_pid) << ", ";
PRINT(read_bytes) << ", ";
PRINT(write_bytes) << ", ";
PRINT(read_char) << ", ";
PRINT(write_char) << ", ";
PRINT(swapin_delay_total) << ", ";
PRINT(blkio_delay_total) << ", ";
PRINT(cpu_delay_total) << ", ";
PRINT(ac_pid) << ", ";
PRINT(ac_utime) << ", ";
PRINT(ac_stime) << ", ";
#undef PRINT
return stream;
}
using namespace DB;
void do_io(size_t id)
{
::taskstats stat;
int tid = TaskStatsInfoGetter::getCurrentTID();
TaskStatsInfoGetter get_info;
if (!get_info.tryGetStat(stat, tid))
{
std::lock_guard<std::mutex> lock(mutex);
std::cerr << "#" << id << ", tid " << tid << ". Can't get stat\n";
}
else
{
std::lock_guard<std::mutex> lock(mutex);
std::cerr << "#" << id << ", tid " << tid << ", intitial\n" << stat << "\n";
}
size_t copy_size = DBMS_DEFAULT_BUFFER_SIZE * (1 + id);
std::string path_dst = "test_out_" + std::to_string(id);
{
ReadBufferFromFile rb("/dev/urandom");
WriteBufferFromFile wb(path_dst, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC | O_DIRECT);
copyData(rb, wb, copy_size);
}
get_info.getStat(stat, tid);
{
std::lock_guard<std::mutex> lock(mutex);
std::cerr << "#" << id << ", tid " << tid << ", step1\n" << stat << "\n";
}
{
ReadBufferFromFile rb(path_dst);
WriteBufferFromOwnString wb;
copyData(rb, wb, copy_size);
}
get_info.getStat(stat, tid);
{
std::lock_guard<std::mutex> lock(mutex);
std::cerr << "#" << id << ", tid " << tid << ", step2\n" << stat << "\n";
}
{
ReadBufferFromFile rb(path_dst);
WriteBufferFromOwnString wb;
copyData(rb, wb, copy_size);
}
get_info.getStat(stat, tid);
{
std::lock_guard<std::mutex> lock(mutex);
std::cerr << "#" << id << ", tid " << tid << ", step3\n" << stat << "\n";
}
Poco::File(path_dst).remove(false);
}
void test_perf()
{
::taskstats stat;
int tid = TaskStatsInfoGetter::getCurrentTID();
TaskStatsInfoGetter get_info;
rusage rusage;
constexpr size_t num_samples = 1000000;
{
Stopwatch watch;
for (size_t i = 0; i < num_samples; ++i)
getrusage(RUSAGE_THREAD, &rusage);
auto ms = watch.elapsedMilliseconds();
if (ms > 0)
std::cerr << "RUsage: " << double(ms) / num_samples << " ms per call, " << 1000 * num_samples / ms << " calls per second\n";
}
{
Stopwatch watch;
for (size_t i = 0; i < num_samples; ++i)
get_info.getStat(stat, tid);
auto ms = watch.elapsedMilliseconds();
if (ms > 0)
std::cerr << "Netlink: " << double(ms) / num_samples << " ms per call, " << 1000 * num_samples / ms << " calls per second\n";
}
std::cerr << stat << "\n";
}
int main()
try
{
std::cerr << "pid " << getpid() << "\n";
size_t num_threads = 2;
ThreadPool pool(num_threads);
for (size_t i = 0; i < num_threads; ++i)
pool.schedule([i]() { do_io(i); });
pool.wait();
test_perf();
return 0;
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(true);
return -1;
}

View File

@ -61,6 +61,8 @@ const char * ASTSystemQuery::typeToString(Type type)
return "STOP REPLICATION QUEUES";
case Type::START_REPLICATION_QUEUES:
return "START REPLICATION QUEUES";
case Type::FLUSH_SYSTEM_TABLES:
return "FLUSH SYSTEM TABLES";
default:
throw Exception("Unknown SYSTEM query command", ErrorCodes::BAD_TYPE_OF_FIELD);
}

View File

@ -35,6 +35,7 @@ public:
START_REPLICATEDS_SENDS,
STOP_REPLICATION_QUEUES,
START_REPLICATION_QUEUES,
FLUSH_SYSTEM_TABLES,
END
};

View File

@ -3,6 +3,8 @@
#include <optional>
#include <Core/NamesAndTypes.h>
#include <Storages/ColumnsDescription.h>
#include <optional>
namespace DB

View File

@ -23,6 +23,7 @@
#include <Common/ProfileEvents.h>
#include <Common/MemoryTracker.h>
#include <Common/escapeForFileName.h>
#include <Common/CurrentThread.h>
#include <common/logger_useful.h>
#include <ext/range.h>
#include <ext/scope_guard.h>
@ -34,6 +35,7 @@
#include <mutex>
namespace CurrentMetrics
{
extern const Metric DistributedSend;
@ -191,9 +193,12 @@ void DistributedBlockOutputStream::waitForJobs()
ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobReplica & job, const Block & current_block)
{
auto memory_tracker = current_memory_tracker;
return [this, memory_tracker, &job, &current_block]()
auto thread_group = CurrentThread::getGroup();
return [this, thread_group, &job, &current_block]()
{
CurrentThread::attachToIfDetached(thread_group);
setThreadName("DistrOutStrProc");
++job.blocks_started;
SCOPE_EXIT({
@ -204,12 +209,6 @@ ThreadPool::Job DistributedBlockOutputStream::runWritingJob(DistributedBlockOutp
job.max_elapsed_time_for_block_ms = std::max(job.max_elapsed_time_for_block_ms, elapsed_time_for_block_ms);
});
if (!current_memory_tracker)
{
current_memory_tracker = memory_tracker;
setThreadName("DistrOutStrProc");
}
const auto & shard_info = cluster->getShardsInfo()[job.shard_index];
size_t num_shards = cluster->getShardsInfo().size();
auto & shard_job = per_shard_jobs[job.shard_index];

View File

@ -6,8 +6,10 @@
#include <IO/WriteHelpers.h>
#include <common/logger_useful.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <Common/CurrentThread.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <ext/scope_guard.h>
#include <pcg_random.hpp>
#include <random>
@ -56,6 +58,12 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_)
{
LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads");
/// Put all threads to one thread group
/// The master thread exits immediately
CurrentThread::initializeQuery();
thread_group = CurrentThread::getGroup();
CurrentThread::detachQuery();
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
@ -114,9 +122,11 @@ void BackgroundProcessingPool::threadFunction()
{
setThreadName("BackgrProcPool");
MemoryTracker memory_tracker;
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
current_memory_tracker = &memory_tracker;
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
pcg64 rng(randomSeed());
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
@ -201,8 +211,6 @@ void BackgroundProcessingPool::threadFunction()
task->iterator = tasks.emplace(next_time_to_execute, task);
}
}
current_memory_tracker = nullptr;
}
}

View File

@ -12,6 +12,8 @@
#include <Poco/Event.h>
#include <Poco/Timestamp.h>
#include <Core/Types.h>
#include <Common/CurrentThread.h>
namespace DB
{
@ -64,6 +66,8 @@ protected:
std::atomic<bool> shutdown {false};
std::condition_variable wake_event;
/// Thread group used for profiling purposes
ThreadGroupStatusPtr thread_group;
void threadFunction();
};

View File

@ -1,6 +1,8 @@
#include <Storages/MergeTree/MergeList.h>
#include <Common/CurrentMetrics.h>
#include <Poco/Ext/ThreadNumber.h>
#include <Common/CurrentThread.h>
namespace CurrentMetrics
{
@ -20,11 +22,12 @@ MergeListElement::MergeListElement(const std::string & database, const std::stri
source_part_names.emplace_back(source_part->name);
/// Each merge is executed into separate background processing pool thread
background_pool_task_memory_tracker = current_memory_tracker;
if (background_pool_task_memory_tracker)
background_thread_memory_tracker = &CurrentThread::getMemoryTracker();
if (background_thread_memory_tracker)
{
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingForMerges);
background_pool_task_memory_tracker->setNext(&memory_tracker);
background_thread_memory_tracker_prev_parent = background_thread_memory_tracker->getParent();
background_thread_memory_tracker->setParent(&memory_tracker);
}
}
@ -56,8 +59,8 @@ MergeInfo MergeListElement::getInfo() const
MergeListElement::~MergeListElement()
{
/// Unplug memory_tracker from current background processing pool thread
if (background_pool_task_memory_tracker)
background_pool_task_memory_tracker->setNext(nullptr);
if (background_thread_memory_tracker)
background_thread_memory_tracker->setParent(background_thread_memory_tracker_prev_parent);
}
}

View File

@ -65,8 +65,9 @@ struct MergeListElement : boost::noncopyable
/// Updated only for Vertical algorithm
std::atomic<UInt64> columns_written{};
MemoryTracker memory_tracker;
MemoryTracker * background_pool_task_memory_tracker;
MemoryTracker memory_tracker{VariableContext::Process};
MemoryTracker * background_thread_memory_tracker;
MemoryTracker * background_thread_memory_tracker_prev_parent = nullptr;
/// Poco thread number used in logs
UInt32 thread_number;

View File

@ -267,7 +267,7 @@ void MergeTreeReader::Stream::loadMarks()
auto load = [&]() -> MarkCache::MappedPtr
{
/// Memory for marks must not be accounted as memory usage for query, because they are stored in shared cache.
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
size_t file_size = Poco::File(path).getSize();
size_t expected_file_size = sizeof(MarkInCompressedFile) * marks_count;

View File

@ -471,7 +471,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
* And otherwise it will look like excessively growing memory consumption in context of query.
* (observed in long INSERT SELECTs)
*/
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
/// Write index. The index contains Primary Key value for each `index_granularity` row.
for (size_t i = index_offset; i < rows; i += storage.index_granularity)

View File

@ -178,7 +178,7 @@ static void appendBlock(const Block & from, Block & to)
try
{
/// Avoid "memory limit exceeded" exceptions during rollback.
TemporarilyDisableMemoryTracker temporarily_disable_memory_tracker;
auto temporarily_disable_memory_tracker = getCurrentMemoryTrackerActionLock();
for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no)
{

View File

@ -18,7 +18,7 @@ void StorageSystemEvents::fillData(MutableColumns & res_columns, const Context &
{
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
{
UInt64 value = ProfileEvents::counters[i];
UInt64 value = ProfileEvents::global_counters[i];
if (0 != value)
{

View File

@ -1,8 +1,14 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Interpreters/Context.h>
#include <DataTypes/DataTypeArray.h>
#include <Interpreters/ProcessList.h>
#include <Storages/System/StorageSystemProcesses.h>
#include <Interpreters/Context.h>
#include <Interpreters/Settings.h>
#include <Interpreters/ProfileEventsExt.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>
namespace DB
@ -48,13 +54,19 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
{"memory_usage", std::make_shared<DataTypeInt64>()},
{"peak_memory_usage", std::make_shared<DataTypeInt64>()},
{"query", std::make_shared<DataTypeString>()},
{ "thread_numbers", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt32>()) },
{ "ProfileEvents.Names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
{ "ProfileEvents.Values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()) },
{ "Settings.Names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
{ "Settings.Values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()) },
};
}
void StorageSystemProcesses::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
{
ProcessList::Info info = context.getProcessList().getInfo();
ProcessList::Info info = context.getProcessList().getInfo(true, true, true);
for (const auto & process : info)
{
@ -89,6 +101,40 @@ void StorageSystemProcesses::fillData(MutableColumns & res_columns, const Contex
res_columns[i++]->insert(process.memory_usage);
res_columns[i++]->insert(process.peak_memory_usage);
res_columns[i++]->insert(process.query);
{
Array threads_array;
threads_array.reserve(process.thread_numbers.size());
for (const UInt32 thread_number : process.thread_numbers)
threads_array.emplace_back(UInt64(thread_number));
res_columns[i++]->insert(threads_array);
}
{
IColumn * column_profile_events_names = res_columns[i++].get();
IColumn * column_profile_events_values = res_columns[i++].get();
if (process.profile_counters)
ProfileEvents::dumpToArrayColumns(*process.profile_counters, column_profile_events_names, column_profile_events_values, true);
else
{
column_profile_events_names->insertDefault();
column_profile_events_values->insertDefault();
}
}
{
IColumn * column_settings_names = res_columns[i++].get();
IColumn * column_settings_values = res_columns[i++].get();
if (process.query_settings)
process.query_settings->dumpToArrayColumns(column_settings_names, column_settings_values, true);
else
{
column_settings_names->insertDefault();
column_settings_values->insertDefault();
}
}
}
}

View File

@ -73,7 +73,7 @@ def main(args):
base_dir = os.path.abspath(args.queries)
tmp_dir = os.path.abspath(args.tmp)
# Keep same default values as in queries/0_stateless/00000_sh_lib.sh
# Keep same default values as in queries/shell_config.sh
os.environ.setdefault("CLICKHOUSE_BINARY", args.binary)
os.environ.setdefault("CLICKHOUSE_CLIENT", args.client)
os.environ.setdefault("CLICKHOUSE_CONFIG", args.configserver)
@ -81,6 +81,12 @@ def main(args):
os.environ.setdefault("CLICKHOUSE_CONFIG_CLIENT", args.configclient)
os.environ.setdefault("CLICKHOUSE_TMP", tmp_dir)
# Force to print server warnings in stderr
# Shell scripts could change logging level
server_logs_level = "warning"
os.environ.setdefault("CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL", server_logs_level)
# TODO ! use clickhouse-extract-from-config here:
if args.zookeeper is None:
code, out = commands.getstatusoutput(args.binary + "-extract-from-config --try --config " + args.configserver + ' --key zookeeper | grep . | wc -l')
try:
@ -151,12 +157,12 @@ def main(args):
break
case_file = os.path.join(suite_dir, case)
if os.path.isfile(case_file) and (case.endswith('.sh') or case.endswith('.py') or case.endswith('.sql')):
(name, ext) = os.path.splitext(case)
(name, ext) = os.path.splitext(case)
if os.path.isfile(case_file) and (ext == '.sql' or ext == '.sh' or ext == '.py'):
report_testcase = et.Element("testcase", attrib = {"name": name})
try:
print "{0:70}".format(name + ": "),
sys.stdout.flush()
@ -194,9 +200,9 @@ def main(args):
stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr'
if ext == '.sql':
command = "{0} --testmode --multiquery < {1} > {2} 2> {3}".format(args.client, case_file, stdout_file, stderr_file)
command = "{0} --send_logs_level={1} --testmode --multiquery < {2} > {3} 2> {4}".format(args.client, server_logs_level, case_file, stdout_file, stderr_file)
else:
command = "{0} > {1} 2> {2}".format(case_file, stdout_file, stderr_file)
command = "{} > {} 2> {}".format(case_file, stdout_file, stderr_file)
proc = Popen(command, shell = True)
start_time = datetime.now()

View File

@ -1,3 +1,5 @@
SET send_logs_level = 'none';
SELECT * FROM system.numbers LIMIT 3;
SELECT sys_num.number FROM system.numbers AS sys_num WHERE number > 2 LIMIT 2;
SELECT number FROM system.numbers WHERE number >= 5 LIMIT 2;

Some files were not shown because too many files have changed in this diff Show More