mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
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:
commit
9c2aad6c17
@ -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") \
|
||||
|
@ -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.") \
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
|
@ -0,0 +1 @@
|
||||
1
|
16
tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh
Executable file
16
tests/queries/0_stateless/01939_network_receive_bytes_metrics.sh
Executable 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"
|
@ -0,0 +1 @@
|
||||
1
|
16
tests/queries/0_stateless/01939_network_send_bytes_metrics.sh
Executable file
16
tests/queries/0_stateless/01939_network_send_bytes_metrics.sh
Executable 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"
|
Loading…
Reference in New Issue
Block a user