Skip to content

Commit 8c1f396

Browse files
authored
Merge pull request #23245 from michael-redpanda/CORE-7245-Fix-segfault-mtls-audit
kafka: Fixed segfault issue with auditing and mTLS
2 parents dde149c + a5f56c3 commit 8c1f396

File tree

4 files changed

+138
-3
lines changed

4 files changed

+138
-3
lines changed

src/v/kafka/server/connection_context.cc

+1-1
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ ss::future<> connection_context::start() {
203203
}
204204

205205
ss::future<> connection_context::stop() {
206-
if (_hook) {
206+
if (_hook && is_linked()) {
207207
_hook.value().get().erase(_hook.value().get().iterator_to(*this));
208208
}
209209
if (conn) {

src/v/kafka/server/server.cc

+4-1
Original file line numberDiff line numberDiff line change
@@ -347,6 +347,10 @@ ss::future<> server::apply(ss::lw_shared_ptr<net::connection> conn) {
347347

348348
std::exception_ptr eptr;
349349
try {
350+
co_await ctx->start();
351+
// Must call start() to ensure `ctx` is inserted into the `_connections`
352+
// list. Otherwise if enqueing the audit message fails and `stop()` is
353+
// called, this will result in a segfault.
350354
if (authn_method == config::broker_authn_method::mtls_identity) {
351355
auto authn_event = make_auth_event_options(mtls_state.value(), ctx);
352356
if (!ctx->server().audit_mgr().enqueue_authn_event(
@@ -356,7 +360,6 @@ ss::future<> server::apply(ss::lw_shared_ptr<net::connection> conn) {
356360
"system error");
357361
}
358362
}
359-
co_await ctx->start();
360363
co_await ctx->process();
361364
} catch (...) {
362365
eptr = std::current_exception();

src/v/security/audit/audit_log_manager.cc

+3-1
Original file line numberDiff line numberDiff line change
@@ -297,7 +297,9 @@ ss::future<> audit_client::update_status(kafka::error_code errc) {
297297
}
298298
} else if (_last_errc == kafka::error_code::illegal_sasl_state) {
299299
/// The status changed from erraneous to anything else
300-
if (errc != kafka::error_code::illegal_sasl_state) {
300+
if (
301+
errc != kafka::error_code::illegal_sasl_state
302+
&& errc != kafka::error_code::broker_not_available) {
301303
co_await _sink->update_auth_status(
302304
audit_sink::auth_misconfigured_t::no);
303305
}

tests/rptest/tests/audit_log_test.py

+130
Original file line numberDiff line numberDiff line change
@@ -464,6 +464,12 @@ def change_max_buffer_size_per_shard(self, new_size: int):
464464
self._modify_cluster_config(
465465
{'audit_queue_max_buffer_size_per_shard': new_size})
466466

467+
def modify_audit_enabled(self, enabled: bool):
468+
"""
469+
Modifies value of audit_enabled
470+
"""
471+
self._modify_cluster_config({'audit_enabled': enabled})
472+
467473
def modify_node_config(self, node, update_fn, skip_readiness_check=True):
468474
"""Modifies the current node configuration, restarts the node for
469475
changes to take effect
@@ -1649,6 +1655,130 @@ def test_no_audit_user_authn(self):
16491655
pass
16501656

16511657

1658+
class AuditLogTestInvalidConfigBase(AuditLogTestBase):
1659+
username = 'test'
1660+
password = 'test12345'
1661+
algorithm = 'SCRAM-SHA-256'
1662+
"""
1663+
Tests situations where audit log client is not properly configured
1664+
"""
1665+
def __init__(self,
1666+
test_context,
1667+
audit_log_config=AuditLogConfig(enabled=False,
1668+
num_partitions=1,
1669+
event_types=[]),
1670+
log_config=LoggingConfig('info',
1671+
logger_levels={
1672+
'auditing': 'trace',
1673+
'kafka': 'trace',
1674+
'security': 'trace'
1675+
}),
1676+
**kwargs):
1677+
self.test_context = test_context
1678+
# The 'none' below will cause the audit log client to not be configured properly
1679+
self._audit_log_client_config = redpanda.AuditLogConfig(
1680+
listener_port=9192, listener_authn_method='none')
1681+
self._audit_log_client_config.require_client_auth = False
1682+
self._audit_log_client_config.enable_broker_tls = False
1683+
1684+
super(AuditLogTestInvalidConfigBase, self).__init__(
1685+
test_context=test_context,
1686+
audit_log_config=audit_log_config,
1687+
log_config=log_config,
1688+
audit_log_client_config=self._audit_log_client_config,
1689+
**kwargs)
1690+
1691+
def setUp(self):
1692+
super().setUp()
1693+
self.admin.create_user(self.username, self.password, self.algorithm)
1694+
self.get_super_rpk().acl_create_allow_cluster(self.username, 'All')
1695+
# Following is important so the rest of ducktape functions correctly
1696+
self.modify_audit_excluded_principals(['admin'])
1697+
self.modify_audit_event_types(['authenticate'])
1698+
self.modify_audit_enabled(True)
1699+
# Waits for all audit clients to enter the same state where any attempt
1700+
# to enqueue an event will be rejected because the client is misconfigured
1701+
wait_until(lambda: self.redpanda.search_log_all(
1702+
'error_code: illegal_sasl_state'),
1703+
timeout_sec=30,
1704+
backoff_sec=2,
1705+
err_msg="Did not see illegal_sasl_state error message")
1706+
1707+
1708+
class AuditLogTestInvalidConfig(AuditLogTestInvalidConfigBase):
1709+
def __init__(self, test_context):
1710+
super(AuditLogTestInvalidConfig, self).__init__(
1711+
test_context=test_context,
1712+
security=AuditLogTestSecurityConfig(user_creds=(self.username,
1713+
self.password,
1714+
self.algorithm)))
1715+
1716+
@ok_to_fail_fips
1717+
@cluster(num_nodes=4,
1718+
log_allow_list=[
1719+
r'Failed to append authentication event to audit log',
1720+
r'Failed to audit.*'
1721+
])
1722+
def test_invalid_config(self):
1723+
"""
1724+
Test validates that the topic is failed to get created if audit
1725+
system is not configured correctly.
1726+
"""
1727+
try:
1728+
self.get_rpk().create_topic('test')
1729+
assert False, "Should not have created a topic"
1730+
except RpkException as e:
1731+
assert "audit system failure: BROKER_NOT_AVAILABLE" in str(
1732+
e
1733+
), f'{str(e)} does not contain "audit system failure: BROKER_NOT_AVAILABLE"'
1734+
1735+
1736+
class AuditLogTestInvalidConfigMTLS(AuditLogTestInvalidConfigBase):
1737+
"""
1738+
Tests situations where audit log client is not properly configured and mTLS enabled
1739+
"""
1740+
def __init__(self, test_context):
1741+
self.test_context = test_context
1742+
self.tls = tls.TLSCertManager(self.logger)
1743+
self.user_cert = self.tls.create_cert(socket.gethostname(),
1744+
common_name=self.username,
1745+
name='base_client')
1746+
self.admin_user_cert = self.tls.create_cert(
1747+
socket.gethostname(),
1748+
common_name=RedpandaServiceBase.SUPERUSER_CREDENTIALS[0],
1749+
name='admin_client')
1750+
self._security_config = AuditLogTestSecurityConfig(
1751+
admin_cert=self.admin_user_cert, user_cert=self.user_cert)
1752+
self._security_config.tls_provider = MTLSProvider(self.tls)
1753+
self._security_config.principal_mapping_rules = 'RULE:.*CN=(.*).*/$1/'
1754+
1755+
super(AuditLogTestInvalidConfigMTLS,
1756+
self).__init__(test_context=test_context,
1757+
security=self._security_config)
1758+
1759+
@cluster(
1760+
num_nodes=4,
1761+
log_allow_list=[
1762+
r'Failed to append authentication event to audit log',
1763+
r'Failed to audit.*',
1764+
r'Failed to enqueue mTLS authentication event - audit log system error'
1765+
])
1766+
def test_invalid_config_mtls(self):
1767+
"""
1768+
Validates that mTLS authn is rejected when audit client is misconfigured.
1769+
Also ensures there is no segfault: https://redpandadata.atlassian.net/browse/CORE-7245
1770+
"""
1771+
try:
1772+
self.get_rpk().create_topic('test')
1773+
assert False, "Should not have created a topic"
1774+
except RpkException as e:
1775+
pass
1776+
1777+
assert self.redpanda.search_log_any(
1778+
'Failed to enqueue mTLS authentication event - audit log system error'
1779+
)
1780+
1781+
16521782
class AuditLogTestKafkaTlsApi(AuditLogTestBase):
16531783
"""
16541784
Tests that validate audit log messages for users authenticated via mTLS

0 commit comments

Comments
 (0)