Skip to content

Commit 68c8fa4

Browse files
authored
Merge pull request dpkp#794 from dpkp/conn_metrics
Complete metrics instrumentation
2 parents 3c9b1b6 + 025b69e commit 68c8fa4

13 files changed

+253
-49
lines changed

kafka/client_async.py

+27-3
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
from .conn import BrokerConnection, ConnectionStates, collect_hosts, get_ip_port_afi
2525
from . import errors as Errors
2626
from .future import Future
27+
from .metrics import AnonMeasurable
2728
from .metrics.stats import Avg, Count, Rate
2829
from .metrics.stats.rate import TimeUnit
2930
from .protocol.metadata import MetadataRequest
@@ -187,10 +188,13 @@ def __init__(self, **configs):
187188
self._wake_lock = threading.Lock()
188189
self._selector.register(self._wake_r, selectors.EVENT_READ)
189190
self._closed = False
190-
self._bootstrap(collect_hosts(self.config['bootstrap_servers']))
191191
self._sensors = None
192192
if self.config['metrics']:
193-
self._sensors = KafkaClientMetrics(self.config['metrics'], self.config['metric_group_prefix'])
193+
self._sensors = KafkaClientMetrics(self.config['metrics'],
194+
self.config['metric_group_prefix'],
195+
self._conns)
196+
197+
self._bootstrap(collect_hosts(self.config['bootstrap_servers']))
194198

195199
# Check Broker Version if not set explicitly
196200
if self.config['api_version'] is None:
@@ -218,6 +222,7 @@ def _bootstrap(self, hosts):
218222
cb = functools.partial(self._conn_state_change, 'bootstrap')
219223
bootstrap = BrokerConnection(host, port, afi,
220224
state_change_callback=cb,
225+
node_id='bootstrap',
221226
**self.config)
222227
bootstrap.connect()
223228
while bootstrap.connecting():
@@ -273,6 +278,8 @@ def _conn_state_change(self, node_id, conn):
273278
except KeyError:
274279
pass
275280
self._selector.register(conn._sock, selectors.EVENT_READ, conn)
281+
if self._sensors:
282+
self._sensors.connection_created.record()
276283

277284
if 'bootstrap' in self._conns and node_id != 'bootstrap':
278285
bootstrap = self._conns.pop('bootstrap')
@@ -289,6 +296,8 @@ def _conn_state_change(self, node_id, conn):
289296
self._selector.unregister(conn._sock)
290297
except KeyError:
291298
pass
299+
if self._sensors:
300+
self._sensors.connection_closed.record()
292301
if self._refresh_on_disconnects and not self._closed:
293302
log.warning("Node %s connection failed -- refreshing metadata", node_id)
294303
self.cluster.request_update()
@@ -305,6 +314,7 @@ def _maybe_connect(self, node_id):
305314
cb = functools.partial(self._conn_state_change, node_id)
306315
self._conns[node_id] = BrokerConnection(host, broker.port, afi,
307316
state_change_callback=cb,
317+
node_id=node_id,
308318
**self.config)
309319
conn = self._conns[node_id]
310320
if conn.connected():
@@ -888,10 +898,19 @@ def pop_ready(self):
888898

889899

890900
class KafkaClientMetrics(object):
891-
def __init__(self, metrics, metric_group_prefix):
901+
def __init__(self, metrics, metric_group_prefix, conns):
892902
self.metrics = metrics
893903
self.metric_group_name = metric_group_prefix + '-metrics'
894904

905+
self.connection_closed = metrics.sensor('connections-closed')
906+
self.connection_closed.add(metrics.metric_name(
907+
'connection-close-rate', self.metric_group_name,
908+
'Connections closed per second in the window.'), Rate())
909+
self.connection_created = metrics.sensor('connections-created')
910+
self.connection_created.add(metrics.metric_name(
911+
'connection-creation-rate', self.metric_group_name,
912+
'New connections established per second in the window.'), Rate())
913+
895914
self.select_time = metrics.sensor('select-time')
896915
self.select_time.add(metrics.metric_name(
897916
'select-rate', self.metric_group_name,
@@ -915,3 +934,8 @@ def __init__(self, metrics, metric_group_prefix):
915934
'io-ratio', self.metric_group_name,
916935
'The fraction of time the I/O thread spent doing I/O'),
917936
Rate(time_unit=TimeUnit.NANOSECONDS))
937+
938+
metrics.add_metric(metrics.metric_name(
939+
'connection-count', self.metric_group_name,
940+
'The current number of active connections.'), AnonMeasurable(
941+
lambda config, now: len(conns)))

