Skip to content

Fix decoding bug in AWS_MSK_IAM mechanism #2639

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 4 commits into from
Jun 4, 2025
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
17 changes: 14 additions & 3 deletions kafka/sasl/msk.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@
import hashlib
import hmac
import json
import logging
import string

# needed for AWS_MSK_IAM authentication:
Expand All @@ -13,10 +14,14 @@
# no botocore available, will disable AWS_MSK_IAM mechanism
BotoSession = None

from kafka.errors import KafkaConfigurationError
from kafka.sasl.abc import SaslMechanism
from kafka.vendor.six.moves import urllib


log = logging.getLogger(__name__)


class SaslMechanismAwsMskIam(SaslMechanism):
def __init__(self, **config):
assert BotoSession is not None, 'AWS_MSK_IAM requires the "botocore" package'
Expand All @@ -27,22 +32,28 @@ def __init__(self, **config):
self._is_done = False
self._is_authenticated = False

def auth_bytes(self):
def _build_client(self):
session = BotoSession()
credentials = session.get_credentials().get_frozen_credentials()
client = AwsMskIamClient(
if not session.get_config_variable('region'):
raise KafkaConfigurationError('Unable to determine region for AWS MSK cluster. Is AWS_DEFAULT_REGION set?')
return AwsMskIamClient(
host=self.host,
access_key=credentials.access_key,
secret_key=credentials.secret_key,
region=session.get_config_variable('region'),
token=credentials.token,
)

def auth_bytes(self):
client = self._build_client()
log.debug("Generating auth token for MSK scope: %s", client._scope)
return client.first_message()

def receive(self, auth_bytes):
self._is_done = True
self._is_authenticated = auth_bytes != b''
self._auth = auth_bytes.deode('utf-8')
self._auth = auth_bytes.decode('utf-8')

def is_done(self):
return self._is_done
Expand Down
16 changes: 15 additions & 1 deletion test/sasl/test_msk.py
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,7 @@
import json
import sys

from kafka.sasl.msk import AwsMskIamClient
from kafka.sasl.msk import AwsMskIamClient, SaslMechanismAwsMskIam

try:
from unittest import mock
Expand Down Expand Up @@ -69,3 +69,17 @@ def test_aws_msk_iam_client_temporary_credentials():
'x-amz-security-token': 'XXXXX',
}
assert actual == expected


def test_aws_msk_iam_sasl_mechanism():
with mock.patch('kafka.sasl.msk.BotoSession'):
sasl = SaslMechanismAwsMskIam(security_protocol='SASL_SSL', host='localhost')
with mock.patch.object(sasl, '_build_client', return_value=client_factory(token=None)):
assert sasl.auth_bytes() != b''
assert not sasl.is_done()
assert not sasl.is_authenticated()
sasl.receive(b'foo')
assert sasl._auth == 'foo'
assert sasl.is_done()
assert sasl.is_authenticated()
assert sasl.auth_details()
Loading