Skip to content

Commit 252e0bd

Browse files
authored
Improve client networking backoff / retry (#2480)
* conn: no connection delays between dns entries; merge blacked_out w/ connection_delay * conn: next_ifr_request_timeout_ms() returns delay until next req timeout * Drop poll timeout reset when no in flight requests * Do not mark conn as sending if future immediately resolves (error) * client poll: do not set 100ms timeout for unfinished futures * Improve metadata refresh backoff/retry -- respect connection delays * conn: honor reconnect backoff in connection_delay when connecting * Log connection delay for not-ready nodes in producer sender loop * Increase default reconnect_backoff_max_ms to 30000 (30 secs)
1 parent 85eb8ad commit 252e0bd

File tree

9 files changed

+107
-66
lines changed

9 files changed

+107
-66
lines changed

kafka/admin/client.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ class KafkaAdminClient(object):
7272
reconnection attempts will continue periodically with this fixed
7373
rate. To avoid connection storms, a randomization factor of 0.2
7474
will be applied to the backoff resulting in a random range between
75-
20% below and 20% above the computed value. Default: 1000.
75+
20% below and 20% above the computed value. Default: 30000.
7676
request_timeout_ms (int): Client request timeout in milliseconds.
7777
Default: 30000.
7878
connections_max_idle_ms: Close idle connections after the number of
@@ -156,7 +156,7 @@ class KafkaAdminClient(object):
156156
'request_timeout_ms': 30000,
157157
'connections_max_idle_ms': 9 * 60 * 1000,
158158
'reconnect_backoff_ms': 50,
159-
'reconnect_backoff_max_ms': 1000,
159+
'reconnect_backoff_max_ms': 30000,
160160
'max_in_flight_requests_per_connection': 5,
161161
'receive_buffer_bytes': None,
162162
'send_buffer_bytes': None,

kafka/client_async.py

+38-21
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ class KafkaClient(object):
7575
reconnection attempts will continue periodically with this fixed
7676
rate. To avoid connection storms, a randomization factor of 0.2
7777
will be applied to the backoff resulting in a random range between
78-
20% below and 20% above the computed value. Default: 1000.
78+
20% below and 20% above the computed value. Default: 30000.
7979
request_timeout_ms (int): Client request timeout in milliseconds.
8080
Default: 30000.
8181
connections_max_idle_ms: Close idle connections after the number of
@@ -164,7 +164,7 @@ class KafkaClient(object):
164164
'wakeup_timeout_ms': 3000,
165165
'connections_max_idle_ms': 9 * 60 * 1000,
166166
'reconnect_backoff_ms': 50,
167-
'reconnect_backoff_max_ms': 1000,
167+
'reconnect_backoff_max_ms': 30000,
168168
'max_in_flight_requests_per_connection': 5,
169169
'receive_buffer_bytes': None,
170170
'send_buffer_bytes': None,
@@ -464,9 +464,8 @@ def is_disconnected(self, node_id):
464464
def connection_delay(self, node_id):
465465
"""
466466
Return the number of milliseconds to wait, based on the connection
467-
state, before attempting to send data. When disconnected, this respects
468-
the reconnect backoff time. When connecting, returns 0 to allow
469-
non-blocking connect to finish. When connected, returns a very large
467+
state, before attempting to send data. When connecting or disconnected,
468+
this respects the reconnect backoff time. When connected, returns a very large
470469
number to handle slow/stalled connections.
471470
472471
Arguments:
@@ -537,7 +536,8 @@ def send(self, node_id, request, wakeup=True):
537536
# we will need to call send_pending_requests()
538537
# to trigger network I/O
539538
future = conn.send(request, blocking=False)
540-
self._sending.add(conn)
539+
if not future.is_done:
540+
self._sending.add(conn)
541541

542542
# Wakeup signal is useful in case another thread is
543543
# blocked waiting for incoming network traffic while holding
@@ -563,9 +563,7 @@ def poll(self, timeout_ms=None, future=None):
563563
Returns:
564564
list: responses received (can be empty)
565565
"""
566-
if future is not None:
567-
timeout_ms = 100
568-
elif timeout_ms is None:
566+
if timeout_ms is None:
569567
timeout_ms = self.config['request_timeout_ms']
570568
elif not isinstance(timeout_ms, (int, float)):
571569
raise TypeError('Invalid type for timeout: %s' % type(timeout_ms))
@@ -577,26 +575,25 @@ def poll(self, timeout_ms=None, future=None):
577575
if self._closed:
578576
break
579577

578+
# Send a metadata request if needed (or initiate new connection)
579+
metadata_timeout_ms = self._maybe_refresh_metadata()
580+
580581
# Attempt to complete pending connections
581582
for node_id in list(self._connecting):
582583
self._maybe_connect(node_id)
583584

584-
# Send a metadata request if needed
585-
metadata_timeout_ms = self._maybe_refresh_metadata()
586-
587585
# If we got a future that is already done, don't block in _poll
588586
if future is not None and future.is_done:
589587
timeout = 0
590588
else:
591589
idle_connection_timeout_ms = self._idle_expiry_manager.next_check_ms()
590+
request_timeout_ms = self._next_ifr_request_timeout_ms()
591+
log.debug("Timeouts: user %f, metadata %f, idle connection %f, request %f", timeout_ms, metadata_timeout_ms, idle_connection_timeout_ms, request_timeout_ms)
592592
timeout = min(
593593
timeout_ms,
594594
metadata_timeout_ms,
595595
idle_connection_timeout_ms,
596-
self.config['request_timeout_ms'])
597-
# if there are no requests in flight, do not block longer than the retry backoff
598-
if self.in_flight_request_count() == 0:
599-
timeout = min(timeout, self.config['retry_backoff_ms'])
596+
request_timeout_ms)
600597
timeout = max(0, timeout) # avoid negative timeouts
601598

