Merge pull request #25962 from ClickHouse/network-instrumentation

More instrumentation for network interaction: add counters for recv/send bytes; add gauges for recvs/sends.
This commit is contained in:
alexey-milovidov 2021-07-08 17:20:16 +03:00 committed by GitHub
commit 9c2aad6c17
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 61 additions and 2 deletions

View File

@ -30,6 +30,8 @@
M(OpenFileForWrite, "Number of files open for writing") \
M(Read, "Number of read (read, pread, io_getevents, etc.) syscalls in fly") \
M(Write, "Number of write (write, pwrite, io_getevents, etc.) syscalls in fly") \
M(NetworkReceive, "Number of threads receiving data from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSend, "Number of threads sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(SendScalars, "Number of connections that are sending data for scalars to remote servers.") \
M(SendExternalTables, "Number of connections that are sending data for external tables to remote servers. External tables are used to implement GLOBAL IN and GLOBAL JOIN operators with distributed subqueries.") \
M(QueryThread, "Number of query processing threads") \

View File

@ -49,8 +49,10 @@
M(CreatedReadBufferMMapFailed, "") \
M(DiskReadElapsedMicroseconds, "Total time spent waiting for read syscall. This include reads from page cache.") \
M(DiskWriteElapsedMicroseconds, "Total time spent waiting for write syscall. This include writes to page cache.") \
M(NetworkReceiveElapsedMicroseconds, "") \
M(NetworkSendElapsedMicroseconds, "") \
M(NetworkReceiveElapsedMicroseconds, "Total time spent waiting for data to receive or receiving data from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \
M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \
M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform the 'max_network_bandwidth' setting.") \
\
M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \

View File

@ -5,11 +5,19 @@
#include <Common/Exception.h>
#include <Common/NetException.h>
#include <Common/Stopwatch.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
namespace ProfileEvents
{
extern const Event NetworkReceiveElapsedMicroseconds;
extern const Event NetworkReceiveBytes;
}
namespace CurrentMetrics
{
extern const Metric NetworkReceive;
}
@ -31,6 +39,8 @@ bool ReadBufferFromPocoSocket::nextImpl()
/// Add more details to exceptions.
try
{
CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkReceive);
/// If async_callback is specified, and read will block, run async_callback and try again later.
/// It is expected that file descriptor may be polled externally.
/// Note that receive timeout is not checked here. External code should check it while polling.
@ -57,6 +67,7 @@ bool ReadBufferFromPocoSocket::nextImpl()
/// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one
ProfileEvents::increment(ProfileEvents::NetworkReceiveElapsedMicroseconds, watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::NetworkReceiveBytes, bytes_read);
if (bytes_read)
working_buffer.resize(bytes_read);

View File

@ -6,11 +6,19 @@
#include <Common/NetException.h>
#include <Common/Stopwatch.h>
#include <Common/MemoryTracker.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
namespace ProfileEvents
{
extern const Event NetworkSendElapsedMicroseconds;
extern const Event NetworkSendBytes;
}
namespace CurrentMetrics
{
extern const Metric NetworkSend;
}
@ -40,6 +48,7 @@ void WriteBufferFromPocoSocket::nextImpl()
/// Add more details to exceptions.
try
{
CurrentMetrics::Increment metric_increment(CurrentMetrics::NetworkSend);
res = socket.impl()->sendBytes(working_buffer.begin() + bytes_written, offset() - bytes_written);
}
catch (const Poco::Net::NetException & e)
@ -62,6 +71,7 @@ void WriteBufferFromPocoSocket::nextImpl()
}
ProfileEvents::increment(ProfileEvents::NetworkSendElapsedMicroseconds, watch.elapsedMicroseconds());
ProfileEvents::increment(ProfileEvents::NetworkSendBytes, bytes_written);
}
WriteBufferFromPocoSocket::WriteBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size)

View File

@ -0,0 +1,16 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;"
seq 1 1000 | ${CLICKHOUSE_CLIENT} --query "INSERT INTO t FORMAT TSV"
${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS;
WITH ProfileEvents['NetworkReceiveBytes'] AS bytes
SELECT bytes >= 8000 AND bytes < 9000 ? 1 : bytes FROM system.query_log
WHERE current_database = currentDatabase() AND query_kind = 'Insert' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;"
${CLICKHOUSE_CLIENT} --query "DROP TABLE t"

View File

@ -0,0 +1,16 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --multiquery --query "DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt64) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query "SELECT number FROM numbers(1000)" > /dev/null
${CLICKHOUSE_CLIENT} --multiquery --query "SYSTEM FLUSH LOGS;
WITH ProfileEvents['NetworkSendBytes'] AS bytes
SELECT bytes >= 8000 AND bytes < 9000 ? 1 : bytes FROM system.query_log
WHERE current_database = currentDatabase() AND query_kind = 'Select' AND event_date >= yesterday() AND type = 2 ORDER BY event_time DESC LIMIT 1;"
${CLICKHOUSE_CLIENT} --query "DROP TABLE t"