Skip to content

Read all available socket bytes #1332

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

Merged
merged 3 commits into from
Jan 11, 2018
Merged
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
2 changes: 2 additions & 0 deletions kafka/client_async.py
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,8 @@ class KafkaClient(object):
'receive_buffer_bytes': None,
'send_buffer_bytes': None,
'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
'sock_chunk_bytes': 4096, # undocumented experimental option
'sock_chunk_buffer_count': 1000, # undocumented experimental option
'retry_backoff_ms': 100,
'metadata_max_age_ms': 300000,
'security_protocol': 'PLAINTEXT',
Expand Down
44 changes: 22 additions & 22 deletions kafka/conn.py
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,8 @@ class BrokerConnection(object):
'receive_buffer_bytes': None,
'send_buffer_bytes': None,
'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
'sock_chunk_bytes': 4096, # undocumented experimental option
'sock_chunk_buffer_count': 1000, # undocumented experimental option
'security_protocol': 'PLAINTEXT',
'ssl_context': None,
'ssl_check_hostname': True,
Expand Down Expand Up @@ -748,19 +750,21 @@ def recv(self):
return responses

def _recv(self):
responses = []
SOCK_CHUNK_BYTES = 4096
while True:
"""Take all available bytes from socket, return list of any responses from parser"""
recvd = []
while len(recvd) < self.config['sock_chunk_buffer_count']:
try:
data = self._sock.recv(SOCK_CHUNK_BYTES)
# We expect socket.recv to raise an exception if there is not
# enough data to read the full bytes_to_read
data = self._sock.recv(self.config['sock_chunk_bytes'])
# We expect socket.recv to raise an exception if there are no
# bytes available to read from the socket in non-blocking mode.
# but if the socket is disconnected, we will get empty data
# without an exception raised
if not data:
log.error('%s: socket disconnected', self)
self.close(error=Errors.ConnectionError('socket disconnected'))
break
return []
else:
recvd.append(data)

except SSLWantReadError:
break
Expand All @@ -770,27 +774,23 @@ def _recv(self):
log.exception('%s: Error receiving network data'
' closing socket', self)
self.close(error=Errors.ConnectionError(e))
break
return []
except BlockingIOError:
if six.PY3:
break
raise

if self._sensors:
self._sensors.bytes_received.record(len(data))

try:
more_responses = self._protocol.receive_bytes(data)
except Errors.KafkaProtocolError as e:
self.close(e)
break
else:
responses.extend([resp for (_, resp) in more_responses])

if len(data) < SOCK_CHUNK_BYTES:
break
recvd_data = b''.join(recvd)
if self._sensors:
self._sensors.bytes_received.record(len(recvd_data))

return responses
try:
responses = self._protocol.receive_bytes(recvd_data)
except Errors.KafkaProtocolError as e:
self.close(e)
return []
else:
return [resp for (_, resp) in responses] # drop correlation id

def requests_timed_out(self):
if self.in_flight_requests:
Expand Down
2 changes: 2 additions & 0 deletions kafka/consumer/group.py
Original file line number Diff line number Diff line change
Expand Up @@ -270,6 +270,8 @@ class KafkaConsumer(six.Iterator):
'receive_buffer_bytes': None,
'send_buffer_bytes': None,
'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
'sock_chunk_bytes': 4096, # undocumented experimental option
'sock_chunk_buffer_count': 1000, # undocumented experimental option
'consumer_timeout_ms': float('inf'),
'skip_double_compressed_messages': False,
'security_protocol': 'PLAINTEXT',
Expand Down
2 changes: 2 additions & 0 deletions kafka/producer/kafka.py
Original file line number Diff line number Diff line change
Expand Up @@ -292,6 +292,8 @@ class KafkaProducer(object):
'receive_buffer_bytes': None,
'send_buffer_bytes': None,
'socket_options': [(socket.IPPROTO_TCP, socket.TCP_NODELAY, 1)],
'sock_chunk_bytes': 4096, # undocumented experimental option
'sock_chunk_buffer_count': 1000, # undocumented experimental option
'reconnect_backoff_ms': 50,
'reconnect_backoff_max': 1000,
'max_in_flight_requests_per_connection': 5,
Expand Down