Skip to content

Commit

Permalink
Merge pull request #4939 from abhijat/cleanup-si-logs
Browse files Browse the repository at this point in the history
cloud_storage: redacts fields from header
  • Loading branch information
abhijat committed May 26, 2022
2 parents 6bdabfa + 41bafd8 commit 5bb252b
Show file tree
Hide file tree
Showing 4 changed files with 57 additions and 14 deletions.
28 changes: 26 additions & 2 deletions src/v/http/client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@

#include "bytes/details/io_iterator_consumer.h"
#include "bytes/iobuf.h"
#include "config/base_property.h"
#include "http/logger.h"
#include "ssx/sformat.h"
#include "vlog.h"
Expand Down Expand Up @@ -76,10 +77,12 @@ ss::future<client::request_response_t> client::make_request(
auto verb = header.method();
auto target = header.target();
ss::sstring target_str(target.data(), target.size());
prefix_logger ctxlog(http_log, ssx::sformat("[{}]", target_str));
vlog(ctxlog.trace, "client.make_request {}", redacted_header(header));

auto req = ss::make_shared<request_stream>(this, std::move(header));
auto res = ss::make_shared<response_stream>(this, verb, target_str);
prefix_logger ctxlog(http_log, ssx::sformat("[{}]", target_str));
vlog(ctxlog.trace, "client.make_request {}", header);

auto now = ss::lowres_clock::now();
auto age = _last_response == ss::lowres_clock::time_point::min()
? ss::lowres_clock::duration::max()
Expand Down Expand Up @@ -589,4 +592,25 @@ ss::input_stream<char> client::response_stream::as_input_stream() {
return ss::input_stream<char>(std::move(ds));
}

client::request_header redacted_header(client::request_header original) {
using field_type = std::variant<boost::beast::http::field, std::string>;

static const std::unordered_set<field_type> redacted_fields{
boost::beast::http::field::authorization, "x-amz-content-sha256"};

auto h{std::move(original)};

for (const auto& field : redacted_fields) {
std::visit(
[&h](const auto& f) {
if (h.find(f) != h.end()) {
h.set(f, std::string{config::secret_placeholder});
}
},
field);
}

return h;
}

} // namespace http
7 changes: 7 additions & 0 deletions src/v/http/client.h
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,13 @@ class client : protected net::base_transport {
ss::lowres_clock::duration _max_idle_time;
};

/// Utility function for producing a copy of the request header with some
/// fields redacted for logging.
///
/// \param original a request header with potentially sensitive header values
/// \return redacted header with sensitive values removed
client::request_header redacted_header(client::request_header original);

template<class BufferSeq>
inline ss::future<> client::forward(client* client, BufferSeq&& seq) {
auto scattered = iobuf_as_scattered(std::forward<BufferSeq>(seq));
Expand Down
27 changes: 21 additions & 6 deletions src/v/s3/client.cc
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,7 @@ request_creator::make_list_objects_v2_request(
header.insert(aws_header_names::start_after, std::to_string(*max_keys));
}
auto ec = _sign.sign_header(header, aws_signatures::emptysig);
vlog(s3_log.trace, "ListObjectsV2:\n {}", header);
vlog(s3_log.trace, "ListObjectsV2:\n {}", http::redacted_header(header));
if (ec) {
return ec;
}
Expand Down Expand Up @@ -506,7 +506,10 @@ ss::future<http::client::response_stream_ref> client::get_object(
return ss::make_exception_future<http::client::response_stream_ref>(
std::system_error(header.error()));
}
vlog(s3_log.trace, "send https request:\n{}", header);
vlog(
s3_log.trace,
"send https request:\n{}",
http::redacted_header(header.value()));
return _client.request(std::move(header.value()), timeout)
.then([](http::client::response_stream_ref&& ref) {
// here we didn't receive any bytes from the socket and
Expand Down Expand Up @@ -543,7 +546,10 @@ ss::future<client::head_object_result> client::head_object(
return ss::make_exception_future<client::head_object_result>(
std::system_error(header.error()));
}
vlog(s3_log.trace, "send https request:\n{}", header);
vlog(
s3_log.trace,
"send https request:\n{}",
http::redacted_header(header.value()));
return _client.request(std::move(header.value()), timeout)
.then(
[key](const http::client::response_stream_ref& ref)
Expand Down Expand Up @@ -590,7 +596,10 @@ ss::future<> client::put_object(
if (!header) {
return ss::make_exception_future<>(std::system_error(header.error()));
}
vlog(s3_log.trace, "send https request:\n{}", header);
vlog(
s3_log.trace,
"send https request:\n{}",
http::redacted_header(header.value()));
return ss::do_with(
std::move(body),
[this, timeout, header = std::move(header)](
Expand Down Expand Up @@ -631,7 +640,10 @@ ss::future<client::list_bucket_result> client::list_objects_v2(
return ss::make_exception_future<list_bucket_result>(
std::system_error(header.error()));
}
vlog(s3_log.trace, "send https request:\n{}", header);
vlog(
s3_log.trace,
"send https request:\n{}",
http::redacted_header(header.value()));
return _client.request(std::move(header.value()), timeout)
.then([](const http::client::response_stream_ref& resp) mutable {
// chunked encoding is used so we don't know output size in
Expand Down Expand Up @@ -674,7 +686,10 @@ ss::future<> client::delete_object(
if (!header) {
return ss::make_exception_future<>(std::system_error(header.error()));
}
vlog(s3_log.trace, "send https request:\n{}", header);
vlog(
s3_log.trace,
"send https request:\n{}",
http::redacted_header(header.value()));
return _client.request(std::move(header.value()), timeout)
.then([](const http::client::response_stream_ref& ref) {
return drain_response_stream(ref).then([ref](iobuf&& res) {
Expand Down
9 changes: 3 additions & 6 deletions src/v/s3/signature.cc
Original file line number Diff line number Diff line change
Expand Up @@ -335,11 +335,7 @@ std::error_code signature_v4::sign_header(
auto sign_key = gen_sig_key(_private_key(), date_str, _region(), service);
auto cred_scope = ssx::sformat(
"{}/{}/{}/aws4_request", date_str, _region(), service);
vlog(
s3_log.trace,
"Credentials updated:\n[signing key]\n{}\n[scope]\n{}\n",
hexdigest(sign_key),
cred_scope);
vlog(s3_log.trace, "Credentials updated:\n[scope]\n{}\n", cred_scope);
auto amz_date = _sig_time.format_datetime();
header.set("x-amz-date", {amz_date.data(), amz_date.size()});
header.set("x-amz-content-sha256", {sha256.data(), sha256.size()});
Expand All @@ -364,7 +360,8 @@ std::error_code signature_v4::sign_header(
canonical_headers.value().signed_headers,
hexdigest(digest));
header.set(boost::beast::http::field::authorization, auth_header);
vlog(s3_log.trace, "\n[signed-header]\n\n{}", header);
vlog(
s3_log.trace, "\n[signed-header]\n\n{}", http::redacted_header(header));
return {};
}

Expand Down

0 comments on commit 5bb252b

Please sign in to comment.