Skip to content

Commit

Permalink
dt: Added tests to verify SASL/PLAIN config
Browse files Browse the repository at this point in the history
Tested that a log message appears when SASL/PLAIN is enabled and that
PLAIN cannot be the only SASL mechanism.

Signed-off-by: Michael Boquard <michael@redpanda.com>
  • Loading branch information
michael-redpanda committed Dec 11, 2024
1 parent cd9f608 commit 7771b04
Showing 1 changed file with 104 additions and 1 deletion.
105 changes: 104 additions & 1 deletion tests/rptest/tests/scram_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -7,20 +7,24 @@
# the Business Source License, use of this software will be governed
# by the Apache License, Version 2.0
from enum import IntEnum
import json
import random
import socket
import string
import requests
from requests.exceptions import HTTPError
import socket
import time
import urllib.parse
import re

from confluent_kafka import KafkaException, KafkaError

from ducktape.cluster.cluster import ClusterNode
from ducktape.mark import parametrize, matrix
from ducktape.utils.util import wait_until
from ducktape.errors import TimeoutError
from ducktape.services.service import Service

from rptest.services.cluster import cluster
from rptest.tests.redpanda_test import RedpandaTest
Expand All @@ -30,7 +34,8 @@
from rptest.clients.rpk import RpkTool, RpkException
from rptest.clients.python_librdkafka import PythonLibrdkafka
from rptest.services.admin import Admin
from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig
from rptest.services.redpanda import SecurityConfig, SaslCredentials, SecurityConfig, TLSProvider
from rptest.services.tls import Certificate, CertificateAuthority, TLSCertManager
from rptest.tests.sasl_reauth_test import get_sasl_metrics, REAUTH_METRIC, EXPIRATION_METRIC
from rptest.util import expect_http_error
from rptest.utils.utf8 import CONTROL_CHARS, CONTROL_CHARS_MAP, generate_string_with_control_character
Expand Down Expand Up @@ -465,6 +470,104 @@ def test_plain_authn(self, client_type, scram_type, sasl_plain_enabled):
self._make_topic(client, sasl_plain_enabled)


class SaslPlainTLSProvider(TLSProvider):
def __init__(self, tls: TLSCertManager):
self._tls = tls

@property
def ca(self) -> CertificateAuthority:
return self._tls.ca

def create_broker_cert(self, service: Service,
node: ClusterNode) -> Certificate:
assert node in service.nodes
return self._tls.create_cert(node.name)

def create_service_client_cert(self, _: Service, name: str) -> Certificate:
return self._tls.create_cert(socket.gethostname(), name=name)


class SaslPlainConfigTest(BaseScramTest):
"""
These tests verify the behavior of Redpanda in different
configurations with SASL/PLAIN enabled
"""

LICENSE_CHECK_INTERVAL_SEC = 1

def __init__(self, test_context):
self.security = SecurityConfig()
self.security.enable_sasl = True
super(SaslPlainConfigTest, self).__init__(test_context,
num_brokers=3,
security=self.security)
self.redpanda.set_environment({
'__REDPANDA_LICENSE_CHECK_INTERVAL_SEC':
f'{self.LICENSE_CHECK_INTERVAL_SEC}'
})
self.tls = TLSCertManager(self.logger)

def setUp(self):
pass

def _start_cluster(self, enable_tls: bool):
if enable_tls:
self.security.tls_provider = SaslPlainTLSProvider(tls=self.tls)
self.redpanda.set_security_settings(self.security)
super().setUp()

@cluster(num_nodes=3)
def test_cannot_enable_only_plain(self):
"""
This test verifies that a user cannot select PLAIN as the only
sasl_mechanism
"""
self._start_cluster(enable_tls=False)
admin = Admin(self.redpanda)
try:
admin.patch_cluster_config(upsert={'sasl_mechanisms': ['PLAIN']})
assert False, "Should not be able to enable only PLAIN"
except HTTPError as e:
assert e.response.status_code == 400, f"Expected 400, got {e.response.status_code}"
response = json.loads(e.response.text)
assert 'sasl_mechanisms' in response, f'Response missing "sasl_mechanisms": {response}'
assert "When PLAIN is enabled, at least one other mechanism must be enabled" == response[
'sasl_mechanisms'], f"Invalid message in response: {response['sasl_mechanisms']}"

def _license_nag_is_set(self):
return self.redpanda.search_log_all(
f"Overriding default license log annoy interval to: {self.LICENSE_CHECK_INTERVAL_SEC}s"
)

@cluster(num_nodes=3, log_allow_list=[re.compile('SASL/PLAIN is enabled')])
@parametrize(enable_tls=True)
@parametrize(enable_tls=False)
def test_sasl_plain_log(self, enable_tls: bool):
"""
This test verifies that a log message is emitted when SASL/PLAIN is enabled
"""
self._start_cluster(enable_tls=enable_tls)
wait_until(self._license_nag_is_set,
timeout_sec=30,
err_msg="Failed to set license nag interval")
admin = Admin(self.redpanda)
admin.patch_cluster_config(
upsert={'sasl_mechanisms': ['SCRAM', 'PLAIN']})

self.logger.debug("Waiting for SASL/PLAIN message")

def has_sasl_plain_log():
# There is always at least one Kafka API with TLS disabled meaning
# this will always log at the error level
return self.redpanda.search_log_all(
r"^ERROR.*SASL/PLAIN is enabled\. This is insecure and not recommended for production\.$"
)

wait_until(has_sasl_plain_log,
timeout_sec=self.LICENSE_CHECK_INTERVAL_SEC * 2,
err_msg="Failed to find SASL/PLAIN log message")


class ScramLiveUpdateTest(RedpandaTest):
def __init__(self, test_context):
super(ScramLiveUpdateTest, self).__init__(test_context, num_brokers=1)
Expand Down

0 comments on commit 7771b04

Please sign in to comment.