m2m模型翻译
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

1015 lines
47 KiB

6 months ago
  1. from __future__ import absolute_import
  2. import collections
  3. import copy
  4. import logging
  5. import random
  6. import sys
  7. import time
  8. from kafka.vendor import six
  9. import kafka.errors as Errors
  10. from kafka.future import Future
  11. from kafka.metrics.stats import Avg, Count, Max, Rate
  12. from kafka.protocol.fetch import FetchRequest
  13. from kafka.protocol.offset import (
  14. OffsetRequest, OffsetResetStrategy, UNKNOWN_OFFSET
  15. )
  16. from kafka.record import MemoryRecords
  17. from kafka.serializer import Deserializer
  18. from kafka.structs import TopicPartition, OffsetAndTimestamp
  19. log = logging.getLogger(__name__)
  20. # Isolation levels
  21. READ_UNCOMMITTED = 0
  22. READ_COMMITTED = 1
  23. ConsumerRecord = collections.namedtuple("ConsumerRecord",
  24. ["topic", "partition", "offset", "timestamp", "timestamp_type",
  25. "key", "value", "headers", "checksum", "serialized_key_size", "serialized_value_size", "serialized_header_size"])
  26. CompletedFetch = collections.namedtuple("CompletedFetch",
  27. ["topic_partition", "fetched_offset", "response_version",
  28. "partition_data", "metric_aggregator"])
  29. class NoOffsetForPartitionError(Errors.KafkaError):
  30. pass
  31. class RecordTooLargeError(Errors.KafkaError):
  32. pass
  33. class Fetcher(six.Iterator):
  34. DEFAULT_CONFIG = {
  35. 'key_deserializer': None,
  36. 'value_deserializer': None,
  37. 'fetch_min_bytes': 1,
  38. 'fetch_max_wait_ms': 500,
  39. 'fetch_max_bytes': 52428800,
  40. 'max_partition_fetch_bytes': 1048576,
  41. 'max_poll_records': sys.maxsize,
  42. 'check_crcs': True,
  43. 'iterator_refetch_records': 1, # undocumented -- interface may change
  44. 'metric_group_prefix': 'consumer',
  45. 'api_version': (0, 8, 0),
  46. 'retry_backoff_ms': 100
  47. }
  48. def __init__(self, client, subscriptions, metrics, **configs):
  49. """Initialize a Kafka Message Fetcher.
  50. Keyword Arguments:
  51. key_deserializer (callable): Any callable that takes a
  52. raw message key and returns a deserialized key.
  53. value_deserializer (callable, optional): Any callable that takes a
  54. raw message value and returns a deserialized value.
  55. fetch_min_bytes (int): Minimum amount of data the server should
  56. return for a fetch request, otherwise wait up to
  57. fetch_max_wait_ms for more data to accumulate. Default: 1.
  58. fetch_max_wait_ms (int): The maximum amount of time in milliseconds
  59. the server will block before answering the fetch request if
  60. there isn't sufficient data to immediately satisfy the
  61. requirement given by fetch_min_bytes. Default: 500.
  62. fetch_max_bytes (int): The maximum amount of data the server should
  63. return for a fetch request. This is not an absolute maximum, if
  64. the first message in the first non-empty partition of the fetch
  65. is larger than this value, the message will still be returned
  66. to ensure that the consumer can make progress. NOTE: consumer
  67. performs fetches to multiple brokers in parallel so memory
  68. usage will depend on the number of brokers containing
  69. partitions for the topic.
  70. Supported Kafka version >= 0.10.1.0. Default: 52428800 (50 MB).
  71. max_partition_fetch_bytes (int): The maximum amount of data
  72. per-partition the server will return. The maximum total memory
  73. used for a request = #partitions * max_partition_fetch_bytes.
  74. This size must be at least as large as the maximum message size
  75. the server allows or else it is possible for the producer to
  76. send messages larger than the consumer can fetch. If that
  77. happens, the consumer can get stuck trying to fetch a large
  78. message on a certain partition. Default: 1048576.
  79. check_crcs (bool): Automatically check the CRC32 of the records
  80. consumed. This ensures no on-the-wire or on-disk corruption to
  81. the messages occurred. This check adds some overhead, so it may
  82. be disabled in cases seeking extreme performance. Default: True
  83. """
  84. self.config = copy.copy(self.DEFAULT_CONFIG)
  85. for key in self.config:
  86. if key in configs:
  87. self.config[key] = configs[key]
  88. self._client = client
  89. self._subscriptions = subscriptions
  90. self._completed_fetches = collections.deque() # Unparsed responses
  91. self._next_partition_records = None # Holds a single PartitionRecords until fully consumed
  92. self._iterator = None
  93. self._fetch_futures = collections.deque()
  94. self._sensors = FetchManagerMetrics(metrics, self.config['metric_group_prefix'])
  95. self._isolation_level = READ_UNCOMMITTED
  96. def send_fetches(self):
  97. """Send FetchRequests for all assigned partitions that do not already have
  98. an in-flight fetch or pending fetch data.
  99. Returns:
  100. List of Futures: each future resolves to a FetchResponse
  101. """
  102. futures = []
  103. for node_id, request in six.iteritems(self._create_fetch_requests()):
  104. if self._client.ready(node_id):
  105. log.debug("Sending FetchRequest to node %s", node_id)
  106. future = self._client.send(node_id, request, wakeup=False)
  107. future.add_callback(self._handle_fetch_response, request, time.time())
  108. future.add_errback(log.error, 'Fetch to node %s failed: %s', node_id)
  109. futures.append(future)
  110. self._fetch_futures.extend(futures)
  111. self._clean_done_fetch_futures()
  112. return futures
  113. def reset_offsets_if_needed(self, partitions):
  114. """Lookup and set offsets for any partitions which are awaiting an
  115. explicit reset.
  116. Arguments:
  117. partitions (set of TopicPartitions): the partitions to reset
  118. """
  119. for tp in partitions:
  120. # TODO: If there are several offsets to reset, we could submit offset requests in parallel
  121. if self._subscriptions.is_assigned(tp) and self._subscriptions.is_offset_reset_needed(tp):
  122. self._reset_offset(tp)
  123. def _clean_done_fetch_futures(self):
  124. while True:
  125. if not self._fetch_futures:
  126. break
  127. if not self._fetch_futures[0].is_done:
  128. break
  129. self._fetch_futures.popleft()
  130. def in_flight_fetches(self):
  131. """Return True if there are any unprocessed FetchRequests in flight."""
  132. self._clean_done_fetch_futures()
  133. return bool(self._fetch_futures)
  134. def update_fetch_positions(self, partitions):
  135. """Update the fetch positions for the provided partitions.
  136. Arguments:
  137. partitions (list of TopicPartitions): partitions to update
  138. Raises:
  139. NoOffsetForPartitionError: if no offset is stored for a given
  140. partition and no reset policy is available
  141. """
  142. # reset the fetch position to the committed position
  143. for tp in partitions:
  144. if not self._subscriptions.is_assigned(tp):
  145. log.warning("partition %s is not assigned - skipping offset"
  146. " update", tp)
  147. continue
  148. elif self._subscriptions.is_fetchable(tp):
  149. log.warning("partition %s is still fetchable -- skipping offset"
  150. " update", tp)
  151. continue
  152. if self._subscriptions.is_offset_reset_needed(tp):
  153. self._reset_offset(tp)
  154. elif self._subscriptions.assignment[tp].committed is None:
  155. # there's no committed position, so we need to reset with the
  156. # default strategy
  157. self._subscriptions.need_offset_reset(tp)
  158. self._reset_offset(tp)
  159. else:
  160. committed = self._subscriptions.assignment[tp].committed.offset
  161. log.debug("Resetting offset for partition %s to the committed"
  162. " offset %s", tp, committed)
  163. self._subscriptions.seek(tp, committed)
  164. def get_offsets_by_times(self, timestamps, timeout_ms):
  165. offsets = self._retrieve_offsets(timestamps, timeout_ms)
  166. for tp in timestamps:
  167. if tp not in offsets:
  168. offsets[tp] = None
  169. else:
  170. offset, timestamp = offsets[tp]
  171. offsets[tp] = OffsetAndTimestamp(offset, timestamp)
  172. return offsets
  173. def beginning_offsets(self, partitions, timeout_ms):
  174. return self.beginning_or_end_offset(
  175. partitions, OffsetResetStrategy.EARLIEST, timeout_ms)
  176. def end_offsets(self, partitions, timeout_ms):
  177. return self.beginning_or_end_offset(
  178. partitions, OffsetResetStrategy.LATEST, timeout_ms)
  179. def beginning_or_end_offset(self, partitions, timestamp, timeout_ms):
  180. timestamps = dict([(tp, timestamp) for tp in partitions])
  181. offsets = self._retrieve_offsets(timestamps, timeout_ms)
  182. for tp in timestamps:
  183. offsets[tp] = offsets[tp][0]
  184. return offsets
  185. def _reset_offset(self, partition):
  186. """Reset offsets for the given partition using the offset reset strategy.
  187. Arguments:
  188. partition (TopicPartition): the partition that needs reset offset
  189. Raises:
  190. NoOffsetForPartitionError: if no offset reset strategy is defined
  191. """
  192. timestamp = self._subscriptions.assignment[partition].reset_strategy
  193. if timestamp is OffsetResetStrategy.EARLIEST:
  194. strategy = 'earliest'
  195. elif timestamp is OffsetResetStrategy.LATEST:
  196. strategy = 'latest'
  197. else:
  198. raise NoOffsetForPartitionError(partition)
  199. log.debug("Resetting offset for partition %s to %s offset.",
  200. partition, strategy)
  201. offsets = self._retrieve_offsets({partition: timestamp})
  202. if partition in offsets:
  203. offset = offsets[partition][0]
  204. # we might lose the assignment while fetching the offset,
  205. # so check it is still active
  206. if self._subscriptions.is_assigned(partition):
  207. self._subscriptions.seek(partition, offset)
  208. else:
  209. log.debug("Could not find offset for partition %s since it is probably deleted" % (partition,))
  210. def _retrieve_offsets(self, timestamps, timeout_ms=float("inf")):
  211. """Fetch offset for each partition passed in ``timestamps`` map.
  212. Blocks until offsets are obtained, a non-retriable exception is raised
  213. or ``timeout_ms`` passed.
  214. Arguments:
  215. timestamps: {TopicPartition: int} dict with timestamps to fetch
  216. offsets by. -1 for the latest available, -2 for the earliest
  217. available. Otherwise timestamp is treated as epoch milliseconds.
  218. Returns:
  219. {TopicPartition: (int, int)}: Mapping of partition to
  220. retrieved offset and timestamp. If offset does not exist for
  221. the provided timestamp, that partition will be missing from
  222. this mapping.
  223. """
  224. if not timestamps:
  225. return {}
  226. start_time = time.time()
  227. remaining_ms = timeout_ms
  228. timestamps = copy.copy(timestamps)
  229. while remaining_ms > 0:
  230. if not timestamps:
  231. return {}
  232. future = self._send_offset_requests(timestamps)
  233. self._client.poll(future=future, timeout_ms=remaining_ms)
  234. if future.succeeded():
  235. return future.value
  236. if not future.retriable():
  237. raise future.exception # pylint: disable-msg=raising-bad-type
  238. elapsed_ms = (time.time() - start_time) * 1000
  239. remaining_ms = timeout_ms - elapsed_ms
  240. if remaining_ms < 0:
  241. break
  242. if future.exception.invalid_metadata:
  243. refresh_future = self._client.cluster.request_update()
  244. self._client.poll(future=refresh_future, timeout_ms=remaining_ms)
  245. # Issue #1780
  246. # Recheck partition existence after after a successful metadata refresh
  247. if refresh_future.succeeded() and isinstance(future.exception, Errors.StaleMetadata):
  248. log.debug("Stale metadata was raised, and we now have an updated metadata. Rechecking partition existence")
  249. unknown_partition = future.exception.args[0] # TopicPartition from StaleMetadata
  250. if self._client.cluster.leader_for_partition(unknown_partition) is None:
  251. log.debug("Removed partition %s from offsets retrieval" % (unknown_partition, ))
  252. timestamps.pop(unknown_partition)
  253. else:
  254. time.sleep(self.config['retry_backoff_ms'] / 1000.0)
  255. elapsed_ms = (time.time() - start_time) * 1000
  256. remaining_ms = timeout_ms - elapsed_ms
  257. raise Errors.KafkaTimeoutError(
  258. "Failed to get offsets by timestamps in %s ms" % (timeout_ms,))
  259. def fetched_records(self, max_records=None, update_offsets=True):
  260. """Returns previously fetched records and updates consumed offsets.
  261. Arguments:
  262. max_records (int): Maximum number of records returned. Defaults
  263. to max_poll_records configuration.
  264. Raises:
  265. OffsetOutOfRangeError: if no subscription offset_reset_strategy
  266. CorruptRecordException: if message crc validation fails (check_crcs
  267. must be set to True)
  268. RecordTooLargeError: if a message is larger than the currently
  269. configured max_partition_fetch_bytes
  270. TopicAuthorizationError: if consumer is not authorized to fetch
  271. messages from the topic
  272. Returns: (records (dict), partial (bool))
  273. records: {TopicPartition: [messages]}
  274. partial: True if records returned did not fully drain any pending
  275. partition requests. This may be useful for choosing when to
  276. pipeline additional fetch requests.
  277. """
  278. if max_records is None:
  279. max_records = self.config['max_poll_records']
  280. assert max_records > 0
  281. drained = collections.defaultdict(list)
  282. records_remaining = max_records
  283. while records_remaining > 0:
  284. if not self._next_partition_records:
  285. if not self._completed_fetches:
  286. break
  287. completion = self._completed_fetches.popleft()
  288. self._next_partition_records = self._parse_fetched_data(completion)
  289. else:
  290. records_remaining -= self._append(drained,
  291. self._next_partition_records,
  292. records_remaining,
  293. update_offsets)
  294. return dict(drained), bool(self._completed_fetches)
  295. def _append(self, drained, part, max_records, update_offsets):
  296. if not part:
  297. return 0
  298. tp = part.topic_partition
  299. fetch_offset = part.fetch_offset
  300. if not self._subscriptions.is_assigned(tp):
  301. # this can happen when a rebalance happened before
  302. # fetched records are returned to the consumer's poll call
  303. log.debug("Not returning fetched records for partition %s"
  304. " since it is no longer assigned", tp)
  305. else:
  306. # note that the position should always be available
  307. # as long as the partition is still assigned
  308. position = self._subscriptions.assignment[tp].position
  309. if not self._subscriptions.is_fetchable(tp):
  310. # this can happen when a partition is paused before
  311. # fetched records are returned to the consumer's poll call
  312. log.debug("Not returning fetched records for assigned partition"
  313. " %s since it is no longer fetchable", tp)
  314. elif fetch_offset == position:
  315. # we are ensured to have at least one record since we already checked for emptiness
  316. part_records = part.take(max_records)
  317. next_offset = part_records[-1].offset + 1
  318. log.log(0, "Returning fetched records at offset %d for assigned"
  319. " partition %s and update position to %s", position,
  320. tp, next_offset)
  321. for record in part_records:
  322. drained[tp].append(record)
  323. if update_offsets:
  324. self._subscriptions.assignment[tp].position = next_offset
  325. return len(part_records)
  326. else:
  327. # these records aren't next in line based on the last consumed
  328. # position, ignore them they must be from an obsolete request
  329. log.debug("Ignoring fetched records for %s at offset %s since"
  330. " the current position is %d", tp, part.fetch_offset,
  331. position)
  332. part.discard()
  333. return 0
  334. def _message_generator(self):
  335. """Iterate over fetched_records"""
  336. while self._next_partition_records or self._completed_fetches:
  337. if not self._next_partition_records:
  338. completion = self._completed_fetches.popleft()
  339. self._next_partition_records = self._parse_fetched_data(completion)
  340. continue
  341. # Send additional FetchRequests when the internal queue is low
  342. # this should enable moderate pipelining
  343. if len(self._completed_fetches) <= self.config['iterator_refetch_records']:
  344. self.send_fetches()
  345. tp = self._next_partition_records.topic_partition
  346. # We can ignore any prior signal to drop pending message sets
  347. # because we are starting from a fresh one where fetch_offset == position
  348. # i.e., the user seek()'d to this position
  349. self._subscriptions.assignment[tp].drop_pending_message_set = False
  350. for msg in self._next_partition_records.take():
  351. # Because we are in a generator, it is possible for
  352. # subscription state to change between yield calls
  353. # so we need to re-check on each loop
  354. # this should catch assignment changes, pauses
  355. # and resets via seek_to_beginning / seek_to_end
  356. if not self._subscriptions.is_fetchable(tp):
  357. log.debug("Not returning fetched records for partition %s"
  358. " since it is no longer fetchable", tp)
  359. self._next_partition_records = None
  360. break
  361. # If there is a seek during message iteration,
  362. # we should stop unpacking this message set and
  363. # wait for a new fetch response that aligns with the
  364. # new seek position
  365. elif self._subscriptions.assignment[tp].drop_pending_message_set:
  366. log.debug("Skipping remainder of message set for partition %s", tp)
  367. self._subscriptions.assignment[tp].drop_pending_message_set = False
  368. self._next_partition_records = None
  369. break
  370. # Compressed messagesets may include earlier messages
  371. elif msg.offset < self._subscriptions.assignment[tp].position:
  372. log.debug("Skipping message offset: %s (expecting %s)",
  373. msg.offset,
  374. self._subscriptions.assignment[tp].position)
  375. continue
  376. self._subscriptions.assignment[tp].position = msg.offset + 1
  377. yield msg
  378. self._next_partition_records = None
  379. def _unpack_message_set(self, tp, records):
  380. try:
  381. batch = records.next_batch()
  382. while batch is not None:
  383. # LegacyRecordBatch cannot access either base_offset or last_offset_delta
  384. try:
  385. self._subscriptions.assignment[tp].last_offset_from_message_batch = batch.base_offset + \
  386. batch.last_offset_delta
  387. except AttributeError:
  388. pass
  389. for record in batch:
  390. key_size = len(record.key) if record.key is not None else -1
  391. value_size = len(record.value) if record.value is not None else -1
  392. key = self._deserialize(
  393. self.config['key_deserializer'],
  394. tp.topic, record.key)
  395. value = self._deserialize(
  396. self.config['value_deserializer'],
  397. tp.topic, record.value)
  398. headers = record.headers
  399. header_size = sum(
  400. len(h_key.encode("utf-8")) + (len(h_val) if h_val is not None else 0) for h_key, h_val in
  401. headers) if headers else -1
  402. yield ConsumerRecord(
  403. tp.topic, tp.partition, record.offset, record.timestamp,
  404. record.timestamp_type, key, value, headers, record.checksum,
  405. key_size, value_size, header_size)
  406. batch = records.next_batch()
  407. # If unpacking raises StopIteration, it is erroneously
  408. # caught by the generator. We want all exceptions to be raised
  409. # back to the user. See Issue 545
  410. except StopIteration as e:
  411. log.exception('StopIteration raised unpacking messageset')
  412. raise RuntimeError('StopIteration raised unpacking messageset')
  413. def __iter__(self): # pylint: disable=non-iterator-returned
  414. return self
  415. def __next__(self):
  416. if not self._iterator:
  417. self._iterator = self._message_generator()
  418. try:
  419. return next(self._iterator)
  420. except StopIteration:
  421. self._iterator = None
  422. raise
  423. def _deserialize(self, f, topic, bytes_):
  424. if not f:
  425. return bytes_
  426. if isinstance(f, Deserializer):
  427. return f.deserialize(topic, bytes_)
  428. return f(bytes_)
  429. def _send_offset_requests(self, timestamps):
  430. """Fetch offsets for each partition in timestamps dict. This may send
  431. request to multiple nodes, based on who is Leader for partition.
  432. Arguments:
  433. timestamps (dict): {TopicPartition: int} mapping of fetching
  434. timestamps.
  435. Returns:
  436. Future: resolves to a mapping of retrieved offsets
  437. """
  438. timestamps_by_node = collections.defaultdict(dict)
  439. for partition, timestamp in six.iteritems(timestamps):
  440. node_id = self._client.cluster.leader_for_partition(partition)
  441. if node_id is None:
  442. self._client.add_topic(partition.topic)
  443. log.debug("Partition %s is unknown for fetching offset,"
  444. " wait for metadata refresh", partition)
  445. return Future().failure(Errors.StaleMetadata(partition))
  446. elif node_id == -1:
  447. log.debug("Leader for partition %s unavailable for fetching "
  448. "offset, wait for metadata refresh", partition)
  449. return Future().failure(
  450. Errors.LeaderNotAvailableError(partition))
  451. else:
  452. timestamps_by_node[node_id][partition] = timestamp
  453. # Aggregate results until we have all
  454. list_offsets_future = Future()
  455. responses = []
  456. node_count = len(timestamps_by_node)
  457. def on_success(value):
  458. responses.append(value)
  459. if len(responses) == node_count:
  460. offsets = {}
  461. for r in responses:
  462. offsets.update(r)
  463. list_offsets_future.success(offsets)
  464. def on_fail(err):
  465. if not list_offsets_future.is_done:
  466. list_offsets_future.failure(err)
  467. for node_id, timestamps in six.iteritems(timestamps_by_node):
  468. _f = self._send_offset_request(node_id, timestamps)
  469. _f.add_callback(on_success)
  470. _f.add_errback(on_fail)
  471. return list_offsets_future
  472. def _send_offset_request(self, node_id, timestamps):
  473. by_topic = collections.defaultdict(list)
  474. for tp, timestamp in six.iteritems(timestamps):
  475. if self.config['api_version'] >= (0, 10, 1):
  476. data = (tp.partition, timestamp)
  477. else:
  478. data = (tp.partition, timestamp, 1)
  479. by_topic[tp.topic].append(data)
  480. if self.config['api_version'] >= (0, 10, 1):
  481. request = OffsetRequest[1](-1, list(six.iteritems(by_topic)))
  482. else:
  483. request = OffsetRequest[0](-1, list(six.iteritems(by_topic)))
  484. # Client returns a future that only fails on network issues
  485. # so create a separate future and attach a callback to update it
  486. # based on response error codes
  487. future = Future()
  488. _f = self._client.send(node_id, request)
  489. _f.add_callback(self._handle_offset_response, future)
  490. _f.add_errback(lambda e: future.failure(e))
  491. return future
  492. def _handle_offset_response(self, future, response):
  493. """Callback for the response of the list offset call above.
  494. Arguments:
  495. future (Future): the future to update based on response
  496. response (OffsetResponse): response from the server
  497. Raises:
  498. AssertionError: if response does not match partition
  499. """
  500. timestamp_offset_map = {}
  501. for topic, part_data in response.topics:
  502. for partition_info in part_data:
  503. partition, error_code = partition_info[:2]
  504. partition = TopicPartition(topic, partition)
  505. error_type = Errors.for_code(error_code)
  506. if error_type is Errors.NoError:
  507. if response.API_VERSION == 0:
  508. offsets = partition_info[2]
  509. assert len(offsets) <= 1, 'Expected OffsetResponse with one offset'
  510. if not offsets:
  511. offset = UNKNOWN_OFFSET
  512. else:
  513. offset = offsets[0]
  514. log.debug("Handling v0 ListOffsetResponse response for %s. "
  515. "Fetched offset %s", partition, offset)
  516. if offset != UNKNOWN_OFFSET:
  517. timestamp_offset_map[partition] = (offset, None)
  518. else:
  519. timestamp, offset = partition_info[2:]
  520. log.debug("Handling ListOffsetResponse response for %s. "
  521. "Fetched offset %s, timestamp %s",
  522. partition, offset, timestamp)
  523. if offset != UNKNOWN_OFFSET:
  524. timestamp_offset_map[partition] = (offset, timestamp)
  525. elif error_type is Errors.UnsupportedForMessageFormatError:
  526. # The message format on the broker side is before 0.10.0,
  527. # we simply put None in the response.
  528. log.debug("Cannot search by timestamp for partition %s because the"
  529. " message format version is before 0.10.0", partition)
  530. elif error_type is Errors.NotLeaderForPartitionError:
  531. log.debug("Attempt to fetch offsets for partition %s failed due"
  532. " to obsolete leadership information, retrying.",
  533. partition)
  534. future.failure(error_type(partition))
  535. return
  536. elif error_type is Errors.UnknownTopicOrPartitionError:
  537. log.warning("Received unknown topic or partition error in ListOffset "
  538. "request for partition %s. The topic/partition " +
  539. "may not exist or the user may not have Describe access "
  540. "to it.", partition)
  541. future.failure(error_type(partition))
  542. return
  543. else:
  544. log.warning("Attempt to fetch offsets for partition %s failed due to:"
  545. " %s", partition, error_type)
  546. future.failure(error_type(partition))
  547. return
  548. if not future.is_done:
  549. future.success(timestamp_offset_map)
  550. def _fetchable_partitions(self):
  551. fetchable = self._subscriptions.fetchable_partitions()
  552. # do not fetch a partition if we have a pending fetch response to process
  553. current = self._next_partition_records
  554. pending = copy.copy(self._completed_fetches)
  555. if current:
  556. fetchable.discard(current.topic_partition)
  557. for fetch in pending:
  558. fetchable.discard(fetch.topic_partition)
  559. return fetchable
  560. def _create_fetch_requests(self):
  561. """Create fetch requests for all assigned partitions, grouped by node.
  562. FetchRequests skipped if no leader, or node has requests in flight
  563. Returns:
  564. dict: {node_id: FetchRequest, ...} (version depends on api_version)
  565. """
  566. # create the fetch info as a dict of lists of partition info tuples
  567. # which can be passed to FetchRequest() via .items()
  568. fetchable = collections.defaultdict(lambda: collections.defaultdict(list))
  569. for partition in self._fetchable_partitions():
  570. node_id = self._client.cluster.leader_for_partition(partition)
  571. # advance position for any deleted compacted messages if required
  572. if self._subscriptions.assignment[partition].last_offset_from_message_batch:
  573. next_offset_from_batch_header = self._subscriptions.assignment[partition].last_offset_from_message_batch + 1
  574. if next_offset_from_batch_header > self._subscriptions.assignment[partition].position:
  575. log.debug(
  576. "Advance position for partition %s from %s to %s (last message batch location plus one)"
  577. " to correct for deleted compacted messages",
  578. partition, self._subscriptions.assignment[partition].position, next_offset_from_batch_header)
  579. self._subscriptions.assignment[partition].position = next_offset_from_batch_header
  580. position = self._subscriptions.assignment[partition].position
  581. # fetch if there is a leader and no in-flight requests
  582. if node_id is None or node_id == -1:
  583. log.debug("No leader found for partition %s."
  584. " Requesting metadata update", partition)
  585. self._client.cluster.request_update()
  586. elif self._client.in_flight_request_count(node_id) == 0:
  587. partition_info = (
  588. partition.partition,
  589. position,
  590. self.config['max_partition_fetch_bytes']
  591. )
  592. fetchable[node_id][partition.topic].append(partition_info)
  593. log.debug("Adding fetch request for partition %s at offset %d",
  594. partition, position)
  595. else:
  596. log.log(0, "Skipping fetch for partition %s because there is an inflight request to node %s",
  597. partition, node_id)
  598. if self.config['api_version'] >= (0, 11, 0):
  599. version = 4
  600. elif self.config['api_version'] >= (0, 10, 1):
  601. version = 3
  602. elif self.config['api_version'] >= (0, 10):
  603. version = 2
  604. elif self.config['api_version'] == (0, 9):
  605. version = 1
  606. else:
  607. version = 0
  608. requests = {}
  609. for node_id, partition_data in six.iteritems(fetchable):
  610. if version < 3:
  611. requests[node_id] = FetchRequest[version](
  612. -1, # replica_id
  613. self.config['fetch_max_wait_ms'],
  614. self.config['fetch_min_bytes'],
  615. partition_data.items())
  616. else:
  617. # As of version == 3 partitions will be returned in order as
  618. # they are requested, so to avoid starvation with
  619. # `fetch_max_bytes` option we need this shuffle
  620. # NOTE: we do have partition_data in random order due to usage
  621. # of unordered structures like dicts, but that does not
  622. # guarantee equal distribution, and starting in Python3.6
  623. # dicts retain insert order.
  624. partition_data = list(partition_data.items())
  625. random.shuffle(partition_data)
  626. if version == 3:
  627. requests[node_id] = FetchRequest[version](
  628. -1, # replica_id
  629. self.config['fetch_max_wait_ms'],
  630. self.config['fetch_min_bytes'],
  631. self.config['fetch_max_bytes'],
  632. partition_data)
  633. else:
  634. requests[node_id] = FetchRequest[version](
  635. -1, # replica_id
  636. self.config['fetch_max_wait_ms'],
  637. self.config['fetch_min_bytes'],
  638. self.config['fetch_max_bytes'],
  639. self._isolation_level,
  640. partition_data)
  641. return requests
  642. def _handle_fetch_response(self, request, send_time, response):
  643. """The callback for fetch completion"""
  644. fetch_offsets = {}
  645. for topic, partitions in request.topics:
  646. for partition_data in partitions:
  647. partition, offset = partition_data[:2]
  648. fetch_offsets[TopicPartition(topic, partition)] = offset
  649. partitions = set([TopicPartition(topic, partition_data[0])
  650. for topic, partitions in response.topics
  651. for partition_data in partitions])
  652. metric_aggregator = FetchResponseMetricAggregator(self._sensors, partitions)
  653. # randomized ordering should improve balance for short-lived consumers
  654. random.shuffle(response.topics)
  655. for topic, partitions in response.topics:
  656. random.shuffle(partitions)
  657. for partition_data in partitions:
  658. tp = TopicPartition(topic, partition_data[0])
  659. completed_fetch = CompletedFetch(
  660. tp, fetch_offsets[tp],
  661. response.API_VERSION,
  662. partition_data[1:],
  663. metric_aggregator
  664. )
  665. self._completed_fetches.append(completed_fetch)
  666. if response.API_VERSION >= 1:
  667. self._sensors.fetch_throttle_time_sensor.record(response.throttle_time_ms)
  668. self._sensors.fetch_latency.record((time.time() - send_time) * 1000)
  669. def _parse_fetched_data(self, completed_fetch):
  670. tp = completed_fetch.topic_partition
  671. fetch_offset = completed_fetch.fetched_offset
  672. num_bytes = 0
  673. records_count = 0
  674. parsed_records = None
  675. error_code, highwater = completed_fetch.partition_data[:2]
  676. error_type = Errors.for_code(error_code)
  677. try:
  678. if not self._subscriptions.is_fetchable(tp):
  679. # this can happen when a rebalance happened or a partition
  680. # consumption paused while fetch is still in-flight
  681. log.debug("Ignoring fetched records for partition %s"
  682. " since it is no longer fetchable", tp)
  683. elif error_type is Errors.NoError:
  684. self._subscriptions.assignment[tp].highwater = highwater
  685. # we are interested in this fetch only if the beginning
  686. # offset (of the *request*) matches the current consumed position
  687. # Note that the *response* may return a messageset that starts
  688. # earlier (e.g., compressed messages) or later (e.g., compacted topic)
  689. position = self._subscriptions.assignment[tp].position
  690. if position is None or position != fetch_offset:
  691. log.debug("Discarding fetch response for partition %s"
  692. " since its offset %d does not match the"
  693. " expected offset %d", tp, fetch_offset,
  694. position)
  695. return None
  696. records = MemoryRecords(completed_fetch.partition_data[-1])
  697. if records.has_next():
  698. log.debug("Adding fetched record for partition %s with"
  699. " offset %d to buffered record list", tp,
  700. position)
  701. unpacked = list(self._unpack_message_set(tp, records))
  702. parsed_records = self.PartitionRecords(fetch_offset, tp, unpacked)
  703. last_offset = unpacked[-1].offset
  704. self._sensors.records_fetch_lag.record(highwater - last_offset)
  705. num_bytes = records.valid_bytes()
  706. records_count = len(unpacked)
  707. elif records.size_in_bytes() > 0:
  708. # we did not read a single message from a non-empty
  709. # buffer because that message's size is larger than
  710. # fetch size, in this case record this exception
  711. record_too_large_partitions = {tp: fetch_offset}
  712. raise RecordTooLargeError(
  713. "There are some messages at [Partition=Offset]: %s "
  714. " whose size is larger than the fetch size %s"
  715. " and hence cannot be ever returned."
  716. " Increase the fetch size, or decrease the maximum message"
  717. " size the broker will allow." % (
  718. record_too_large_partitions,
  719. self.config['max_partition_fetch_bytes']),
  720. record_too_large_partitions)
  721. self._sensors.record_topic_fetch_metrics(tp.topic, num_bytes, records_count)
  722. elif error_type in (Errors.NotLeaderForPartitionError,
  723. Errors.UnknownTopicOrPartitionError):
  724. self._client.cluster.request_update()
  725. elif error_type is Errors.OffsetOutOfRangeError:
  726. position = self._subscriptions.assignment[tp].position
  727. if position is None or position != fetch_offset:
  728. log.debug("Discarding stale fetch response for partition %s"
  729. " since the fetched offset %d does not match the"
  730. " current offset %d", tp, fetch_offset, position)
  731. elif self._subscriptions.has_default_offset_reset_policy():
  732. log.info("Fetch offset %s is out of range for topic-partition %s", fetch_offset, tp)
  733. self._subscriptions.need_offset_reset(tp)
  734. else:
  735. raise Errors.OffsetOutOfRangeError({tp: fetch_offset})
  736. elif error_type is Errors.TopicAuthorizationFailedError:
  737. log.warning("Not authorized to read from topic %s.", tp.topic)
  738. raise Errors.TopicAuthorizationFailedError(set(tp.topic))
  739. elif error_type is Errors.UnknownError:
  740. log.warning("Unknown error fetching data for topic-partition %s", tp)
  741. else:
  742. raise error_type('Unexpected error while fetching data')
  743. finally:
  744. completed_fetch.metric_aggregator.record(tp, num_bytes, records_count)
  745. return parsed_records
  746. class PartitionRecords(object):
  747. def __init__(self, fetch_offset, tp, messages):
  748. self.fetch_offset = fetch_offset
  749. self.topic_partition = tp
  750. self.messages = messages
  751. # When fetching an offset that is in the middle of a
  752. # compressed batch, we will get all messages in the batch.
  753. # But we want to start 'take' at the fetch_offset
  754. # (or the next highest offset in case the message was compacted)
  755. for i, msg in enumerate(messages):
  756. if msg.offset < fetch_offset:
  757. log.debug("Skipping message offset: %s (expecting %s)",
  758. msg.offset, fetch_offset)
  759. else:
  760. self.message_idx = i
  761. break
  762. else:
  763. self.message_idx = 0
  764. self.messages = None
  765. # For truthiness evaluation we need to define __len__ or __nonzero__
  766. def __len__(self):
  767. if self.messages is None or self.message_idx >= len(self.messages):
  768. return 0
  769. return len(self.messages) - self.message_idx
  770. def discard(self):
  771. self.messages = None
  772. def take(self, n=None):
  773. if not len(self):
  774. return []
  775. if n is None or n > len(self):
  776. n = len(self)
  777. next_idx = self.message_idx + n
  778. res = self.messages[self.message_idx:next_idx]
  779. self.message_idx = next_idx
  780. # fetch_offset should be incremented by 1 to parallel the
  781. # subscription position (also incremented by 1)
  782. self.fetch_offset = max(self.fetch_offset, res[-1].offset + 1)
  783. return res
  784. class FetchResponseMetricAggregator(object):
  785. """
  786. Since we parse the message data for each partition from each fetch
  787. response lazily, fetch-level metrics need to be aggregated as the messages
  788. from each partition are parsed. This class is used to facilitate this
  789. incremental aggregation.
  790. """
  791. def __init__(self, sensors, partitions):
  792. self.sensors = sensors
  793. self.unrecorded_partitions = partitions
  794. self.total_bytes = 0
  795. self.total_records = 0
  796. def record(self, partition, num_bytes, num_records):
  797. """
  798. After each partition is parsed, we update the current metric totals
  799. with the total bytes and number of records parsed. After all partitions
  800. have reported, we write the metric.
  801. """
  802. self.unrecorded_partitions.remove(partition)
  803. self.total_bytes += num_bytes
  804. self.total_records += num_records
  805. # once all expected partitions from the fetch have reported in, record the metrics
  806. if not self.unrecorded_partitions:
  807. self.sensors.bytes_fetched.record(self.total_bytes)
  808. self.sensors.records_fetched.record(self.total_records)
  809. class FetchManagerMetrics(object):
  810. def __init__(self, metrics, prefix):
  811. self.metrics = metrics
  812. self.group_name = '%s-fetch-manager-metrics' % (prefix,)
  813. self.bytes_fetched = metrics.sensor('bytes-fetched')
  814. self.bytes_fetched.add(metrics.metric_name('fetch-size-avg', self.group_name,
  815. 'The average number of bytes fetched per request'), Avg())
  816. self.bytes_fetched.add(metrics.metric_name('fetch-size-max', self.group_name,
  817. 'The maximum number of bytes fetched per request'), Max())
  818. self.bytes_fetched.add(metrics.metric_name('bytes-consumed-rate', self.group_name,
  819. 'The average number of bytes consumed per second'), Rate())
  820. self.records_fetched = self.metrics.sensor('records-fetched')
  821. self.records_fetched.add(metrics.metric_name('records-per-request-avg', self.group_name,
  822. 'The average number of records in each request'), Avg())
  823. self.records_fetched.add(metrics.metric_name('records-consumed-rate', self.group_name,
  824. 'The average number of records consumed per second'), Rate())
  825. self.fetch_latency = metrics.sensor('fetch-latency')
  826. self.fetch_latency.add(metrics.metric_name('fetch-latency-avg', self.group_name,
  827. 'The average time taken for a fetch request.'), Avg())
  828. self.fetch_latency.add(metrics.metric_name('fetch-latency-max', self.group_name,
  829. 'The max time taken for any fetch request.'), Max())
  830. self.fetch_latency.add(metrics.metric_name('fetch-rate', self.group_name,
  831. 'The number of fetch requests per second.'), Rate(sampled_stat=Count()))
  832. self.records_fetch_lag = metrics.sensor('records-lag')
  833. self.records_fetch_lag.add(metrics.metric_name('records-lag-max', self.group_name,
  834. 'The maximum lag in terms of number of records for any partition in self window'), Max())
  835. self.fetch_throttle_time_sensor = metrics.sensor('fetch-throttle-time')
  836. self.fetch_throttle_time_sensor.add(metrics.metric_name('fetch-throttle-time-avg', self.group_name,
  837. 'The average throttle time in ms'), Avg())
  838. self.fetch_throttle_time_sensor.add(metrics.metric_name('fetch-throttle-time-max', self.group_name,
  839. 'The maximum throttle time in ms'), Max())
  840. def record_topic_fetch_metrics(self, topic, num_bytes, num_records):
  841. # record bytes fetched
  842. name = '.'.join(['topic', topic, 'bytes-fetched'])
  843. bytes_fetched = self.metrics.get_sensor(name)
  844. if not bytes_fetched:
  845. metric_tags = {'topic': topic.replace('.', '_')}
  846. bytes_fetched = self.metrics.sensor(name)
  847. bytes_fetched.add(self.metrics.metric_name('fetch-size-avg',
  848. self.group_name,
  849. 'The average number of bytes fetched per request for topic %s' % (topic,),
  850. metric_tags), Avg())
  851. bytes_fetched.add(self.metrics.metric_name('fetch-size-max',
  852. self.group_name,
  853. 'The maximum number of bytes fetched per request for topic %s' % (topic,),
  854. metric_tags), Max())
  855. bytes_fetched.add(self.metrics.metric_name('bytes-consumed-rate',
  856. self.group_name,
  857. 'The average number of bytes consumed per second for topic %s' % (topic,),
  858. metric_tags), Rate())
  859. bytes_fetched.record(num_bytes)
  860. # record records fetched
  861. name = '.'.join(['topic', topic, 'records-fetched'])
  862. records_fetched = self.metrics.get_sensor(name)
  863. if not records_fetched:
  864. metric_tags = {'topic': topic.replace('.', '_')}
  865. records_fetched = self.metrics.sensor(name)
  866. records_fetched.add(self.metrics.metric_name('records-per-request-avg',
  867. self.group_name,
  868. 'The average number of records in each request for topic %s' % (topic,),
  869. metric_tags), Avg())
  870. records_fetched.add(self.metrics.metric_name('records-consumed-rate',
  871. self.group_name,
  872. 'The average number of records consumed per second for topic %s' % (topic,),
  873. metric_tags), Rate())
  874. records_fetched.record(num_records)