Skip to content

Commit

Permalink
Merge pull request #18766 from michael-redpanda/CORE-3167-Relax-fips-…
Browse files Browse the repository at this point in the history
…enabled

CORE-3167 relax fips enabled behavior
  • Loading branch information
michael-redpanda authored Jun 5, 2024
2 parents fd2759d + 82b4733 commit 086dc22
Show file tree
Hide file tree
Showing 14 changed files with 273 additions and 44 deletions.
3 changes: 2 additions & 1 deletion src/v/cloud_storage/types.cc
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
#include "cloud_storage/configuration.h"
#include "cloud_storage/logger.h"
#include "config/node_config.h"
#include "config/types.h"

#include <absl/container/node_hash_set.h>

Expand Down Expand Up @@ -428,7 +429,7 @@ ss::future<configuration> 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);
Expand Down
3 changes: 2 additions & 1 deletion src/v/cloud_storage_clients/s3_client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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");

Expand Down
2 changes: 1 addition & 1 deletion src/v/cluster/cluster_utils.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
27 changes: 27 additions & 0 deletions src/v/config/convert.h
Original file line number Diff line number Diff line change
Expand Up @@ -574,4 +574,31 @@ struct convert<model::recovery_validation_mode> {
}
};

template<>
struct convert<config::fips_mode_flag> {
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<std::string>();
if (
std::find(acceptable_values.begin(), acceptable_values.end(), value)
== acceptable_values.end()) {
return false;
}

rhs = string_switch<type>(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
16 changes: 12 additions & 4 deletions src/v/config/node_config.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@
#include "node_config.h"

#include "config/configuration.h"
#include "config/types.h"
#include "utils/unresolved_address.h"

namespace config {
Expand Down Expand Up @@ -193,19 +194,26 @@ 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 "
"and TLSv1.3, and of those, only the ones that use NIST-approved "
"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",
Expand Down
2 changes: 1 addition & 1 deletion src/v/config/node_config.h
Original file line number Diff line number Diff line change
Expand Up @@ -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<bool> fips_mode;
enum_property<fips_mode_flag> fips_mode;

// Path to the OpenSSL config file
property<std::optional<std::filesystem::path>> openssl_config_file;
Expand Down
2 changes: 2 additions & 0 deletions src/v/config/property.h
Original file line number Diff line number Diff line change
Expand Up @@ -651,6 +651,8 @@ consteval std::string_view property_type_name() {
} else if constexpr (std::
is_same_v<type, model::recovery_validation_mode>) {
return "recovery_validation_mode";
} else if constexpr (std::is_same_v<type, config::fips_mode_flag>) {
return "string";
} else {
static_assert(
base::unsupported_type<T>::value, "Type name not defined");
Expand Down
7 changes: 7 additions & 0 deletions src/v/config/rjson_serialization.cc
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,8 @@

#include "config/rjson_serialization.h"

#include "config/types.h"

namespace json {

void rjson_serialize(
Expand Down Expand Up @@ -210,4 +212,9 @@ void rjson_serialize(
w.EndObject();
}

void rjson_serialize(
json::Writer<json::StringBuffer>& w, const config::fips_mode_flag& f) {
stringize(w, f);
}

} // namespace json
5 changes: 5 additions & 0 deletions src/v/config/rjson_serialization.h
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -114,4 +115,8 @@ void rjson_serialize(

void rjson_serialize(
json::Writer<json::StringBuffer>&, const model::recovery_validation_mode&);

void rjson_serialize(
json::Writer<json::StringBuffer>&, const config::fips_mode_flag& f);

} // namespace json
48 changes: 48 additions & 0 deletions src/v/config/types.h
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@
*/
#pragma once

#include "strings/string_switch.h"

#include <ostream>

/*
Expand Down Expand Up @@ -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 <seastar/core/sstring.hh>

#include <base/seastarx.h>
namespace config {

enum class s3_url_style { virtual_host = 0, path };
Expand All @@ -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<fips_mode_flag>(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
71 changes: 53 additions & 18 deletions src/v/redpanda/application.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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();
}
Expand Down Expand Up @@ -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,
Expand Down
Loading

0 comments on commit 086dc22

Please sign in to comment.