Skip to content

Do not block writing to wake socket #1767

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
35 changes: 23 additions & 12 deletions kafka/client_async.py
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,6 @@ class KafkaClient(object):
'bootstrap_topics_filter': set(),
'client_id': 'kafka-python-' + __version__,
'request_timeout_ms': 30000,
'wakeup_timeout_ms': 3000,
'connections_max_idle_ms': 9 * 60 * 1000,
'reconnect_backoff_ms': 50,
'reconnect_backoff_max_ms': 1000,
Expand Down Expand Up @@ -210,10 +209,11 @@ def __init__(self, **configs):
self._refresh_on_disconnects = True
self._last_bootstrap = 0
self._bootstrap_fails = 0
self._wake_r, self._wake_w = socket.socketpair()
self._wake_r.setblocking(False)
self._wake_w.settimeout(self.config['wakeup_timeout_ms'] / 1000.0)
self._wake_r = None
self._wake_w = None
self._need_wakeup = False
self._wake_lock = threading.Lock()
self._init_wakeup_socketpair()

self._lock = threading.RLock()

Expand All @@ -222,7 +222,6 @@ def __init__(self, **configs):
# lock above.
self._pending_completion = collections.deque()

self._selector.register(self._wake_r, selectors.EVENT_READ)
self._idle_expiry_manager = IdleConnectionManager(self.config['connections_max_idle_ms'])
self._closed = False
self._sensors = None
Expand All @@ -238,6 +237,18 @@ def __init__(self, **configs):
check_timeout = self.config['api_version_auto_timeout_ms'] / 1000
self.config['api_version'] = self.check_version(timeout=check_timeout)

def _init_wakeup_socketpair(self):
self._wake_r, self._wake_w = socket.socketpair()
self._wake_r.setblocking(False)
self._wake_w.setblocking(False)
log.debug("Wakeup socketpair (send): %s, bufsize: %s",
self._wake_w,
self._wake_w.getsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF))
log.debug("Wakeup socketpair (recv): %s, bufsize: %s",
self._wake_r,
self._wake_r.getsockopt(socket.SOL_SOCKET, socket.SO_SNDBUF))
self._selector.register(self._wake_r, selectors.EVENT_READ)

def _can_bootstrap(self):
effective_failures = self._bootstrap_fails // self._num_bootstrap_hosts
backoff_factor = 2 ** effective_failures
Expand Down Expand Up @@ -881,20 +892,20 @@ def check_version(self, node_id=None, timeout=2, strict=False):
def wakeup(self):
with self._wake_lock:
try:
self._wake_w.sendall(b'x')
except socket.timeout:
log.warning('Timeout to send to wakeup socket!')
raise Errors.KafkaTimeoutError()
except socket.error:
log.warning('Unable to send to wakeup socket!')
if not self._need_wakeup:
self._wake_w.send(b'x')
self._need_wakeup = True
except socket.error as e:
log.warning('Unable to send to wakeup socket! (%s)', e)

def _clear_wake_fd(self):
# reading from wake socket should only happen in a single thread
while True:
try:
self._wake_r.recv(1024)
self._wake_r.recv(4096)
except socket.error:
break
self._need_wakeup = False

def _maybe_close_oldest_connection(self):
expired_connection = self._idle_expiry_manager.poll_expired_connection()
Expand Down
1 change: 0 additions & 1 deletion kafka/producer/kafka.py
Original file line number Diff line number Diff line change
Expand Up @@ -377,7 +377,6 @@ def __init__(self, **configs):
self._metrics = Metrics(metric_config, reporters)

client = KafkaClient(metrics=self._metrics, metric_group_prefix='producer',
wakeup_timeout_ms=self.config['max_block_ms'],
**self.config)

# Get auto-discovered version from client if necessary
Expand Down