|
4 | 4 | import sys |
5 | 5 | import traceback |
6 | 6 | import warnings |
| 7 | +from contextlib import AsyncExitStack |
7 | 8 |
|
8 | 9 | from aiokafka import __version__ |
9 | 10 | from aiokafka.abc import ConsumerRebalanceListener |
@@ -335,7 +336,6 @@ def __init__( |
335 | 336 |
|
336 | 337 | if loop.get_debug(): |
337 | 338 | self._source_traceback = traceback.extract_stack(sys._getframe(1)) |
338 | | - self._closed = False |
339 | 339 |
|
340 | 340 | if topics: |
341 | 341 | topics = self._validate_topics(topics) |
@@ -368,83 +368,98 @@ async def start(self): |
368 | 368 | self._loop is get_running_loop() |
369 | 369 | ), "Please create objects with the same loop as running with" |
370 | 370 | assert self._fetcher is None, "Did you call `start` twice?" |
371 | | - await self._client.bootstrap() |
372 | | - await self._wait_topics() |
373 | 371 |
|
374 | | - if self._client.api_version < (0, 9): |
375 | | - raise ValueError(f"Unsupported Kafka version: {self._client.api_version}") |
| 372 | + self._exit_stack = AsyncExitStack() |
| 373 | + try: |
| 374 | + await self._client.bootstrap() |
| 375 | + self._exit_stack.push_async_callback(self._client.close) |
| 376 | + await self._wait_topics() |
376 | 377 |
|
377 | | - if ( |
378 | | - self._isolation_level == "read_committed" |
379 | | - and self._client.api_version < (0, 11) # fmt: skip |
380 | | - ): |
381 | | - raise UnsupportedVersionError( |
382 | | - "`read_committed` isolation_level available only for Brokers " |
383 | | - "0.11 and above" |
384 | | - ) |
| 378 | + if self._client.api_version < (0, 9): |
| 379 | + raise ValueError( |
| 380 | + f"Unsupported Kafka version: {self._client.api_version}" |
| 381 | + ) |
385 | 382 |
|
386 | | - self._fetcher = Fetcher( |
387 | | - self._client, |
388 | | - self._subscription, |
389 | | - key_deserializer=self._key_deserializer, |
390 | | - value_deserializer=self._value_deserializer, |
391 | | - fetch_min_bytes=self._fetch_min_bytes, |
392 | | - fetch_max_bytes=self._fetch_max_bytes, |
393 | | - fetch_max_wait_ms=self._fetch_max_wait_ms, |
394 | | - max_partition_fetch_bytes=self._max_partition_fetch_bytes, |
395 | | - check_crcs=self._check_crcs, |
396 | | - fetcher_timeout=self._consumer_timeout, |
397 | | - retry_backoff_ms=self._retry_backoff_ms, |
398 | | - auto_offset_reset=self._auto_offset_reset, |
399 | | - isolation_level=self._isolation_level, |
400 | | - ) |
| 383 | + if ( |
| 384 | + self._isolation_level == "read_committed" |
| 385 | + and self._client.api_version < (0, 11) # fmt: skip |
| 386 | + ): |
| 387 | + raise UnsupportedVersionError( |
| 388 | + "`read_committed` isolation_level available only for Brokers " |
| 389 | + "0.11 and above" |
| 390 | + ) |
401 | 391 |
|
402 | | - if self._group_id is not None: |
403 | | - # using group coordinator for automatic partitions assignment |
404 | | - self._coordinator = GroupCoordinator( |
| 392 | + self._fetcher = Fetcher( |
405 | 393 | self._client, |
406 | 394 | self._subscription, |
407 | | - group_id=self._group_id, |
408 | | - group_instance_id=self._group_instance_id, |
409 | | - heartbeat_interval_ms=self._heartbeat_interval_ms, |
410 | | - session_timeout_ms=self._session_timeout_ms, |
| 395 | + key_deserializer=self._key_deserializer, |
| 396 | + value_deserializer=self._value_deserializer, |
| 397 | + fetch_min_bytes=self._fetch_min_bytes, |
| 398 | + fetch_max_bytes=self._fetch_max_bytes, |
| 399 | + fetch_max_wait_ms=self._fetch_max_wait_ms, |
| 400 | + max_partition_fetch_bytes=self._max_partition_fetch_bytes, |
| 401 | + check_crcs=self._check_crcs, |
| 402 | + fetcher_timeout=self._consumer_timeout, |
411 | 403 | retry_backoff_ms=self._retry_backoff_ms, |
412 | | - enable_auto_commit=self._enable_auto_commit, |
413 | | - auto_commit_interval_ms=self._auto_commit_interval_ms, |
414 | | - assignors=self._partition_assignment_strategy, |
415 | | - exclude_internal_topics=self._exclude_internal_topics, |
416 | | - rebalance_timeout_ms=self._rebalance_timeout_ms, |
417 | | - max_poll_interval_ms=self._max_poll_interval_ms, |
| 404 | + auto_offset_reset=self._auto_offset_reset, |
| 405 | + isolation_level=self._isolation_level, |
418 | 406 | ) |
419 | | - if self._subscription.subscription is not None: |
420 | | - if self._subscription.partitions_auto_assigned(): |
| 407 | + self._exit_stack.push_async_callback(self._fetcher.close) |
| 408 | + |
| 409 | + if self._group_id is not None: |
| 410 | + # using group coordinator for automatic partitions assignment |
| 411 | + self._coordinator = GroupCoordinator( |
| 412 | + self._client, |
| 413 | + self._subscription, |
| 414 | + group_id=self._group_id, |
| 415 | + group_instance_id=self._group_instance_id, |
| 416 | + heartbeat_interval_ms=self._heartbeat_interval_ms, |
| 417 | + session_timeout_ms=self._session_timeout_ms, |
| 418 | + retry_backoff_ms=self._retry_backoff_ms, |
| 419 | + enable_auto_commit=self._enable_auto_commit, |
| 420 | + auto_commit_interval_ms=self._auto_commit_interval_ms, |
| 421 | + assignors=self._partition_assignment_strategy, |
| 422 | + exclude_internal_topics=self._exclude_internal_topics, |
| 423 | + rebalance_timeout_ms=self._rebalance_timeout_ms, |
| 424 | + max_poll_interval_ms=self._max_poll_interval_ms, |
| 425 | + ) |
| 426 | + self._exit_stack.push_async_callback(self._coordinator.close) |
| 427 | + |
| 428 | + if self._subscription.subscription is not None: |
| 429 | + if self._subscription.partitions_auto_assigned(): |
| 430 | + # Either we passed `topics` to constructor or `subscribe` |
| 431 | + # was called before `start` |
| 432 | + await self._subscription.wait_for_assignment() |
| 433 | + else: |
| 434 | + # `assign` was called before `start`. We did not start |
| 435 | + # this task on that call, as coordinator was yet to be |
| 436 | + # created |
| 437 | + self._coordinator.start_commit_offsets_refresh_task( |
| 438 | + self._subscription.subscription.assignment |
| 439 | + ) |
| 440 | + else: |
| 441 | + # Using a simple assignment coordinator for reassignment on |
| 442 | + # metadata changes |
| 443 | + self._coordinator = NoGroupCoordinator( |
| 444 | + self._client, |
| 445 | + self._subscription, |
| 446 | + exclude_internal_topics=self._exclude_internal_topics, |
| 447 | + ) |
| 448 | + self._exit_stack.push_async_callback(self._coordinator.close) |
| 449 | + |
| 450 | + if ( |
| 451 | + self._subscription.subscription is not None |
| 452 | + and self._subscription.partitions_auto_assigned() |
| 453 | + ): |
421 | 454 | # Either we passed `topics` to constructor or `subscribe` |
422 | 455 | # was called before `start` |
423 | | - await self._subscription.wait_for_assignment() |
424 | | - else: |
425 | | - # `assign` was called before `start`. We did not start |
426 | | - # this task on that call, as coordinator was yet to be |
427 | | - # created |
428 | | - self._coordinator.start_commit_offsets_refresh_task( |
429 | | - self._subscription.subscription.assignment |
430 | | - ) |
431 | | - else: |
432 | | - # Using a simple assignment coordinator for reassignment on |
433 | | - # metadata changes |
434 | | - self._coordinator = NoGroupCoordinator( |
435 | | - self._client, |
436 | | - self._subscription, |
437 | | - exclude_internal_topics=self._exclude_internal_topics, |
438 | | - ) |
| 456 | + await self._client.force_metadata_update() |
| 457 | + self._coordinator.assign_all_partitions(check_unknown=True) |
439 | 458 |
|
440 | | - if ( |
441 | | - self._subscription.subscription is not None |
442 | | - and self._subscription.partitions_auto_assigned() |
443 | | - ): |
444 | | - # Either we passed `topics` to constructor or `subscribe` |
445 | | - # was called before `start` |
446 | | - await self._client.force_metadata_update() |
447 | | - self._coordinator.assign_all_partitions(check_unknown=True) |
| 459 | + self._closed = False |
| 460 | + except: |
| 461 | + await self._exit_stack.aclose() |
| 462 | + raise |
448 | 463 |
|
449 | 464 | async def _wait_topics(self): |
450 | 465 | if self._subscription.subscription is not None: |
@@ -514,11 +529,7 @@ async def stop(self): |
514 | 529 | return |
515 | 530 | log.debug("Closing the KafkaConsumer.") |
516 | 531 | self._closed = True |
517 | | - if self._coordinator: |
518 | | - await self._coordinator.close() |
519 | | - if self._fetcher: |
520 | | - await self._fetcher.close() |
521 | | - await self._client.close() |
| 532 | + await self._exit_stack.aclose() |
522 | 533 | log.debug("The KafkaConsumer has closed.") |
523 | 534 |
|
524 | 535 | async def commit(self, offsets=None): |
|
0 commit comments