Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit 87fb1bb

Browse files
committed
Improve/refactor bootstrap_connected
1 parent 7a7a890 commit 87fb1bb

File tree

4 files changed

+22
-14
lines changed

4 files changed

+22
-14
lines changed

kafka/client_async.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -951,6 +951,16 @@ def _maybe_close_oldest_connection(self):
951951
log.info('Closing idle connection %s, last active %d ms ago', conn_id, idle_ms)
952952
self.close(node_id=conn_id)
953953

954+
def bootstrap_connected(self):
955+
"""Return True if a bootstrap node is connected"""
956+
for node_id in self._conns:
957+
if not self.cluster.is_bootstrap(node_id):
958+
continue
959+
if self._conns[node_id].connected():
960+
return True
961+
else:
962+
return False
963+
954964

955965
# OrderedDict requires python2.7+
956966
try:

kafka/consumer/group.py

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -393,9 +393,7 @@ def __init__(self, *topics, **configs):
393393

394394
def bootstrap_connected(self):
395395
"""Return True if the bootstrap is connected."""
396-
if self._client._bootstrap_fails > 0:
397-
return False
398-
return True
396+
return self._client.bootstrap_connected()
399397

400398
def assign(self, partitions):
401399
"""Manually assign a list of TopicPartitions to this consumer.

kafka/producer/kafka.py

Lines changed: 8 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
from kafka.serializer import Serializer
2424
from kafka.structs import TopicPartition
2525

26+
2627
log = logging.getLogger(__name__)
2728
PRODUCER_CLIENT_ID_SEQUENCE = AtomicInteger()
2829

@@ -375,13 +376,13 @@ def __init__(self, **configs):
375376
reporters = [reporter() for reporter in self.config['metric_reporters']]
376377
self._metrics = Metrics(metric_config, reporters)
377378

378-
self._client = KafkaClient(metrics=self._metrics, metric_group_prefix='producer',
379-
wakeup_timeout_ms=self.config['max_block_ms'],
380-
**self.config)
379+
client = KafkaClient(metrics=self._metrics, metric_group_prefix='producer',
380+
wakeup_timeout_ms=self.config['max_block_ms'],
381+
**self.config)
381382

382383
# Get auto-discovered version from client if necessary
383384
if self.config['api_version'] is None:
384-
self.config['api_version'] = self._client.config['api_version']
385+
self.config['api_version'] = client.config['api_version']
385386

386387
if self.config['compression_type'] == 'lz4':
387388
assert self.config['api_version'] >= (0, 8, 2), 'LZ4 Requires >= Kafka 0.8.2 Brokers'
@@ -397,9 +398,9 @@ def __init__(self, **configs):
397398

398399
message_version = self._max_usable_produce_magic()
399400
self._accumulator = RecordAccumulator(message_version=message_version, metrics=self._metrics, **self.config)
400-
self._metadata = self._client.cluster
401+
self._metadata = client.cluster
401402
guarantee_message_order = bool(self.config['max_in_flight_requests_per_connection'] == 1)
402-
self._sender = Sender(self._client, self._metadata,
403+
self._sender = Sender(client, self._metadata,
403404
self._accumulator, self._metrics,
404405
guarantee_message_order=guarantee_message_order,
405406
**self.config)
@@ -413,20 +414,16 @@ def __init__(self, **configs):
413414

414415
def bootstrap_connected(self):
415416
"""Return True if the bootstrap is connected."""
416-
if self._client._bootstrap_fails > 0:
417-
return False
418-
return True
417+
return self._sender.bootstrap_connected()
419418

420419
def _cleanup_factory(self):
421420
"""Build a cleanup clojure that doesn't increase our ref count"""
422421
_self = weakref.proxy(self)
423-
424422
def wrapper():
425423
try:
426424
_self.close(timeout=0)
427425
except (ReferenceError, AttributeError):
428426
pass
429-
430427
return wrapper
431428

432429
def _unregister_cleanup(self):

kafka/producer/sender.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -315,6 +315,9 @@ def wakeup(self):
315315
"""Wake up the selector associated with this send thread."""
316316
self._client.wakeup()
317317

318+
def bootstrap_connected(self):
319+
return self._client.bootstrap_connected()
320+
318321

319322
class SenderMetrics(object):
320323

0 commit comments

Comments
 (0)