forked from dpkp/kafka-python
-
Notifications
You must be signed in to change notification settings - Fork 8
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
There is some interest in supporting various SASL mechanisms not currently included in the library: * dpkp#2110 (DMS) * dpkp#2204 (SSPI) * dpkp#2232 (AWS_MSK_IAM) Adding these mechanisms in the core library may be undesirable due to: * Increased maintenance burden. * Unavailable testing environments. * Vendor specificity. This commit provides a quick prototype for a pluggable SASL system. --- **Example** To define a custom SASL mechanism a module must implement two methods: ```py def validate_config(conn): # Check configuration values, available libraries, etc. assert conn.config['vendor_specific_setting'] is not None, ( 'vendor_specific_setting required when sasl_mechanism=MY_SASL' ) def try_authenticate(conn, future): # Do authentication routine and return resolved Future with failed # or succeeded state. ``` And then the custom mechanism should be registered before initializing a KafkaAdminClient, KafkaConsumer, or KafkaProducer: ```py import kafka.sasl from kafka import KafkaProducer import my_sasl kafka.sasl.register_mechanism('MY_SASL', my_sasl) producer = KafkaProducer(sasl_mechanism='MY_SASL') ``` --- **Notes** **ABCs** This prototype does not implement an ABC for custom SASL mechanisms. Using an ABC would reduce a few of the explicit assertions involved with registering a mechanism and is a viable option. Due to differing feature sets between py2/py3 this option was not explored, but shouldn't be difficult. **Private Methods** This prototype relies on some methods that are currently marked as **private** in `BrokerConnection`. * `._can_send_recv` * `._lock` * `._recv_bytes_blocking` * `._send_bytes_blocking` A pluggable system would require stable interfaces for these actions. **Alternative Approach** If the module-scoped dict modification in `register_mechanism` feels too clunky maybe the addtional mechanisms can be specified via an argument when initializing one of the `Kafka*` classes?
- Loading branch information
1 parent
f0a57a6
commit 03c357f
Showing
6 changed files
with
378 additions
and
255 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,53 @@ | ||
import logging | ||
|
||
from kafka.sasl import gssapi, oauthbearer, plain, scram | ||
|
||
log = logging.getLogger(__name__) | ||
|
||
MECHANISMS = { | ||
'GSSAPI': gssapi, | ||
'OAUTHBEARER': oauthbearer, | ||
'PLAIN': plain, | ||
'SCRAM-SHA-256': scram, | ||
'SCRAM-SHA-512': scram, | ||
} | ||
|
||
|
||
def register_mechanism(key, module): | ||
""" | ||
Registers a custom SASL mechanism that can be used via sasl_mechanism={key}. | ||
Example: | ||
import kakfa.sasl | ||
from kafka import KafkaProducer | ||
from mymodule import custom_sasl | ||
kafka.sasl.register_mechanism('CUSTOM_SASL', custom_sasl) | ||
producer = KafkaProducer(sasl_mechanism='CUSTOM_SASL') | ||
Arguments: | ||
key (str): The name of the mechanism returned by the broker and used | ||
in the sasl_mechanism config value. | ||
module (module): A module that implements the following methods... | ||
def validate_config(conn: BrokerConnection): -> None: | ||
# Raises an AssertionError for missing or invalid conifg values. | ||
def try_authenticate(conn: BrokerConncetion, future: -> Future): | ||
# Executes authentication routine and returns a resolved Future. | ||
Raises: | ||
AssertionError: The registered module does not define a required method. | ||
""" | ||
assert callable(getattr(module, 'validate_config', None)), ( | ||
'Custom SASL mechanism {} must implement method #validate_config()' | ||
.format(key) | ||
) | ||
assert callable(getattr(module, 'try_authenticate', None)), ( | ||
'Custom SASL mechanism {} must implement method #try_authenticate()' | ||
.format(key) | ||
) | ||
if key in MECHANISMS: | ||
log.warning('Overriding existing SASL mechanism {}'.format(key)) | ||
|
||
MECHANISMS[key] = module |
Oops, something went wrong.