group.py 51 KB

12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109
  1. from __future__ import absolute_import
  2. import copy
  3. import logging
  4. import socket
  5. import time
  6. from kafka.errors import KafkaConfigurationError, UnsupportedVersionError
  7. from kafka.vendor import six
  8. from kafka.client_async import KafkaClient, selectors
  9. from kafka.consumer.fetcher import Fetcher
  10. from kafka.consumer.subscription_state import SubscriptionState
  11. from kafka.coordinator.consumer import ConsumerCoordinator
  12. from kafka.coordinator.assignors.range import RangePartitionAssignor
  13. from kafka.coordinator.assignors.roundrobin import RoundRobinPartitionAssignor
  14. from kafka.metrics import MetricConfig, Metrics
  15. from kafka.protocol.offset import OffsetResetStrategy
  16. from kafka.structs import TopicPartition
  17. from kafka.version import __version__
  18. log = logging.getLogger(__name__)
  19. class KafkaConsumer(six.Iterator):
  20. """Consume records from a Kafka cluster.
  21. The consumer will transparently handle the failure of servers in the Kafka
  22. cluster, and adapt as topic-partitions are created or migrate between
  23. brokers. It also interacts with the assigned kafka Group Coordinator node
  24. to allow multiple consumers to load balance consumption of topics (requires
  25. kafka >= 0.9.0.0).
  26. The consumer is not thread safe and should not be shared across threads.
  27. Arguments:
  28. *topics (str): optional list of topics to subscribe to. If not set,
  29. call :meth:`~kafka.KafkaConsumer.subscribe` or
  30. :meth:`~kafka.KafkaConsumer.assign` before consuming records.
  31. Keyword Arguments:
  32. bootstrap_servers: 'host[:port]' string (or list of 'host[:port]'
  33. strings) that the consumer should contact to bootstrap initial
  34. cluster metadata. This does not have to be the full node list.
  35. It just needs to have at least one broker that will respond to a
  36. Metadata API Request. Default port is 9092. If no servers are
  37. specified, will default to localhost:9092.
  38. client_id (str): A name for this client. This string is passed in
  39. each request to servers and can be used to identify specific
  40. server-side log entries that correspond to this client. Also
  41. submitted to GroupCoordinator for logging with respect to
  42. consumer group administration. Default: 'kafka-python-{version}'
  43. group_id (str or None): The name of the consumer group to join for dynamic
  44. partition assignment (if enabled), and to use for fetching and
  45. committing offsets. If None, auto-partition assignment (via
  46. group coordinator) and offset commits are disabled.
  47. Default: None
  48. key_deserializer (callable): Any callable that takes a
  49. raw message key and returns a deserialized key.
  50. value_deserializer (callable): Any callable that takes a
  51. raw message value and returns a deserialized value.
  52. fetch_min_bytes (int): Minimum amount of data the server should
  53. return for a fetch request, otherwise wait up to
  54. fetch_max_wait_ms for more data to accumulate. Default: 1.
  55. fetch_max_wait_ms (int): The maximum amount of time in milliseconds
  56. the server will block before answering the fetch request if
  57. there isn't sufficient data to immediately satisfy the
  58. requirement given by fetch_min_bytes. Default: 500.
  59. fetch_max_bytes (int): The maximum amount of data the server should
  60. return for a fetch request. This is not an absolute maximum, if the
  61. first message in the first non-empty partition of the fetch is
  62. larger than this value, the message will still be returned to
  63. ensure that the consumer can make progress. NOTE: consumer performs
  64. fetches to multiple brokers in parallel so memory usage will depend
  65. on the number of brokers containing partitions for the topic.
  66. Supported Kafka version >= 0.10.1.0. Default: 52428800 (50 Mb).
  67. max_partition_fetch_bytes (int): The maximum amount of data
  68. per-partition the server will return. The maximum total memory
  69. used for a request = #partitions * max_partition_fetch_bytes.
  70. This size must be at least as large as the maximum message size
  71. the server allows or else it is possible for the producer to
  72. send messages larger than the consumer can fetch. If that
  73. happens, the consumer can get stuck trying to fetch a large
  74. message on a certain partition. Default: 1048576.
  75. request_timeout_ms (int): Client request timeout in milliseconds.
  76. Default: 40000.
  77. retry_backoff_ms (int): Milliseconds to backoff when retrying on
  78. errors. Default: 100.
  79. reconnect_backoff_ms (int): The amount of time in milliseconds to
  80. wait before attempting to reconnect to a given host.
  81. Default: 50.
  82. reconnect_backoff_max_ms (int): The maximum amount of time in
  83. milliseconds to wait when reconnecting to a broker that has
  84. repeatedly failed to connect. If provided, the backoff per host
  85. will increase exponentially for each consecutive connection
  86. failure, up to this maximum. To avoid connection storms, a
  87. randomization factor of 0.2 will be applied to the backoff
  88. resulting in a random range between 20% below and 20% above
  89. the computed value. Default: 1000.
  90. max_in_flight_requests_per_connection (int): Requests are pipelined
  91. to kafka brokers up to this number of maximum requests per
  92. broker connection. Default: 5.
  93. auto_offset_reset (str): A policy for resetting offsets on
  94. OffsetOutOfRange errors: 'earliest' will move to the oldest
  95. available message, 'latest' will move to the most recent. Any
  96. other value will raise the exception. Default: 'latest'.
  97. enable_auto_commit (bool): If True , the consumer's offset will be
  98. periodically committed in the background. Default: True.
  99. auto_commit_interval_ms (int): Number of milliseconds between automatic
  100. offset commits, if enable_auto_commit is True. Default: 5000.
  101. default_offset_commit_callback (callable): Called as
  102. callback(offsets, response) response will be either an Exception
  103. or an OffsetCommitResponse struct. This callback can be used to
  104. trigger custom actions when a commit request completes.
  105. check_crcs (bool): Automatically check the CRC32 of the records
  106. consumed. This ensures no on-the-wire or on-disk corruption to
  107. the messages occurred. This check adds some overhead, so it may
  108. be disabled in cases seeking extreme performance. Default: True
  109. metadata_max_age_ms (int): The period of time in milliseconds after
  110. which we force a refresh of metadata, even if we haven't seen any
  111. partition leadership changes to proactively discover any new
  112. brokers or partitions. Default: 300000
  113. partition_assignment_strategy (list): List of objects to use to
  114. distribute partition ownership amongst consumer instances when
  115. group management is used.
  116. Default: [RangePartitionAssignor, RoundRobinPartitionAssignor]
  117. heartbeat_interval_ms (int): The expected time in milliseconds
  118. between heartbeats to the consumer coordinator when using
  119. Kafka's group management feature. Heartbeats are used to ensure
  120. that the consumer's session stays active and to facilitate
  121. rebalancing when new consumers join or leave the group. The
  122. value must be set lower than session_timeout_ms, but typically
  123. should be set no higher than 1/3 of that value. It can be
  124. adjusted even lower to control the expected time for normal
  125. rebalances. Default: 3000
  126. session_timeout_ms (int): The timeout used to detect failures when
  127. using Kafka's group management facilities. Default: 30000
  128. max_poll_records (int): The maximum number of records returned in a
  129. single call to :meth:`~kafka.KafkaConsumer.poll`. Default: 500
  130. receive_buffer_bytes (int): The size of the TCP receive buffer
  131. (SO_RCVBUF) to use when reading data. Default: None (relies on
  132. system defaults). The java client defaults to 32768.
  133. send_buffer_bytes (int): The size of the TCP send buffer
  134. (SO_SNDBUF) to use when sending data. Default: None (relies on
  135. system defaults). The java client defaults to 131072.
  136. socket_options (list): List of tuple-arguments to socket.setsockopt
  137. to apply to broker connection sockets. Default:
  138. [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)]
  139. consumer_timeout_ms (int): number of milliseconds to block during
  140. message iteration before raising StopIteration (i.e., ending the
  141. iterator). Default block forever [float('inf')].
  142. skip_double_compressed_messages (bool): A bug in KafkaProducer <= 1.2.4
  143. caused some messages to be corrupted via double-compression.
  144. By default, the fetcher will return these messages as a compressed
  145. blob of bytes with a single offset, i.e. how the message was
  146. actually published to the cluster. If you prefer to have the
  147. fetcher automatically detect corrupt messages and skip them,
  148. set this option to True. Default: False.
  149. security_protocol (str): Protocol used to communicate with brokers.
  150. Valid values are: PLAINTEXT, SSL. Default: PLAINTEXT.
  151. ssl_context (ssl.SSLContext): Pre-configured SSLContext for wrapping
  152. socket connections. If provided, all other ssl_* configurations
  153. will be ignored. Default: None.
  154. ssl_check_hostname (bool): Flag to configure whether ssl handshake
  155. should verify that the certificate matches the brokers hostname.
  156. Default: True.
  157. ssl_cafile (str): Optional filename of ca file to use in certificate
  158. verification. Default: None.
  159. ssl_certfile (str): Optional filename of file in pem format containing
  160. the client certificate, as well as any ca certificates needed to
  161. establish the certificate's authenticity. Default: None.
  162. ssl_keyfile (str): Optional filename containing the client private key.
  163. Default: None.
  164. ssl_password (str): Optional password to be used when loading the
  165. certificate chain. Default: None.
  166. ssl_crlfile (str): Optional filename containing the CRL to check for
  167. certificate expiration. By default, no CRL check is done. When
  168. providing a file, only the leaf certificate will be checked against
  169. this CRL. The CRL can only be checked with Python 3.4+ or 2.7.9+.
  170. Default: None.
  171. api_version (tuple): Specify which Kafka API version to use. If set to
  172. None, the client will attempt to infer the broker version by probing
  173. various APIs. Different versions enable different functionality.
  174. Examples:
  175. (0, 9) enables full group coordination features with automatic
  176. partition assignment and rebalancing,
  177. (0, 8, 2) enables kafka-storage offset commits with manual
  178. partition assignment only,
  179. (0, 8, 1) enables zookeeper-storage offset commits with manual
  180. partition assignment only,
  181. (0, 8, 0) enables basic functionality but requires manual
  182. partition assignment and offset management.
  183. For the full list of supported versions, see
  184. KafkaClient.API_VERSIONS. Default: None
  185. api_version_auto_timeout_ms (int): number of milliseconds to throw a
  186. timeout exception from the constructor when checking the broker
  187. api version. Only applies if api_version set to 'auto'
  188. metric_reporters (list): A list of classes to use as metrics reporters.
  189. Implementing the AbstractMetricsReporter interface allows plugging
  190. in classes that will be notified of new metric creation. Default: []
  191. metrics_num_samples (int): The number of samples maintained to compute
  192. metrics. Default: 2
  193. metrics_sample_window_ms (int): The maximum age in milliseconds of
  194. samples used to compute metrics. Default: 30000
  195. selector (selectors.BaseSelector): Provide a specific selector
  196. implementation to use for I/O multiplexing.
  197. Default: selectors.DefaultSelector
  198. exclude_internal_topics (bool): Whether records from internal topics
  199. (such as offsets) should be exposed to the consumer. If set to True
  200. the only way to receive records from an internal topic is
  201. subscribing to it. Requires 0.10+ Default: True
  202. sasl_mechanism (str): String picking sasl mechanism when security_protocol
  203. is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
  204. Default: None
  205. sasl_plain_username (str): Username for sasl PLAIN authentication.
  206. Default: None
  207. sasl_plain_password (str): Password for sasl PLAIN authentication.
  208. Default: None
  209. Note:
  210. Configuration parameters are described in more detail at
  211. https://kafka.apache.org/documentation/#newconsumerconfigs
  212. """
  213. DEFAULT_CONFIG = {
  214. 'bootstrap_servers': 'localhost',
  215. 'client_id': 'kafka-python-' + __version__,
  216. 'group_id': None,
  217. 'key_deserializer': None,
  218. 'value_deserializer': None,
  219. 'fetch_max_wait_ms': 500,
  220. 'fetch_min_bytes': 1,
  221. 'fetch_max_bytes': 52428800,
  222. 'max_partition_fetch_bytes': 1 * 1024 * 1024,
  223. 'request_timeout_ms': 40 * 1000,
  224. 'retry_backoff_ms': 100,
  225. 'reconnect_backoff_ms': 50,
  226. 'reconnect_backoff_max_ms': 1000,
  227. 'max_in_flight_requests_per_connection': 5,
  228. 'auto_offset_reset': 'latest',
  229. 'enable_auto_commit': True,
  230. 'auto_commit_interval_ms': 5000,
  231. 'default_offset_commit_callback': lambda offsets, response: True,
  232. 'check_crcs': True,
  233. 'metadata_max_age_ms': 5 * 60 * 1000,
  234. 'partition_assignment_strategy': (RangePartitionAssignor, RoundRobinPartitionAssignor),
  235. 'heartbeat_interval_ms': 3000,
  236. 'session_timeout_ms': 30000,
  237. 'max_poll_records': 500,
  238. 'receive_buffer_bytes': None,
  239. 'send_buffer_bytes': None,
  240. 'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
  241. 'consumer_timeout_ms': float('inf'),
  242. 'skip_double_compressed_messages': False,
  243. 'security_protocol': 'PLAINTEXT',
  244. 'ssl_context': None,
  245. 'ssl_check_hostname': True,
  246. 'ssl_cafile': None,
  247. 'ssl_certfile': None,
  248. 'ssl_keyfile': None,
  249. 'ssl_crlfile': None,
  250. 'ssl_password': None,
  251. 'api_version': None,
  252. 'api_version_auto_timeout_ms': 2000,
  253. 'connections_max_idle_ms': 9 * 60 * 1000,
  254. 'metric_reporters': [],
  255. 'metrics_num_samples': 2,
  256. 'metrics_sample_window_ms': 30000,
  257. 'metric_group_prefix': 'consumer',
  258. 'selector': selectors.DefaultSelector,
  259. 'exclude_internal_topics': True,
  260. 'sasl_mechanism': None,
  261. 'sasl_plain_username': None,
  262. 'sasl_plain_password': None,
  263. }
  264. def __init__(self, *topics, **configs):
  265. self.config = copy.copy(self.DEFAULT_CONFIG)
  266. for key in self.config:
  267. if key in configs:
  268. self.config[key] = configs.pop(key)
  269. # Only check for extra config keys in top-level class
  270. assert not configs, 'Unrecognized configs: %s' % configs
  271. deprecated = {'smallest': 'earliest', 'largest': 'latest'}
  272. if self.config['auto_offset_reset'] in deprecated:
  273. new_config = deprecated[self.config['auto_offset_reset']]
  274. log.warning('use auto_offset_reset=%s (%s is deprecated)',
  275. new_config, self.config['auto_offset_reset'])
  276. self.config['auto_offset_reset'] = new_config
  277. request_timeout_ms = self.config['request_timeout_ms']
  278. session_timeout_ms = self.config['session_timeout_ms']
  279. fetch_max_wait_ms = self.config['fetch_max_wait_ms']
  280. if request_timeout_ms <= session_timeout_ms:
  281. raise KafkaConfigurationError(
  282. "Request timeout (%s) must be larger than session timeout (%s)" %
  283. (request_timeout_ms, session_timeout_ms))
  284. if request_timeout_ms <= fetch_max_wait_ms:
  285. raise KafkaConfigurationError("Request timeout (%s) must be larger than fetch-max-wait-ms (%s)" %
  286. (request_timeout_ms, fetch_max_wait_ms))
  287. metrics_tags = {'client-id': self.config['client_id']}
  288. metric_config = MetricConfig(samples=self.config['metrics_num_samples'],
  289. time_window_ms=self.config['metrics_sample_window_ms'],
  290. tags=metrics_tags)
  291. reporters = [reporter() for reporter in self.config['metric_reporters']]
  292. self._metrics = Metrics(metric_config, reporters)
  293. # TODO _metrics likely needs to be passed to KafkaClient, etc.
  294. # api_version was previously a str. Accept old format for now
  295. if isinstance(self.config['api_version'], str):
  296. str_version = self.config['api_version']
  297. if str_version == 'auto':
  298. self.config['api_version'] = None
  299. else:
  300. self.config['api_version'] = tuple(map(int, str_version.split('.')))
  301. log.warning('use api_version=%s [tuple] -- "%s" as str is deprecated',
  302. str(self.config['api_version']), str_version)
  303. self._client = KafkaClient(metrics=self._metrics, **self.config)
  304. # Get auto-discovered version from client if necessary
  305. if self.config['api_version'] is None:
  306. self.config['api_version'] = self._client.config['api_version']
  307. self._subscription = SubscriptionState(self.config['auto_offset_reset'])
  308. self._fetcher = Fetcher(
  309. self._client, self._subscription, self._metrics, **self.config)
  310. self._coordinator = ConsumerCoordinator(
  311. self._client, self._subscription, self._metrics,
  312. assignors=self.config['partition_assignment_strategy'],
  313. **self.config)
  314. self._closed = False
  315. self._iterator = None
  316. self._consumer_timeout = float('inf')
  317. if topics:
  318. self._subscription.subscribe(topics=topics)
  319. self._client.set_topics(topics)
  320. def assign(self, partitions):
  321. """Manually assign a list of TopicPartitions to this consumer.
  322. Arguments:
  323. partitions (list of TopicPartition): Assignment for this instance.
  324. Raises:
  325. IllegalStateError: If consumer has already called
  326. :meth:`~kafka.KafkaConsumer.subscribe`.
  327. Warning:
  328. It is not possible to use both manual partition assignment with
  329. :meth:`~kafka.KafkaConsumer.assign` and group assignment with
  330. :meth:`~kafka.KafkaConsumer.subscribe`.
  331. Note:
  332. This interface does not support incremental assignment and will
  333. replace the previous assignment (if there was one).
  334. Note:
  335. Manual topic assignment through this method does not use the
  336. consumer's group management functionality. As such, there will be
  337. no rebalance operation triggered when group membership or cluster
  338. and topic metadata change.
  339. """
  340. self._subscription.assign_from_user(partitions)
  341. self._client.set_topics([tp.topic for tp in partitions])
  342. def assignment(self):
  343. """Get the TopicPartitions currently assigned to this consumer.
  344. If partitions were directly assigned using
  345. :meth:`~kafka.KafkaConsumer.assign`, then this will simply return the
  346. same partitions that were previously assigned. If topics were
  347. subscribed using :meth:`~kafka.KafkaConsumer.subscribe`, then this will
  348. give the set of topic partitions currently assigned to the consumer
  349. (which may be None if the assignment hasn't happened yet, or if the
  350. partitions are in the process of being reassigned).
  351. Returns:
  352. set: {TopicPartition, ...}
  353. """
  354. return self._subscription.assigned_partitions()
  355. def close(self, autocommit=True):
  356. """Close the consumer, waiting indefinitely for any needed cleanup.
  357. Keyword Arguments:
  358. autocommit (bool): If auto-commit is configured for this consumer,
  359. this optional flag causes the consumer to attempt to commit any
  360. pending consumed offsets prior to close. Default: True
  361. """
  362. if self._closed:
  363. return
  364. log.debug("Closing the KafkaConsumer.")
  365. self._closed = True
  366. self._coordinator.close(autocommit=autocommit)
  367. self._metrics.close()
  368. self._client.close()
  369. try:
  370. self.config['key_deserializer'].close()
  371. except AttributeError:
  372. pass
  373. try:
  374. self.config['value_deserializer'].close()
  375. except AttributeError:
  376. pass
  377. log.debug("The KafkaConsumer has closed.")
  378. def commit_async(self, offsets=None, callback=None):
  379. """Commit offsets to kafka asynchronously, optionally firing callback.
  380. This commits offsets only to Kafka. The offsets committed using this API
  381. will be used on the first fetch after every rebalance and also on
  382. startup. As such, if you need to store offsets in anything other than
  383. Kafka, this API should not be used. To avoid re-processing the last
  384. message read if a consumer is restarted, the committed offset should be
  385. the next message your application should consume, i.e.: last_offset + 1.
  386. This is an asynchronous call and will not block. Any errors encountered
  387. are either passed to the callback (if provided) or discarded.
  388. Arguments:
  389. offsets (dict, optional): {TopicPartition: OffsetAndMetadata} dict
  390. to commit with the configured group_id. Defaults to currently
  391. consumed offsets for all subscribed partitions.
  392. callback (callable, optional): Called as callback(offsets, response)
  393. with response as either an Exception or an OffsetCommitResponse
  394. struct. This callback can be used to trigger custom actions when
  395. a commit request completes.
  396. Returns:
  397. kafka.future.Future
  398. """
  399. assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
  400. assert self.config['group_id'] is not None, 'Requires group_id'
  401. if offsets is None:
  402. offsets = self._subscription.all_consumed_offsets()
  403. log.debug("Committing offsets: %s", offsets)
  404. future = self._coordinator.commit_offsets_async(
  405. offsets, callback=callback)
  406. return future
  407. def commit(self, offsets=None):
  408. """Commit offsets to kafka, blocking until success or error.
  409. This commits offsets only to Kafka. The offsets committed using this API
  410. will be used on the first fetch after every rebalance and also on
  411. startup. As such, if you need to store offsets in anything other than
  412. Kafka, this API should not be used. To avoid re-processing the last
  413. message read if a consumer is restarted, the committed offset should be
  414. the next message your application should consume, i.e.: last_offset + 1.
  415. Blocks until either the commit succeeds or an unrecoverable error is
  416. encountered (in which case it is thrown to the caller).
  417. Currently only supports kafka-topic offset storage (not zookeeper).
  418. Arguments:
  419. offsets (dict, optional): {TopicPartition: OffsetAndMetadata} dict
  420. to commit with the configured group_id. Defaults to currently
  421. consumed offsets for all subscribed partitions.
  422. """
  423. assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
  424. assert self.config['group_id'] is not None, 'Requires group_id'
  425. if offsets is None:
  426. offsets = self._subscription.all_consumed_offsets()
  427. self._coordinator.commit_offsets_sync(offsets)
  428. def committed(self, partition):
  429. """Get the last committed offset for the given partition.
  430. This offset will be used as the position for the consumer
  431. in the event of a failure.
  432. This call may block to do a remote call if the partition in question
  433. isn't assigned to this consumer or if the consumer hasn't yet
  434. initialized its cache of committed offsets.
  435. Arguments:
  436. partition (TopicPartition): The partition to check.
  437. Returns:
  438. The last committed offset, or None if there was no prior commit.
  439. """
  440. assert self.config['api_version'] >= (0, 8, 1), 'Requires >= Kafka 0.8.1'
  441. assert self.config['group_id'] is not None, 'Requires group_id'
  442. if not isinstance(partition, TopicPartition):
  443. raise TypeError('partition must be a TopicPartition namedtuple')
  444. if self._subscription.is_assigned(partition):
  445. committed = self._subscription.assignment[partition].committed
  446. if committed is None:
  447. self._coordinator.refresh_committed_offsets_if_needed()
  448. committed = self._subscription.assignment[partition].committed
  449. else:
  450. commit_map = self._coordinator.fetch_committed_offsets([partition])
  451. if partition in commit_map:
  452. committed = commit_map[partition].offset
  453. else:
  454. committed = None
  455. return committed
  456. def topics(self):
  457. """Get all topics the user is authorized to view.
  458. Returns:
  459. set: topics
  460. """
  461. cluster = self._client.cluster
  462. if self._client._metadata_refresh_in_progress and self._client._topics:
  463. future = cluster.request_update()
  464. self._client.poll(future=future)
  465. stash = cluster.need_all_topic_metadata
  466. cluster.need_all_topic_metadata = True
  467. future = cluster.request_update()
  468. self._client.poll(future=future)
  469. cluster.need_all_topic_metadata = stash
  470. return cluster.topics()
  471. def partitions_for_topic(self, topic):
  472. """Get metadata about the partitions for a given topic.
  473. Arguments:
  474. topic (str): Topic to check.
  475. Returns:
  476. set: Partition ids
  477. """
  478. return self._client.cluster.partitions_for_topic(topic)
  479. def poll(self, timeout_ms=0, max_records=None):
  480. """Fetch data from assigned topics / partitions.
  481. Records are fetched and returned in batches by topic-partition.
  482. On each poll, consumer will try to use the last consumed offset as the
  483. starting offset and fetch sequentially. The last consumed offset can be
  484. manually set through :meth:`~kafka.KafkaConsumer.seek` or automatically
  485. set as the last committed offset for the subscribed list of partitions.
  486. Incompatible with iterator interface -- use one or the other, not both.
  487. Arguments:
  488. timeout_ms (int, optional): Milliseconds spent waiting in poll if
  489. data is not available in the buffer. If 0, returns immediately
  490. with any records that are available currently in the buffer,
  491. else returns empty. Must not be negative. Default: 0
  492. max_records (int, optional): The maximum number of records returned
  493. in a single call to :meth:`~kafka.KafkaConsumer.poll`.
  494. Default: Inherit value from max_poll_records.
  495. Returns:
  496. dict: Topic to list of records since the last fetch for the
  497. subscribed list of topics and partitions.
  498. """
  499. assert timeout_ms >= 0, 'Timeout must not be negative'
  500. if max_records is None:
  501. max_records = self.config['max_poll_records']
  502. # Poll for new data until the timeout expires
  503. start = time.time()
  504. remaining = timeout_ms
  505. while True:
  506. records = self._poll_once(remaining, max_records)
  507. if records:
  508. return records
  509. elapsed_ms = (time.time() - start) * 1000
  510. remaining = timeout_ms - elapsed_ms
  511. if remaining <= 0:
  512. return {}
  513. def _poll_once(self, timeout_ms, max_records):
  514. """Do one round of polling. In addition to checking for new data, this does
  515. any needed heart-beating, auto-commits, and offset updates.
  516. Arguments:
  517. timeout_ms (int): The maximum time in milliseconds to block.
  518. Returns:
  519. dict: Map of topic to list of records (may be empty).
  520. """
  521. if self._use_consumer_group():
  522. self._coordinator.ensure_coordinator_known()
  523. self._coordinator.ensure_active_group()
  524. # 0.8.2 brokers support kafka-backed offset storage via group coordinator
  525. elif self.config['group_id'] is not None and self.config['api_version'] >= (0, 8, 2):
  526. self._coordinator.ensure_coordinator_known()
  527. # Fetch positions if we have partitions we're subscribed to that we
  528. # don't know the offset for
  529. if not self._subscription.has_all_fetch_positions():
  530. self._update_fetch_positions(self._subscription.missing_fetch_positions())
  531. # If data is available already, e.g. from a previous network client
  532. # poll() call to commit, then just return it immediately
  533. records, partial = self._fetcher.fetched_records(max_records)
  534. if records:
  535. # Before returning the fetched records, we can send off the
  536. # next round of fetches and avoid block waiting for their
  537. # responses to enable pipelining while the user is handling the
  538. # fetched records.
  539. if not partial:
  540. self._fetcher.send_fetches()
  541. return records
  542. # Send any new fetches (won't resend pending fetches)
  543. self._fetcher.send_fetches()
  544. self._client.poll(timeout_ms=timeout_ms)
  545. records, _ = self._fetcher.fetched_records(max_records)
  546. return records
  547. def position(self, partition):
  548. """Get the offset of the next record that will be fetched
  549. Arguments:
  550. partition (TopicPartition): Partition to check
  551. Returns:
  552. int: Offset
  553. """
  554. if not isinstance(partition, TopicPartition):
  555. raise TypeError('partition must be a TopicPartition namedtuple')
  556. assert self._subscription.is_assigned(partition), 'Partition is not assigned'
  557. offset = self._subscription.assignment[partition].position
  558. if offset is None:
  559. self._update_fetch_positions([partition])
  560. offset = self._subscription.assignment[partition].position
  561. return offset
  562. def highwater(self, partition):
  563. """Last known highwater offset for a partition.
  564. A highwater offset is the offset that will be assigned to the next
  565. message that is produced. It may be useful for calculating lag, by
  566. comparing with the reported position. Note that both position and
  567. highwater refer to the *next* offset -- i.e., highwater offset is
  568. one greater than the newest available message.
  569. Highwater offsets are returned in FetchResponse messages, so will
  570. not be available if no FetchRequests have been sent for this partition
  571. yet.
  572. Arguments:
  573. partition (TopicPartition): Partition to check
  574. Returns:
  575. int or None: Offset if available
  576. """
  577. if not isinstance(partition, TopicPartition):
  578. raise TypeError('partition must be a TopicPartition namedtuple')
  579. assert self._subscription.is_assigned(partition), 'Partition is not assigned'
  580. return self._subscription.assignment[partition].highwater
  581. def pause(self, *partitions):
  582. """Suspend fetching from the requested partitions.
  583. Future calls to :meth:`~kafka.KafkaConsumer.poll` will not return any
  584. records from these partitions until they have been resumed using
  585. :meth:`~kafka.KafkaConsumer.resume`.
  586. Note: This method does not affect partition subscription. In particular,
  587. it does not cause a group rebalance when automatic assignment is used.
  588. Arguments:
  589. *partitions (TopicPartition): Partitions to pause.
  590. """
  591. if not all([isinstance(p, TopicPartition) for p in partitions]):
  592. raise TypeError('partitions must be TopicPartition namedtuples')
  593. for partition in partitions:
  594. log.debug("Pausing partition %s", partition)
  595. self._subscription.pause(partition)
  596. def paused(self):
  597. """Get the partitions that were previously paused using
  598. :meth:`~kafka.KafkaConsumer.pause`.
  599. Returns:
  600. set: {partition (TopicPartition), ...}
  601. """
  602. return self._subscription.paused_partitions()
  603. def resume(self, *partitions):
  604. """Resume fetching from the specified (paused) partitions.
  605. Arguments:
  606. *partitions (TopicPartition): Partitions to resume.
  607. """
  608. if not all([isinstance(p, TopicPartition) for p in partitions]):
  609. raise TypeError('partitions must be TopicPartition namedtuples')
  610. for partition in partitions:
  611. log.debug("Resuming partition %s", partition)
  612. self._subscription.resume(partition)
  613. def seek(self, partition, offset):
  614. """Manually specify the fetch offset for a TopicPartition.
  615. Overrides the fetch offsets that the consumer will use on the next
  616. :meth:`~kafka.KafkaConsumer.poll`. If this API is invoked for the same
  617. partition more than once, the latest offset will be used on the next
  618. :meth:`~kafka.KafkaConsumer.poll`.
  619. Note: You may lose data if this API is arbitrarily used in the middle of
  620. consumption to reset the fetch offsets.
  621. Arguments:
  622. partition (TopicPartition): Partition for seek operation
  623. offset (int): Message offset in partition
  624. Raises:
  625. AssertionError: If offset is not an int >= 0; or if partition is not
  626. currently assigned.
  627. """
  628. if not isinstance(partition, TopicPartition):
  629. raise TypeError('partition must be a TopicPartition namedtuple')
  630. assert isinstance(offset, int) and offset >= 0, 'Offset must be >= 0'
  631. assert partition in self._subscription.assigned_partitions(), 'Unassigned partition'
  632. log.debug("Seeking to offset %s for partition %s", offset, partition)
  633. self._subscription.assignment[partition].seek(offset)
  634. def seek_to_beginning(self, *partitions):
  635. """Seek to the oldest available offset for partitions.
  636. Arguments:
  637. *partitions: Optionally provide specific TopicPartitions, otherwise
  638. default to all assigned partitions.
  639. Raises:
  640. AssertionError: If any partition is not currently assigned, or if
  641. no partitions are assigned.
  642. """
  643. if not all([isinstance(p, TopicPartition) for p in partitions]):
  644. raise TypeError('partitions must be TopicPartition namedtuples')
  645. if not partitions:
  646. partitions = self._subscription.assigned_partitions()
  647. assert partitions, 'No partitions are currently assigned'
  648. else:
  649. for p in partitions:
  650. assert p in self._subscription.assigned_partitions(), 'Unassigned partition'
  651. for tp in partitions:
  652. log.debug("Seeking to beginning of partition %s", tp)
  653. self._subscription.need_offset_reset(tp, OffsetResetStrategy.EARLIEST)
  654. def seek_to_end(self, *partitions):
  655. """Seek to the most recent available offset for partitions.
  656. Arguments:
  657. *partitions: Optionally provide specific TopicPartitions, otherwise
  658. default to all assigned partitions.
  659. Raises:
  660. AssertionError: If any partition is not currently assigned, or if
  661. no partitions are assigned.
  662. """
  663. if not all([isinstance(p, TopicPartition) for p in partitions]):
  664. raise TypeError('partitions must be TopicPartition namedtuples')
  665. if not partitions:
  666. partitions = self._subscription.assigned_partitions()
  667. assert partitions, 'No partitions are currently assigned'
  668. else:
  669. for p in partitions:
  670. assert p in self._subscription.assigned_partitions(), 'Unassigned partition'
  671. for tp in partitions:
  672. log.debug("Seeking to end of partition %s", tp)
  673. self._subscription.need_offset_reset(tp, OffsetResetStrategy.LATEST)
  674. def subscribe(self, topics=(), pattern=None, listener=None):
  675. """Subscribe to a list of topics, or a topic regex pattern.
  676. Partitions will be dynamically assigned via a group coordinator.
  677. Topic subscriptions are not incremental: this list will replace the
  678. current assignment (if there is one).
  679. This method is incompatible with :meth:`~kafka.KafkaConsumer.assign`.
  680. Arguments:
  681. topics (list): List of topics for subscription.
  682. pattern (str): Pattern to match available topics. You must provide
  683. either topics or pattern, but not both.
  684. listener (ConsumerRebalanceListener): Optionally include listener
  685. callback, which will be called before and after each rebalance
  686. operation.
  687. As part of group management, the consumer will keep track of the
  688. list of consumers that belong to a particular group and will
  689. trigger a rebalance operation if one of the following events
  690. trigger:
  691. * Number of partitions change for any of the subscribed topics
  692. * Topic is created or deleted
  693. * An existing member of the consumer group dies
  694. * A new member is added to the consumer group
  695. When any of these events are triggered, the provided listener
  696. will be invoked first to indicate that the consumer's assignment
  697. has been revoked, and then again when the new assignment has
  698. been received. Note that this listener will immediately override
  699. any listener set in a previous call to subscribe. It is
  700. guaranteed, however, that the partitions revoked/assigned
  701. through this interface are from topics subscribed in this call.
  702. Raises:
  703. IllegalStateError: If called after previously calling
  704. :meth:`~kafka.KafkaConsumer.assign`.
  705. AssertionError: If neither topics or pattern is provided.
  706. TypeError: If listener is not a ConsumerRebalanceListener.
  707. """
  708. # SubscriptionState handles error checking
  709. self._subscription.subscribe(topics=topics,
  710. pattern=pattern,
  711. listener=listener)
  712. # Regex will need all topic metadata
  713. if pattern is not None:
  714. self._client.cluster.need_all_topic_metadata = True
  715. self._client.set_topics([])
  716. self._client.cluster.request_update()
  717. log.debug("Subscribed to topic pattern: %s", pattern)
  718. else:
  719. self._client.cluster.need_all_topic_metadata = False
  720. self._client.set_topics(self._subscription.group_subscription())
  721. log.debug("Subscribed to topic(s): %s", topics)
  722. def subscription(self):
  723. """Get the current topic subscription.
  724. Returns:
  725. set: {topic, ...}
  726. """
  727. return self._subscription.subscription.copy()
  728. def unsubscribe(self):
  729. """Unsubscribe from all topics and clear all assigned partitions."""
  730. self._subscription.unsubscribe()
  731. self._coordinator.close()
  732. self._client.cluster.need_all_topic_metadata = False
  733. self._client.set_topics([])
  734. log.debug("Unsubscribed all topics or patterns and assigned partitions")
  735. def metrics(self, raw=False):
  736. """Warning: this is an unstable interface.
  737. It may change in future releases without warning"""
  738. if raw:
  739. return self._metrics.metrics
  740. metrics = {}
  741. for k, v in self._metrics.metrics.items():
  742. if k.group not in metrics:
  743. metrics[k.group] = {}
  744. if k.name not in metrics[k.group]:
  745. metrics[k.group][k.name] = {}
  746. metrics[k.group][k.name] = v.value()
  747. return metrics
  748. def offsets_for_times(self, timestamps):
  749. """Look up the offsets for the given partitions by timestamp. The
  750. returned offset for each partition is the earliest offset whose
  751. timestamp is greater than or equal to the given timestamp in the
  752. corresponding partition.
  753. This is a blocking call. The consumer does not have to be assigned the
  754. partitions.
  755. If the message format version in a partition is before 0.10.0, i.e.
  756. the messages do not have timestamps, ``None`` will be returned for that
  757. partition. ``None`` will also be returned for the partition if there
  758. are no messages in it.
  759. Note:
  760. This method may block indefinitely if the partition does not exist.
  761. Arguments:
  762. timestamps (dict): ``{TopicPartition: int}`` mapping from partition
  763. to the timestamp to look up. Unit should be milliseconds since
  764. beginning of the epoch (midnight Jan 1, 1970 (UTC))
  765. Returns:
  766. ``{TopicPartition: OffsetAndTimestamp}``: mapping from partition
  767. to the timestamp and offset of the first message with timestamp
  768. greater than or equal to the target timestamp.
  769. Raises:
  770. ValueError: If the target timestamp is negative
  771. UnsupportedVersionError: If the broker does not support looking
  772. up the offsets by timestamp.
  773. KafkaTimeoutError: If fetch failed in request_timeout_ms
  774. """
  775. if self.config['api_version'] <= (0, 10, 0):
  776. raise UnsupportedVersionError(
  777. "offsets_for_times API not supported for cluster version {}"
  778. .format(self.config['api_version']))
  779. for tp, ts in timestamps.items():
  780. timestamps[tp] = int(ts)
  781. if ts < 0:
  782. raise ValueError(
  783. "The target time for partition {} is {}. The target time "
  784. "cannot be negative.".format(tp, ts))
  785. return self._fetcher.get_offsets_by_times(
  786. timestamps, self.config['request_timeout_ms'])
  787. def beginning_offsets(self, partitions):
  788. """Get the first offset for the given partitions.
  789. This method does not change the current consumer position of the
  790. partitions.
  791. Note:
  792. This method may block indefinitely if the partition does not exist.
  793. Arguments:
  794. partitions (list): List of TopicPartition instances to fetch
  795. offsets for.
  796. Returns:
  797. ``{TopicPartition: int}``: The earliest available offsets for the
  798. given partitions.
  799. Raises:
  800. UnsupportedVersionError: If the broker does not support looking
  801. up the offsets by timestamp.
  802. KafkaTimeoutError: If fetch failed in request_timeout_ms.
  803. """
  804. offsets = self._fetcher.beginning_offsets(
  805. partitions, self.config['request_timeout_ms'])
  806. return offsets
  807. def end_offsets(self, partitions):
  808. """Get the last offset for the given partitions. The last offset of a
  809. partition is the offset of the upcoming message, i.e. the offset of the
  810. last available message + 1.
  811. This method does not change the current consumer position of the
  812. partitions.
  813. Note:
  814. This method may block indefinitely if the partition does not exist.
  815. Arguments:
  816. partitions (list): List of TopicPartition instances to fetch
  817. offsets for.
  818. Returns:
  819. ``{TopicPartition: int}``: The end offsets for the given partitions.
  820. Raises:
  821. UnsupportedVersionError: If the broker does not support looking
  822. up the offsets by timestamp.
  823. KafkaTimeoutError: If fetch failed in request_timeout_ms
  824. """
  825. offsets = self._fetcher.end_offsets(
  826. partitions, self.config['request_timeout_ms'])
  827. return offsets
  828. def _use_consumer_group(self):
  829. """Return True iff this consumer can/should join a broker-coordinated group."""
  830. if self.config['api_version'] < (0, 9):
  831. return False
  832. elif self.config['group_id'] is None:
  833. return False
  834. elif not self._subscription.partitions_auto_assigned():
  835. return False
  836. return True
  837. def _update_fetch_positions(self, partitions):
  838. """Set the fetch position to the committed position (if there is one)
  839. or reset it using the offset reset policy the user has configured.
  840. Arguments:
  841. partitions (List[TopicPartition]): The partitions that need
  842. updating fetch positions.
  843. Raises:
  844. NoOffsetForPartitionError: If no offset is stored for a given
  845. partition and no offset reset policy is defined.
  846. """
  847. if (self.config['api_version'] >= (0, 8, 1) and
  848. self.config['group_id'] is not None):
  849. # Refresh commits for all assigned partitions
  850. self._coordinator.refresh_committed_offsets_if_needed()
  851. # Then, do any offset lookups in case some positions are not known
  852. self._fetcher.update_fetch_positions(partitions)
  853. def _message_generator(self):
  854. assert self.assignment() or self.subscription() is not None, 'No topic subscription or manual partition assignment'
  855. while time.time() < self._consumer_timeout:
  856. if self._use_consumer_group():
  857. self._coordinator.ensure_coordinator_known()
  858. self._coordinator.ensure_active_group()
  859. # 0.8.2 brokers support kafka-backed offset storage via group coordinator
  860. elif self.config['group_id'] is not None and self.config['api_version'] >= (0, 8, 2):
  861. self._coordinator.ensure_coordinator_known()
  862. # Fetch offsets for any subscribed partitions that we arent tracking yet
  863. if not self._subscription.has_all_fetch_positions():
  864. partitions = self._subscription.missing_fetch_positions()
  865. self._update_fetch_positions(partitions)
  866. poll_ms = 1000 * (self._consumer_timeout - time.time())
  867. if not self._fetcher.in_flight_fetches():
  868. poll_ms = 0
  869. self._client.poll(timeout_ms=poll_ms)
  870. # We need to make sure we at least keep up with scheduled tasks,
  871. # like heartbeats, auto-commits, and metadata refreshes
  872. timeout_at = self._next_timeout()
  873. # Because the consumer client poll does not sleep unless blocking on
  874. # network IO, we need to explicitly sleep when we know we are idle
  875. # because we haven't been assigned any partitions to fetch / consume
  876. if self._use_consumer_group() and not self.assignment():
  877. sleep_time = max(timeout_at - time.time(), 0)
  878. if sleep_time > 0 and not self._client.in_flight_request_count():
  879. log.debug('No partitions assigned; sleeping for %s', sleep_time)
  880. time.sleep(sleep_time)
  881. continue
  882. # Short-circuit the fetch iterator if we are already timed out
  883. # to avoid any unintentional interaction with fetcher setup
  884. if time.time() > timeout_at:
  885. continue
  886. for msg in self._fetcher:
  887. yield msg
  888. if time.time() > timeout_at:
  889. log.debug("internal iterator timeout - breaking for poll")
  890. break
  891. if self._client.in_flight_request_count():
  892. self._client.poll(timeout_ms=0)
  893. # An else block on a for loop only executes if there was no break
  894. # so this should only be called on a StopIteration from the fetcher
  895. # We assume that it is safe to init_fetches when fetcher is done
  896. # i.e., there are no more records stored internally
  897. else:
  898. self._fetcher.send_fetches()
  899. def _next_timeout(self):
  900. timeout = min(self._consumer_timeout,
  901. self._client._delayed_tasks.next_at() + time.time(),
  902. self._client.cluster.ttl() / 1000.0 + time.time())
  903. # Although the delayed_tasks timeout above should cover processing
  904. # HeartbeatRequests, it is still possible that HeartbeatResponses
  905. # are left unprocessed during a long _fetcher iteration without
  906. # an intermediate poll(). And because tasks are responsible for
  907. # rescheduling themselves, an unprocessed response will prevent
  908. # the next heartbeat from being sent. This check should help
  909. # avoid that.
  910. if self._use_consumer_group():
  911. heartbeat = time.time() + self._coordinator.heartbeat.ttl()
  912. timeout = min(timeout, heartbeat)
  913. return timeout
  914. def __iter__(self): # pylint: disable=non-iterator-returned
  915. return self
  916. def __next__(self):
  917. if not self._iterator:
  918. self._iterator = self._message_generator()
  919. self._set_consumer_timeout()
  920. try:
  921. return next(self._iterator)
  922. except StopIteration:
  923. self._iterator = None
  924. raise
  925. def _set_consumer_timeout(self):
  926. # consumer_timeout_ms can be used to stop iteration early
  927. if self.config['consumer_timeout_ms'] >= 0:
  928. self._consumer_timeout = time.time() + (
  929. self.config['consumer_timeout_ms'] / 1000.0)
  930. # Old KafkaConsumer methods are deprecated
  931. def configure(self, **configs):
  932. raise NotImplementedError(
  933. 'deprecated -- initialize a new consumer')
  934. def set_topic_partitions(self, *topics):
  935. raise NotImplementedError(
  936. 'deprecated -- use subscribe() or assign()')
  937. def fetch_messages(self):
  938. raise NotImplementedError(
  939. 'deprecated -- use poll() or iterator interface')
  940. def get_partition_offsets(self, topic, partition,
  941. request_time_ms, max_num_offsets):
  942. raise NotImplementedError(
  943. 'deprecated -- send an OffsetRequest with KafkaClient')
  944. def offsets(self, group=None):
  945. raise NotImplementedError('deprecated -- use committed(partition)')
  946. def task_done(self, message):
  947. raise NotImplementedError(
  948. 'deprecated -- commit offsets manually if needed')