From ad7f522a8e30aefd2918d0265a9d1c3b3abf2535 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 13:42:33 -0700 Subject: [PATCH 01/18] chore: add missing header files Subsequent commits clean up header dependencies, but some break. This adds in the missing headers that would otherwise cause subsequent commits to not compile. Signed-off-by: Noah Watkins --- src/v/cloud_storage/partition_manifest.cc | 1 + src/v/cloud_storage_clients/configuration.h | 1 + src/v/http/tests/http_client_test.cc | 1 + src/v/net/include/net/connection.h | 1 + src/v/raft/coordinated_recovery_throttle.cc | 1 + src/v/security/authorizer.cc | 1 + src/v/transform/transform_manager.cc | 2 ++ src/v/wasm/engine_probe.cc | 1 + src/v/wasm/engine_probe.h | 1 + src/v/wasm/wasmtime.cc | 4 ++++ 10 files changed, 14 insertions(+) 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/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/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/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/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/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/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/wasmtime.cc b/src/v/wasm/wasmtime.cc index 9736ab9c45ebd..8c95b2b4eae4d 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,6 +27,7 @@ #include "storage/parser_utils.h" #include "transform_module.h" #include "utils/human.h" +#include "utils/to_string.h" #include "utils/type_traits.h" #include "wasi.h" #include "wasm/api.h" @@ -49,6 +52,7 @@ #include #include +#include #include #include From 98bc4584d00ea47e65bc7fcecf600dcf4ad8a291 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Thu, 23 May 2024 21:15:47 -0700 Subject: [PATCH 02/18] base: move type traits from utils Signed-off-by: Noah Watkins --- src/v/{utils => base/include/base}/type_traits.h | 0 src/v/container/tests/bench_utils.h | 2 +- src/v/container/tests/vector_bench.cc | 2 +- src/v/kafka/server/handlers/describe_configs.cc | 2 +- src/v/redpanda/admin/server.h | 2 +- src/v/serde/rw/chrono.h | 2 +- src/v/transform/rpc/client.cc | 2 +- src/v/wasm/ffi.h | 2 +- src/v/wasm/tests/wasm_probe_test.cc | 2 +- src/v/wasm/wasmtime.cc | 1 - 10 files changed, 8 insertions(+), 9 deletions(-) rename src/v/{utils => base/include/base}/type_traits.h (100%) diff --git a/src/v/utils/type_traits.h b/src/v/base/include/base/type_traits.h similarity index 100% rename from src/v/utils/type_traits.h rename to src/v/base/include/base/type_traits.h diff --git a/src/v/container/tests/bench_utils.h b/src/v/container/tests/bench_utils.h index 91025d04a141a..9d4d6c5525e17 100644 --- a/src/v/container/tests/bench_utils.h +++ b/src/v/container/tests/bench_utils.h @@ -11,7 +11,7 @@ #pragma once #include "random/generators.h" #include "utils/functional.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include diff --git a/src/v/container/tests/vector_bench.cc b/src/v/container/tests/vector_bench.cc index bb6291f762e01..c0c2664107043 100644 --- a/src/v/container/tests/vector_bench.cc +++ b/src/v/container/tests/vector_bench.cc @@ -13,7 +13,7 @@ #include "container/tests/bench_utils.h" #include "random/generators.h" #include "utils/functional.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include diff --git a/src/v/kafka/server/handlers/describe_configs.cc b/src/v/kafka/server/handlers/describe_configs.cc index 68ff3476d6b87..8f36588a7605f 100644 --- a/src/v/kafka/server/handlers/describe_configs.cc +++ b/src/v/kafka/server/handlers/describe_configs.cc @@ -28,7 +28,7 @@ #include "reflection/type_traits.h" #include "security/acl.h" #include "ssx/sformat.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include #include diff --git a/src/v/redpanda/admin/server.h b/src/v/redpanda/admin/server.h index a5ffb9bcbd5b0..75b57f557b253 100644 --- a/src/v/redpanda/admin/server.h +++ b/src/v/redpanda/admin/server.h @@ -32,7 +32,7 @@ #include "security/types.h" #include "storage/node.h" #include "transform/fwd.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include #include diff --git a/src/v/serde/rw/chrono.h b/src/v/serde/rw/chrono.h index 68d7a50bd8e8b..c3bf484464ebb 100644 --- a/src/v/serde/rw/chrono.h +++ b/src/v/serde/rw/chrono.h @@ -11,7 +11,7 @@ #include "serde/logger.h" #include "serde/rw/rw.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include diff --git a/src/v/transform/rpc/client.cc b/src/v/transform/rpc/client.cc index 2e7e5ad969bd0..3e4a0f3c83574 100644 --- a/src/v/transform/rpc/client.cc +++ b/src/v/transform/rpc/client.cc @@ -28,7 +28,7 @@ #include "transform/rpc/deps.h" #include "transform/rpc/rpc_service.h" #include "transform/rpc/serde.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include #include diff --git a/src/v/wasm/ffi.h b/src/v/wasm/ffi.h index 2994d10af9626..8628e0075f22c 100644 --- a/src/v/wasm/ffi.h +++ b/src/v/wasm/ffi.h @@ -16,7 +16,7 @@ #include "bytes/iobuf.h" #include "reflection/type_traits.h" #include "utils/named_type.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include #include diff --git a/src/v/wasm/tests/wasm_probe_test.cc b/src/v/wasm/tests/wasm_probe_test.cc index 3d63f404bacbc..da8ef1acfc4d9 100644 --- a/src/v/wasm/tests/wasm_probe_test.cc +++ b/src/v/wasm/tests/wasm_probe_test.cc @@ -12,7 +12,7 @@ #include "base/units.h" #include "gmock/gmock.h" #include "metrics/metrics.h" -#include "utils/type_traits.h" +#include "base/type_traits.h" #include "wasm/logger.h" #include diff --git a/src/v/wasm/wasmtime.cc b/src/v/wasm/wasmtime.cc index 8c95b2b4eae4d..233b31aaf27ec 100644 --- a/src/v/wasm/wasmtime.cc +++ b/src/v/wasm/wasmtime.cc @@ -28,7 +28,6 @@ #include "transform_module.h" #include "utils/human.h" #include "utils/to_string.h" -#include "utils/type_traits.h" #include "wasi.h" #include "wasm/api.h" #include "wasm/errc.h" From 912c6ec9fa35d338025acea8523e9dd2598b4170 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Thu, 23 May 2024 21:21:00 -0700 Subject: [PATCH 03/18] base: move specialization type trait to base Signed-off-by: Noah Watkins --- src/v/base/include/base/type_traits.h | 12 ++++++++++++ src/v/reflection/type_traits.h | 9 +-------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/v/base/include/base/type_traits.h b/src/v/base/include/base/type_traits.h index ed8732c15abd7..2939ef754a36a 100644 --- a/src/v/base/include/base/type_traits.h +++ b/src/v/base/include/base/type_traits.h @@ -58,3 +58,15 @@ template struct unsupported_value : std::false_type {}; } // namespace utils + +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 diff --git a/src/v/reflection/type_traits.h b/src/v/reflection/type_traits.h index c4d09212a2b21..cde09c9334f46 100644 --- a/src/v/reflection/type_traits.h +++ b/src/v/reflection/type_traits.h @@ -12,6 +12,7 @@ #pragma once #include "base/seastarx.h" +#include "base/type_traits.h" #include "container/fragmented_vector.h" #include "tristate.h" #include "utils/named_type.h" @@ -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> From 7ccb68f3aa003dfcb0b681adaae88c24489bee19 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Thu, 23 May 2024 21:22:51 -0700 Subject: [PATCH 04/18] base: move is_optional concept to base Signed-off-by: Noah Watkins --- src/v/base/include/base/type_traits.h | 8 ++++++++ src/v/reflection/type_traits.h | 3 --- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/v/base/include/base/type_traits.h b/src/v/base/include/base/type_traits.h index 2939ef754a36a..9888f5d91397b 100644 --- a/src/v/base/include/base/type_traits.h +++ b/src/v/base/include/base/type_traits.h @@ -11,6 +11,7 @@ #pragma once +#include #include namespace utils { @@ -70,3 +71,10 @@ 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/reflection/type_traits.h b/src/v/reflection/type_traits.h index cde09c9334f46..8295e09ef1274 100644 --- a/src/v/reflection/type_traits.h +++ b/src/v/reflection/type_traits.h @@ -69,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; From 57615df6fdc0eefe49a8e951095cb0ce61493549 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Thu, 23 May 2024 21:28:22 -0700 Subject: [PATCH 05/18] config: remove dependence on reflection library Signed-off-by: Noah Watkins --- src/v/config/property.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/v/config/property.h b/src/v/config/property.h index 619281684dcae..d22293e8d6347 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 From 45124052666469f90f9c9c8c761d9a4f7e44f21c Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Wed, 22 May 2024 22:46:01 -0700 Subject: [PATCH 06/18] utils: move static delete fn to utils Signed-off-by: Noah Watkins --- src/v/compression/include/compression/stream_zstd.h | 2 +- src/v/compression/internal/lz4_frame_compressor.cc | 2 +- src/v/utils/hdr_hist.h | 2 +- src/v/{ => utils}/static_deleter_fn.h | 0 4 files changed, 3 insertions(+), 3 deletions(-) rename src/v/{ => utils}/static_deleter_fn.h (100%) 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/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 From 66d1f71e09247251d2109a8015bf07d529597a77 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Thu, 23 May 2024 21:38:04 -0700 Subject: [PATCH 07/18] config: remove unused headers Signed-off-by: Noah Watkins --- src/v/config/broker_authn_endpoint.cc | 1 - src/v/config/configuration.cc | 1 - src/v/config/rest_authn_endpoint.cc | 1 - 3 files changed, 3 deletions(-) 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..e68564bffc4d5 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -16,7 +16,6 @@ #include "config/validators.h" #include "model/metadata.h" #include "model/namespace.h" -#include "pandaproxy/schema_registry/schema_id_validation.h" #include "security/gssapi_principal_mapper.h" #include "security/mtls.h" #include "security/oidc_principal_mapping.h" 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" From d2e89d6b392559acb32a47c3c562b752cebc15c4 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 11:20:08 -0700 Subject: [PATCH 08/18] config: use full header path Signed-off-by: Noah Watkins --- src/v/config/throughput_control_group.cc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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" From f30b0fb4288a0c2605d49f0769d350800d3b471c Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 11:20:19 -0700 Subject: [PATCH 09/18] metrics: remove unused header Signed-off-by: Noah Watkins --- src/v/metrics/metrics.h | 1 - 1 file changed, 1 deletion(-) 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" From 9721d1915f64e2a4e41b9d2e13532a79175ea059 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 13:53:43 -0700 Subject: [PATCH 10/18] metrics: add missing copyright header Signed-off-by: Noah Watkins --- src/v/metrics/metrics.cc | 13 +++++++++++++ src/v/rpc/connection_set.cc | 1 + 2 files changed, 14 insertions(+) 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/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 { From 5b0a9b6e4c8c779cd556d352214e5f9171290b26 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 11:21:35 -0700 Subject: [PATCH 11/18] utils: move tristate into utils Signed-off-by: Noah Watkins --- src/v/cluster/tests/serialization_rt_test.cc | 2 +- src/v/cluster/tx_topic_manager.cc | 2 +- src/v/cluster/types.cc | 2 +- src/v/cluster/types.h | 2 +- src/v/kafka/server/snc_quota_manager.cc | 2 +- src/v/model/adl_serde.h | 2 +- src/v/pandaproxy/json/requests/produce.h | 2 +- src/v/reflection/type_traits.h | 2 +- src/v/serde/rw/tristate_rw.h | 2 +- src/v/serde/test/serde_test.cc | 2 +- src/v/storage/ntp_config.h | 2 +- src/v/storage/types.h | 2 +- src/v/test_utils/randoms.h | 2 +- src/v/utils/tests/tristate_test.cc | 2 +- src/v/{ => utils}/tristate.h | 0 15 files changed, 14 insertions(+), 14 deletions(-) rename src/v/{ => utils}/tristate.h (100%) 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..294b5b9bdbe5c 100644 --- a/src/v/cluster/types.cc +++ b/src/v/cluster/types.cc @@ -17,7 +17,7 @@ #include "model/timestamp.h" #include "reflection/adl.h" #include "security/acl.h" -#include "tristate.h" +#include "utils/tristate.h" #include "utils/to_string.h" #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/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/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/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/reflection/type_traits.h b/src/v/reflection/type_traits.h index 8295e09ef1274..6f18664a228b8 100644 --- a/src/v/reflection/type_traits.h +++ b/src/v/reflection/type_traits.h @@ -14,7 +14,7 @@ #include "base/seastarx.h" #include "base/type_traits.h" #include "container/fragmented_vector.h" -#include "tristate.h" +#include "utils/tristate.h" #include "utils/named_type.h" #include 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/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/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/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 From cf7eb7740bec8cd24a8928013f0fc7fd5009f879 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 14:37:09 -0700 Subject: [PATCH 12/18] config: remove bottomless bucket max width It's nice that the bottomless bucket exposes a max width that can be used in configuration system, but its not scalable to do this for everything in the tree otherwise the configuration subsystem is going to have too many dependencies. In this case the max width seems to be about type compatibility in that its not clear that choosing max int would ever be a reasonable choice anyway. So external configuration facing validation is just different and can live in config system. Signed-off-by: Noah Watkins --- src/v/config/configuration.cc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index e68564bffc4d5..563c3be9f4f30 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -23,8 +23,8 @@ #include "ssx/sformat.h" #include "storage/chunk_cache.h" #include "storage/segment_appender.h" -#include "utils/bottomless_token_bucket.h" +#include #include #include @@ -2978,7 +2978,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", From 8ee2081ebba6e6695fade33a237e3dac6ab39493 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 14:39:42 -0700 Subject: [PATCH 13/18] storage: factor out config specific storage bits This follows along the path that security::config library has taken which is to create a dependency-free library in the subsystem that exports the bits that are shared between the subsystem and the configuration. Signed-off-by: Noah Watkins --- src/v/config/configuration.cc | 5 ++--- src/v/storage/config.h | 38 ++++++++++++++++++++++++++++++++ src/v/storage/segment_appender.h | 18 +++------------ 3 files changed, 43 insertions(+), 18 deletions(-) create mode 100644 src/v/storage/config.h diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 563c3be9f4f30..268f569af677e 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -21,8 +21,7 @@ #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 "storage/config.h" #include #include @@ -1170,7 +1169,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", 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/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; } From fd6c0fd5c2f9e15194714b755434790f17aca794 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 12:16:27 -0700 Subject: [PATCH 14/18] security: create config-specific header The security_config library contains specific bits that are shared between security module and configuration. However, the header declarations for those bits are in headers that aren't easily included in configuration system because of all the additional things brought along. So this commit splits these out into a specific config.h header that is intended to be included by configuration system. Signed-off-by: Noah Watkins --- src/v/config/configuration.cc | 3 +-- src/v/security/config.h | 32 +++++++++++++++++++++++++ src/v/security/mtls.cc | 3 --- src/v/security/mtls.h | 3 --- src/v/security/oidc_principal_mapping.h | 2 -- 5 files changed, 33 insertions(+), 10 deletions(-) create mode 100644 src/v/security/config.h diff --git a/src/v/config/configuration.cc b/src/v/config/configuration.cc index 268f569af677e..0de6b13a781fd 100644 --- a/src/v/config/configuration.cc +++ b/src/v/config/configuration.cc @@ -16,9 +16,8 @@ #include "config/validators.h" #include "model/metadata.h" #include "model/namespace.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/config.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/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 From 962d1655a6a8f067df31bcfb39b6684062e43d8a Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 16:32:13 -0700 Subject: [PATCH 15/18] chore: apply clang format Signed-off-by: Noah Watkins --- src/v/cluster/types.cc | 2 +- src/v/container/tests/bench_utils.h | 2 +- src/v/container/tests/vector_bench.cc | 2 +- src/v/kafka/server/handlers/describe_configs.cc | 2 +- src/v/redpanda/admin/server.h | 2 +- src/v/reflection/type_traits.h | 2 +- src/v/serde/rw/chrono.h | 2 +- src/v/transform/rpc/client.cc | 2 +- src/v/wasm/ffi.h | 2 +- src/v/wasm/tests/wasm_probe_test.cc | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/v/cluster/types.cc b/src/v/cluster/types.cc index 294b5b9bdbe5c..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 "utils/tristate.h" #include "utils/to_string.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/container/tests/bench_utils.h b/src/v/container/tests/bench_utils.h index 9d4d6c5525e17..cc3fb589dcb65 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 "base/type_traits.h" #include diff --git a/src/v/container/tests/vector_bench.cc b/src/v/container/tests/vector_bench.cc index c0c2664107043..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 "base/type_traits.h" #include diff --git a/src/v/kafka/server/handlers/describe_configs.cc b/src/v/kafka/server/handlers/describe_configs.cc index 8f36588a7605f..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 "base/type_traits.h" #include #include diff --git a/src/v/redpanda/admin/server.h b/src/v/redpanda/admin/server.h index 75b57f557b253..122ebde04a5d1 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 "base/type_traits.h" #include #include diff --git a/src/v/reflection/type_traits.h b/src/v/reflection/type_traits.h index 6f18664a228b8..8eb7ff1f4e267 100644 --- a/src/v/reflection/type_traits.h +++ b/src/v/reflection/type_traits.h @@ -14,8 +14,8 @@ #include "base/seastarx.h" #include "base/type_traits.h" #include "container/fragmented_vector.h" -#include "utils/tristate.h" #include "utils/named_type.h" +#include "utils/tristate.h" #include #include diff --git a/src/v/serde/rw/chrono.h b/src/v/serde/rw/chrono.h index c3bf484464ebb..7212533c7f8e1 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 "base/type_traits.h" #include diff --git a/src/v/transform/rpc/client.cc b/src/v/transform/rpc/client.cc index 3e4a0f3c83574..8cbd295f05a8d 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 "base/type_traits.h" #include #include diff --git a/src/v/wasm/ffi.h b/src/v/wasm/ffi.h index 8628e0075f22c..a9bf40120144e 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 "base/type_traits.h" #include #include diff --git a/src/v/wasm/tests/wasm_probe_test.cc b/src/v/wasm/tests/wasm_probe_test.cc index da8ef1acfc4d9..6f56d1080b5cc 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 "base/type_traits.h" #include "wasm/logger.h" #include From e6cb520b2c86926567b0495a69d6a3d302176679 Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 22:05:05 -0700 Subject: [PATCH 16/18] rpc: add missing header Signed-off-by: Noah Watkins --- src/v/rpc/reconnect_transport.cc | 1 + 1 file changed, 1 insertion(+) 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 From b2392355caed75c5d31b27cf35395cb663e285dc Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Fri, 24 May 2024 22:13:28 -0700 Subject: [PATCH 17/18] base: remove utils namespace Signed-off-by: Noah Watkins --- src/v/base/include/base/type_traits.h | 8 ++++---- src/v/config/property.h | 2 +- src/v/container/tests/bench_utils.h | 2 +- .../server/handlers/configs/config_response_utils.cc | 2 +- src/v/redpanda/admin/server.h | 2 +- src/v/security/acl.h | 2 +- src/v/security/audit/schemas/utils.h | 2 +- src/v/serde/rw/chrono.h | 4 ++-- src/v/transform/rpc/client.cc | 2 +- src/v/wasm/ffi.h | 4 ++-- src/v/wasm/tests/wasm_probe_test.cc | 2 +- src/v/wasm/wasmtime.cc | 4 ++-- 12 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/v/base/include/base/type_traits.h b/src/v/base/include/base/type_traits.h index 9888f5d91397b..c48aff6f761cd 100644 --- a/src/v/base/include/base/type_traits.h +++ b/src/v/base/include/base/type_traits.h @@ -14,7 +14,7 @@ #include #include -namespace utils { +namespace base { /** * A utility for statically asserting false. @@ -27,7 +27,7 @@ namespace utils { * if constexpr (...) { * // ... * } else { - * static_assert(utils::unsupported_type::value, "unsupported type"); + * static_assert(base::unsupported_type::value, "unsupported type"); * } * } * @@ -51,14 +51,14 @@ 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 { diff --git a/src/v/config/property.h b/src/v/config/property.h index d22293e8d6347..25ea581b4e5f8 100644 --- a/src/v/config/property.h +++ b/src/v/config/property.h @@ -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/container/tests/bench_utils.h b/src/v/container/tests/bench_utils.h index cc3fb589dcb65..736dd5bffe5d8 100644 --- a/src/v/container/tests/bench_utils.h +++ b/src/v/container/tests/bench_utils.h @@ -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/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/redpanda/admin/server.h b/src/v/redpanda/admin/server.h index 122ebde04a5d1..868bdb2fb7085 100644 --- a/src/v/redpanda/admin/server.h +++ b/src/v/redpanda/admin/server.h @@ -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/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/serde/rw/chrono.h b/src/v/serde/rw/chrono.h index 7212533c7f8e1..e114f25d51fbb 100644 --- a/src/v/serde/rw/chrono.h +++ b/src/v/serde/rw/chrono.h @@ -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/transform/rpc/client.cc b/src/v/transform/rpc/client.cc index 8cbd295f05a8d..46a3aeea98fd1 100644 --- a/src/v/transform/rpc/client.cc +++ b/src/v/transform/rpc/client.cc @@ -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/wasm/ffi.h b/src/v/wasm/ffi.h index a9bf40120144e..d902eb4f9c8eb 100644 --- a/src/v/wasm/ffi.h +++ b/src/v/wasm/ffi.h @@ -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 6f56d1080b5cc..6870aca9fd7d1 100644 --- a/src/v/wasm/tests/wasm_probe_test.cc +++ b/src/v/wasm/tests/wasm_probe_test.cc @@ -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 233b31aaf27ec..19677e33a0754 100644 --- a/src/v/wasm/wasmtime.cc +++ b/src/v/wasm/wasmtime.cc @@ -344,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"); } } @@ -552,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"); } } From 1f74fc39f95015c688ac7a5d87f98d9a8e2b3bce Mon Sep 17 00:00:00 2001 From: Noah Watkins Date: Sat, 25 May 2024 09:25:57 -0700 Subject: [PATCH 18/18] security: add missing header Signed-off-by: Noah Watkins --- src/v/security/config_bsl.cc | 1 + src/v/security/config_rcl.cc | 1 + 2 files changed, 2 insertions(+) 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"