602599
self._poll(timeout / 1000)
@@ -615,6 +612,8 @@ def poll(self, timeout_ms=None, future=None):
615612
def _register_send_sockets(self):
616613
while self._sending:
617614
conn = self._sending.pop()
615+
if conn._sock is None:
616+
continue
618617
try:
619618
key = self._selector.get_key(conn._sock)
620619
events = key.events | selectors.EVENT_WRITE
@@ -772,6 +771,17 @@ def least_loaded_node(self):
772771

773772
return found
774773

774+
def least_loaded_node_refresh_ms(self):
775+
"""Return connection delay in milliseconds for next available node.
776+
777+
This method is used primarily for retry/backoff during metadata refresh
778+
during / after a cluster outage, in which there are no available nodes.
779+
780+
Returns:
781+
float: delay_ms
782+
"""
783+
return min([self.connection_delay(broker.nodeId) for broker in self.cluster.brokers()])
784+
775785
def set_topics(self, topics):
776786
"""Set specific topics to track for metadata.
777787
@@ -803,12 +813,18 @@ def add_topic(self, topic):
803813
self._topics.add(topic)
804814
return self.cluster.request_update()
805815

816+
def _next_ifr_request_timeout_ms(self):
817+
if self._conns:
818+
return min([conn.next_ifr_request_timeout_ms() for conn in six.itervalues(self._conns)])
819+
else:
820+
return float('inf')
821+
806822
# This method should be locked when running multi-threaded
807823
def _maybe_refresh_metadata(self, wakeup=False):
808824
"""Send a metadata request if needed.
809825
810826
Returns:
811-
int: milliseconds until next refresh
827+
float: milliseconds until next refresh
812828
"""
813829
ttl = self.cluster.ttl()
814830
wait_for_in_progress_ms = self.config['request_timeout_ms'] if self._metadata_refresh_in_progress else 0
@@ -822,8 +838,9 @@ def _maybe_refresh_metadata(self, wakeup=False):
822838
# least_loaded_node()
823839
node_id = self.least_loaded_node()
824840
if node_id is None:
825-
log.debug("Give up sending metadata request since no node is available");
826-
return self.config['reconnect_backoff_ms']
841+
next_connect_ms = self.least_loaded_node_refresh_ms()
842+
log.debug("Give up sending metadata request since no node is available. (reconnect delay %d ms)", next_connect_ms)
843+
return next_connect_ms
827844

828845
if self._can_send_request(node_id):
829846
topics = list(self._topics)
@@ -850,11 +867,11 @@ def refresh_done(val_or_error):
850867
# the client from unnecessarily connecting to additional nodes while a previous connection
851868
# attempt has not been completed.
852869
if self._connecting:
853-
return self.config['reconnect_backoff_ms']
870+
return float('inf')
854871

855872
if self.maybe_connect(node_id, wakeup=wakeup):
856873
log.debug("Initializing connection to node %s for metadata request", node_id)
857-
return self.config['reconnect_backoff_ms']
874+
return float('inf')
858875

859876
# connected but can't send more, OR connecting
860877
# In either case we just need to wait for a network event

kafka/conn.py

+22-14
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,7 @@ class BrokerConnection(object):
120120
reconnection attempts will continue periodically with this fixed
121121
rate. To avoid connection storms, a randomization factor of 0.2
122122
will be applied to the backoff resulting in a random range between
123-
20% below and 20% above the computed value. Default: 1000.
123+
20% below and 20% above the computed value. Default: 30000.
124124
request_timeout_ms (int): Client request timeout in milliseconds.
125125
Default: 30000.
126126
max_in_flight_requests_per_connection (int): Requests are pipelined
@@ -198,7 +198,7 @@ class BrokerConnection(object):
198198
'node_id': 0,
199199
'request_timeout_ms': 30000,
200200
'reconnect_backoff_ms': 50,
201-
'reconnect_backoff_max_ms': 1000,
201+
'reconnect_backoff_max_ms': 30000,
202202
'max_in_flight_requests_per_connection': 5,
203203
'receive_buffer_bytes': None,
204204
'send_buffer_bytes': None,
@@ -848,20 +848,22 @@ def blacked_out(self):
848848
re-establish a connection yet
849849
"""
850850
if self.state is ConnectionStates.DISCONNECTED:
851-
if time.time() < self.last_attempt + self._reconnect_backoff:
852-
return True
851+
return self.connection_delay() > 0
853852
return False
854853

