Skip to content

Commit 709ee3b

Browse files
committed
Support for PLAIN sasl authentication (PR #779)
Merge squashed branch 'larsjsol-sasl_plain'
2 parents c693709 + 787e8b2 commit 709ee3b

File tree

6 files changed

+179
-5
lines changed

6 files changed

+179
-5
lines changed

kafka/client_async.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -70,6 +70,9 @@ class KafkaClient(object):
7070
'selector': selectors.DefaultSelector,
7171
'metrics': None,
7272
'metric_group_prefix': '',
73+
'sasl_mechanism': None,
74+
'sasl_plain_username': None,
75+
'sasl_plain_password': None,
7376
}
7477
API_VERSIONS = [
7578
(0, 10),
@@ -150,6 +153,13 @@ def __init__(self, **configs):
150153
metrics (kafka.metrics.Metrics): Optionally provide a metrics
151154
instance for capturing network IO stats. Default: None.
152155
metric_group_prefix (str): Prefix for metric names. Default: ''
156+
sasl_mechanism (str): string picking sasl mechanism when security_protocol
157+
is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
158+
Default: None
159+
sasl_plain_username (str): username for sasl PLAIN authentication.
160+
Default: None
161+
sasl_plain_password (str): passowrd for sasl PLAIN authentication.
162+
Defualt: None
153163
"""
154164
self.config = copy.copy(self.DEFAULT_CONFIG)
155165
for key in self.config:

kafka/conn.py

Lines changed: 106 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,7 @@
1515
import kafka.errors as Errors
1616
from kafka.future import Future
1717
from kafka.protocol.api import RequestHeader
18+
from kafka.protocol.admin import SaslHandShakeRequest, SaslHandShakeResponse
1819
from kafka.protocol.commit import GroupCoordinatorResponse
1920
from kafka.protocol.types import Int32
2021
from kafka.version import __version__
@@ -48,7 +49,7 @@ class ConnectionStates(object):
4849
CONNECTING = '<connecting>'
4950
HANDSHAKE = '<handshake>'
5051
CONNECTED = '<connected>'
51-
52+
AUTHENTICATING = '<authenticating>'
5253

5354
InFlightRequest = collections.namedtuple('InFlightRequest',
5455
['request', 'response_type', 'correlation_id', 'future', 'timestamp'])
@@ -73,7 +74,11 @@ class BrokerConnection(object):
7374
'ssl_password': None,
7475
'api_version': (0, 8, 2), # default to most restrictive
7576
'state_change_callback': lambda conn: True,
77+
'sasl_mechanism': 'PLAIN',
78+
'sasl_plain_username': None,
79+
'sasl_plain_password': None
7680
}
81+
SASL_MECHANISMS = ('PLAIN',)
7782

7883
def __init__(self, host, port, afi, **configs):
7984
self.host = host
@@ -96,11 +101,19 @@ def __init__(self, host, port, afi, **configs):
96101
(socket.SOL_SOCKET, socket.SO_SNDBUF,
97102
self.config['send_buffer_bytes']))
98103

104+
if self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL'):
105+
assert self.config['sasl_mechanism'] in self.SASL_MECHANISMS, (
106+
'sasl_mechanism must be in ' + self.SASL_MECHANISMS)
107+
if self.config['sasl_mechanism'] == 'PLAIN':
108+
assert self.config['sasl_plain_username'] is not None, 'sasl_plain_username required for PLAIN sasl'
109+
assert self.config['sasl_plain_password'] is not None, 'sasl_plain_password required for PLAIN sasl'
110+
99111
self.state = ConnectionStates.DISCONNECTED
100112
self._sock = None
101113
self._ssl_context = None
102114
if self.config['ssl_context'] is not None:
103115
self._ssl_context = self.config['ssl_context']
116+
self._sasl_auth_future = None
104117
self._rbuffer = io.BytesIO()
105118
self._receiving = False
106119
self._next_payload_bytes = 0
@@ -188,6 +201,8 @@ def connect(self):
188201
if self.config['security_protocol'] in ('SSL', 'SASL_SSL'):
189202
log.debug('%s: initiating SSL handshake', str(self))
190203
self.state = ConnectionStates.HANDSHAKE
204+
elif self.config['security_protocol'] == 'SASL_PLAINTEXT':
205+
self.state = ConnectionStates.AUTHENTICATING
191206
else:
192207
self.state = ConnectionStates.CONNECTED
193208
self.config['state_change_callback'](self)
@@ -211,6 +226,16 @@ def connect(self):
211226
if self.state is ConnectionStates.HANDSHAKE:
212227
if self._try_handshake():
213228
log.debug('%s: completed SSL handshake.', str(self))
229+
if self.config['security_protocol'] == 'SASL_SSL':
230+
self.state = ConnectionStates.AUTHENTICATING
231+
else:
232+
self.state = ConnectionStates.CONNECTED
233+
self.config['state_change_callback'](self)
234+
235+
if self.state is ConnectionStates.AUTHENTICATING:
236+
assert self.config['security_protocol'] in ('SASL_PLAINTEXT', 'SASL_SSL')
237+
if self._try_authenticate():
238+
log.info('%s: Authenticated as %s', str(self), self.config['sasl_plain_username'])
214239
self.state = ConnectionStates.CONNECTED
215240
self.config['state_change_callback'](self)
216241

@@ -273,6 +298,75 @@ def _try_handshake(self):
273298

274299
return False
275300

301+
def _try_authenticate(self):
302+
assert self.config['api_version'] is None or self.config['api_version'] >= (0, 10)
303+
304+
if self._sasl_auth_future is None:
305+
# Build a SaslHandShakeRequest message
306+
request = SaslHandShakeRequest[0](self.config['sasl_mechanism'])
307+
future = Future()
308+
sasl_response = self._send(request)
309+
sasl_response.add_callback(self._handle_sasl_handshake_response, future)
310+
sasl_response.add_errback(lambda f, e: f.failure(e), future)
311+
self._sasl_auth_future = future
312+
self._recv()
313+
if self._sasl_auth_future.failed():
314+
raise self._sasl_auth_future.exception # pylint: disable-msg=raising-bad-type
315+
return self._sasl_auth_future.succeeded()
316+
317+
def _handle_sasl_handshake_response(self, future, response):
318+
error_type = Errors.for_code(response.error_code)
319+
if error_type is not Errors.NoError:
320+
error = error_type(self)
321+
self.close(error=error)
322+
return future.failure(error_type(self))
323+
324+
if self.config['sasl_mechanism'] == 'PLAIN':
325+
return self._try_authenticate_plain(future)
326+
else:
327+
return future.failure(
328+
Errors.UnsupportedSaslMechanismError(
329+
'kafka-python does not support SASL mechanism %s' %
330+
self.config['sasl_mechanism']))
331+
332+
def _try_authenticate_plain(self, future):
333+
if self.config['security_protocol'] == 'SASL_PLAINTEXT':
334+
log.warning('%s: Sending username and password in the clear', str(self))
335+
336+
data = b''
337+
try:
338+
self._sock.setblocking(True)
339+
# Send PLAIN credentials per RFC-4616
340+
msg = bytes('\0'.join([self.config['sasl_plain_username'],
341+
self.config['sasl_plain_username'],
342+
self.config['sasl_plain_password']]).encode('utf-8'))
343+
size = Int32.encode(len(msg))
344+
self._sock.sendall(size + msg)
345+
346+
# The server will send a zero sized message (that is Int32(0)) on success.
347+
# The connection is closed on failure
348+
while len(data) < 4:
349+
fragment = self._sock.recv(4 - len(data))
350+
if not fragment:
351+
log.error('%s: Authentication failed for user %s', self, self.config['sasl_plain_username'])
352+
error = Errors.AuthenticationFailedError(
353+
'Authentication failed for user {0}'.format(
354+
self.config['sasl_plain_username']))
355+
future.failure(error)
356+
raise error
357+
data += fragment
358+
self._sock.setblocking(False)
359+
except (AssertionError, ConnectionError) as e:
360+
log.exception("%s: Error receiving reply from server", self)
361+
error = Errors.ConnectionError("%s: %s" % (str(self), e))
362+
future.failure(error)
363+
self.close(error=error)
364+
365+
if data != b'\x00\x00\x00\x00':
366+
return future.failure(Errors.AuthenticationFailedError())
367+
368+
return future.success(True)
369+
276370
def blacked_out(self):
277371
"""
278372
Return true if we are disconnected from the given node and can't
@@ -292,7 +386,8 @@ def connecting(self):
292386
"""Returns True if still connecting (this may encompass several
293387
different states, such as SSL handshake, authorization, etc)."""
294388
return self.state in (ConnectionStates.CONNECTING,
295-
ConnectionStates.HANDSHAKE)
389+
ConnectionStates.HANDSHAKE,
390+
ConnectionStates.AUTHENTICATING)
296391

