diff --git a/src/v/cloud_io/tests/s3_imposter.cc b/src/v/cloud_io/tests/s3_imposter.cc index 3ab278b4e4e1f..b1584cd083ca6 100644 --- a/src/v/cloud_io/tests/s3_imposter.cc +++ b/src/v/cloud_io/tests/s3_imposter.cc @@ -15,6 +15,7 @@ #include "bytes/iobuf_parser.h" #include "cloud_storage_clients/client.h" #include "cloud_storage_clients/client_probe.h" +#include "http/tests/utils.h" #include "test_utils/async.h" #include "test_utils/test_macros.h" @@ -501,11 +502,11 @@ void s3_imposter_fixture::set_routes( using reply = ss::http::reply; _content_handler = ss::make_shared( expectations, *this, std::move(headers_to_store)); - _handler = std::make_unique( - [this](const_req req, reply& repl) { + _handler = std::make_unique( + [this](const_req req, reply& repl, [[maybe_unused]] ss::sstring& type) { return _content_handler->handle(req, repl); }, - "txt"); + "xml"); r.add_default_handler(_handler.get()); } diff --git a/src/v/cloud_storage/tests/anomalies_detector_test.cc b/src/v/cloud_storage/tests/anomalies_detector_test.cc index 24bdce88cebe1..f7671377bada5 100644 --- a/src/v/cloud_storage/tests/anomalies_detector_test.cc +++ b/src/v/cloud_storage/tests/anomalies_detector_test.cc @@ -424,7 +424,10 @@ class bucket_view_fixture : public http_imposter_fixture { .request(full_path) .with_method(ss::httpd::operation_type::GET) .then_reply_with( - not_found_response, ss::http::reply::status_type::not_found); + not_found_response, + std::vector>{ + {"Content-Type", "application/xml"}}, + ss::http::reply::status_type::not_found); } void parse_manifests( diff --git a/src/v/cloud_storage_clients/abs_client.cc b/src/v/cloud_storage_clients/abs_client.cc index f5fc44f1878ad..32c6fc27ff3fc 100644 --- a/src/v/cloud_storage_clients/abs_client.cc +++ b/src/v/cloud_storage_clients/abs_client.cc @@ -50,7 +50,6 @@ constexpr boost::beast::string_view delete_snapshot_name constexpr boost::beast::string_view is_hns_enabled_name = "x-ms-is-hns-enabled"; constexpr boost::beast::string_view delete_snapshot_value = "include"; constexpr boost::beast::string_view error_code_name = "x-ms-error-code"; -constexpr boost::beast::string_view content_type_name = "Content-Type"; constexpr boost::beast::string_view expiry_option_name = "x-ms-expiry-option"; constexpr boost::beast::string_view expiry_option_value = "RelativeToNow"; constexpr boost::beast::string_view expiry_time_name = "x-ms-expiry-time"; @@ -74,23 +73,6 @@ bool is_error_retryable( namespace cloud_storage_clients { -enum class response_content_type : int8_t { unknown, xml, json }; - -static response_content_type -get_response_content_type(const http::client::response_header& headers) { - if (auto iter = headers.find(content_type_name); iter != headers.end()) { - if (iter->value().find("json") != std::string_view::npos) { - return response_content_type::json; - } - - if (iter->value().find("xml") != std::string_view::npos) { - return response_content_type::xml; - } - } - - return response_content_type::unknown; -} - static abs_rest_error_response parse_xml_rest_error_response(boost::beast::http::status result, iobuf buf) { using namespace cloud_storage_clients; @@ -628,7 +610,7 @@ ss::future abs_client::do_get_object( response_stream->get_headers()); } - const auto content_type = get_response_content_type( + const auto content_type = util::get_response_content_type( response_stream->get_headers()); auto buf = co_await util::drain_response_stream( std::move(response_stream)); @@ -686,7 +668,7 @@ ss::future<> abs_client::do_put_object( if (const auto is_no_content_and_accepted = accept_no_content && status == no_content; status != created && !is_no_content_and_accepted) { - const auto content_type = get_response_content_type( + const auto content_type = util::get_response_content_type( response_stream->get_headers()); auto buf = co_await util::drain_response_stream( std::move(response_stream)); @@ -803,7 +785,7 @@ ss::future<> abs_client::do_delete_object( const auto status = response_stream->get_headers().result(); if (status != boost::beast::http::status::accepted) { - const auto content_type = get_response_content_type( + const auto content_type = util::get_response_content_type( response_stream->get_headers()); auto buf = co_await util::drain_response_stream( std::move(response_stream)); @@ -884,7 +866,7 @@ ss::future abs_client::do_list_objects( const auto status = response_stream->get_headers().result(); if (status != boost::beast::http::status::ok) { - const auto content_type = get_response_content_type( + const auto content_type = util::get_response_content_type( response_stream->get_headers()); iobuf buf = co_await util::drain_response_stream(response_stream); throw parse_rest_error_response(content_type, status, std::move(buf)); @@ -1047,7 +1029,7 @@ ss::future<> abs_client::do_delete_file( if ( status != boost::beast::http::status::accepted && status != boost::beast::http::status::ok) { - const auto content_type = get_response_content_type( + const auto content_type = util::get_response_content_type( response_stream->get_headers()); auto buf = co_await util::drain_response_stream( std::move(response_stream)); diff --git a/src/v/cloud_storage_clients/s3_client.cc b/src/v/cloud_storage_clients/s3_client.cc index 6adf74e0149b3..45a4ce5e7290f 100644 --- a/src/v/cloud_storage_clients/s3_client.cc +++ b/src/v/cloud_storage_clients/s3_client.cc @@ -412,36 +412,47 @@ status_to_error_code(boost::beast::http::status s) { return cloud_storage_clients::s3_error_code::_unknown; } -template -ss::future -parse_rest_error_response(boost::beast::http::status result, iobuf&& buf) { - if (buf.empty()) { - // AWS errors occasionally come with an empty body - // (See https://github.com/redpanda-data/redpanda/issues/6061) - // Without a proper code, we treat it as a hint to gracefully retry - // (synthesize the slow_down code). - rest_error_response err( - fmt::format("{}", status_to_error_code(result)), - fmt::format("Empty error response, status code {}", result), - "", - ""); - return ss::make_exception_future(err); - } else { - try { - auto resp = util::iobuf_to_ptree(std::move(buf), s3_log); - constexpr const char* empty = ""; - auto code = resp.get("Error.Code", empty); - auto msg = resp.get("Error.Message", empty); - auto rid = resp.get("Error.RequestId", empty); - auto res = resp.get("Error.Resource", empty); - rest_error_response err(code, msg, rid, res); - return ss::make_exception_future(err); - } catch (...) { - vlog( - s3_log.error, "!!error parse error {}", std::current_exception()); - throw; +rest_error_response parse_xml_rest_error_response(iobuf&& buf) { + try { + auto resp = util::iobuf_to_ptree(std::move(buf), s3_log); + constexpr const char* empty = ""; + auto code = resp.get("Error.Code", empty); + auto msg = resp.get("Error.Message", empty); + auto rid = resp.get("Error.RequestId", empty); + auto res = resp.get("Error.Resource", empty); + rest_error_response err(code, msg, rid, res); + return err; + } catch (...) { + vlog(s3_log.error, "!!error parse error {}", std::current_exception()); + throw; + } +} + +template +ss::future parse_rest_error_response( + response_content_type type, boost::beast::http::status result, iobuf&& buf) { + // AWS errors occasionally come with an empty body + // (See https://github.com/redpanda-data/redpanda/issues/6061) + // Without a proper code, we treat it as a hint to gracefully retry + // (synthesize the slow_down code). + if (!buf.empty()) { + if (type == response_content_type::xml) { + // Error responses from S3 _should_ have the Content-Type header set + // with `application/xml`- however, certain responses (such as `503 + // Service Unavailable`) may not be of this form. + // https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html + return ss::make_exception_future( + parse_xml_rest_error_response(std::move(buf))); } } + + auto result_prefix = buf.empty() ? "Empty error response, " : ""; + rest_error_response err( + fmt::format("{}", status_to_error_code(result)), + fmt::format("{}status code {}", result_prefix, result), + "", + ""); + return ss::make_exception_future(err); } /// Head response doesn't give us an XML encoded error object in @@ -708,11 +719,13 @@ ss::future s3_client::do_get_object( ref->get_headers().result(), ref->get_headers()); } + const auto content_type = util::get_response_content_type( + ref->get_headers()); return util::drain_response_stream(std::move(ref)) - .then([result](iobuf&& res) { + .then([content_type, result](iobuf&& res) { return parse_rest_error_response< http::client::response_stream_ref>( - result, std::move(res)); + content_type, result, std::move(res)); }); } return ss::make_ready_future( @@ -839,8 +852,11 @@ ss::future<> s3_client::do_put_object( id, status, ref->get_headers()); + const auto content_type + = util::get_response_content_type( + ref->get_headers()); return parse_rest_error_response<>( - status, std::move(res)); + content_type, status, std::move(res)); } return ss::now(); }); @@ -924,12 +940,14 @@ ss::future s3_client::do_list_objects_v2( header.result(), header); + const auto content_type = util::get_response_content_type( + header); // In the error path we drain the response stream fully, the // error response should not be very large. return util::drain_chunked_response_stream(resp).then( - [result = header.result()](iobuf buf) { + [result = header.result(), content_type](iobuf buf) { return parse_rest_error_response( - result, std::move(buf)); + content_type, result, std::move(buf)); }); } @@ -1016,7 +1034,10 @@ ss::future<> s3_client::do_delete_object( key, status, ref->get_headers()); - return parse_rest_error_response<>(status, std::move(res)); + const auto content_type = util::get_response_content_type( + ref->get_headers()); + return parse_rest_error_response<>( + content_type, status, std::move(res)); } return ss::now(); }); @@ -1106,8 +1127,10 @@ auto s3_client::do_delete_objects( [response](iobuf&& res) { auto status = response->get_headers().result(); if (status != boost::beast::http::status::ok) { + const auto content_type = util::get_response_content_type( + response->get_headers()); return parse_rest_error_response( - status, std::move(res)); + content_type, status, std::move(res)); } auto parse_result = iobuf_to_delete_objects_result( std::move(res)); diff --git a/src/v/cloud_storage_clients/tests/BUILD b/src/v/cloud_storage_clients/tests/BUILD index e9031998f11e6..8b2d87797e65f 100644 --- a/src/v/cloud_storage_clients/tests/BUILD +++ b/src/v/cloud_storage_clients/tests/BUILD @@ -43,6 +43,7 @@ redpanda_cc_btest( "//src/v/bytes:iostream", "//src/v/cloud_storage_clients", "//src/v/hashing:secure", + "//src/v/http/tests:utils", "//src/v/net", "//src/v/test_utils:seastar_boost", "//src/v/utils:base64", diff --git a/src/v/cloud_storage_clients/tests/CMakeLists.txt b/src/v/cloud_storage_clients/tests/CMakeLists.txt index ff30ea1616955..9688eaf7bd35d 100644 --- a/src/v/cloud_storage_clients/tests/CMakeLists.txt +++ b/src/v/cloud_storage_clients/tests/CMakeLists.txt @@ -10,7 +10,7 @@ rp_test( exception_test.cc util_test.cc DEFINITIONS BOOST_TEST_DYN_LINK - LIBRARIES v::seastar_testing_main Boost::unit_test_framework v::http v::cloud_storage_clients v::cloud_roles + LIBRARIES v::seastar_testing_main Boost::unit_test_framework v::http v::cloud_storage_clients v::cloud_roles v::http_test_utils ARGS "-- -c 1" LABELS s3 ) diff --git a/src/v/cloud_storage_clients/tests/s3_client_test.cc b/src/v/cloud_storage_clients/tests/s3_client_test.cc index 2a5f2a4cda508..7ffc0038cd94c 100644 --- a/src/v/cloud_storage_clients/tests/s3_client_test.cc +++ b/src/v/cloud_storage_clients/tests/s3_client_test.cc @@ -15,6 +15,7 @@ #include "cloud_storage_clients/client_pool.h" #include "cloud_storage_clients/s3_client.h" #include "hashing/secure.h" +#include "http/tests/utils.h" #include "net/dns.h" #include "net/types.h" #include "test_utils/fixture.h" @@ -31,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -125,6 +127,8 @@ static constexpr auto no_such_config_payload = R"xml( void set_routes(ss::httpd::routes& r) { using namespace ss::httpd; using reply = ss::http::reply; + using flexible_function_handler + = http::test_utils::flexible_function_handler; auto empty_put_response = new function_handler( [](const_req req) { BOOST_REQUIRE(!req.get_header("x-amz-content-sha256").empty()); @@ -132,24 +136,30 @@ void set_routes(ss::httpd::routes& r) { return ""; }, "txt"); - auto erroneous_put_response = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + auto erroneous_put_response = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::internal_server_error); return error_payload; }, - "txt"); + "xml"); auto get_response = new function_handler( [](const_req req) { BOOST_REQUIRE(!req.get_header("x-amz-content-sha256").empty()); return ss::sstring(expected_payload, expected_payload_size); }, "txt"); - auto erroneous_get_response = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + auto erroneous_get_response = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::internal_server_error); return error_payload; }, - "txt"); + "xml"); auto empty_delete_response = new function_handler( [](const_req req, reply& reply) { BOOST_REQUIRE(!req.get_header("x-amz-content-sha256").empty()); @@ -157,14 +167,17 @@ void set_routes(ss::httpd::routes& r) { return ""; }, "txt"); - auto erroneous_delete_response = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + auto erroneous_delete_response = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::internal_server_error); return error_payload; }, - "txt"); - auto list_objects_response = new function_handler( - [](const_req req, reply& reply) { + "xml"); + auto list_objects_response = new flexible_function_handler( + [](const_req req, reply& reply, [[maybe_unused]] ss::sstring& type) { BOOST_REQUIRE(!req.get_header("x-amz-content-sha256").empty()); BOOST_REQUIRE_EQUAL(req.get_query_param("list-type"), "2"); auto prefix = req.get_query_param("prefix"); @@ -180,27 +193,33 @@ void set_routes(ss::httpd::routes& r) { req.get_query_param("continuation-token"), "ctok"); return list_objects_payload; } - return ""; + return "none"; }, - "txt"); + "xml"); auto unexpected_error_response = new function_handler( []([[maybe_unused]] const_req req, reply& reply) { reply.set_status(reply::status_type::internal_server_error); return "unexpected!"; }, "txt"); - auto key_not_found_response = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + auto key_not_found_response = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::not_found); return no_such_key_payload; }, - "txt"); - auto bucket_not_found_response = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + "xml"); + auto bucket_not_found_response = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::not_found); return no_such_bucket_payload; }, - "txt"); + "xml"); auto delete_objects_response = new function_handler( [](const_req req, reply& reply) -> std::string { if (!req.query_parameters.contains("delete")) { @@ -282,12 +301,15 @@ void set_routes(ss::httpd::routes& r) { return ""; }, "txt"); - auto no_such_config = new function_handler( - []([[maybe_unused]] const_req req, reply& reply) { + auto no_such_config = new flexible_function_handler( + []( + [[maybe_unused]] const_req req, + reply& reply, + [[maybe_unused]] ss::sstring& type) { reply.set_status(reply::status_type::not_found); return no_such_config_payload; }, - "txt"); + "xml"); r.add(operation_type::PUT, url("/test"), empty_put_response); r.add(operation_type::PUT, url("/test-error"), erroneous_put_response); r.add(operation_type::GET, url("/test"), get_response); diff --git a/src/v/cloud_storage_clients/types.h b/src/v/cloud_storage_clients/types.h index 7ef8e9ecdb008..2a2e07a4f7229 100644 --- a/src/v/cloud_storage_clients/types.h +++ b/src/v/cloud_storage_clients/types.h @@ -93,6 +93,8 @@ from_config(std::optional us) { return std::nullopt; } +enum class response_content_type : int8_t { unknown, xml, json }; + } // namespace cloud_storage_clients namespace std { diff --git a/src/v/cloud_storage_clients/util.cc b/src/v/cloud_storage_clients/util.cc index bd8ce1ca956d7..59c83e2466bb5 100644 --- a/src/v/cloud_storage_clients/util.cc +++ b/src/v/cloud_storage_clients/util.cc @@ -274,4 +274,21 @@ void url_encode_target(http::client::request_header& header) { } } +response_content_type +get_response_content_type(const http::client::response_header& headers) { + static constexpr boost::beast::string_view content_type_name + = "Content-Type"; + if (auto iter = headers.find(content_type_name); iter != headers.end()) { + if (iter->value().find("json") != std::string_view::npos) { + return response_content_type::json; + } + + if (iter->value().find("xml") != std::string_view::npos) { + return response_content_type::xml; + } + } + + return response_content_type::unknown; +} + } // namespace cloud_storage_clients::util diff --git a/src/v/cloud_storage_clients/util.h b/src/v/cloud_storage_clients/util.h index 28bf8a753620f..74f58837f3639 100644 --- a/src/v/cloud_storage_clients/util.h +++ b/src/v/cloud_storage_clients/util.h @@ -57,4 +57,7 @@ std::vector all_paths_to_file(const object_key& path); // TODO: This should be replaced after we will represent URIs as structs void url_encode_target(http::client::request_header& header); +response_content_type +get_response_content_type(const http::client::response_header& headers); + } // namespace cloud_storage_clients::util diff --git a/src/v/http/tests/BUILD b/src/v/http/tests/BUILD index 4fa5421bc75ba..924b812883cb0 100644 --- a/src/v/http/tests/BUILD +++ b/src/v/http/tests/BUILD @@ -5,10 +5,12 @@ redpanda_test_cc_library( srcs = [ "http_imposter.cc", "registered_urls.cc", + "utils.cc", ], hdrs = [ "http_imposter.h", "registered_urls.h", + "utils.h", ], include_prefix = "http/tests", visibility = ["//visibility:public"], diff --git a/src/v/http/tests/CMakeLists.txt b/src/v/http/tests/CMakeLists.txt index 09ad032a401da..3e156048933f3 100644 --- a/src/v/http/tests/CMakeLists.txt +++ b/src/v/http/tests/CMakeLists.txt @@ -1,6 +1,6 @@ v_cc_library( NAME http_test_utils - SRCS registered_urls.cc http_imposter.cc + SRCS registered_urls.cc http_imposter.cc utils.cc DEPS Seastar::seastar_testing v::http) rp_test( diff --git a/src/v/http/tests/http_imposter.cc b/src/v/http/tests/http_imposter.cc index 1671beece383c..71346f0fc7203 100644 --- a/src/v/http/tests/http_imposter.cc +++ b/src/v/http/tests/http_imposter.cc @@ -11,10 +11,9 @@ #include "http/tests/http_imposter.h" #include "base/vlog.h" +#include "http/tests/utils.h" #include "utils/uuid.h" -#include - #include static ss::logger http_imposter_log("http_imposter"); // NOLINT @@ -94,8 +93,9 @@ void http_imposter_fixture::listen() { void http_imposter_fixture::set_routes(ss::httpd::routes& r) { using namespace ss::httpd; - _handler = std::make_unique( - [this](const_req req, ss::http::reply& repl) -> ss::sstring { + _handler = std::make_unique( + [this](const_req req, ss::http::reply& repl, ss::sstring& content_type) + -> ss::sstring { if (_masking_active) { if ( ss::lowres_clock::now() - _masking_active->started @@ -153,6 +153,7 @@ void http_imposter_fixture::set_routes(ss::httpd::routes& r) { } else if ( req._method == "POST" && req.query_parameters.contains("delete")) { // Delete objects + content_type = "xml"; return R"xml()xml"; } else { auto lookup_r = ri; @@ -162,12 +163,14 @@ void http_imposter_fixture::set_routes(ss::httpd::routes& r) { repl.set_status(response.status); for (const auto& [k, v] : response.headers) { repl.add_header(k, v); + if (k == "Content-Type" && v == "application/xml") { + content_type = "xml"; + } } return response.body; } - }, - "txt"); + }); r.add_default_handler(_handler.get()); } diff --git a/src/v/http/tests/registered_urls.cc b/src/v/http/tests/registered_urls.cc index 556235c273eef..863de8cb8490a 100644 --- a/src/v/http/tests/registered_urls.cc +++ b/src/v/http/tests/registered_urls.cc @@ -60,6 +60,24 @@ void registered_urls::add_mapping::add_mapping_when::then_reply_with( .headers = std::move(headers), .status = status}; } +void registered_urls::add_mapping::add_mapping_when::then_reply_with( + ss::sstring content, + std::vector> headers, + ss::http::reply::status_type status) { + if (!r.contains(url)) { + r[url] = method_reply_map{}; + } + + if (!r[url].contains(method)) { + r[url][method] = content_reply_map{}; + } + + r[url][method][request_content] = response{ + .body = std::move(content), + .headers = std::move(headers), + .status = status}; +} + registered_urls::add_mapping::add_mapping_when& registered_urls::add_mapping::add_mapping_when::with_method( ss::httpd::operation_type m) { diff --git a/src/v/http/tests/registered_urls.h b/src/v/http/tests/registered_urls.h index 5f98e0f0f4a90..2776ed277e06d 100644 --- a/src/v/http/tests/registered_urls.h +++ b/src/v/http/tests/registered_urls.h @@ -95,6 +95,11 @@ struct registered_urls { void then_reply_with( ss::sstring content, ss::http::reply::status_type status); + void then_reply_with( + ss::sstring content, + std::vector> headers, + ss::http::reply::status_type status); + void then_reply_with( std::vector> headers, ss::http::reply::status_type status); diff --git a/src/v/http/tests/utils.cc b/src/v/http/tests/utils.cc new file mode 100644 index 0000000000000..bdb3deafcf3c6 --- /dev/null +++ b/src/v/http/tests/utils.cc @@ -0,0 +1,60 @@ +/* + * Copyright 2025 Redpanda Data, Inc. + * + * Licensed as a Redpanda Enterprise file under the Redpanda Community + * License (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md + */ + +#include "http/tests/utils.h" + +#include "base/seastarx.h" + +#include +#include +#include +#include +#include + +namespace http { + +namespace test_utils { + +flexible_function_handler::flexible_function_handler( + const flexible_handle_function& f_handle, ss::sstring content_type) + : _f_handle([this, f_handle]( + std::unique_ptr req, + std::unique_ptr rep) { + rep->_content += f_handle( + *req.get(), *rep.get(), std::ref(_content_type)); + return ss::make_ready_future>( + std::move(rep)); + }) + , _content_type(std::move(content_type)) {} + +ss::future> flexible_function_handler::handle( + [[maybe_unused]] const ss::sstring& path, + std::unique_ptr req, + std::unique_ptr rep) { + return _f_handle(std::move(req), std::move(rep)) + .then([this](std::unique_ptr rep) { + if (_content_type == "xml") { + // Because `application/xml` is not implemented as a mapping + // in `http/mime_types.cc`, in order to construct a reply with + // the `Content-Type` header set to `application/xml`, we + // need to hard code a path here. + rep->set_mime_type("application/xml"); + rep->done(); + } else { + rep->done(_content_type); + } + return ss::make_ready_future>( + std::move(rep)); + }); +} + +} // namespace test_utils + +} // namespace http diff --git a/src/v/http/tests/utils.h b/src/v/http/tests/utils.h new file mode 100644 index 0000000000000..623f2cc3e2b28 --- /dev/null +++ b/src/v/http/tests/utils.h @@ -0,0 +1,54 @@ +/* + * Copyright 2025 Redpanda Data, Inc. + * + * Licensed as a Redpanda Enterprise file under the Redpanda Community + * License (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * https://github.com/redpanda-data/redpanda/blob/master/licenses/rcl.md + */ + +#pragma once + +#include "base/seastarx.h" + +#include +#include +#include +#include +#include + +namespace http { + +namespace test_utils { + +// `ss::httpd::function_handler` calls `rep->done(_type)` on the reply before +// returning it. Because certain code paths in imposter services may want to +// return a different `Content-Type` in their response, this impl's `handle()` +// function allows for modifying the `type` set in the function handler's body +// before the final call to `done(type)` is performed. +// +// This class also handles an oversight in `mime_types.cc`, in which `xml` is +// not handled as a possible mime type. +class flexible_function_handler : public ss::httpd::handler_base { + using flexible_handle_function = std::function; + +public: + flexible_function_handler( + const flexible_handle_function& f_handle, + ss::sstring content_type = "txt"); + + ss::future> handle( + [[maybe_unused]] const ss::sstring& path, + std::unique_ptr req, + std::unique_ptr rep) override; + +private: + ss::httpd::future_handler_function _f_handle; + ss::sstring _content_type; +}; + +} // namespace test_utils + +} // namespace http