diff --git a/src/v/utils/type_traits.h b/src/v/base/include/base/type_traits.h similarity index 57% rename from src/v/utils/type_traits.h rename to src/v/base/include/base/type_traits.h index ed8732c15abd7..c48aff6f761cd 100644 --- a/src/v/utils/type_traits.h +++ b/src/v/base/include/base/type_traits.h @@ -11,9 +11,10 @@ #pragma once +#include #include -namespace utils { +namespace base { /** * A utility for statically asserting false. @@ -26,7 +27,7 @@ namespace utils { * if constexpr (...) { * // ... * } else { - * static_assert(utils::unsupported_type::value, "unsupported type"); + * static_assert(base::unsupported_type::value, "unsupported type"); * } * } * @@ -50,11 +51,30 @@ struct unsupported_type : std::false_type {}; * } else if constexpr (value == foo::baz) { * // ... * } else { - * static_assert(utils::unsupported_value::value, "supported foo"); + * static_assert(base::unsupported_value::value, "supported foo"); * } * } */ template struct unsupported_value : std::false_type {}; -} // namespace utils +} // namespace base + +namespace detail { + +template class C> +struct is_specialization_of : std::false_type {}; +template class C, typename... Args> +struct is_specialization_of, C> : std::true_type {}; +template class C> +inline constexpr bool is_specialization_of_v + = is_specialization_of::value; + +} // namespace detail + +namespace reflection { + +template +concept is_std_optional = ::detail::is_specialization_of_v; + +} diff --git a/src/v/cloud_storage/partition_manifest.cc b/src/v/cloud_storage/partition_manifest.cc index 00e81b8f3e2f9..c7a1b7730b9e8 100644 --- a/src/v/cloud_storage/partition_manifest.cc +++ b/src/v/cloud_storage/partition_manifest.cc @@ -26,6 +26,7 @@ #include "model/fundamental.h" #include "model/timestamp.h" #include "reflection/to_tuple.h" +#include "reflection/type_traits.h" #include "serde/envelope.h" #include "serde/envelope_for_each_field.h" #include "serde/serde.h" diff --git a/src/v/cloud_storage_clients/configuration.h b/src/v/cloud_storage_clients/configuration.h index 487b62ef2a9b0..618ad8a13d7f5 100644 --- a/src/v/cloud_storage_clients/configuration.h +++ b/src/v/cloud_storage_clients/configuration.h @@ -12,6 +12,7 @@ #include "cloud_storage_clients/client_probe.h" #include "cloud_storage_clients/types.h" +#include "model/metadata.h" #include "net/transport.h" #include "net/types.h" diff --git a/src/v/cluster/tests/serialization_rt_test.cc b/src/v/cluster/tests/serialization_rt_test.cc index afc300aa58395..93cac9bf352f7 100644 --- a/src/v/cluster/tests/serialization_rt_test.cc +++ b/src/v/cluster/tests/serialization_rt_test.cc @@ -34,7 +34,7 @@ #include "storage/types.h" #include "test_utils/randoms.h" #include "test_utils/rpc.h" -#include "tristate.h" +#include "utils/tristate.h" #include "v8_engine/data_policy.h" #include diff --git a/src/v/cluster/tx_topic_manager.cc b/src/v/cluster/tx_topic_manager.cc index c291c14c66f5a..8b463cd14a00f 100644 --- a/src/v/cluster/tx_topic_manager.cc +++ b/src/v/cluster/tx_topic_manager.cc @@ -19,7 +19,7 @@ #include "features/feature_table.h" #include "model/namespace.h" #include "ssx/future-util.h" -#include "tristate.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/cluster/types.cc b/src/v/cluster/types.cc index 7eb7da4ff80fd..3507ab836a63b 100644 --- a/src/v/cluster/types.cc +++ b/src/v/cluster/types.cc @@ -17,8 +17,8 @@ #include "model/timestamp.h" #include "reflection/adl.h" #include "security/acl.h" -#include "tristate.h" #include "utils/to_string.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/cluster/types.h b/src/v/cluster/types.h index 809f889870172..e0a680a390538 100644 --- a/src/v/cluster/types.h +++ b/src/v/cluster/types.h @@ -33,7 +33,7 @@ #include "security/types.h" #include "serde/envelope.h" #include "storage/ntp_config.h" -#include "tristate.h" +#include "utils/tristate.h" #include "v8_engine/data_policy.h" #include diff --git a/src/v/compression/include/compression/stream_zstd.h b/src/v/compression/include/compression/stream_zstd.h index 159e03091abf1..65a02aed5d932 100644 --- a/src/v/compression/include/compression/stream_zstd.h +++ b/src/v/compression/include/compression/stream_zstd.h @@ -11,7 +11,7 @@ #pragma once #include "bytes/iobuf.h" -#include "static_deleter_fn.h" +#include "utils/static_deleter_fn.h" #include #include diff --git a/src/v/compression/internal/lz4_frame_compressor.cc b/src/v/compression/internal/lz4_frame_compressor.cc index 2c6a1226c1c2b..a7b42054f48c3 100644 --- a/src/v/compression/internal/lz4_frame_compressor.cc +++ b/src/v/compression/internal/lz4_frame_compressor.cc @@ -11,7 +11,7 @@ #include "base/vassert.h" #include "compression/lz4_decompression_buffers.h" -#include "static_deleter_fn.h" +#include "utils/static_deleter_fn.h" #include diff --git a/src/v/config/broker_authn_endpoint.cc b/src/v/config/broker_authn_endpoint.cc index ec0cbaedecb6d..6e9ff370c2743 100644 --- a/src/v/config/broker_authn_endpoint.cc +++ b/src/v/config/broker_authn_endpoint.cc @@ -9,7 +9,6 @@ #include "config/broker_authn_endpoint.h" -#include "kafka/client/exceptions.h" #include "model/metadata.h" #include "strings/string_switch.h" diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index fc3b700071b8e..0de6b13a781fd 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -16,16 +16,13 @@ #include "config/validators.h" #include "model/metadata.h" #include "model/namespace.h" -#include "pandaproxy/schema_registry/schema_id_validation.h" +#include "security/config.h" #include "security/gssapi_principal_mapper.h" -#include "security/mtls.h" -#include "security/oidc_principal_mapping.h" #include "security/oidc_url_parser.h" #include "ssx/sformat.h" -#include "storage/chunk_cache.h" -#include "storage/segment_appender.h" -#include "utils/bottomless_token_bucket.h" +#include "storage/config.h" +#include #include #include @@ -1171,7 +1168,7 @@ configuration::configuration() .example = "32768", .visibility = visibility::tunable}, 32_MiB, - storage::segment_appender::validate_fallocation_step) + storage::validate_fallocation_step) , storage_target_replay_bytes( *this, "storage_target_replay_bytes", @@ -2979,7 +2976,8 @@ configuration::configuration() "balancer, in milliseconds", {.needs_restart = needs_restart::no, .visibility = visibility::user}, 5000ms, - {.min = 1ms, .max = bottomless_token_bucket::max_width}) + {.min = 1ms, + .max = std::chrono::milliseconds(std::numeric_limits::max())}) , kafka_quota_balancer_node_period( *this, "kafka_quota_balancer_node_period_ms", diff --git a/src/v/config/property.h b/src/v/config/property.h index 619281684dcae..25ea581b4e5f8 100644 --- a/src/v/config/property.h +++ b/src/v/config/property.h @@ -11,13 +11,13 @@ #pragma once #include "base/oncore.h" +#include "base/type_traits.h" #include "config/base_property.h" #include "config/rjson_serialization.h" #include "container/intrusive_list_helpers.h" #include "json/stringbuffer.h" #include "json/writer.h" #include "pandaproxy/schema_registry/schema_id_validation.h" -#include "reflection/type_traits.h" #include "utils/to_string.h" #include @@ -653,7 +653,7 @@ consteval std::string_view property_type_name() { return "recovery_validation_mode"; } else { static_assert( - utils::unsupported_type::value, "Type name not defined"); + base::unsupported_type::value, "Type name not defined"); } } diff --git a/src/v/config/rest_authn_endpoint.cc b/src/v/config/rest_authn_endpoint.cc index 23657bb58a71b..33091c7cf891f 100644 --- a/src/v/config/rest_authn_endpoint.cc +++ b/src/v/config/rest_authn_endpoint.cc @@ -9,7 +9,6 @@ #include "config/rest_authn_endpoint.h" -#include "kafka/client/exceptions.h" #include "model/metadata.h" #include "strings/string_switch.h" diff --git a/src/v/config/throughput_control_group.cc b/src/v/config/throughput_control_group.cc index ea4072559e624..7e1827c3494b3 100644 --- a/src/v/config/throughput_control_group.cc +++ b/src/v/config/throughput_control_group.cc @@ -9,7 +9,7 @@ * by the Apache License, Version 2.0 */ -#include "throughput_control_group.h" +#include "config/throughput_control_group.h" #include "config/convert.h" #include "ssx/sformat.h" diff --git a/src/v/container/tests/bench_utils.h b/src/v/container/tests/bench_utils.h index 91025d04a141a..736dd5bffe5d8 100644 --- a/src/v/container/tests/bench_utils.h +++ b/src/v/container/tests/bench_utils.h @@ -9,9 +9,9 @@ * by the Apache License, Version 2.0 */ #pragma once +#include "base/type_traits.h" #include "random/generators.h" #include "utils/functional.h" -#include "utils/type_traits.h" #include @@ -56,6 +56,6 @@ static auto make_value() { .okdone = random_generators::get_int(), }; } else { - static_assert(utils::unsupported_type::value, "unsupported"); + static_assert(base::unsupported_type::value, "unsupported"); } } diff --git a/src/v/container/tests/vector_bench.cc b/src/v/container/tests/vector_bench.cc index bb6291f762e01..e60832960760b 100644 --- a/src/v/container/tests/vector_bench.cc +++ b/src/v/container/tests/vector_bench.cc @@ -9,11 +9,11 @@ * by the Apache License, Version 2.0 */ +#include "base/type_traits.h" #include "container/fragmented_vector.h" #include "container/tests/bench_utils.h" #include "random/generators.h" #include "utils/functional.h" -#include "utils/type_traits.h" #include diff --git a/src/v/http/tests/http_client_test.cc b/src/v/http/tests/http_client_test.cc index 505a35b999247..fd739a63ec0dd 100644 --- a/src/v/http/tests/http_client_test.cc +++ b/src/v/http/tests/http_client_test.cc @@ -15,6 +15,7 @@ #include "http/client.h" #include "http/logger.h" #include "json/document.h" +#include "json/json.h" #include "net/dns.h" #include "net/transport.h" diff --git a/src/v/kafka/server/handlers/configs/config_response_utils.cc b/src/v/kafka/server/handlers/configs/config_response_utils.cc index 9c653888c29db..a5107b5acb674 100644 --- a/src/v/kafka/server/handlers/configs/config_response_utils.cc +++ b/src/v/kafka/server/handlers/configs/config_response_utils.cc @@ -134,7 +134,7 @@ consteval describe_configs_type property_config_type() { return describe_configs_type::list; } else { static_assert( - utils::unsupported_type::value, + base::unsupported_type::value, "Type name is not supported in describe_configs_type"); } } diff --git a/src/v/kafka/server/handlers/describe_configs.cc b/src/v/kafka/server/handlers/describe_configs.cc index 68ff3476d6b87..9d989340bdde6 100644 --- a/src/v/kafka/server/handlers/describe_configs.cc +++ b/src/v/kafka/server/handlers/describe_configs.cc @@ -9,6 +9,7 @@ #include "kafka/server/handlers/describe_configs.h" +#include "base/type_traits.h" #include "cluster/metadata_cache.h" #include "cluster/types.h" #include "config/configuration.h" @@ -28,7 +29,6 @@ #include "reflection/type_traits.h" #include "security/acl.h" #include "ssx/sformat.h" -#include "utils/type_traits.h" #include #include diff --git a/src/v/kafka/server/snc_quota_manager.cc b/src/v/kafka/server/snc_quota_manager.cc index 6f85ced636064..0286354b898eb 100644 --- a/src/v/kafka/server/snc_quota_manager.cc +++ b/src/v/kafka/server/snc_quota_manager.cc @@ -14,7 +14,7 @@ #include "prometheus/prometheus_sanitize.h" #include "ssx/future-util.h" #include "ssx/sharded_ptr.h" -#include "tristate.h" +#include "utils/tristate.h" #include diff --git a/src/v/metrics/metrics.cc b/src/v/metrics/metrics.cc index 197b842e4370f..e224a0df3d07c 100644 --- a/src/v/metrics/metrics.cc +++ b/src/v/metrics/metrics.cc @@ -1,5 +1,18 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ #include "metrics.h" +#include "base/vassert.h" +#include "config/configuration.h" + namespace metrics { internal_metric_groups& internal_metric_groups::add_group( diff --git a/src/v/metrics/metrics.h b/src/v/metrics/metrics.h index 6de5ead01d487..0fbd3f60d247c 100644 --- a/src/v/metrics/metrics.h +++ b/src/v/metrics/metrics.h @@ -11,7 +11,6 @@ #pragma once -#include "config/configuration.h" #include "metrics/metrics_registry.h" #include "ssx/sformat.h" #include "utils/hdr_hist.h" diff --git a/src/v/model/adl_serde.h b/src/v/model/adl_serde.h index 19419b39fdd2a..87cbf386bdad3 100644 --- a/src/v/model/adl_serde.h +++ b/src/v/model/adl_serde.h @@ -17,7 +17,7 @@ #include "model/record.h" #include "model/timeout_clock.h" #include "reflection/adl.h" -#include "tristate.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/net/include/net/connection.h b/src/v/net/include/net/connection.h index 162a3de626230..6979b6cd308e5 100644 --- a/src/v/net/include/net/connection.h +++ b/src/v/net/include/net/connection.h @@ -19,6 +19,7 @@ #include #include #include +#include #include diff --git a/src/v/pandaproxy/json/requests/produce.h b/src/v/pandaproxy/json/requests/produce.h index 83dfeb5aac782..9ec9b62c36e1b 100644 --- a/src/v/pandaproxy/json/requests/produce.h +++ b/src/v/pandaproxy/json/requests/produce.h @@ -22,7 +22,7 @@ #include "kafka/protocol/produce.h" #include "pandaproxy/json/iobuf.h" #include "pandaproxy/json/types.h" -#include "tristate.h" +#include "utils/tristate.h" #include diff --git a/src/v/raft/coordinated_recovery_throttle.cc b/src/v/raft/coordinated_recovery_throttle.cc index f058c68d266a7..b2a87e55888f3 100644 --- a/src/v/raft/coordinated_recovery_throttle.cc +++ b/src/v/raft/coordinated_recovery_throttle.cc @@ -11,6 +11,7 @@ #include "raft/coordinated_recovery_throttle.h" +#include "config/configuration.h" #include "prometheus/prometheus_sanitize.h" #include "raft/logger.h" diff --git a/src/v/redpanda/admin/server.h b/src/v/redpanda/admin/server.h index a5ffb9bcbd5b0..868bdb2fb7085 100644 --- a/src/v/redpanda/admin/server.h +++ b/src/v/redpanda/admin/server.h @@ -12,6 +12,7 @@ #pragma once #include "base/seastarx.h" +#include "base/type_traits.h" #include "cloud_storage/fwd.h" #include "cluster/fwd.h" #include "cluster/tx_gateway_frontend.h" @@ -32,7 +33,6 @@ #include "security/types.h" #include "storage/node.h" #include "transform/fwd.h" -#include "utils/type_traits.h" #include #include @@ -172,7 +172,7 @@ class admin_server { auth_state.pass(); } else { static_assert( - utils::unsupported_value::value, + base::unsupported_value::value, "Invalid auth_level"); } diff --git a/src/v/reflection/type_traits.h b/src/v/reflection/type_traits.h index c4d09212a2b21..8eb7ff1f4e267 100644 --- a/src/v/reflection/type_traits.h +++ b/src/v/reflection/type_traits.h @@ -12,9 +12,10 @@ #pragma once #include "base/seastarx.h" +#include "base/type_traits.h" #include "container/fragmented_vector.h" -#include "tristate.h" #include "utils/named_type.h" +#include "utils/tristate.h" #include #include @@ -29,14 +30,6 @@ namespace detail { -template class C> -struct is_specialization_of : std::false_type {}; -template class C, typename... Args> -struct is_specialization_of, C> : std::true_type {}; -template class C> -inline constexpr bool is_specialization_of_v - = is_specialization_of::value; - template class C> struct is_specialization_of_sized : std::false_type {}; template class C, class T, size_t N> @@ -76,9 +69,6 @@ template concept is_ss_circular_buffer = ::detail::is_specialization_of_v; -template -concept is_std_optional = ::detail::is_specialization_of_v; - template concept is_rp_named_type = ::detail::is_specialization_of_v; diff --git a/src/v/rpc/connection_set.cc b/src/v/rpc/connection_set.cc index 8fa4b72c6ab19..2ec7e3a7e5885 100644 --- a/src/v/rpc/connection_set.cc +++ b/src/v/rpc/connection_set.cc @@ -11,6 +11,7 @@ #include "rpc/connection_set.h" +#include "config/configuration.h" #include "rpc/rpc_utils.h" namespace rpc { diff --git a/src/v/rpc/reconnect_transport.cc b/src/v/rpc/reconnect_transport.cc index 92383fdd23cfd..bf59f2374c886 100644 --- a/src/v/rpc/reconnect_transport.cc +++ b/src/v/rpc/reconnect_transport.cc @@ -15,6 +15,7 @@ #include "rpc/logger.h" #include "rpc/transport.h" #include "rpc/types.h" +#include "utils/to_string.h" #include diff --git a/src/v/security/acl.h b/src/v/security/acl.h index d819bd0fa30cb..22745b9840c89 100644 --- a/src/v/security/acl.h +++ b/src/v/security/acl.h @@ -58,7 +58,7 @@ consteval resource_type get_resource_type() { } else if constexpr (std::is_same_v) { return resource_type::transactional_id; } else { - static_assert(utils::unsupported_type::value, "Unsupported type"); + static_assert(base::unsupported_type::value, "Unsupported type"); } } diff --git a/src/v/security/audit/schemas/utils.h b/src/v/security/audit/schemas/utils.h index 6ca207878c23c..900bb2cb7b7c7 100644 --- a/src/v/security/audit/schemas/utils.h +++ b/src/v/security/audit/schemas/utils.h @@ -81,7 +81,7 @@ consteval audit_resource_type get_audit_resource_type() { } else if constexpr (std::is_same_v) { return audit_resource_type::acl_binding_filter; } else { - static_assert(utils::unsupported_type::value, "Unsupported type"); + static_assert(base::unsupported_type::value, "Unsupported type"); } } diff --git a/src/v/security/authorizer.cc b/src/v/security/authorizer.cc index 7507f28f033f0..a8b5426541ac1 100644 --- a/src/v/security/authorizer.cc +++ b/src/v/security/authorizer.cc @@ -12,6 +12,7 @@ #include "security/authorizer.h" #include "acl_store.h" +#include "config/configuration.h" #include "metrics/metrics.h" #include "prometheus/prometheus_sanitize.h" #include "security/role.h" diff --git a/src/v/security/config.h b/src/v/security/config.h new file mode 100644 index 0000000000000..02a15ab898e3f --- /dev/null +++ b/src/v/security/config.h @@ -0,0 +1,32 @@ +/* + * Copyright 2024 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#pragma once + +#include "base/seastarx.h" + +#include + +#include +#include + +namespace security::tls { + +std::optional +validate_rules(const std::optional>& r) noexcept; + +} + +namespace security::oidc { + +std::optional +validate_principal_mapping_rule(ss::sstring const& rule); + +} diff --git a/src/v/security/config_bsl.cc b/src/v/security/config_bsl.cc index 0499f567c8d21..ec5a767a0bf4b 100644 --- a/src/v/security/config_bsl.cc +++ b/src/v/security/config_bsl.cc @@ -9,6 +9,7 @@ * by the Apache License, Version 2.0 */ +#include "security/config.h" #include "security/mtls.h" #include diff --git a/src/v/security/config_rcl.cc b/src/v/security/config_rcl.cc index d07570ec35106..b03f39a3b32f9 100644 --- a/src/v/security/config_rcl.cc +++ b/src/v/security/config_rcl.cc @@ -8,6 +8,7 @@ * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md */ +#include "security/config.h" #include "security/gssapi_principal_mapper.h" #include "security/mtls.h" #include "security/oidc_error.h" diff --git a/src/v/security/mtls.cc b/src/v/security/mtls.cc index a255e4bd0f4af..175fb9c51d063 100644 --- a/src/v/security/mtls.cc +++ b/src/v/security/mtls.cc @@ -27,9 +27,6 @@ parse_rules(std::optional> unparsed_rules); } // namespace detail -std::optional -validate_rules(const std::optional>& r) noexcept; - std::ostream& operator<<(std::ostream& os, const rule& r) { fmt::print(os, "{}", r); return os; diff --git a/src/v/security/mtls.h b/src/v/security/mtls.h index 52e1f901a74a0..231a84bc4bbc5 100644 --- a/src/v/security/mtls.h +++ b/src/v/security/mtls.h @@ -86,9 +86,6 @@ class mtls_state { std::optional _subject; }; -std::optional -validate_rules(const std::optional>& r) noexcept; - } // namespace security::tls template<> diff --git a/src/v/security/oidc_principal_mapping.h b/src/v/security/oidc_principal_mapping.h index b9598f7171f84..6de648ec632fc 100644 --- a/src/v/security/oidc_principal_mapping.h +++ b/src/v/security/oidc_principal_mapping.h @@ -40,7 +40,5 @@ class principal_mapping_rule { }; result parse_principal_mapping_rule(std::string_view); -std::optional -validate_principal_mapping_rule(ss::sstring const& rule); } // namespace security::oidc diff --git a/src/v/serde/rw/chrono.h b/src/v/serde/rw/chrono.h index 68d7a50bd8e8b..e114f25d51fbb 100644 --- a/src/v/serde/rw/chrono.h +++ b/src/v/serde/rw/chrono.h @@ -9,9 +9,9 @@ #pragma once +#include "base/type_traits.h" #include "serde/logger.h" #include "serde/rw/rw.h" -#include "utils/type_traits.h" #include @@ -116,7 +116,7 @@ void tag_invoke( template void write(iobuf&, std::chrono::time_point t) { static_assert( - utils::unsupported_type::value, + base::unsupported_type::value, "Time point serialization is risky and can have unintended " "consequences. Check with Redpanda team before fixing this."); } @@ -127,7 +127,7 @@ void read( std::chrono::time_point& t, std::size_t const /* bytes_left_limit */) { static_assert( - utils::unsupported_type::value, + base::unsupported_type::value, "Time point serialization is risky and can have unintended " "consequences. Check with Redpanda team before fixing this."); } diff --git a/src/v/serde/rw/tristate_rw.h b/src/v/serde/rw/tristate_rw.h index f1eb72adc5897..bdb685ee85078 100644 --- a/src/v/serde/rw/tristate_rw.h +++ b/src/v/serde/rw/tristate_rw.h @@ -10,7 +10,7 @@ #pragma once #include "serde/rw/rw.h" -#include "tristate.h" +#include "utils/tristate.h" namespace serde { diff --git a/src/v/serde/test/serde_test.cc b/src/v/serde/test/serde_test.cc index 97e0ec7d78bb7..99d91f8b24b13 100644 --- a/src/v/serde/test/serde_test.cc +++ b/src/v/serde/test/serde_test.cc @@ -16,7 +16,7 @@ #include "serde/rw/variant.h" #include "serde/serde.h" #include "test_utils/randoms.h" -#include "tristate.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/storage/config.h b/src/v/storage/config.h new file mode 100644 index 0000000000000..417953d32e25d --- /dev/null +++ b/src/v/storage/config.h @@ -0,0 +1,38 @@ +/* + * Copyright 2023 Redpanda Data, Inc. + * + * Use of this software is governed by the Business Source License + * included in the file licenses/BSL.md + * + * As of the Change Date specified in that file, in accordance with + * the Business Source License, use of this software will be governed + * by the Apache License, Version 2.0 + */ +#pragma once + +#include "base/seastarx.h" +#include "base/units.h" + +#include + +#include + +namespace storage { + +inline constexpr const size_t segment_appender_fallocation_alignment = 4_KiB; + +/** Validator for fallocation step configuration setting */ +inline std::optional +validate_fallocation_step(const size_t& value) { + if (value % segment_appender_fallocation_alignment != 0) { + return "Fallocation step must be multiple of 4096"; + } else if (value < segment_appender_fallocation_alignment) { + return "Fallocation step must be at least 4 KiB (4096)"; + } else if (value > 1_GiB) { + return "Fallocation step can't be larger than 1 GiB (1073741824)"; + } else { + return std::nullopt; + } +} + +} // namespace storage diff --git a/src/v/storage/ntp_config.h b/src/v/storage/ntp_config.h index 85fb54a2e5b31..7a69a0e07011a 100644 --- a/src/v/storage/ntp_config.h +++ b/src/v/storage/ntp_config.h @@ -15,7 +15,7 @@ #include "model/metadata.h" #include "model/namespace.h" #include "ssx/sformat.h" -#include "tristate.h" +#include "utils/tristate.h" #include diff --git a/src/v/storage/segment_appender.h b/src/v/storage/segment_appender.h index 446680f350253..3dd8a875d9f39 100644 --- a/src/v/storage/segment_appender.h +++ b/src/v/storage/segment_appender.h @@ -19,6 +19,7 @@ #include "container/intrusive_list_helpers.h" #include "model/record.h" #include "ssx/semaphore.h" +#include "storage/config.h" #include "storage/fwd.h" #include "storage/segment_appender_chunk.h" #include "storage/storage_resources.h" @@ -53,7 +54,8 @@ class segment_appender { public: using chunk = segment_appender_chunk; - static constexpr const size_t fallocation_alignment = 4_KiB; + static constexpr const size_t fallocation_alignment + = segment_appender_fallocation_alignment; static constexpr const size_t write_behind_memory = 1_MiB; struct options { @@ -131,20 +133,6 @@ class segment_appender { void set_callbacks(callbacks* callbacks) { _callbacks = callbacks; } - /** Validator for fallocation step configuration setting */ - static std::optional - validate_fallocation_step(const size_t& value) { - if (value % segment_appender::fallocation_alignment != 0) { - return "Fallocation step must be multiple of 4096"; - } else if (value < segment_appender::fallocation_alignment) { - return "Fallocation step must be at least 4 KiB (4096)"; - } else if (value > 1_GiB) { - return "Fallocation step can't be larger than 1 GiB (1073741824)"; - } else { - return std::nullopt; - } - } - constexpr ss::io_priority_class get_priority_class() const { return _opts.priority; } diff --git a/src/v/storage/types.h b/src/v/storage/types.h index bcad9850cda93..ab91de341693e 100644 --- a/src/v/storage/types.h +++ b/src/v/storage/types.h @@ -21,7 +21,7 @@ #include "storage/fwd.h" #include "storage/key_offset_map.h" #include "storage/scoped_file_tracker.h" -#include "tristate.h" +#include "utils/tristate.h" #include #include //io_priority diff --git a/src/v/test_utils/randoms.h b/src/v/test_utils/randoms.h index 2964afdb8ed6e..6e35f31f0efb2 100644 --- a/src/v/test_utils/randoms.h +++ b/src/v/test_utils/randoms.h @@ -14,7 +14,7 @@ #include "cluster/producer_state.h" #include "container/fragmented_vector.h" #include "random/generators.h" -#include "tristate.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/transform/rpc/client.cc b/src/v/transform/rpc/client.cc index 2e7e5ad969bd0..46a3aeea98fd1 100644 --- a/src/v/transform/rpc/client.cc +++ b/src/v/transform/rpc/client.cc @@ -11,6 +11,7 @@ #include "transform/rpc/client.h" +#include "base/type_traits.h" #include "cluster/errc.h" #include "cluster/types.h" #include "config/configuration.h" @@ -28,7 +29,6 @@ #include "transform/rpc/deps.h" #include "transform/rpc/rpc_service.h" #include "transform/rpc/serde.h" -#include "utils/type_traits.h" #include #include @@ -146,7 +146,7 @@ std::invoke_result_t retry_with_backoff(Func func, ss::abort_source* as) { ec = r.ec; } else { static_assert( - utils::unsupported_type::value, + base::unsupported_type::value, "unsupported response type"); } switch (ec) { diff --git a/src/v/transform/transform_manager.cc b/src/v/transform/transform_manager.cc index fecae8d17b6a3..1eee9f5d1d0ac 100644 --- a/src/v/transform/transform_manager.cc +++ b/src/v/transform/transform_manager.cc @@ -32,6 +32,8 @@ #include #include +#include +#include #include #include diff --git a/src/v/utils/hdr_hist.h b/src/v/utils/hdr_hist.h index 60581a9255f88..2d77855dd75e3 100644 --- a/src/v/utils/hdr_hist.h +++ b/src/v/utils/hdr_hist.h @@ -18,7 +18,7 @@ // vectorized types. needed comment to allow clang-format // header sorting to not resort cstdint #include "base/seastarx.h" -#include "static_deleter_fn.h" +#include "utils/static_deleter_fn.h" #include #include diff --git a/src/v/static_deleter_fn.h b/src/v/utils/static_deleter_fn.h similarity index 100% rename from src/v/static_deleter_fn.h rename to src/v/utils/static_deleter_fn.h diff --git a/src/v/utils/tests/tristate_test.cc b/src/v/utils/tests/tristate_test.cc index 2b13b97fa0519..f85ac2c232800 100644 --- a/src/v/utils/tests/tristate_test.cc +++ b/src/v/utils/tests/tristate_test.cc @@ -7,7 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0 -#include "tristate.h" +#include "utils/tristate.h" #include diff --git a/src/v/tristate.h b/src/v/utils/tristate.h similarity index 100% rename from src/v/tristate.h rename to src/v/utils/tristate.h diff --git a/src/v/wasm/engine_probe.cc b/src/v/wasm/engine_probe.cc index c3d94cc6056e7..0521e9967fea2 100644 --- a/src/v/wasm/engine_probe.cc +++ b/src/v/wasm/engine_probe.cc @@ -11,6 +11,7 @@ #include "engine_probe.h" +#include "base/vassert.h" #include "metrics/metrics.h" #include "prometheus/prometheus_sanitize.h" diff --git a/src/v/wasm/engine_probe.h b/src/v/wasm/engine_probe.h index 199eab9b42da1..412df693bbf29 100644 --- a/src/v/wasm/engine_probe.h +++ b/src/v/wasm/engine_probe.h @@ -16,6 +16,7 @@ #include #include +#include #include #include diff --git a/src/v/wasm/ffi.h b/src/v/wasm/ffi.h index 2994d10af9626..d902eb4f9c8eb 100644 --- a/src/v/wasm/ffi.h +++ b/src/v/wasm/ffi.h @@ -11,12 +11,12 @@ #pragma once +#include "base/type_traits.h" #include "base/vassert.h" #include "bytes/bytes.h" #include "bytes/iobuf.h" #include "reflection/type_traits.h" #include "utils/named_type.h" -#include "utils/type_traits.h" #include #include @@ -244,7 +244,7 @@ void transform_type(std::vector& types) { } else if constexpr (ss::is_future::value) { transform_type(types); } else { - static_assert(utils::unsupported_type::value, "Unknown type"); + static_assert(base::unsupported_type::value, "Unknown type"); } } @@ -293,7 +293,7 @@ std::tuple extract_parameter( mem, raw_params, idx); return std::tuple(underlying); } else { - static_assert(utils::unsupported_type::value, "Unknown type"); + static_assert(base::unsupported_type::value, "Unknown type"); } } diff --git a/src/v/wasm/tests/wasm_probe_test.cc b/src/v/wasm/tests/wasm_probe_test.cc index 3d63f404bacbc..6870aca9fd7d1 100644 --- a/src/v/wasm/tests/wasm_probe_test.cc +++ b/src/v/wasm/tests/wasm_probe_test.cc @@ -9,10 +9,10 @@ * by the Apache License, Version 2.0 */ +#include "base/type_traits.h" #include "base/units.h" #include "gmock/gmock.h" #include "metrics/metrics.h" -#include "utils/type_traits.h" #include "wasm/logger.h" #include @@ -55,7 +55,7 @@ std::optional find_metric_value( } else if constexpr (std::is_same_v) { return sample.ui(); } else { - static_assert(utils::unsupported_type::value, "unsupported type"); + static_assert(base::unsupported_type::value, "unsupported type"); } } diff --git a/src/v/wasm/wasmtime.cc b/src/v/wasm/wasmtime.cc index 9736ab9c45ebd..19677e33a0754 100644 --- a/src/v/wasm/wasmtime.cc +++ b/src/v/wasm/wasmtime.cc @@ -11,7 +11,9 @@ #include "wasmtime.h" #include "allocator.h" +#include "base/type_traits.h" #include "base/vassert.h" +#include "base/vlog.h" #include "engine_probe.h" #include "ffi.h" #include "logger.h" @@ -25,7 +27,7 @@ #include "storage/parser_utils.h" #include "transform_module.h" #include "utils/human.h" -#include "utils/type_traits.h" +#include "utils/to_string.h" #include "wasi.h" #include "wasm/api.h" #include "wasm/errc.h" @@ -49,6 +51,7 @@ #include #include +#include #include #include @@ -341,7 +344,7 @@ wasmtime_val_t convert_to_wasmtime(T value) { .kind = WASMTIME_I32, .of = {.i32 = static_cast(value)}}; } else { static_assert( - utils::unsupported_type::value, "Unsupported wasm result type"); + base::unsupported_type::value, "Unsupported wasm result type"); } } @@ -549,7 +552,7 @@ class wasmtime_engine : public engine { return &_sr_module; } else { static_assert( - utils::unsupported_type::value, "unsupported module"); + base::unsupported_type::value, "unsupported module"); } }