From 822216a27f4986f23b652baea26fe89f51ca3aca Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 3 Jun 2024 21:48:22 -0400 Subject: [PATCH 1/6] config: Introduce fips_mode_flag Signed-off-by: Michael Boquard --- src/v/config/types.h | 48 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/src/v/config/types.h b/src/v/config/types.h index 41d5cf2dcc980..461bf94a95e2e 100644 --- a/src/v/config/types.h +++ b/src/v/config/types.h @@ -10,6 +10,8 @@ */ #pragma once +#include "strings/string_switch.h" + #include /* @@ -38,6 +40,10 @@ * - defines a `T from_config(config::s3_url_style)` conversion type used * to convert from the configuration option type to the sub-system type. */ + +#include + +#include namespace config { enum class s3_url_style { virtual_host = 0, path }; @@ -51,4 +57,46 @@ inline std::ostream& operator<<(std::ostream& os, const s3_url_style& us) { } } +enum class fips_mode_flag { + // FIPS mode disabled + disabled = 0, + // FIPS mode enabled with strict environment checks + enabled, + // FIPS mode enabled with permissive environment checks + permissive +}; + +constexpr std::string_view to_string_view(fips_mode_flag f) { + switch (f) { + case fips_mode_flag::disabled: + return "disabled"; + case fips_mode_flag::enabled: + return "enabled"; + case fips_mode_flag::permissive: + return "permissive"; + } +} + +inline std::ostream& operator<<(std::ostream& o, fips_mode_flag f) { + return o << to_string_view(f); +} + +inline std::istream& operator>>(std::istream& i, fips_mode_flag& f) { + ss::sstring s; + i >> s; + f = string_switch(s) + .match( + to_string_view(fips_mode_flag::disabled), fips_mode_flag::disabled) + .match( + to_string_view(fips_mode_flag::enabled), fips_mode_flag::enabled) + .match( + to_string_view(fips_mode_flag::permissive), + fips_mode_flag::permissive); + return i; +} + +inline bool fips_mode_enabled(fips_mode_flag f) { + return f != fips_mode_flag::disabled; +} + } // namespace config From edc0e7860e1438674a50912797e7a38d1e425db5 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 3 Jun 2024 21:50:28 -0400 Subject: [PATCH 2/6] config: Conversion functions for fips_mode_flag Signed-off-by: Michael Boquard --- src/v/config/convert.h | 27 +++++++++++++++++++++++++++ src/v/config/property.h | 2 ++ src/v/config/rjson_serialization.cc | 7 +++++++ src/v/config/rjson_serialization.h | 5 +++++ 4 files changed, 41 insertions(+) diff --git a/src/v/config/convert.h b/src/v/config/convert.h index 291ae54d78a4b..b079864a7a836 100644 --- a/src/v/config/convert.h +++ b/src/v/config/convert.h @@ -574,4 +574,31 @@ struct convert { } }; +template<> +struct convert { + using type = config::fips_mode_flag; + + static constexpr auto acceptable_values = std::to_array( + {to_string_view(type::disabled), + to_string_view(type::enabled), + to_string_view(type::permissive)}); + + static Node encode(const type& rhs) { return Node(fmt::format("{}", rhs)); } + static bool decode(const Node& node, type& rhs) { + auto value = node.as(); + if ( + std::find(acceptable_values.begin(), acceptable_values.end(), value) + == acceptable_values.end()) { + return false; + } + + rhs = string_switch(std::string_view{value}) + .match(to_string_view(type::disabled), type::disabled) + .match(to_string_view(type::enabled), type::enabled) + .match(to_string_view(type::permissive), type::permissive); + + return true; + } +}; + } // namespace YAML diff --git a/src/v/config/property.h b/src/v/config/property.h index 25ea581b4e5f8..46c999f0b9f95 100644 --- a/src/v/config/property.h +++ b/src/v/config/property.h @@ -651,6 +651,8 @@ consteval std::string_view property_type_name() { } else if constexpr (std:: is_same_v) { return "recovery_validation_mode"; + } else if constexpr (std::is_same_v) { + return "string"; } else { static_assert( base::unsupported_type::value, "Type name not defined"); diff --git a/src/v/config/rjson_serialization.cc b/src/v/config/rjson_serialization.cc index 537503c2b91e0..c005d3220f157 100644 --- a/src/v/config/rjson_serialization.cc +++ b/src/v/config/rjson_serialization.cc @@ -9,6 +9,8 @@ #include "config/rjson_serialization.h" +#include "config/types.h" + namespace json { void rjson_serialize( @@ -210,4 +212,9 @@ void rjson_serialize( w.EndObject(); } +void rjson_serialize( + json::Writer& w, const config::fips_mode_flag& f) { + stringize(w, f); +} + } // namespace json diff --git a/src/v/config/rjson_serialization.h b/src/v/config/rjson_serialization.h index 46ed6a2c7d612..d500e43b52924 100644 --- a/src/v/config/rjson_serialization.h +++ b/src/v/config/rjson_serialization.h @@ -16,6 +16,7 @@ #include "config/endpoint_tls_config.h" #include "config/seed_server.h" #include "config/tls_config.h" +#include "config/types.h" #include "json/json.h" #include "json/stringbuffer.h" #include "json/writer.h" @@ -114,4 +115,8 @@ void rjson_serialize( void rjson_serialize( json::Writer&, const model::recovery_validation_mode&); + +void rjson_serialize( + json::Writer&, const config::fips_mode_flag& f); + } // namespace json From f573e5b4c67a273b7aef9550480b91578286a662 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Mon, 3 Jun 2024 21:51:39 -0400 Subject: [PATCH 3/6] config: Convert fips_mode to fips_mode_flag Signed-off-by: Michael Boquard --- src/v/cloud_storage/types.cc | 3 +- src/v/cloud_storage_clients/s3_client.cc | 3 +- src/v/cluster/cluster_utils.cc | 2 +- src/v/config/node_config.cc | 16 ++++-- src/v/config/node_config.h | 2 +- src/v/redpanda/application.cc | 71 ++++++++++++++++++------ tests/rptest/services/redpanda.py | 2 +- 7 files changed, 72 insertions(+), 27 deletions(-) diff --git a/src/v/cloud_storage/types.cc b/src/v/cloud_storage/types.cc index 7d50c1287a7d7..4eeadc731336f 100644 --- a/src/v/cloud_storage/types.cc +++ b/src/v/cloud_storage/types.cc @@ -14,6 +14,7 @@ #include "cloud_storage/configuration.h" #include "cloud_storage/logger.h" #include "config/node_config.h" +#include "config/types.h" #include @@ -428,7 +429,7 @@ ss::future configuration::get_s3_config() { region, bucket_name, cloud_storage_clients::from_config(url_style), - config::node().fips_mode.value(), + config::fips_mode_enabled(config::node().fips_mode.value()), get_default_overrides(), disable_metrics, disable_public_metrics); diff --git a/src/v/cloud_storage_clients/s3_client.cc b/src/v/cloud_storage_clients/s3_client.cc index 9479e5680be25..e98e50b788dac 100644 --- a/src/v/cloud_storage_clients/s3_client.cc +++ b/src/v/cloud_storage_clients/s3_client.cc @@ -19,6 +19,7 @@ #include "cloud_storage_clients/xml_sax_parser.h" #include "config/configuration.h" #include "config/node_config.h" +#include "config/types.h" #include "hashing/secure.h" #include "http/client.h" #include "net/types.h" @@ -602,7 +603,7 @@ s3_client::self_configure() { // fips mode can only work in virtual_host mode, so if the above test failed // the TS service is likely misconfigured vassert( - !config::node().fips_mode.value(), + !config::fips_mode_enabled(config::node().fips_mode.value()), "fips_mode requires the bucket to configured in virtual_host mode, but " "the connectivity test failed"); diff --git a/src/v/cluster/cluster_utils.cc b/src/v/cluster/cluster_utils.cc index 6bf1e3446dc6f..0e189a8d03e44 100644 --- a/src/v/cluster/cluster_utils.cc +++ b/src/v/cluster/cluster_utils.cc @@ -83,7 +83,7 @@ model::broker make_self_broker(const config::node_config& node_cfg) { .available_memory_gb = total_mem_gb, .available_disk_gb = disk_gb, .available_memory_bytes = total_mem, - .in_fips_mode = node_cfg.fips_mode()}); + .in_fips_mode = config::fips_mode_enabled(node_cfg.fips_mode())}); } bool are_replica_sets_equal( diff --git a/src/v/config/node_config.cc b/src/v/config/node_config.cc index 4bbbb33e99538..6a814e1228bf9 100644 --- a/src/v/config/node_config.cc +++ b/src/v/config/node_config.cc @@ -10,6 +10,7 @@ #include "node_config.h" #include "config/configuration.h" +#include "config/types.h" #include "utils/unresolved_address.h" namespace config { @@ -193,11 +194,15 @@ node_config::node_config() noexcept , fips_mode( *this, "fips_mode", - "Controls whether or not Redpanda starts in FIPS mode. In the FIPS " - "mode of operation, Redpanda first verifies that the operating system " + "Controls whether Redpanda starts in FIPS mode. This property " + "allows for three values: 'disabled', 'enabled', and 'permissive'. With " + "'enabled', Redpanda first verifies that the operating " + "system " "is enabled for FIPS by checking /proc/sys/crypto/fips_enabled. If the " "file does not exist or does not return '1', Redpanda immediately " - "exits. After the check is complete, Redpanda loads the " + "exits. With 'permissive', the same check is performed " + "but a WARNING is logged and Redpanda continues to run. After " + "the check is complete, Redpanda loads the " "OpenSSL FIPS provider into the OpenSSL library. After this is " "complete, Redpanda is operating in FIPS mode, which means that the " "TLS cipher suites available to users are limited to TLSv1.2 " @@ -205,7 +210,10 @@ node_config::node_config() noexcept "cryptographic methods. For more information about FIPS, refer to " "Redpanda documentation.", {.visibility = visibility::user}, - false) + fips_mode_flag::disabled, + {fips_mode_flag::disabled, + fips_mode_flag::enabled, + fips_mode_flag::permissive}) , openssl_config_file( *this, "openssl_config_file", diff --git a/src/v/config/node_config.h b/src/v/config/node_config.h index 7af93bf6d4465..60eaa54a131a4 100644 --- a/src/v/config/node_config.h +++ b/src/v/config/node_config.h @@ -88,7 +88,7 @@ struct node_config final : public config_store { verbose_logging_timeout_sec_max; // Flag indicating whether or not Redpanda will start in FIPS mode - property fips_mode; + enum_property fips_mode; // Path to the OpenSSL config file property> openssl_config_file; diff --git a/src/v/redpanda/application.cc b/src/v/redpanda/application.cc index fcf901c034abf..052e228022744 100644 --- a/src/v/redpanda/application.cc +++ b/src/v/redpanda/application.cc @@ -76,6 +76,7 @@ #include "config/endpoint_tls_config.h" #include "config/node_config.h" #include "config/seed_server.h" +#include "config/types.h" #include "crypto/ossl_context_service.h" #include "features/feature_table_snapshot.h" #include "features/fwd.h" @@ -887,21 +888,53 @@ void application::check_environment() { } } - if (config::node().fips_mode()) { + if (config::fips_mode_enabled(config::node().fips_mode())) { if (!ss::file_exists(fips_enabled_file).get()) { - throw std::runtime_error(fmt::format( - "File '{}' does not exist. Redpanda cannot start in FIPS mode", - fips_enabled_file)); - } - - auto fd = ss::file_desc::open(fips_enabled_file.data(), O_RDONLY); - char buf[1]; - fd.read(buf, 1); - if (buf[0] != '1') { - throw std::runtime_error(fmt::format( - "File '{}' not reporting '1'. Redpanda cannot start in FIPS " - "mode", - fips_enabled_file)); + if (config::node().fips_mode() == config::fips_mode_flag::enabled) { + throw std::runtime_error(fmt::format( + "File '{}' does not exist. Redpanda cannot start in FIPS " + "mode", + fips_enabled_file)); + } else if ( + config::node().fips_mode() + == config::fips_mode_flag::permissive) { + vlog( + _log.warn, + "File '{}' does not exist. Redpanda will start in FIPS mode " + "but this is not a support configuration", + fips_enabled_file); + } else { + vassert( + false, + "Should not be performing environment check for FIPS when " + "fips_mode flag is {}", + config::node().fips_mode()); + } + } else { + auto fd = ss::file_desc::open(fips_enabled_file.data(), O_RDONLY); + char buf[1]; + fd.read(buf, 1); + if (buf[0] != '1') { + auto msg = fmt::format( + "File '{}' not reporting '1'. Redpanda cannot start in FIPS " + "mode", + fips_enabled_file); + if ( + config::node().fips_mode() + == config::fips_mode_flag::enabled) { + throw std::runtime_error(msg); + } else if ( + config::node().fips_mode() + == config::fips_mode_flag::permissive) { + vlog(_log.warn, "{}", msg); + } else { + vassert( + false, + "Should not be performing environment check for FIPS " + "when fips_mode flag is {}", + config::node().fips_mode()); + } + } } syschecks::systemd_message("Starting Redpanda in FIPS mode").get(); } @@ -2136,15 +2169,17 @@ void application::wire_up_and_start_crypto_services() { std::ref(*thread_worker), ss::sstring{config::node().openssl_config_file().value_or("")}, ss::sstring{config::node().openssl_module_directory().value_or("")}, - config::node().fips_mode() ? crypto::is_fips_mode::yes - : crypto::is_fips_mode::no) + config::fips_mode_enabled(config::node().fips_mode()) + ? crypto::is_fips_mode::yes + : crypto::is_fips_mode::no) .get(); ossl_context_service.invoke_on_all(&crypto::ossl_context_service::start) .get(); ossl_context_service.map([](auto& s) { return s.fips_mode(); }) .then([](auto fips_mode_vals) { - auto expected = config::node().fips_mode() ? crypto::is_fips_mode::yes - : crypto::is_fips_mode::no; + auto expected = config::fips_mode_enabled(config::node().fips_mode()) + ? crypto::is_fips_mode::yes + : crypto::is_fips_mode::no; for (auto fips_mode : fips_mode_vals) { vassert( fips_mode == expected, diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index e887811a332ad..dbf30aae5d3a4 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -3959,7 +3959,7 @@ def is_fips_capable(node) -> bool: ) doc = yaml.full_load(conf) doc["redpanda"].update( - dict(fips_mode=True, + dict(fips_mode="enabled", openssl_config_file=RedpandaService.OPENSSL_CONFIG_FILE, openssl_module_directory=RedpandaService. OPENSSL_MODULES_PATH)) From 22181ed22d93ceeb8e02ed372146e83e036dc481 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Tue, 4 Jun 2024 07:19:25 -0400 Subject: [PATCH 4/6] dt: Moved settings FIPS mode flag in node config This ensures that a test that wishes to override the FIPS mode flag doesn't get that overwritten later. Signed-off-by: Michael Boquard --- tests/rptest/services/redpanda.py | 32 +++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index dbf30aae5d3a4..abbb9491e94d2 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -3920,6 +3920,22 @@ def write_node_conf_file(self, sasl_enabled=self.sasl_enabled(), endpoint_authn_method=self.endpoint_authn_method(), auto_auth=self._security.auto_auth) + + def is_fips_capable(node) -> bool: + cur_ver = self._installer.installed_version(node) + return cur_ver == RedpandaInstaller.HEAD or cur_ver >= (24, 2, 1) + + if in_fips_environment() and is_fips_capable(node): + self.logger.info( + "Operating in FIPS environment, enabling FIPS mode for Redpanda" + ) + doc = yaml.full_load(conf) + doc["redpanda"].update( + dict(fips_mode="enabled", + openssl_config_file=RedpandaService.OPENSSL_CONFIG_FILE, + openssl_module_directory=RedpandaService. + OPENSSL_MODULES_PATH)) + conf = yaml.dump(doc) if override_cfg_params or node in self._extra_node_conf: doc = yaml.full_load(conf) @@ -3949,22 +3965,6 @@ def write_node_conf_file(self, doc["redpanda"].update(dict(kafka_api_tls=tls_config)) conf = yaml.dump(doc) - def is_fips_capable(node) -> bool: - cur_ver = self._installer.installed_version(node) - return cur_ver == RedpandaInstaller.HEAD or cur_ver >= (24, 2, 1) - - if in_fips_environment() and is_fips_capable(node): - self.logger.info( - "Operating in FIPS environment, enabling FIPS mode for Redpanda" - ) - doc = yaml.full_load(conf) - doc["redpanda"].update( - dict(fips_mode="enabled", - openssl_config_file=RedpandaService.OPENSSL_CONFIG_FILE, - openssl_module_directory=RedpandaService. - OPENSSL_MODULES_PATH)) - conf = yaml.dump(doc) - self.logger.info("Writing Redpanda node config file: {}".format( RedpandaService.NODE_CONFIG_FILE)) self.logger.debug(conf) From f3036b0b8adb443a9599a7d130db83d0a9599929 Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Tue, 4 Jun 2024 12:05:34 -0400 Subject: [PATCH 5/6] dt: Handle FIPS config for DT When RP is not installed to /opt/redpanda, some of the config files will point to non-existant files. This change will make it so one can run Redpanda in FIPS mode in DT. Signed-off-by: Michael Boquard --- tests/rptest/services/redpanda.py | 50 ++++++++++++++++++--- tests/rptest/services/templates/openssl.cnf | 9 ++++ 2 files changed, 53 insertions(+), 6 deletions(-) create mode 100644 tests/rptest/services/templates/openssl.cnf diff --git a/tests/rptest/services/redpanda.py b/tests/rptest/services/redpanda.py index abbb9491e94d2..bef6fc5d36933 100644 --- a/tests/rptest/services/redpanda.py +++ b/tests/rptest/services/redpanda.py @@ -1241,6 +1241,7 @@ class RedpandaServiceBase(RedpandaServiceABC, Service): BACKTRACE_CAPTURE = os.path.join(PERSISTENT_ROOT, "redpanda_backtrace.log") COVERAGE_PROFRAW_CAPTURE = os.path.join(PERSISTENT_ROOT, "redpanda.profraw") + TEMP_OSSL_CONFIG_FILE = "/etc/openssl.cnf" DEFAULT_NODE_READY_TIMEOUT_SEC = 20 NODE_READY_TIMEOUT_MIN_SEC_KEY = "node_ready_timeout_min_sec" DEFAULT_CLOUD_STORAGE_SCRUB_TIMEOUT_SEC = 60 @@ -1256,8 +1257,8 @@ class RedpandaServiceBase(RedpandaServiceABC, Service): FAILURE_INJECTION_CONFIG_PATH = "/etc/redpanda/failure_injection_config.json" - OPENSSL_CONFIG_FILE = "/opt/redpanda/openssl/openssl.cnf" - OPENSSL_MODULES_PATH = "/opt/redpanda/lib/ossl-modules/" + OPENSSL_CONFIG_FILE_BASE = "openssl/openssl.cnf" + OPENSSL_MODULES_PATH_BASE = "lib/ossl-modules/" # When configuring multiple listeners for testing, a secondary port to use # instead of the default. @@ -3007,6 +3008,8 @@ def start_node(self, node.account.mkdirs(RedpandaService.DATA_DIR) node.account.mkdirs(os.path.dirname(RedpandaService.NODE_CONFIG_FILE)) + self.write_openssl_config_file(node) + if write_config: self.write_node_conf_file( node, @@ -3819,6 +3822,9 @@ def clean_node(self, node.account.remove(RedpandaService.SYSTEM_TLS_CA_CRT_FILE) node.account.ssh(f"update-ca-certificates") + if node.account.exists(RedpandaService.TEMP_OSSL_CONFIG_FILE): + node.account.remove(RedpandaService.TEMP_OSSL_CONFIG_FILE) + if not preserve_current_install or not self._installer._started: # Reset the binaries to use the original binaries. # NOTE: if the installer hasn't been started, there is no @@ -3867,6 +3873,38 @@ def get_node_fqdn(node): timeout_sec=10).decode('utf-8').split(' ')[0] return fqdn + def write_openssl_config_file(self, node): + conf = self.render("openssl.cnf", + fips_conf_file=os.path.join( + self.rp_install_path(), + "openssl/fipsmodule.cnf")) + self.logger.debug( + f'Writing {RedpandaService.TEMP_OSSL_CONFIG_FILE} to {node.name}:\n{conf}' + ) + node.account.create_file(RedpandaService.TEMP_OSSL_CONFIG_FILE, conf) + + def get_openssl_config_file_path(self) -> str: + path = os.path.join(self.rp_install_path(), + self.OPENSSL_CONFIG_FILE_BASE) + if self.rp_install_path() != "/opt/redpanda": + # If we aren't using an 'installed' Redpanda instance, the openssl config file + # located in the install path will not point to the correct location of the FIPS + # module config file. We generate an openssl config file just for this purpose + # see write_openssl_config_file above + path = RedpandaService.TEMP_OSSL_CONFIG_FILE + + self.logger.debug( + f'OpenSSL Config File Path: {path} ({self.rp_install_path()})') + return path + + def get_openssl_modules_directory(self) -> str: + path = os.path.join(self.rp_install_path(), + self.OPENSSL_MODULES_PATH_BASE) + + self.logger.debug( + f'OpenSSL Modules Directory: {path} ({self.rp_install_path()})') + return path + def write_node_conf_file(self, node, override_cfg_params=None, @@ -3920,7 +3958,7 @@ def write_node_conf_file(self, sasl_enabled=self.sasl_enabled(), endpoint_authn_method=self.endpoint_authn_method(), auto_auth=self._security.auto_auth) - + def is_fips_capable(node) -> bool: cur_ver = self._installer.installed_version(node) return cur_ver == RedpandaInstaller.HEAD or cur_ver >= (24, 2, 1) @@ -3932,9 +3970,9 @@ def is_fips_capable(node) -> bool: doc = yaml.full_load(conf) doc["redpanda"].update( dict(fips_mode="enabled", - openssl_config_file=RedpandaService.OPENSSL_CONFIG_FILE, - openssl_module_directory=RedpandaService. - OPENSSL_MODULES_PATH)) + openssl_config_file=self.get_openssl_config_file_path(), + openssl_module_directory=self. + get_openssl_modules_directory)) conf = yaml.dump(doc) if override_cfg_params or node in self._extra_node_conf: diff --git a/tests/rptest/services/templates/openssl.cnf b/tests/rptest/services/templates/openssl.cnf new file mode 100644 index 0000000000000..ef708e469c099 --- /dev/null +++ b/tests/rptest/services/templates/openssl.cnf @@ -0,0 +1,9 @@ +config_diagnostics = 1 +openssl_conf = openssl_init +.include {{fips_conf_file}} + +[openssl_init] +providers = provider_sect + +[provider_sect] +fips = fips_sect \ No newline at end of file From 82b473366da58efb730b4b5aaa4b08201a945c3c Mon Sep 17 00:00:00 2001 From: Michael Boquard Date: Tue, 4 Jun 2024 12:07:16 -0400 Subject: [PATCH 6/6] dt: Added test for Redpanda in FIPS permissive mode Signed-off-by: Michael Boquard --- tests/rptest/tests/redpanda_startup_test.py | 48 +++++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/tests/rptest/tests/redpanda_startup_test.py b/tests/rptest/tests/redpanda_startup_test.py index 577c55f25c8f6..70cfb1c48f546 100644 --- a/tests/rptest/tests/redpanda_startup_test.py +++ b/tests/rptest/tests/redpanda_startup_test.py @@ -7,7 +7,10 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0 +import os + from rptest.services.cluster import cluster +from rptest.services.redpanda import RedpandaService, in_fips_environment from rptest.tests.redpanda_test import RedpandaTest @@ -23,3 +26,48 @@ def __init__(self, test_context): @cluster(num_nodes=3) def test_startup(self): pass + + +class RedpandaFIPSStartupTest(RedpandaTest): + """ + Tests that Redpanda can start up in FIPS permissive mode + """ + def __init__(self, test_context): + super(RedpandaFIPSStartupTest, + self).__init__(test_context=test_context) + + for node in self.redpanda.nodes: + self.redpanda.set_extra_node_conf( + node, { + "fips_mode": + "permissive", + "openssl_config_file": + self.redpanda.get_openssl_config_file_path(), + "openssl_module_directory": + self.redpanda.get_openssl_modules_directory() + }) + + @cluster(num_nodes=3) + def test_startup(self): + """ + This test will validate that Redpanda can come up in permissive mode + and that the proper warning messages are printed + """ + fips_enabled_file = '/proc/sys/crypto/fips_enabled' + file_does_not_exist_log = f"File '{fips_enabled_file}' does not exist." + file_not_one_log = f"File '{fips_enabled_file}' not reporting '1'" + if in_fips_environment(): + # Exception to the test here - if we're in a FIPS environment, no log messages should appear + self.logger.debug("In FIPS environment, no log expected") + assert not self.redpanda.search_log_all(file_does_not_exist_log) + assert not self.redpanda.search_log_all(file_not_one_log) + elif not os.path.isfile(fips_enabled_file): + self.logger.debug( + f"Not in FIPS environment and '{fips_enabled_file}' doesn't exist" + ) + assert self.redpanda.search_log_all(file_does_not_exist_log) + else: + self.logger.debug( + f"Not in FIPS environment and '{fips_enabled_file}' file exists" + ) + assert self.redpanda.search_log_all(file_not_one_log)