855854
def connection_delay(self):
856855
"""
857856
Return the number of milliseconds to wait, based on the connection
858-
state, before attempting to send data. When disconnected, this respects
859-
the reconnect backoff time. When connecting or connected, returns a very
857+
state, before attempting to send data. When connecting or disconnected,
858+
this respects the reconnect backoff time. When connected, returns a very
860859
large number to handle slow/stalled connections.
861860
"""
862-
time_waited = time.time() - (self.last_attempt or 0)
863-
if self.state is ConnectionStates.DISCONNECTED:
864-
return max(self._reconnect_backoff - time_waited, 0) * 1000
861+
if self.disconnected() or self.connecting():
862+
if len(self._gai) > 0:
863+
return 0
864+
else:
865+
time_waited = time.time() - self.last_attempt
866+
return max(self._reconnect_backoff - time_waited, 0) * 1000
865867
else:
866868
# When connecting or connected, we should be able to delay
867869
# indefinitely since other events (connection or data acked) will
@@ -887,6 +889,9 @@ def _reset_reconnect_backoff(self):
887889
self._failures = 0
888890
self._reconnect_backoff = self.config['reconnect_backoff_ms'] / 1000.0
889891

892+
def _reconnect_jitter_pct(self):
893+
return uniform(0.8, 1.2)
894+
890895
def _update_reconnect_backoff(self):
891896
# Do not mark as failure if there are more dns entries available to try
892897
if len(self._gai) > 0:
@@ -895,7 +900,7 @@ def _update_reconnect_backoff(self):
895900
self._failures += 1
896901
self._reconnect_backoff = self.config['reconnect_backoff_ms'] * 2 ** (self._failures - 1)
897902
self._reconnect_backoff = min(self._reconnect_backoff, self.config['reconnect_backoff_max_ms'])
898-
self._reconnect_backoff *= uniform(0.8, 1.2)
903+
self._reconnect_backoff *= self._reconnect_jitter_pct()
899904
self._reconnect_backoff /= 1000.0
900905
log.debug('%s: reconnect backoff %s after %s failures', self, self._reconnect_backoff, self._failures)
901906

@@ -1136,15 +1141,18 @@ def _recv(self):
11361141
return ()
11371142

11381143
def requests_timed_out(self):
1144+
return self.next_ifr_request_timeout_ms() == 0
1145+
1146+
def next_ifr_request_timeout_ms(self):
11391147
with self._lock:
11401148
if self.in_flight_requests:
11411149
get_timestamp = lambda v: v[1]
11421150
oldest_at = min(map(get_timestamp,
11431151
self.in_flight_requests.values()))
1144-
timeout = self.config['request_timeout_ms'] / 1000.0
1145-
if time.time() >= oldest_at + timeout:
1146-
return True
1147-
return False
1152+
next_timeout = oldest_at + self.config['request_timeout_ms'] / 1000.0
1153+
return max(0, (next_timeout - time.time()) * 1000)
1154+
else:
1155+
return float('inf')
11481156

11491157
def _handle_api_version_response(self, response):
11501158
error_type = Errors.for_code(response.error_code)