297392
def disconnected(self):
298393
"""Return True iff socket is closed"""
@@ -337,6 +432,10 @@ def send(self, request, expect_response=True):
337432
return future.failure(Errors.ConnectionError(str(self)))
338433
elif not self.can_send_more():
339434
return future.failure(Errors.TooManyInFlightRequests(str(self)))
435+
return self._send(request, expect_response=expect_response)
436+
437+
def _send(self, request, expect_response=True):
438+
future = Future()
340439
correlation_id = self._next_correlation_id()
341440
header = RequestHeader(request,
342441
correlation_id=correlation_id,
@@ -385,7 +484,7 @@ def recv(self):
385484
Return response if available
386485
"""
387486
assert not self._processing, 'Recursion not supported'
388-
if not self.connected():
487+
if not self.connected() and not self.state is ConnectionStates.AUTHENTICATING:
389488
log.warning('%s cannot recv: socket not connected', self)
390489
# If requests are pending, we should close the socket and
391490
# fail all the pending request futures
@@ -405,6 +504,9 @@ def recv(self):
405504
self.config['request_timeout_ms']))
406505
return None
407506

507+
return self._recv()
508+
509+
def _recv(self):
408510
# Not receiving is the state of reading the payload header
409511
if not self._receiving:
410512
try:
@@ -452,7 +554,7 @@ def recv(self):
452554
# enough data to read the full bytes_to_read
453555
# but if the socket is disconnected, we will get empty data
454556
# without an exception raised
455-
if not data:
557+
if bytes_to_read and not data:
456558
log.error('%s: socket disconnected', self)
457559
self.close(error=Errors.ConnectionError('socket disconnected'))
458560
return None

kafka/consumer/group.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -186,6 +186,13 @@ class KafkaConsumer(six.Iterator):
186186
(such as offsets) should be exposed to the consumer. If set to True
187187
the only way to receive records from an internal topic is
188188
subscribing to it. Requires 0.10+ Default: True
189+
sasl_mechanism (str): string picking sasl mechanism when security_protocol
190+
is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
191+
Default: None
192+
sasl_plain_username (str): username for sasl PLAIN authentication.
193+
Default: None
194+
sasl_plain_password (str): passowrd for sasl PLAIN authentication.
195+
Defualt: None
189196
190197
Note:
191198
Configuration parameters are described in more detail at
@@ -234,6 +241,9 @@ class KafkaConsumer(six.Iterator):
234241
'metrics_sample_window_ms': 30000,
235242
'selector': selectors.DefaultSelector,
236243
'exclude_internal_topics': True,
244+
'sasl_mechanism': None,
245+
'sasl_plain_username': None,
246+
'sasl_plain_password': None,
237247
}
238248

239249
def __init__(self, *topics, **configs):

kafka/errors.py

Lines changed: 20 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,14 @@ class CommitFailedError(KafkaError):
5858
pass
5959

6060

61+
class AuthenticationMethodNotSupported(KafkaError):
62+
pass
63+
64+
65+
class AuthenticationFailedError(KafkaError):
66+
retriable = False
67+
68+
6169
class BrokerResponseError(KafkaError):
6270
errno = None
6371
message = None
@@ -328,6 +336,18 @@ class InvalidTimestampError(BrokerResponseError):
328336
description = ('The timestamp of the message is out of acceptable range.')
329337

330338

339+
class UnsupportedSaslMechanismError(BrokerResponseError):
340+
errno = 33
341+
message = 'UNSUPPORTED_SASL_MECHANISM'
342+
description = ('The broker does not support the requested SASL mechanism.')
343+
344+
345+
class IllegalSaslStateError(BrokerResponseError):
346+
errno = 34
347+
message = 'ILLEGAL_SASL_STATE'
348+
description = ('Request is not valid given the current SASL state.')
349+
350+
331351
class KafkaUnavailableError(KafkaError):
332352
pass
333353

kafka/producer/kafka.py

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -199,7 +199,8 @@ class KafkaProducer(object):
199199
to kafka brokers up to this number of maximum requests per
200200
broker connection. Default: 5.
201201
security_protocol (str): Protocol used to communicate with brokers.
202-
Valid values are: PLAINTEXT, SSL. Default: PLAINTEXT.
202+
Valid values are: PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL.
203+
Default: PLAINTEXT.
203204
ssl_context (ssl.SSLContext): pre-configured SSLContext for wrapping
204205
socket connections. If provided, all other ssl_* configurations
205206
will be ignored. Default: None.
@@ -235,6 +236,13 @@ class KafkaProducer(object):
235236
selector (selectors.BaseSelector): Provide a specific selector
236237
implementation to use for I/O multiplexing.
237238
Default: selectors.DefaultSelector
239+
sasl_mechanism (str): string picking sasl mechanism when security_protocol
240+
is SASL_PLAINTEXT or SASL_SSL. Currently only PLAIN is supported.
241+
Default: None
242+
sasl_plain_username (str): username for sasl PLAIN authentication.
243+
Default: None
244+
sasl_plain_password (str): passowrd for sasl PLAIN authentication.
245+
Defualt: None
238246
239247
Note:
240248
Configuration parameters are described in more detail at
@@ -276,6 +284,9 @@ class KafkaProducer(object):
276284
'metrics_num_samples': 2,
277285
'metrics_sample_window_ms': 30000,
278286
'selector': selectors.DefaultSelector,
287+
'sasl_mechanism': None,
288+
'sasl_plain_username': None,
289+
'sasl_plain_password': None,
279290
}
280291

281292
def __init__(self, **configs):

kafka/protocol/admin.py

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -78,3 +78,24 @@ class DescribeGroupsRequest_v0(Struct):
7878

7979
DescribeGroupsRequest = [DescribeGroupsRequest_v0]
8080
DescribeGroupsResponse = [DescribeGroupsResponse_v0]
81+
82+
83+
class SaslHandShakeResponse_v0(Struct):
84+
API_KEY = 17
85+
API_VERSION = 0
86+
SCHEMA = Schema(
87+
('error_code', Int16),
88+
('enabled_mechanisms', Array(String('utf-8')))
89+
)
90+
91+
92+
class SaslHandShakeRequest_v0(Struct):
93+
API_KEY = 17
94+
API_VERSION = 0
95+
RESPONSE_TYPE = SaslHandShakeResponse_v0
96+
SCHEMA = Schema(
97+
('mechanism', String('utf-8'))
98+
)
99+
100+
SaslHandShakeRequest = [SaslHandShakeRequest_v0]
101+
SaslHandShakeResponse = [SaslHandShakeResponse_v0]

0 commit comments

Comments
 (0)