kafka/conn.py

+189-1
Original file line numberDiff line numberDiff line change
@@ -14,8 +14,9 @@
1414

1515
import kafka.errors as Errors
1616
from kafka.future import Future
17+
from kafka.metrics.stats import Avg, Count, Max, Rate
1718
from kafka.protocol.api import RequestHeader
18-
from kafka.protocol.admin import SaslHandShakeRequest, SaslHandShakeResponse
19+
from kafka.protocol.admin import SaslHandShakeRequest
1920
from kafka.protocol.commit import GroupCoordinatorResponse
2021
from kafka.protocol.types import Int32
2122
from kafka.version import __version__
@@ -58,6 +59,7 @@ class ConnectionStates(object):
5859
class BrokerConnection(object):
5960
DEFAULT_CONFIG = {
6061
'client_id': 'kafka-python-' + __version__,
62+
'node_id': 0,
6163
'request_timeout_ms': 40000,
6264
'reconnect_backoff_ms': 50,
6365
'max_in_flight_requests_per_connection': 5,
@@ -74,13 +76,83 @@ class BrokerConnection(object):
7476
'ssl_password': None,
7577
'api_version': (0, 8, 2), # default to most restrictive
7678
'state_change_callback': lambda conn: True,
79+
'metrics': None,
80+
'metric_group_prefix': '',
7781
'sasl_mechanism': 'PLAIN',
7882
'sasl_plain_username': None,
7983
'sasl_plain_password': None
8084
}
8185
SASL_MECHANISMS = ('PLAIN',)
8286

8387
def __init__(self, host, port, afi, **configs):
88+
"""Initialize a kafka broker connection
89+
90+
Keyword Arguments:
91+
client_id (str): a name for this client. This string is passed in
92+
each request to servers and can be used to identify specific
93+
server-side log entries that correspond to this client. Also
94+
submitted to GroupCoordinator for logging with respect to
95+
consumer group administration. Default: 'kafka-python-{version}'
96+
reconnect_backoff_ms (int): The amount of time in milliseconds to
97+
wait before attempting to reconnect to a given host.
98+
Default: 50.
99+
request_timeout_ms (int): Client request timeout in milliseconds.
100+
Default: 40000.
101+
max_in_flight_requests_per_connection (int): Requests are pipelined
102+
to kafka brokers up to this number of maximum requests per
103+
broker connection. Default: 5.
104+
receive_buffer_bytes (int): The size of the TCP receive buffer
105+
(SO_RCVBUF) to use when reading data. Default: None (relies on
106+
system defaults). Java client defaults to 32768.
107+
send_buffer_bytes (int): The size of the TCP send buffer
108+
(SO_SNDBUF) to use when sending data. Default: None (relies on
109+
system defaults). Java client defaults to 131072.
110+
socket_options (list): List of tuple-arguments to socket.setsockopt
111+
to apply to broker connection sockets. Default:
112+
[(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)]
113+
security_protocol (str): Protocol used to communicate with brokers.
114+
Valid values are: PLAINTEXT, SSL. Default: PLAINTEXT.
115+
ssl_context (ssl.SSLContext): pre-configured SSLContext for wrapping
116+
socket connections. If provided, all other ssl_* configurations
117+
will be ignored. Default: None.
118+
ssl_check_hostname (bool): flag to configure whether ssl handshake
119+
should verify that the certificate matches the brokers hostname.
120+
default: True.
121+
ssl_cafile (str): optional filename of ca file to use in certificate
122+
veriication. default: None.
123+
ssl_certfile (str): optional filename of file in pem format containing
124+
the client certificate, as well as any ca certificates needed to
125+
establish the certificate's authenticity. default: None.
126+
ssl_keyfile (str): optional filename containing the client private key.
127+
default: None.
128+
ssl_password (callable, str, bytes, bytearray): optional password or
129+
callable function that returns a password, for decrypting the
130+
client private key. Default: None.
131+
ssl_crlfile (str): optional filename containing the CRL to check for
132+
certificate expiration. By default, no CRL check is done. When
133+
providing a file, only the leaf certificate will be checked against
134+
this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+.
135+
default: None.
136+
api_version (tuple): specify which kafka API version to use. Accepted
137+
values are: (0, 8, 0), (0, 8, 1), (0, 8, 2), (0, 9), (0, 10)
138+
If None, KafkaClient will attempt to infer the broker
139+
version by probing various APIs. Default: None
140+
api_version_auto_timeout_ms (int): number of milliseconds to throw a
141+
timeout exception from the constructor when checking the broker
142+
api version. Only applies if api_version is None
143+
state_chance_callback (callable): function to be called when the
144+
connection state changes from CONNECTING to CONNECTED etc.
145+
metrics (kafka.metrics.Metrics): Optionally provide a metrics
146+
instance for capturing network IO stats. Default: None.
147+
metric_group_prefix (str): Prefix for metric names. Default: ''
148+
sasl_mechanism (str): string picking sasl mechanism when security_protocol
149+
is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
150+
Default: None
151+
sasl_plain_username (str): username for sasl PLAIN authentication.
152+
Default: None
153+
sasl_plain_password (str): passowrd for sasl PLAIN authentication.
154+
Defualt: None
155+
"""
84156
self.host = host
85157
self.hostname = host
86158
self.port = port
@@ -123,6 +195,11 @@ def __init__(self, host, port, afi, **configs):
123195
self._correlation_id = 0
124196
self._gai = None
125197
self._gai_index = 0
198+
self._sensors = None
199+
if self.config['metrics']:
200+
self._sensors = BrokerConnectionMetrics(self.config['metrics'],
201+
self.config['metric_group_prefix'],
202+
self.config['node_id'])
126203

