sender.py 21 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492
  1. from __future__ import absolute_import, division
  2. import collections
  3. import copy
  4. import logging
  5. import threading
  6. import time
  7. from kafka.vendor import six
  8. from .. import errors as Errors
  9. from ..metrics.measurable import AnonMeasurable
  10. from ..metrics.stats import Avg, Max, Rate
  11. from ..protocol.produce import ProduceRequest
  12. from ..structs import TopicPartition
  13. from ..version import __version__
  14. log = logging.getLogger(__name__)
  15. class Sender(threading.Thread):
  16. """
  17. The background thread that handles the sending of produce requests to the
  18. Kafka cluster. This thread makes metadata requests to renew its view of the
  19. cluster and then sends produce requests to the appropriate nodes.
  20. """
  21. DEFAULT_CONFIG = {
  22. 'max_request_size': 1048576,
  23. 'acks': 1,
  24. 'retries': 0,
  25. 'request_timeout_ms': 30000,
  26. 'guarantee_message_order': False,
  27. 'client_id': 'kafka-python-' + __version__,
  28. 'api_version': (0, 8, 0),
  29. }
  30. def __init__(self, client, metadata, accumulator, metrics, **configs):
  31. super(Sender, self).__init__()
  32. self.config = copy.copy(self.DEFAULT_CONFIG)
  33. for key in self.config:
  34. if key in configs:
  35. self.config[key] = configs.pop(key)
  36. self.name = self.config['client_id'] + '-network-thread'
  37. self._client = client
  38. self._accumulator = accumulator
  39. self._metadata = client.cluster
  40. self._running = True
  41. self._force_close = False
  42. self._topics_to_add = set()
  43. self._sensors = SenderMetrics(metrics, self._client, self._metadata)
  44. def run(self):
  45. """The main run loop for the sender thread."""
  46. log.debug("Starting Kafka producer I/O thread.")
  47. # main loop, runs until close is called
  48. while self._running:
  49. try:
  50. self.run_once()
  51. except Exception:
  52. log.exception("Uncaught error in kafka producer I/O thread")
  53. log.debug("Beginning shutdown of Kafka producer I/O thread, sending"
  54. " remaining records.")
  55. # okay we stopped accepting requests but there may still be
  56. # requests in the accumulator or waiting for acknowledgment,
  57. # wait until these are completed.
  58. while (not self._force_close
  59. and (self._accumulator.has_unsent()
  60. or self._client.in_flight_request_count() > 0)):
  61. try:
  62. self.run_once()
  63. except Exception:
  64. log.exception("Uncaught error in kafka producer I/O thread")
  65. if self._force_close:
  66. # We need to fail all the incomplete batches and wake up the
  67. # threads waiting on the futures.
  68. self._accumulator.abort_incomplete_batches()
  69. try:
  70. self._client.close()
  71. except Exception:
  72. log.exception("Failed to close network client")
  73. log.debug("Shutdown of Kafka producer I/O thread has completed.")
  74. def run_once(self):
  75. """Run a single iteration of sending."""
  76. while self._topics_to_add:
  77. self._client.add_topic(self._topics_to_add.pop())
  78. # get the list of partitions with data ready to send
  79. result = self._accumulator.ready(self._metadata)
  80. ready_nodes, next_ready_check_delay, unknown_leaders_exist = result
  81. # if there are any partitions whose leaders are not known yet, force
  82. # metadata update
  83. if unknown_leaders_exist:
  84. log.debug('Unknown leaders exist, requesting metadata update')
  85. self._metadata.request_update()
  86. # remove any nodes we aren't ready to send to
  87. not_ready_timeout = 999999999
  88. for node in list(ready_nodes):
  89. if not self._client.ready(node):
  90. log.debug('Node %s not ready; delaying produce of accumulated batch', node)
  91. ready_nodes.remove(node)
  92. not_ready_timeout = min(not_ready_timeout,
  93. self._client.connection_delay(node))
  94. # create produce requests
  95. batches_by_node = self._accumulator.drain(
  96. self._metadata, ready_nodes, self.config['max_request_size'])
  97. if self.config['guarantee_message_order']:
  98. # Mute all the partitions drained
  99. for batch_list in six.itervalues(batches_by_node):
  100. for batch in batch_list:
  101. self._accumulator.muted.add(batch.topic_partition)
  102. expired_batches = self._accumulator.abort_expired_batches(
  103. self.config['request_timeout_ms'], self._metadata)
  104. for expired_batch in expired_batches:
  105. self._sensors.record_errors(expired_batch.topic_partition.topic, expired_batch.record_count)
  106. self._sensors.update_produce_request_metrics(batches_by_node)
  107. requests = self._create_produce_requests(batches_by_node)
  108. # If we have any nodes that are ready to send + have sendable data,
  109. # poll with 0 timeout so this can immediately loop and try sending more
  110. # data. Otherwise, the timeout is determined by nodes that have
  111. # partitions with data that isn't yet sendable (e.g. lingering, backing
  112. # off). Note that this specifically does not include nodes with
  113. # sendable data that aren't ready to send since they would cause busy
  114. # looping.
  115. poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout)
  116. if ready_nodes:
  117. log.debug("Nodes with data ready to send: %s", ready_nodes) # trace
  118. log.debug("Created %d produce requests: %s", len(requests), requests) # trace
  119. poll_timeout_ms = 0
  120. for node_id, request in six.iteritems(requests):
  121. batches = batches_by_node[node_id]
  122. log.debug('Sending Produce Request: %r', request)
  123. (self._client.send(node_id, request)
  124. .add_callback(
  125. self._handle_produce_response, node_id, time.time(), batches)
  126. .add_errback(
  127. self._failed_produce, batches, node_id))
  128. # if some partitions are already ready to be sent, the select time
  129. # would be 0; otherwise if some partition already has some data
  130. # accumulated but not ready yet, the select time will be the time
  131. # difference between now and its linger expiry time; otherwise the
  132. # select time will be the time difference between now and the
  133. # metadata expiry time
  134. self._client.poll(poll_timeout_ms)
  135. def initiate_close(self):
  136. """Start closing the sender (won't complete until all data is sent)."""
  137. self._running = False
  138. self._accumulator.close()
  139. self.wakeup()
  140. def force_close(self):
  141. """Closes the sender without sending out any pending messages."""
  142. self._force_close = True
  143. self.initiate_close()
  144. def add_topic(self, topic):
  145. # This is generally called from a separate thread
  146. # so this needs to be a thread-safe operation
  147. # we assume that checking set membership across threads
  148. # is ok where self._client._topics should never
  149. # remove topics for a producer instance, only add them.
  150. if topic not in self._client._topics:
  151. self._topics_to_add.add(topic)
  152. self.wakeup()
  153. def _failed_produce(self, batches, node_id, error):
  154. log.debug("Error sending produce request to node %d: %s", node_id, error) # trace
  155. for batch in batches:
  156. self._complete_batch(batch, error, -1, None)
  157. def _handle_produce_response(self, node_id, send_time, batches, response):
  158. """Handle a produce response."""
  159. # if we have a response, parse it
  160. log.debug('Parsing produce response: %r', response)
  161. if response:
  162. batches_by_partition = dict([(batch.topic_partition, batch)
  163. for batch in batches])
  164. for topic, partitions in response.topics:
  165. for partition_info in partitions:
  166. if response.API_VERSION < 2:
  167. partition, error_code, offset = partition_info
  168. ts = None
  169. else:
  170. partition, error_code, offset, ts = partition_info
  171. tp = TopicPartition(topic, partition)
  172. error = Errors.for_code(error_code)
  173. batch = batches_by_partition[tp]
  174. self._complete_batch(batch, error, offset, ts)
  175. if response.API_VERSION > 0:
  176. self._sensors.record_throttle_time(response.throttle_time_ms, node=node_id)
  177. else:
  178. # this is the acks = 0 case, just complete all requests
  179. for batch in batches:
  180. self._complete_batch(batch, None, -1, None)
  181. def _complete_batch(self, batch, error, base_offset, timestamp_ms=None):
  182. """Complete or retry the given batch of records.
  183. Arguments:
  184. batch (RecordBatch): The record batch
  185. error (Exception): The error (or None if none)
  186. base_offset (int): The base offset assigned to the records if successful
  187. timestamp_ms (int, optional): The timestamp returned by the broker for this batch
  188. """
  189. # Standardize no-error to None
  190. if error is Errors.NoError:
  191. error = None
  192. if error is not None and self._can_retry(batch, error):
  193. # retry
  194. log.warning("Got error produce response on topic-partition %s,"
  195. " retrying (%d attempts left). Error: %s",
  196. batch.topic_partition,
  197. self.config['retries'] - batch.attempts - 1,
  198. error)
  199. self._accumulator.reenqueue(batch)
  200. self._sensors.record_retries(batch.topic_partition.topic, batch.record_count)
  201. else:
  202. if error is Errors.TopicAuthorizationFailedError:
  203. error = error(batch.topic_partition.topic)
  204. # tell the user the result of their request
  205. batch.done(base_offset, timestamp_ms, error)
  206. self._accumulator.deallocate(batch)
  207. if error is not None:
  208. self._sensors.record_errors(batch.topic_partition.topic, batch.record_count)
  209. if getattr(error, 'invalid_metadata', False):
  210. self._metadata.request_update()
  211. # Unmute the completed partition.
  212. if self.config['guarantee_message_order']:
  213. self._accumulator.muted.remove(batch.topic_partition)
  214. def _can_retry(self, batch, error):
  215. """
  216. We can retry a send if the error is transient and the number of
  217. attempts taken is fewer than the maximum allowed
  218. """
  219. return (batch.attempts < self.config['retries']
  220. and getattr(error, 'retriable', False))
  221. def _create_produce_requests(self, collated):
  222. """
  223. Transfer the record batches into a list of produce requests on a
  224. per-node basis.
  225. Arguments:
  226. collated: {node_id: [RecordBatch]}
  227. Returns:
  228. dict: {node_id: ProduceRequest} (version depends on api_version)
  229. """
  230. requests = {}
  231. for node_id, batches in six.iteritems(collated):
  232. requests[node_id] = self._produce_request(
  233. node_id, self.config['acks'],
  234. self.config['request_timeout_ms'], batches)
  235. return requests
  236. def _produce_request(self, node_id, acks, timeout, batches):
  237. """Create a produce request from the given record batches.
  238. Returns:
  239. ProduceRequest (version depends on api_version)
  240. """
  241. produce_records_by_partition = collections.defaultdict(dict)
  242. for batch in batches:
  243. topic = batch.topic_partition.topic
  244. partition = batch.topic_partition.partition
  245. # TODO: bytearray / memoryview
  246. buf = batch.records.buffer()
  247. produce_records_by_partition[topic][partition] = buf
  248. if self.config['api_version'] >= (0, 10):
  249. version = 2
  250. elif self.config['api_version'] == (0, 9):
  251. version = 1
  252. else:
  253. version = 0
  254. return ProduceRequest[version](
  255. required_acks=acks,
  256. timeout=timeout,
  257. topics=[(topic, list(partition_info.items()))
  258. for topic, partition_info
  259. in six.iteritems(produce_records_by_partition)]
  260. )
  261. def wakeup(self):
  262. """Wake up the selector associated with this send thread."""
  263. self._client.wakeup()
  264. class SenderMetrics(object):
  265. def __init__(self, metrics, client, metadata):
  266. self.metrics = metrics
  267. self._client = client
  268. self._metadata = metadata
  269. sensor_name = 'batch-size'
  270. self.batch_size_sensor = self.metrics.sensor(sensor_name)
  271. self.add_metric('batch-size-avg', Avg(),
  272. sensor_name=sensor_name,
  273. description='The average number of bytes sent per partition per-request.')
  274. self.add_metric('batch-size-max', Max(),
  275. sensor_name=sensor_name,
  276. description='The max number of bytes sent per partition per-request.')
  277. sensor_name = 'compression-rate'
  278. self.compression_rate_sensor = self.metrics.sensor(sensor_name)
  279. self.add_metric('compression-rate-avg', Avg(),
  280. sensor_name=sensor_name,
  281. description='The average compression rate of record batches.')
  282. sensor_name = 'queue-time'
  283. self.queue_time_sensor = self.metrics.sensor(sensor_name)
  284. self.add_metric('record-queue-time-avg', Avg(),
  285. sensor_name=sensor_name,
  286. description='The average time in ms record batches spent in the record accumulator.')
  287. self.add_metric('record-queue-time-max', Max(),
  288. sensor_name=sensor_name,
  289. description='The maximum time in ms record batches spent in the record accumulator.')
  290. sensor_name = 'produce-throttle-time'
  291. self.produce_throttle_time_sensor = self.metrics.sensor(sensor_name)
  292. self.add_metric('produce-throttle-time-avg', Avg(),
  293. sensor_name=sensor_name,
  294. description='The average throttle time in ms')
  295. self.add_metric('produce-throttle-time-max', Max(),
  296. sensor_name=sensor_name,
  297. description='The maximum throttle time in ms')
  298. sensor_name = 'records-per-request'
  299. self.records_per_request_sensor = self.metrics.sensor(sensor_name)
  300. self.add_metric('record-send-rate', Rate(),
  301. sensor_name=sensor_name,
  302. description='The average number of records sent per second.')
  303. self.add_metric('records-per-request-avg', Avg(),
  304. sensor_name=sensor_name,
  305. description='The average number of records per request.')
  306. sensor_name = 'bytes'
  307. self.byte_rate_sensor = self.metrics.sensor(sensor_name)
  308. self.add_metric('byte-rate', Rate(),
  309. sensor_name=sensor_name,
  310. description='The average number of bytes sent per second.')
  311. sensor_name = 'record-retries'
  312. self.retry_sensor = self.metrics.sensor(sensor_name)
  313. self.add_metric('record-retry-rate', Rate(),
  314. sensor_name=sensor_name,
  315. description='The average per-second number of retried record sends')
  316. sensor_name = 'errors'
  317. self.error_sensor = self.metrics.sensor(sensor_name)
  318. self.add_metric('record-error-rate', Rate(),
  319. sensor_name=sensor_name,
  320. description='The average per-second number of record sends that resulted in errors')
  321. sensor_name = 'record-size-max'
  322. self.max_record_size_sensor = self.metrics.sensor(sensor_name)
  323. self.add_metric('record-size-max', Max(),
  324. sensor_name=sensor_name,
  325. description='The maximum record size across all batches')
  326. self.add_metric('record-size-avg', Avg(),
  327. sensor_name=sensor_name,
  328. description='The average maximum record size per batch')
  329. self.add_metric('requests-in-flight',
  330. AnonMeasurable(lambda *_: self._client.in_flight_request_count()),
  331. description='The current number of in-flight requests awaiting a response.')
  332. self.add_metric('metadata-age',
  333. AnonMeasurable(lambda _, now: (now - self._metadata._last_successful_refresh_ms) / 1000),
  334. description='The age in seconds of the current producer metadata being used.')
  335. def add_metric(self, metric_name, measurable, group_name='producer-metrics',
  336. description=None, tags=None,
  337. sensor_name=None):
  338. m = self.metrics
  339. metric = m.metric_name(metric_name, group_name, description, tags)
  340. if sensor_name:
  341. sensor = m.sensor(sensor_name)
  342. sensor.add(metric, measurable)
  343. else:
  344. m.add_metric(metric, measurable)
  345. def maybe_register_topic_metrics(self, topic):
  346. def sensor_name(name):
  347. return 'topic.{0}.{1}'.format(topic, name)
  348. # if one sensor of the metrics has been registered for the topic,
  349. # then all other sensors should have been registered; and vice versa
  350. if not self.metrics.get_sensor(sensor_name('records-per-batch')):
  351. self.add_metric('record-send-rate', Rate(),
  352. sensor_name=sensor_name('records-per-batch'),
  353. group_name='producer-topic-metrics.' + topic,
  354. description= 'Records sent per second for topic ' + topic)
  355. self.add_metric('byte-rate', Rate(),
  356. sensor_name=sensor_name('bytes'),
  357. group_name='producer-topic-metrics.' + topic,
  358. description='Bytes per second for topic ' + topic)
  359. self.add_metric('compression-rate', Avg(),
  360. sensor_name=sensor_name('compression-rate'),
  361. group_name='producer-topic-metrics.' + topic,
  362. description='Average Compression ratio for topic ' + topic)
  363. self.add_metric('record-retry-rate', Rate(),
  364. sensor_name=sensor_name('record-retries'),
  365. group_name='producer-topic-metrics.' + topic,
  366. description='Record retries per second for topic ' + topic)
  367. self.add_metric('record-error-rate', Rate(),
  368. sensor_name=sensor_name('record-errors'),
  369. group_name='producer-topic-metrics.' + topic,
  370. description='Record errors per second for topic ' + topic)
  371. def update_produce_request_metrics(self, batches_map):
  372. for node_batch in batches_map.values():
  373. records = 0
  374. total_bytes = 0
  375. for batch in node_batch:
  376. # register all per-topic metrics at once
  377. topic = batch.topic_partition.topic
  378. self.maybe_register_topic_metrics(topic)
  379. # per-topic record send rate
  380. topic_records_count = self.metrics.get_sensor(
  381. 'topic.' + topic + '.records-per-batch')
  382. topic_records_count.record(batch.record_count)
  383. # per-topic bytes send rate
  384. topic_byte_rate = self.metrics.get_sensor(
  385. 'topic.' + topic + '.bytes')
  386. topic_byte_rate.record(batch.records.size_in_bytes())
  387. # per-topic compression rate
  388. topic_compression_rate = self.metrics.get_sensor(
  389. 'topic.' + topic + '.compression-rate')
  390. topic_compression_rate.record(batch.records.compression_rate())
  391. # global metrics
  392. self.batch_size_sensor.record(batch.records.size_in_bytes())
  393. if batch.drained:
  394. self.queue_time_sensor.record(batch.drained - batch.created)
  395. self.compression_rate_sensor.record(batch.records.compression_rate())
  396. self.max_record_size_sensor.record(batch.max_record_size)
  397. records += batch.record_count
  398. total_bytes += batch.records.size_in_bytes()
  399. self.records_per_request_sensor.record(records)
  400. self.byte_rate_sensor.record(total_bytes)
  401. def record_retries(self, topic, count):
  402. self.retry_sensor.record(count)
  403. sensor = self.metrics.get_sensor('topic.' + topic + '.record-retries')
  404. if sensor:
  405. sensor.record(count)
  406. def record_errors(self, topic, count):
  407. self.error_sensor.record(count)
  408. sensor = self.metrics.get_sensor('topic.' + topic + '.record-errors')
  409. if sensor:
  410. sensor.record(count)
  411. def record_throttle_time(self, throttle_time_ms, node=None):
  412. self.produce_throttle_time_sensor.record(throttle_time_ms)