Instrument bufferpool-wait-ratio metric in KafkaProducer

This commit is contained in:
Dana Powers
2016-08-04 12:54:53 -07:00
parent 460f0784a3
commit 025b69ef4a
3 changed files with 17 additions and 8 deletions

View File

@@ -9,6 +9,7 @@ from ..codec import (has_gzip, has_snappy, has_lz4,
gzip_encode, snappy_encode, gzip_encode, snappy_encode,
lz4_encode, lz4_encode_old_kafka) lz4_encode, lz4_encode_old_kafka)
from .. import errors as Errors from .. import errors as Errors
from ..metrics.stats import Rate
from ..protocol.types import Int32, Int64 from ..protocol.types import Int32, Int64
from ..protocol.message import MessageSet, Message from ..protocol.message import MessageSet, Message
@@ -135,7 +136,7 @@ class MessageSetBuffer(object):
class SimpleBufferPool(object): class SimpleBufferPool(object):
"""A simple pool of BytesIO objects with a weak memory ceiling.""" """A simple pool of BytesIO objects with a weak memory ceiling."""
def __init__(self, memory, poolable_size): def __init__(self, memory, poolable_size, metrics=None, metric_group_prefix='producer-metrics'):
"""Create a new buffer pool. """Create a new buffer pool.
Arguments: Arguments:
@@ -150,10 +151,13 @@ class SimpleBufferPool(object):
self._free = collections.deque([io.BytesIO() for _ in range(buffers)]) self._free = collections.deque([io.BytesIO() for _ in range(buffers)])
self._waiters = collections.deque() self._waiters = collections.deque()
#self.metrics = metrics; self.wait_time = None
#self.waitTime = this.metrics.sensor("bufferpool-wait-time"); if metrics:
#MetricName metricName = metrics.metricName("bufferpool-wait-ratio", metricGrpName, "The fraction of time an appender waits for space allocation."); self.wait_time = metrics.sensor('bufferpool-wait-time')
#this.waitTime.add(metricName, new Rate(TimeUnit.NANOSECONDS)); self.wait_time.add(metrics.metric_name(
'bufferpool-wait-ratio', metric_group_prefix,
'The fraction of time an appender waits for space allocation.'),
Rate())
def allocate(self, size, max_time_to_block_ms): def allocate(self, size, max_time_to_block_ms):
""" """
@@ -187,7 +191,8 @@ class SimpleBufferPool(object):
start_wait = time.time() start_wait = time.time()
more_memory.wait(max_time_to_block_ms / 1000.0) more_memory.wait(max_time_to_block_ms / 1000.0)
end_wait = time.time() end_wait = time.time()
#this.waitTime.record(endWait - startWait, time.milliseconds()); if self.wait_time:
self.wait_time.record(end_wait - start_wait)
if self._free: if self._free:
buf = self._free.popleft() buf = self._free.popleft()

View File

@@ -335,7 +335,7 @@ class KafkaProducer(object):
assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers' assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers'
message_version = 1 if self.config['api_version'] >= (0, 10) else 0 message_version = 1 if self.config['api_version'] >= (0, 10) else 0
self._accumulator = RecordAccumulator(message_version=message_version, **self.config) self._accumulator = RecordAccumulator(message_version=message_version, metrics=self._metrics, **self.config)
self._metadata = client.cluster self._metadata = client.cluster
guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1) guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1)
self._sender = Sender(client, self._metadata, self._sender = Sender(client, self._metadata,

View File

@@ -162,6 +162,8 @@ class RecordAccumulator(object):
'linger_ms': 0, 'linger_ms': 0,
'retry_backoff_ms': 100, 'retry_backoff_ms': 100,
'message_version': 0, 'message_version': 0,
'metrics': None,
'metric_group_prefix': 'producer-metrics',
} }
def __init__(self, **configs): def __init__(self, **configs):
@@ -176,7 +178,9 @@ class RecordAccumulator(object):
self._batches = collections.defaultdict(collections.deque) # TopicPartition: [RecordBatch] self._batches = collections.defaultdict(collections.deque) # TopicPartition: [RecordBatch]
self._tp_locks = {None: threading.Lock()} # TopicPartition: Lock, plus a lock to add entries self._tp_locks = {None: threading.Lock()} # TopicPartition: Lock, plus a lock to add entries
self._free = SimpleBufferPool(self.config['buffer_memory'], self._free = SimpleBufferPool(self.config['buffer_memory'],
self.config['batch_size']) self.config['batch_size'],
metrics=self.config['metrics'],
metric_group_prefix=self.config['metric_group_prefix'])
self._incomplete = IncompleteRecordBatches() self._incomplete = IncompleteRecordBatches()
# The following variables should only be accessed by the sender thread, # The following variables should only be accessed by the sender thread,
# so we don't need to protect them w/ locking. # so we don't need to protect them w/ locking.