127204
def connect(self):
128205
"""Attempt to connect and return ConnectionState"""
@@ -453,6 +530,8 @@ def _send(self, request, expect_response=True):
453530
sent_bytes = self._sock.send(data[total_sent:])
454531
total_sent += sent_bytes
455532
assert total_sent == len(data)
533+
if self._sensors:
534+
self._sensors.bytes_sent.record(total_sent)
456535
self._sock.setblocking(False)
457536
except (AssertionError, ConnectionError) as e:
458537
log.exception("Error sending %s to %s", request, self)
@@ -583,6 +662,8 @@ def _recv(self):
583662

584663
self._receiving = False
585664
self._next_payload_bytes = 0
665+
if self._sensors:
666+
self._sensors.bytes_received.record(4 + self._rbuffer.tell())
586667
self._rbuffer.seek(0)
587668
response = self._process_response(self._rbuffer)
588669
self._rbuffer.seek(0)
@@ -593,6 +674,8 @@ def _process_response(self, read_buffer):
593674
assert not self._processing, 'Recursion not supported'
594675
self._processing = True
595676
ifr = self.in_flight_requests.popleft()
677+
if self._sensors:
678+
self._sensors.request_time.record((time.time() - ifr.timestamp) * 1000)
596679

597680
# verify send/recv correlation ids match
598681
recv_correlation_id = Int32.decode(read_buffer)
@@ -762,6 +845,111 @@ def __repr__(self):
762845
self.port)
763846

764847