kafka/consumer/group.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -98,7 +98,7 @@ class KafkaConsumer(six.Iterator):
9898
reconnection attempts will continue periodically with this fixed
9999
rate. To avoid connection storms, a randomization factor of 0.2
100100
will be applied to the backoff resulting in a random range between
101-
20% below and 20% above the computed value. Default: 1000.
101+
20% below and 20% above the computed value. Default: 30000.
102102
max_in_flight_requests_per_connection (int): Requests are pipelined
103103
to kafka brokers up to this number of maximum requests per
104104
broker connection. Default: 5.
@@ -263,7 +263,7 @@ class KafkaConsumer(six.Iterator):
263263
'request_timeout_ms': 305000, # chosen to be higher than the default of max_poll_interval_ms
264264
'retry_backoff_ms': 100,
265265
'reconnect_backoff_ms': 50,
266-
'reconnect_backoff_max_ms': 1000,
266+
'reconnect_backoff_max_ms': 30000,
267267
'max_in_flight_requests_per_connection': 5,
268268
'auto_offset_reset': 'latest',
269269
'enable_auto_commit': True,

kafka/producer/kafka.py

+2-2
Original file line numberDiff line numberDiff line change
@@ -216,7 +216,7 @@ class KafkaProducer(object):
216216
reconnection attempts will continue periodically with this fixed
217217
rate. To avoid connection storms, a randomization factor of 0.2
218218
will be applied to the backoff resulting in a random range between
219-
20% below and 20% above the computed value. Default: 1000.
219+
20% below and 20% above the computed value. Default: 30000.
220220
max_in_flight_requests_per_connection (int): Requests are pipelined
221221
to kafka brokers up to this number of maximum requests per
222222
broker connection. Note that if this setting is set to be greater
@@ -311,7 +311,7 @@ class KafkaProducer(object):
311311
'sock_chunk_bytes': 4096, # undocumented experimental option
312312
'sock_chunk_buffer_count': 1000, # undocumented experimental option
313313
'reconnect_backoff_ms': 50,
314-
'reconnect_backoff_max_ms': 1000,
314+
'reconnect_backoff_max_ms': 30000,
315315
'max_in_flight_requests_per_connection': 5,
316316
'security_protocol': 'PLAINTEXT',
317317
'ssl_context': None,

kafka/producer/sender.py

+5-5
Original file line numberDiff line numberDiff line change
@@ -103,14 +103,14 @@ def run_once(self):
103103
self._metadata.request_update()
104104

105105
# remove any nodes we aren't ready to send to
106-
not_ready_timeout = float('inf')
106+
not_ready_timeout_ms = float('inf')
107107
for node in list(ready_nodes):
108108
if not self._client.is_ready(node):
109-
log.debug('Node %s not ready; delaying produce of accumulated batch', node)
109+
node_delay_ms = self._client.connection_delay(node)
110+
log.debug('Node %s not ready; delaying produce of accumulated batch (%f ms)', node, node_delay_ms)
110111
self._client.maybe_connect(node, wakeup=False)
111112
ready_nodes.remove(node)
112-
not_ready_timeout = min(not_ready_timeout,
113-
self._client.connection_delay(node))
113+
not_ready_timeout_ms = min(not_ready_timeout_ms, node_delay_ms)
114114

115115
# create produce requests
116116
batches_by_node = self._accumulator.drain(
@@ -136,7 +136,7 @@ def run_once(self):
136136
# off). Note that this specifically does not include nodes with
137137
# sendable data that aren't ready to send since they would cause busy
138138
# looping.
139-
poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout)
139+
poll_timeout_ms = min(next_ready_check_delay * 1000, not_ready_timeout_ms)
140140
if ready_nodes:
141141
log.debug("Nodes with data ready to send: %s", ready_nodes) # trace
142142
log.debug("Created %d produce requests: %s", len(requests), requests) # trace

test/conftest.py

+2
Original file line numberDiff line numberDiff line change
@@ -137,7 +137,9 @@ def conn(mocker):
137137
MetadataResponse[0](
138138
[(0, 'foo', 12), (1, 'bar', 34)], # brokers
139139
[])) # topics
140+
conn.connection_delay.return_value = 0
140141
conn.blacked_out.return_value = False
142+
conn.next_ifr_request_timeout_ms.return_value = float('inf')
141143
def _set_conn_state(state):
142144
conn.state = state
143145
return state

0 commit comments

Comments
 (0)