client_async.py 42 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056
  1. from __future__ import absolute_import, division
  2. import copy
  3. import functools
  4. import heapq
  5. import itertools
  6. import logging
  7. import random
  8. import threading
  9. # selectors in stdlib as of py3.4
  10. try:
  11. import selectors # pylint: disable=import-error
  12. except ImportError:
  13. # vendored backport module
  14. from .vendor import selectors34 as selectors
  15. import socket
  16. import time
  17. from kafka.vendor import six
  18. from .cluster import ClusterMetadata
  19. from .conn import BrokerConnection, ConnectionStates, collect_hosts, get_ip_port_afi
  20. from . import errors as Errors
  21. from .future import Future
  22. from .metrics import AnonMeasurable
  23. from .metrics.stats import Avg, Count, Rate
  24. from .metrics.stats.rate import TimeUnit
  25. from .protocol.metadata import MetadataRequest
  26. # Although this looks unused, it actually monkey-patches socket.socketpair()
  27. # and should be left in as long as we're using socket.socketpair() in this file
  28. from .vendor import socketpair
  29. from .version import __version__
  30. if six.PY2:
  31. ConnectionError = None
  32. log = logging.getLogger('kafka.client')
  33. class KafkaClient(object):
  34. """
  35. A network client for asynchronous request/response network I/O.
  36. This is an internal class used to implement the user-facing producer and
  37. consumer clients.
  38. This class is not thread-safe!
  39. Attributes:
  40. cluster (:any:`ClusterMetadata`): Local cache of cluster metadata, retrieved
  41. via MetadataRequests during :meth:`~kafka.KafkaClient.poll`.
  42. Keyword Arguments:
  43. bootstrap_servers: 'host[:port]' string (or list of 'host[:port]'
  44. strings) that the consumer should contact to bootstrap initial
  45. cluster metadata. This does not have to be the full node list.
  46. It just needs to have at least one broker that will respond to a
  47. Metadata API Request. Default port is 9092. If no servers are
  48. specified, will default to localhost:9092.
  49. client_id (str): a name for this client. This string is passed in
  50. each request to servers and can be used to identify specific
  51. server-side log entries that correspond to this client. Also
  52. submitted to GroupCoordinator for logging with respect to
  53. consumer group administration. Default: 'kafka-python-{version}'
  54. reconnect_backoff_ms (int): The amount of time in milliseconds to
  55. wait before attempting to reconnect to a given host.
  56. Default: 50.
  57. reconnect_backoff_max_ms (int): The maximum amount of time in
  58. milliseconds to wait when reconnecting to a broker that has
  59. repeatedly failed to connect. If provided, the backoff per host
  60. will increase exponentially for each consecutive connection
  61. failure, up to this maximum. To avoid connection storms, a
  62. randomization factor of 0.2 will be applied to the backoff
  63. resulting in a random range between 20% below and 20% above
  64. the computed value. Default: 1000.
  65. request_timeout_ms (int): Client request timeout in milliseconds.
  66. Default: 40000.
  67. retry_backoff_ms (int): Milliseconds to backoff when retrying on
  68. errors. Default: 100.
  69. max_in_flight_requests_per_connection (int): Requests are pipelined
  70. to kafka brokers up to this number of maximum requests per
  71. broker connection. Default: 5.
  72. receive_buffer_bytes (int): The size of the TCP receive buffer
  73. (SO_RCVBUF) to use when reading data. Default: None (relies on
  74. system defaults). Java client defaults to 32768.
  75. send_buffer_bytes (int): The size of the TCP send buffer
  76. (SO_SNDBUF) to use when sending data. Default: None (relies on
  77. system defaults). Java client defaults to 131072.
  78. socket_options (list): List of tuple-arguments to socket.setsockopt
  79. to apply to broker connection sockets. Default:
  80. [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)]
  81. metadata_max_age_ms (int): The period of time in milliseconds after
  82. which we force a refresh of metadata even if we haven't seen any
  83. partition leadership changes to proactively discover any new
  84. brokers or partitions. Default: 300000
  85. security_protocol (str): Protocol used to communicate with brokers.
  86. Valid values are: PLAINTEXT, SSL. Default: PLAINTEXT.
  87. ssl_context (ssl.SSLContext): pre-configured SSLContext for wrapping
  88. socket connections. If provided, all other ssl_* configurations
  89. will be ignored. Default: None.
  90. ssl_check_hostname (bool): flag to configure whether ssl handshake
  91. should verify that the certificate matches the brokers hostname.
  92. default: true.
  93. ssl_cafile (str): optional filename of ca file to use in certificate
  94. veriication. default: none.
  95. ssl_certfile (str): optional filename of file in pem format containing
  96. the client certificate, as well as any ca certificates needed to
  97. establish the certificate's authenticity. default: none.
  98. ssl_keyfile (str): optional filename containing the client private key.
  99. default: none.
  100. ssl_password (str): optional password to be used when loading the
  101. certificate chain. default: none.
  102. ssl_crlfile (str): optional filename containing the CRL to check for
  103. certificate expiration. By default, no CRL check is done. When
  104. providing a file, only the leaf certificate will be checked against
  105. this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+.
  106. default: none.
  107. api_version (tuple): Specify which Kafka API version to use. If set
  108. to None, KafkaClient will attempt to infer the broker version by
  109. probing various APIs. For the full list of supported versions,
  110. see KafkaClient.API_VERSIONS. Default: None
  111. api_version_auto_timeout_ms (int): number of milliseconds to throw a
  112. timeout exception from the constructor when checking the broker
  113. api version. Only applies if api_version is None
  114. selector (selectors.BaseSelector): Provide a specific selector
  115. implementation to use for I/O multiplexing.
  116. Default: selectors.DefaultSelector
  117. metrics (kafka.metrics.Metrics): Optionally provide a metrics
  118. instance for capturing network IO stats. Default: None.
  119. metric_group_prefix (str): Prefix for metric names. Default: ''
  120. sasl_mechanism (str): string picking sasl mechanism when security_protocol
  121. is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
  122. Default: None
  123. sasl_plain_username (str): username for sasl PLAIN authentication.
  124. Default: None
  125. sasl_plain_password (str): password for sasl PLAIN authentication.
  126. Default: None
  127. """
  128. DEFAULT_CONFIG = {
  129. 'bootstrap_servers': 'localhost',
  130. 'client_id': 'kafka-python-' + __version__,
  131. 'request_timeout_ms': 40000,
  132. 'connections_max_idle_ms': 9 * 60 * 1000,
  133. 'reconnect_backoff_ms': 50,
  134. 'reconnect_backoff_max_ms': 1000,
  135. 'max_in_flight_requests_per_connection': 5,
  136. 'receive_buffer_bytes': None,
  137. 'send_buffer_bytes': None,
  138. 'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
  139. 'retry_backoff_ms': 100,
  140. 'metadata_max_age_ms': 300000,
  141. 'security_protocol': 'PLAINTEXT',
  142. 'ssl_context': None,
  143. 'ssl_check_hostname': True,
  144. 'ssl_cafile': None,
  145. 'ssl_certfile': None,
  146. 'ssl_keyfile': None,
  147. 'ssl_password': None,
  148. 'ssl_crlfile': None,
  149. 'api_version': None,
  150. 'api_version_auto_timeout_ms': 2000,
  151. 'selector': selectors.DefaultSelector,
  152. 'metrics': None,
  153. 'metric_group_prefix': '',
  154. 'sasl_mechanism': None,
  155. 'sasl_plain_username': None,
  156. 'sasl_plain_password': None,
  157. }
  158. API_VERSIONS = [
  159. (0, 10, 1),
  160. (0, 10, 0),
  161. (0, 10),
  162. (0, 9),
  163. (0, 8, 2),
  164. (0, 8, 1),
  165. (0, 8, 0)
  166. ]
  167. def __init__(self, **configs):
  168. self.config = copy.copy(self.DEFAULT_CONFIG)
  169. for key in self.config:
  170. if key in configs:
  171. self.config[key] = configs[key]
  172. if self.config['api_version'] is not None:
  173. assert self.config['api_version'] in self.API_VERSIONS, (
  174. 'api_version [{0}] must be one of: {1}'.format(
  175. self.config['api_version'], str(self.API_VERSIONS)))
  176. self.cluster = ClusterMetadata(**self.config)
  177. self._topics = set() # empty set will fetch all topic metadata
  178. self._metadata_refresh_in_progress = False
  179. self._selector = self.config['selector']()
  180. self._conns = {}
  181. self._connecting = set()
  182. self._refresh_on_disconnects = True
  183. self._delayed_tasks = DelayedTaskQueue()
  184. self._last_bootstrap = 0
  185. self._bootstrap_fails = 0
  186. self._wake_r, self._wake_w = socket.socketpair()
  187. self._wake_r.setblocking(False)
  188. self._wake_lock = threading.Lock()
  189. self._selector.register(self._wake_r, selectors.EVENT_READ)
  190. self._idle_expiry_manager = IdleConnectionManager(self.config['connections_max_idle_ms'])
  191. self._closed = False
  192. self._sensors = None
  193. if self.config['metrics']:
  194. self._sensors = KafkaClientMetrics(self.config['metrics'],
  195. self.config['metric_group_prefix'],
  196. self._conns)
  197. self._bootstrap(collect_hosts(self.config['bootstrap_servers']))
  198. # Check Broker Version if not set explicitly
  199. if self.config['api_version'] is None:
  200. check_timeout = self.config['api_version_auto_timeout_ms'] / 1000
  201. self.config['api_version'] = self.check_version(timeout=check_timeout)
  202. def _bootstrap(self, hosts):
  203. log.info('Bootstrapping cluster metadata from %s', hosts)
  204. # Exponential backoff if bootstrap fails
  205. backoff_ms = self.config['reconnect_backoff_ms'] * 2 ** self._bootstrap_fails
  206. next_at = self._last_bootstrap + backoff_ms / 1000.0
  207. self._refresh_on_disconnects = False
  208. now = time.time()
  209. if next_at > now:
  210. log.debug("Sleeping %0.4f before bootstrapping again", next_at - now)
  211. time.sleep(next_at - now)
  212. self._last_bootstrap = time.time()
  213. if self.config['api_version'] is None or self.config['api_version'] < (0, 10):
  214. metadata_request = MetadataRequest[0]([])
  215. else:
  216. metadata_request = MetadataRequest[1](None)
  217. for host, port, afi in hosts:
  218. log.debug("Attempting to bootstrap via node at %s:%s", host, port)
  219. cb = functools.partial(self._conn_state_change, 'bootstrap')
  220. bootstrap = BrokerConnection(host, port, afi,
  221. state_change_callback=cb,
  222. node_id='bootstrap',
  223. **self.config)
  224. bootstrap.connect()
  225. while bootstrap.connecting():
  226. self._selector.select(1)
  227. bootstrap.connect()
  228. if not bootstrap.connected():
  229. bootstrap.close()
  230. continue
  231. future = bootstrap.send(metadata_request)
  232. while not future.is_done:
  233. self._selector.select(1)
  234. bootstrap.recv()
  235. if future.failed():
  236. bootstrap.close()
  237. continue
  238. self.cluster.update_metadata(future.value)
  239. log.info('Bootstrap succeeded: found %d brokers and %d topics.',
  240. len(self.cluster.brokers()), len(self.cluster.topics()))
  241. # A cluster with no topics can return no broker metadata
  242. # in that case, we should keep the bootstrap connection
  243. if not len(self.cluster.brokers()):
  244. self._conns['bootstrap'] = bootstrap
  245. else:
  246. bootstrap.close()
  247. self._bootstrap_fails = 0
  248. break
  249. # No bootstrap found...
  250. else:
  251. log.error('Unable to bootstrap from %s', hosts)
  252. # Max exponential backoff is 2^12, x4000 (50ms -> 200s)
  253. self._bootstrap_fails = min(self._bootstrap_fails + 1, 12)
  254. self._refresh_on_disconnects = True
  255. def _can_connect(self, node_id):
  256. if node_id not in self._conns:
  257. if self.cluster.broker_metadata(node_id):
  258. return True
  259. return False
  260. conn = self._conns[node_id]
  261. return conn.disconnected() and not conn.blacked_out()
  262. def _conn_state_change(self, node_id, conn):
  263. if conn.connecting():
  264. # SSL connections can enter this state 2x (second during Handshake)
  265. if node_id not in self._connecting:
  266. self._connecting.add(node_id)
  267. self._selector.register(conn._sock, selectors.EVENT_WRITE)
  268. elif conn.connected():
  269. log.debug("Node %s connected", node_id)
  270. if node_id in self._connecting:
  271. self._connecting.remove(node_id)
  272. try:
  273. self._selector.unregister(conn._sock)
  274. except KeyError:
  275. pass
  276. self._selector.register(conn._sock, selectors.EVENT_READ, conn)
  277. if self._sensors:
  278. self._sensors.connection_created.record()
  279. self._idle_expiry_manager.update(node_id)
  280. if 'bootstrap' in self._conns and node_id != 'bootstrap':
  281. bootstrap = self._conns.pop('bootstrap')
  282. # XXX: make conn.close() require error to cause refresh
  283. self._refresh_on_disconnects = False
  284. bootstrap.close()
  285. self._refresh_on_disconnects = True
  286. # Connection failures imply that our metadata is stale, so let's refresh
  287. elif conn.state is ConnectionStates.DISCONNECTING:
  288. if node_id in self._connecting:
  289. self._connecting.remove(node_id)
  290. try:
  291. self._selector.unregister(conn._sock)
  292. except KeyError:
  293. pass
  294. if self._sensors:
  295. self._sensors.connection_closed.record()
  296. idle_disconnect = False
  297. if self._idle_expiry_manager.is_expired(node_id):
  298. idle_disconnect = True
  299. self._idle_expiry_manager.remove(node_id)
  300. if self._refresh_on_disconnects and not self._closed and not idle_disconnect:
  301. log.warning("Node %s connection failed -- refreshing metadata", node_id)
  302. self.cluster.request_update()
  303. def _maybe_connect(self, node_id):
  304. """Idempotent non-blocking connection attempt to the given node id."""
  305. broker = self.cluster.broker_metadata(node_id)
  306. conn = self._conns.get(node_id)
  307. if conn is None:
  308. assert broker, 'Broker id %s not in current metadata' % node_id
  309. log.debug("Initiating connection to node %s at %s:%s",
  310. node_id, broker.host, broker.port)
  311. host, port, afi = get_ip_port_afi(broker.host)
  312. cb = functools.partial(self._conn_state_change, node_id)
  313. conn = BrokerConnection(host, broker.port, afi,
  314. state_change_callback=cb,
  315. node_id=node_id,
  316. **self.config)
  317. self._conns[node_id] = conn
  318. # Check if existing connection should be recreated because host/port changed
  319. elif conn.disconnected() and broker is not None:
  320. host, _, __ = get_ip_port_afi(broker.host)
  321. if conn.host != host or conn.port != broker.port:
  322. log.info("Broker metadata change detected for node %s"
  323. " from %s:%s to %s:%s", node_id, conn.host, conn.port,
  324. broker.host, broker.port)
  325. # Drop old connection object.
  326. # It will be recreated on next _maybe_connect
  327. self._conns.pop(node_id)
  328. return False
  329. elif conn.connected():
  330. return True
  331. conn.connect()
  332. return conn.connected()
  333. def ready(self, node_id, metadata_priority=True):
  334. """Check whether a node is connected and ok to send more requests.
  335. Arguments:
  336. node_id (int): the id of the node to check
  337. metadata_priority (bool): Mark node as not-ready if a metadata
  338. refresh is required. Default: True
  339. Returns:
  340. bool: True if we are ready to send to the given node
  341. """
  342. self._maybe_connect(node_id)
  343. return self.is_ready(node_id, metadata_priority=metadata_priority)
  344. def connected(self, node_id):
  345. """Return True iff the node_id is connected."""
  346. if node_id not in self._conns:
  347. return False
  348. return self._conns[node_id].connected()
  349. def close(self, node_id=None):
  350. """Close one or all broker connections.
  351. Arguments:
  352. node_id (int, optional): the id of the node to close
  353. """
  354. if node_id is None:
  355. self._closed = True
  356. for conn in self._conns.values():
  357. conn.close()
  358. self._wake_r.close()
  359. self._wake_w.close()
  360. self._selector.close()
  361. elif node_id in self._conns:
  362. self._conns[node_id].close()
  363. else:
  364. log.warning("Node %s not found in current connection list; skipping", node_id)
  365. return
  366. def is_disconnected(self, node_id):
  367. """Check whether the node connection has been disconnected or failed.
  368. A disconnected node has either been closed or has failed. Connection
  369. failures are usually transient and can be resumed in the next ready()
  370. call, but there are cases where transient failures need to be caught
  371. and re-acted upon.
  372. Arguments:
  373. node_id (int): the id of the node to check
  374. Returns:
  375. bool: True iff the node exists and is disconnected
  376. """
  377. if node_id not in self._conns:
  378. return False
  379. return self._conns[node_id].disconnected()
  380. def connection_delay(self, node_id):
  381. """
  382. Return the number of milliseconds to wait, based on the connection
  383. state, before attempting to send data. When disconnected, this respects
  384. the reconnect backoff time. When connecting, returns 0 to allow
  385. non-blocking connect to finish. When connected, returns a very large
  386. number to handle slow/stalled connections.
  387. Arguments:
  388. node_id (int): The id of the node to check
  389. Returns:
  390. int: The number of milliseconds to wait.
  391. """
  392. if node_id not in self._conns:
  393. return 0
  394. return self._conns[node_id].connection_delay()
  395. def is_ready(self, node_id, metadata_priority=True):
  396. """Check whether a node is ready to send more requests.
  397. In addition to connection-level checks, this method also is used to
  398. block additional requests from being sent during a metadata refresh.
  399. Arguments:
  400. node_id (int): id of the node to check
  401. metadata_priority (bool): Mark node as not-ready if a metadata
  402. refresh is required. Default: True
  403. Returns:
  404. bool: True if the node is ready and metadata is not refreshing
  405. """
  406. if not self._can_send_request(node_id):
  407. return False
  408. # if we need to update our metadata now declare all requests unready to
  409. # make metadata requests first priority
  410. if metadata_priority:
  411. if self._metadata_refresh_in_progress:
  412. return False
  413. if self.cluster.ttl() == 0:
  414. return False
  415. return True
  416. def _can_send_request(self, node_id):
  417. if node_id not in self._conns:
  418. return False
  419. conn = self._conns[node_id]
  420. return conn.connected() and conn.can_send_more()
  421. def send(self, node_id, request):
  422. """Send a request to a specific node.
  423. Arguments:
  424. node_id (int): destination node
  425. request (Struct): request object (not-encoded)
  426. Raises:
  427. AssertionError: if node_id is not in current cluster metadata
  428. Returns:
  429. Future: resolves to Response struct or Error
  430. """
  431. if not self._maybe_connect(node_id):
  432. return Future().failure(Errors.NodeNotReadyError(node_id))
  433. return self._conns[node_id].send(request)
  434. def poll(self, timeout_ms=None, future=None, delayed_tasks=True):
  435. """Try to read and write to sockets.
  436. This method will also attempt to complete node connections, refresh
  437. stale metadata, and run previously-scheduled tasks.
  438. Arguments:
  439. timeout_ms (int, optional): maximum amount of time to wait (in ms)
  440. for at least one response. Must be non-negative. The actual
  441. timeout will be the minimum of timeout, request timeout and
  442. metadata timeout. Default: request_timeout_ms
  443. future (Future, optional): if provided, blocks until future.is_done
  444. Returns:
  445. list: responses received (can be empty)
  446. """
  447. if timeout_ms is None:
  448. timeout_ms = self.config['request_timeout_ms']
  449. responses = []
  450. # Loop for futures, break after first loop if None
  451. while True:
  452. # Attempt to complete pending connections
  453. for node_id in list(self._connecting):
  454. self._maybe_connect(node_id)
  455. # Send a metadata request if needed
  456. metadata_timeout_ms = self._maybe_refresh_metadata()
  457. # Send scheduled tasks
  458. if delayed_tasks:
  459. for task, task_future in self._delayed_tasks.pop_ready():
  460. try:
  461. result = task()
  462. except Exception as e:
  463. log.error("Task %s failed: %s", task, e)
  464. task_future.failure(e)
  465. else:
  466. task_future.success(result)
  467. # If we got a future that is already done, don't block in _poll
  468. if future and future.is_done:
  469. timeout = 0
  470. else:
  471. idle_connection_timeout_ms = self._idle_expiry_manager.next_check_ms()
  472. timeout = min(
  473. timeout_ms,
  474. metadata_timeout_ms,
  475. self._delayed_tasks.next_at() * 1000,
  476. idle_connection_timeout_ms,
  477. self.config['request_timeout_ms'])
  478. timeout = max(0, timeout / 1000.0) # avoid negative timeouts
  479. responses.extend(self._poll(timeout))
  480. # If all we had was a timeout (future is None) - only do one poll
  481. # If we do have a future, we keep looping until it is done
  482. if not future or future.is_done:
  483. break
  484. return responses
  485. def _poll(self, timeout):
  486. responses = []
  487. processed = set()
  488. start_select = time.time()
  489. ready = self._selector.select(timeout)
  490. end_select = time.time()
  491. if self._sensors:
  492. self._sensors.select_time.record((end_select - start_select) * 1000000000)
  493. for key, events in ready:
  494. if key.fileobj is self._wake_r:
  495. self._clear_wake_fd()
  496. continue
  497. elif not (events & selectors.EVENT_READ):
  498. continue
  499. conn = key.data
  500. processed.add(conn)
  501. if not conn.in_flight_requests:
  502. # if we got an EVENT_READ but there were no in-flight requests, one of
  503. # two things has happened:
  504. #
  505. # 1. The remote end closed the connection (because it died, or because
  506. # a firewall timed out, or whatever)
  507. # 2. The protocol is out of sync.
  508. #
  509. # either way, we can no longer safely use this connection
  510. #
  511. # Do a 1-byte read to check protocol didnt get out of sync, and then close the conn
  512. try:
  513. unexpected_data = key.fileobj.recv(1)
  514. if unexpected_data: # anything other than a 0-byte read means protocol issues
  515. log.warning('Protocol out of sync on %r, closing', conn)
  516. except socket.error:
  517. pass
  518. conn.close(Errors.ConnectionError('Socket EVENT_READ without in-flight-requests'))
  519. continue
  520. self._idle_expiry_manager.update(conn.node_id)
  521. responses.extend(conn.recv()) # Note: conn.recv runs callbacks / errbacks
  522. # Check for additional pending SSL bytes
  523. if self.config['security_protocol'] in ('SSL', 'SASL_SSL'):
  524. # TODO: optimize
  525. for conn in self._conns.values():
  526. if conn not in processed and conn.connected() and conn._sock.pending():
  527. responses.extend(conn.recv())
  528. for conn in six.itervalues(self._conns):
  529. if conn.requests_timed_out():
  530. log.warning('%s timed out after %s ms. Closing connection.',
  531. conn, conn.config['request_timeout_ms'])
  532. conn.close(error=Errors.RequestTimedOutError(
  533. 'Request timed out after %s ms' %
  534. conn.config['request_timeout_ms']))
  535. if self._sensors:
  536. self._sensors.io_time.record((time.time() - end_select) * 1000000000)
  537. self._maybe_close_oldest_connection()
  538. return responses
  539. def in_flight_request_count(self, node_id=None):
  540. """Get the number of in-flight requests for a node or all nodes.
  541. Arguments:
  542. node_id (int, optional): a specific node to check. If unspecified,
  543. return the total for all nodes
  544. Returns:
  545. int: pending in-flight requests for the node, or all nodes if None
  546. """
  547. if node_id is not None:
  548. if node_id not in self._conns:
  549. return 0
  550. return len(self._conns[node_id].in_flight_requests)
  551. else:
  552. return sum([len(conn.in_flight_requests) for conn in self._conns.values()])
  553. def least_loaded_node(self):
  554. """Choose the node with fewest outstanding requests, with fallbacks.
  555. This method will prefer a node with an existing connection and no
  556. in-flight-requests. If no such node is found, a node will be chosen
  557. randomly from disconnected nodes that are not "blacked out" (i.e.,
  558. are not subject to a reconnect backoff).
  559. Returns:
  560. node_id or None if no suitable node was found
  561. """
  562. nodes = [broker.nodeId for broker in self.cluster.brokers()]
  563. random.shuffle(nodes)
  564. inflight = float('inf')
  565. found = None
  566. for node_id in nodes:
  567. conn = self._conns.get(node_id)
  568. connected = conn is not None and conn.connected()
  569. blacked_out = conn is not None and conn.blacked_out()
  570. curr_inflight = len(conn.in_flight_requests) if conn else 0
  571. if connected and curr_inflight == 0:
  572. # if we find an established connection
  573. # with no in-flight requests, we can stop right away
  574. return node_id
  575. elif not blacked_out and curr_inflight < inflight:
  576. # otherwise if this is the best we have found so far, record that
  577. inflight = curr_inflight
  578. found = node_id
  579. if found is not None:
  580. return found
  581. # some broker versions return an empty list of broker metadata
  582. # if there are no topics created yet. the bootstrap process
  583. # should detect this and keep a 'bootstrap' node alive until
  584. # a non-bootstrap node is connected and non-empty broker
  585. # metadata is available
  586. elif 'bootstrap' in self._conns:
  587. return 'bootstrap'
  588. return None
  589. def set_topics(self, topics):
  590. """Set specific topics to track for metadata.
  591. Arguments:
  592. topics (list of str): topics to check for metadata
  593. Returns:
  594. Future: resolves after metadata request/response
  595. """
  596. if set(topics).difference(self._topics):
  597. future = self.cluster.request_update()
  598. else:
  599. future = Future().success(set(topics))
  600. self._topics = set(topics)
  601. return future
  602. def add_topic(self, topic):
  603. """Add a topic to the list of topics tracked via metadata.
  604. Arguments:
  605. topic (str): topic to track
  606. Returns:
  607. Future: resolves after metadata request/response
  608. """
  609. if topic in self._topics:
  610. return Future().success(set(self._topics))
  611. self._topics.add(topic)
  612. return self.cluster.request_update()
  613. # request metadata update on disconnect and timedout
  614. def _maybe_refresh_metadata(self):
  615. """Send a metadata request if needed.
  616. Returns:
  617. int: milliseconds until next refresh
  618. """
  619. ttl = self.cluster.ttl()
  620. wait_for_in_progress_ms = self.config['request_timeout_ms'] if self._metadata_refresh_in_progress else 0
  621. metadata_timeout = max(ttl, wait_for_in_progress_ms)
  622. if metadata_timeout > 0:
  623. return metadata_timeout
  624. # Beware that the behavior of this method and the computation of
  625. # timeouts for poll() are highly dependent on the behavior of
  626. # least_loaded_node()
  627. node_id = self.least_loaded_node()
  628. if node_id is None:
  629. log.debug("Give up sending metadata request since no node is available");
  630. return self.config['reconnect_backoff_ms']
  631. if self._can_send_request(node_id):
  632. topics = list(self._topics)
  633. if self.cluster.need_all_topic_metadata or not topics:
  634. topics = [] if self.config['api_version'] < (0, 10) else None
  635. api_version = 0 if self.config['api_version'] < (0, 10) else 1
  636. request = MetadataRequest[api_version](topics)
  637. log.debug("Sending metadata request %s to node %s", request, node_id)
  638. future = self.send(node_id, request)
  639. future.add_callback(self.cluster.update_metadata)
  640. future.add_errback(self.cluster.failed_update)
  641. self._metadata_refresh_in_progress = True
  642. def refresh_done(val_or_error):
  643. self._metadata_refresh_in_progress = False
  644. future.add_callback(refresh_done)
  645. future.add_errback(refresh_done)
  646. return self.config['request_timeout_ms']
  647. # If there's any connection establishment underway, wait until it completes. This prevents
  648. # the client from unnecessarily connecting to additional nodes while a previous connection
  649. # attempt has not been completed.
  650. if self._connecting:
  651. # Strictly the timeout we should return here is "connect timeout", but as we don't
  652. # have such application level configuration, using request timeout instead.
  653. return self.config['request_timeout_ms']
  654. if self._can_connect(node_id):
  655. log.debug("Initializing connection to node %s for metadata request", node_id)
  656. self._maybe_connect(node_id)
  657. return self.config['reconnect_backoff_ms']
  658. # connected but can't send more, OR connecting
  659. # In either case we just need to wait for a network event
  660. # to let us know the selected connection might be usable again.
  661. return float('inf')
  662. def schedule(self, task, at):
  663. """Schedule a new task to be executed at the given time.
  664. This is "best-effort" scheduling and should only be used for coarse
  665. synchronization. A task cannot be scheduled for multiple times
  666. simultaneously; any previously scheduled instance of the same task
  667. will be cancelled.
  668. Arguments:
  669. task (callable): task to be scheduled
  670. at (float or int): epoch seconds when task should run
  671. Returns:
  672. Future: resolves to result of task call, or exception if raised
  673. """
  674. return self._delayed_tasks.add(task, at)
  675. def unschedule(self, task):
  676. """Unschedule a task.
  677. This will remove all instances of the task from the task queue.
  678. This is a no-op if the task is not scheduled.
  679. Arguments:
  680. task (callable): task to be unscheduled
  681. """
  682. self._delayed_tasks.remove(task)
  683. def check_version(self, node_id=None, timeout=2, strict=False):
  684. """Attempt to guess the version of a Kafka broker.
  685. Note: It is possible that this method blocks longer than the
  686. specified timeout. This can happen if the entire cluster
  687. is down and the client enters a bootstrap backoff sleep.
  688. This is only possible if node_id is None.
  689. Returns: version tuple, i.e. (0, 10), (0, 9), (0, 8, 2), ...
  690. Raises:
  691. NodeNotReadyError (if node_id is provided)
  692. NoBrokersAvailable (if node_id is None)
  693. UnrecognizedBrokerVersion: please file bug if seen!
  694. AssertionError (if strict=True): please file bug if seen!
  695. """
  696. end = time.time() + timeout
  697. while time.time() < end:
  698. # It is possible that least_loaded_node falls back to bootstrap,
  699. # which can block for an increasing backoff period
  700. try_node = node_id or self.least_loaded_node()
  701. if try_node is None:
  702. raise Errors.NoBrokersAvailable()
  703. self._maybe_connect(try_node)
  704. conn = self._conns[try_node]
  705. # We will intentionally cause socket failures
  706. # These should not trigger metadata refresh
  707. self._refresh_on_disconnects = False
  708. try:
  709. remaining = end - time.time()
  710. version = conn.check_version(timeout=remaining, strict=strict)
  711. return version
  712. except Errors.NodeNotReadyError:
  713. # Only raise to user if this is a node-specific request
  714. if node_id is not None:
  715. raise
  716. finally:
  717. self._refresh_on_disconnects = True
  718. # Timeout
  719. else:
  720. raise Errors.NoBrokersAvailable()
  721. def wakeup(self):
  722. with self._wake_lock:
  723. try:
  724. assert self._wake_w.send(b'x') == 1
  725. except (AssertionError, socket.error):
  726. log.warning('Unable to send to wakeup socket!')
  727. def _clear_wake_fd(self):
  728. # reading from wake socket should only happen in a single thread
  729. while True:
  730. try:
  731. self._wake_r.recv(1024)
  732. except socket.error:
  733. break
  734. def _maybe_close_oldest_connection(self):
  735. expired_connection = self._idle_expiry_manager.poll_expired_connection()
  736. if expired_connection:
  737. conn_id, ts = expired_connection
  738. idle_ms = (time.time() - ts) * 1000
  739. log.info('Closing idle connection %s, last active %d ms ago', conn_id, idle_ms)
  740. self.close(node_id=conn_id)
  741. class DelayedTaskQueue(object):
  742. # see https://docs.python.org/2/library/heapq.html
  743. def __init__(self):
  744. self._tasks = [] # list of entries arranged in a heap
  745. self._task_map = {} # mapping of tasks to entries
  746. self._counter = itertools.count() # unique sequence count
  747. def add(self, task, at):
  748. """Add a task to run at a later time.
  749. Arguments:
  750. task: can be anything, but generally a callable
  751. at (float or int): epoch seconds to schedule task
  752. Returns:
  753. Future: a future that will be returned with the task when ready
  754. """
  755. if task in self._task_map:
  756. self.remove(task)
  757. count = next(self._counter)
  758. future = Future()
  759. entry = [at, count, (task, future)]
  760. self._task_map[task] = entry
  761. heapq.heappush(self._tasks, entry)
  762. return future
  763. def remove(self, task):
  764. """Remove a previously scheduled task.
  765. Raises:
  766. KeyError: if task is not found
  767. """
  768. entry = self._task_map.pop(task)
  769. task, future = entry[-1]
  770. future.failure(Errors.Cancelled)
  771. entry[-1] = 'REMOVED'
  772. def _drop_removed(self):
  773. while self._tasks and self._tasks[0][-1] is 'REMOVED':
  774. at, count, task = heapq.heappop(self._tasks)
  775. def _pop_next(self):
  776. self._drop_removed()
  777. if not self._tasks:
  778. raise KeyError('pop from an empty DelayedTaskQueue')
  779. _, _, maybe_task = heapq.heappop(self._tasks)
  780. if maybe_task is 'REMOVED':
  781. raise ValueError('popped a removed tasks from queue - bug')
  782. else:
  783. task, future = maybe_task
  784. del self._task_map[task]
  785. return (task, future)
  786. def next_at(self):
  787. """Number of seconds until next task is ready."""
  788. self._drop_removed()
  789. if not self._tasks:
  790. return 9999999999
  791. else:
  792. return max(self._tasks[0][0] - time.time(), 0)
  793. def pop_ready(self):
  794. """Pop and return a list of all ready (task, future) tuples"""
  795. ready_tasks = []
  796. while self._tasks and self._tasks[0][0] < time.time():
  797. try:
  798. task = self._pop_next()
  799. except KeyError:
  800. break
  801. ready_tasks.append(task)
  802. return ready_tasks
  803. # OrderedDict requires python2.7+
  804. try:
  805. from collections import OrderedDict
  806. except ImportError:
  807. # If we dont have OrderedDict, we'll fallback to dict with O(n) priority reads
  808. OrderedDict = dict
  809. class IdleConnectionManager(object):
  810. def __init__(self, connections_max_idle_ms):
  811. if connections_max_idle_ms > 0:
  812. self.connections_max_idle = connections_max_idle_ms / 1000
  813. else:
  814. self.connections_max_idle = float('inf')
  815. self.next_idle_close_check_time = None
  816. self.update_next_idle_close_check_time(time.time())
  817. self.lru_connections = OrderedDict()
  818. def update(self, conn_id):
  819. # order should reflect last-update
  820. if conn_id in self.lru_connections:
  821. del self.lru_connections[conn_id]
  822. self.lru_connections[conn_id] = time.time()
  823. def remove(self, conn_id):
  824. if conn_id in self.lru_connections:
  825. del self.lru_connections[conn_id]
  826. def is_expired(self, conn_id):
  827. if conn_id not in self.lru_connections:
  828. return None
  829. return time.time() >= self.lru_connections[conn_id] + self.connections_max_idle
  830. def next_check_ms(self):
  831. now = time.time()
  832. if not self.lru_connections:
  833. return float('inf')
  834. elif self.next_idle_close_check_time <= now:
  835. return 0
  836. else:
  837. return int((self.next_idle_close_check_time - now) * 1000)
  838. def update_next_idle_close_check_time(self, ts):
  839. self.next_idle_close_check_time = ts + self.connections_max_idle
  840. def poll_expired_connection(self):
  841. if time.time() < self.next_idle_close_check_time:
  842. return None
  843. if not len(self.lru_connections):
  844. return None
  845. oldest_conn_id = None
  846. oldest_ts = None
  847. if OrderedDict is dict:
  848. for conn_id, ts in self.lru_connections.items():
  849. if oldest_conn_id is None or ts < oldest_ts:
  850. oldest_conn_id = conn_id
  851. oldest_ts = ts
  852. else:
  853. (oldest_conn_id, oldest_ts) = next(iter(self.lru_connections.items()))
  854. self.update_next_idle_close_check_time(oldest_ts)
  855. if time.time() >= oldest_ts + self.connections_max_idle:
  856. return (oldest_conn_id, oldest_ts)
  857. else:
  858. return None
  859. class KafkaClientMetrics(object):
  860. def __init__(self, metrics, metric_group_prefix, conns):
  861. self.metrics = metrics
  862. self.metric_group_name = metric_group_prefix + '-metrics'
  863. self.connection_closed = metrics.sensor('connections-closed')
  864. self.connection_closed.add(metrics.metric_name(
  865. 'connection-close-rate', self.metric_group_name,
  866. 'Connections closed per second in the window.'), Rate())
  867. self.connection_created = metrics.sensor('connections-created')
  868. self.connection_created.add(metrics.metric_name(
  869. 'connection-creation-rate', self.metric_group_name,
  870. 'New connections established per second in the window.'), Rate())
  871. self.select_time = metrics.sensor('select-time')
  872. self.select_time.add(metrics.metric_name(
  873. 'select-rate', self.metric_group_name,
  874. 'Number of times the I/O layer checked for new I/O to perform per'
  875. ' second'), Rate(sampled_stat=Count()))
  876. self.select_time.add(metrics.metric_name(
  877. 'io-wait-time-ns-avg', self.metric_group_name,
  878. 'The average length of time the I/O thread spent waiting for a'
  879. ' socket ready for reads or writes in nanoseconds.'), Avg())
  880. self.select_time.add(metrics.metric_name(
  881. 'io-wait-ratio', self.metric_group_name,
  882. 'The fraction of time the I/O thread spent waiting.'),
  883. Rate(time_unit=TimeUnit.NANOSECONDS))
  884. self.io_time = metrics.sensor('io-time')
  885. self.io_time.add(metrics.metric_name(
  886. 'io-time-ns-avg', self.metric_group_name,
  887. 'The average length of time for I/O per select call in nanoseconds.'),
  888. Avg())
  889. self.io_time.add(metrics.metric_name(
  890. 'io-ratio', self.metric_group_name,
  891. 'The fraction of time the I/O thread spent doing I/O'),
  892. Rate(time_unit=TimeUnit.NANOSECONDS))
  893. metrics.add_metric(metrics.metric_name(
  894. 'connection-count', self.metric_group_name,
  895. 'The current number of active connections.'), AnonMeasurable(
  896. lambda config, now: len(conns)))