848+
class BrokerConnectionMetrics(object):
849+
def __init__(self, metrics, metric_group_prefix, node_id):
850+
self.metrics = metrics
851+
852+
# Any broker may have registered summary metrics already
853+
# but if not, we need to create them so we can set as parents below
854+
all_conns_transferred = metrics.get_sensor('bytes-sent-received')
855+
if not all_conns_transferred:
856+
metric_group_name = metric_group_prefix + '-metrics'
857+
858+
bytes_transferred = metrics.sensor('bytes-sent-received')
859+
bytes_transferred.add(metrics.metric_name(
860+
'network-io-rate', metric_group_name,
861+
'The average number of network operations (reads or writes) on all'
862+
' connections per second.'), Rate(sampled_stat=Count()))
863+
864+
bytes_sent = metrics.sensor('bytes-sent',
865+
parents=[bytes_transferred])
866+
bytes_sent.add(metrics.metric_name(
867+
'outgoing-byte-rate', metric_group_name,
868+
'The average number of outgoing bytes sent per second to all'
869+
' servers.'), Rate())
870+
bytes_sent.add(metrics.metric_name(
871+
'request-rate', metric_group_name,
872+
'The average number of requests sent per second.'),
873+
Rate(sampled_stat=Count()))
874+
bytes_sent.add(metrics.metric_name(
875+
'request-size-avg', metric_group_name,
876+
'The average size of all requests in the window.'), Avg())
877+
bytes_sent.add(metrics.metric_name(
878+
'request-size-max', metric_group_name,
879+
'The maximum size of any request sent in the window.'), Max())
880+
881+
bytes_received = metrics.sensor('bytes-received',
882+
parents=[bytes_transferred])
883+
bytes_received.add(metrics.metric_name(
884+
'incoming-byte-rate', metric_group_name,
885+
'Bytes/second read off all sockets'), Rate())
886+
bytes_received.add(metrics.metric_name(
887+
'response-rate', metric_group_name,
888+
'Responses received sent per second.'),
889+
Rate(sampled_stat=Count()))
890+
891+
request_latency = metrics.sensor('request-latency')
892+
request_latency.add(metrics.metric_name(
893+
'request-latency-avg', metric_group_name,
894+
'The average request latency in ms.'),
895+
Avg())
896+
request_latency.add(metrics.metric_name(
897+
'request-latency-max', metric_group_name,
898+
'The maximum request latency in ms.'),
899+
Max())
900+
901+
# if one sensor of the metrics has been registered for the connection,
902+
# then all other sensors should have been registered; and vice versa
903+
node_str = 'node-{0}'.format(node_id)
904+
node_sensor = metrics.get_sensor(node_str + '.bytes-sent')
905+
if not node_sensor:
906+
metric_group_name = metric_group_prefix + '-node-metrics.' + node_str
907+
908+
self.bytes_sent = metrics.sensor(
909+
node_str + '.bytes-sent',
910+
parents=[metrics.get_sensor('bytes-sent')])
911+
self.bytes_sent.add(metrics.metric_name(
912+
'outgoing-byte-rate', metric_group_name,
913+
'The average number of outgoing bytes sent per second.'),
914+
Rate())
915+
self.bytes_sent.add(metrics.metric_name(
916+
'request-rate', metric_group_name,
917+
'The average number of requests sent per second.'),
918+
Rate(sampled_stat=Count()))
919+
self.bytes_sent.add(metrics.metric_name(
920+
'request-size-avg', metric_group_name,
921+
'The average size of all requests in the window.'),
922+
Avg())
923+
self.bytes_sent.add(metrics.metric_name(
924+
'request-size-max', metric_group_name,
925+
'The maximum size of any request sent in the window.'),
926+
Max())
927+
928+
self.bytes_received = metrics.sensor(
929+
node_str + '.bytes-received',
930+
parents=[metrics.get_sensor('bytes-received')])
931+
self.bytes_received.add(metrics.metric_name(
932+
'incoming-byte-rate', metric_group_name,
933+
'Bytes/second read off node-connection socket'),
934+
Rate())
935+
self.bytes_received.add(metrics.metric_name(
936+
'response-rate', metric_group_name,
937+
'The average number of responses received per second.'),
938+
Rate(sampled_stat=Count()))
939+
940+
self.request_time = self.metrics.sensor(
941+
node_str + '.latency',
942+
parents=[metrics.get_sensor('request-latency')])
943+
self.request_time.add(metrics.metric_name(
944+
'request-latency-avg', metric_group_name,
945+
'The average request latency in ms.'),
946+
Avg())
947+
self.request_time.add(metrics.metric_name(
948+
'request-latency-max', metric_group_name,
949+
'The maximum request latency in ms.'),
950+
Max())
951+
952+
765953
def _address_family(address):
766954
"""
767955
Attempt to determine the family of an address (or hostname)

kafka/consumer/fetcher.py

+3-3
Original file line numberDiff line numberDiff line change
@@ -42,11 +42,11 @@ class Fetcher(six.Iterator):
4242
'check_crcs': True,
4343
'skip_double_compressed_messages': False,
4444
'iterator_refetch_records': 1, # undocumented -- interface may change
45+
'metric_group_prefix': 'consumer',
4546
'api_version': (0, 8, 0),
4647
}
4748

48-
def __init__(self, client, subscriptions, metrics, metric_group_prefix,
49-
**configs):
49+
def __init__(self, client, subscriptions, metrics, **configs):
5050
"""Initialize a Kafka Message Fetcher.
5151
5252
Keyword Arguments:
@@ -94,7 +94,7 @@ def __init__(self, client, subscriptions, metrics, metric_group_prefix,
9494
self._record_too_large_partitions = dict() # {topic_partition: offset}
9595
self._iterator = None
9696
self._fetch_futures = collections.deque()
97-
self._sensors = FetchManagerMetrics(metrics, metric_group_prefix)
97+
self._sensors = FetchManagerMetrics(metrics, self.config['metric_group_prefix'])
9898

9999
def init_fetches(self):
100100
"""Send FetchRequests asynchronously for all assigned partitions.

0 commit comments

Comments
 (0)