Skip to content

Commit

Permalink
cloud_storage: log more context from topic_manifest_downloader
Browse files Browse the repository at this point in the history
The downloader has a lot of context but reduces results into an outcome
or error enum. This updates the downloader to log before returning early
from the class, logging errors at ERROR log level, and anything else at
INFO level (since it should be up to callers whether such code paths are
actually problematic).

This is review follow-up to
redpanda-data#20778
  • Loading branch information
andrwng committed Jul 9, 2024
1 parent e1df87e commit a94f2c5
Showing 1 changed file with 91 additions and 1 deletion.
92 changes: 91 additions & 1 deletion src/v/cloud_storage/topic_manifest_downloader.cc
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@
#include "cloud_storage/topic_manifest_downloader.h"

#include "base/outcome.h"
#include "base/vlog.h"
#include "cloud_storage/logger.h"
#include "cloud_storage/topic_manifest.h"
#include "cloud_storage/topic_path_utils.h"
#include "cloud_storage/types.h"
Expand Down Expand Up @@ -76,12 +78,40 @@ topic_manifest_downloader::download_manifest(
std::nullopt,
bin_manifest_filter);
if (list_res.has_error()) {
vlog(
cst_log.error,
"Labeled topic manifest download resulted in listing error with "
"prefix '{}': {}",
labeled_manifest_filter,
list_res.error());
co_return error_outcome::manifest_download_error;
}
// If there's more than one, callers will need to pass a label (or a more
// specific one).
auto list_contents = std::move(list_res.value().contents);
if (list_contents.size() > 1) {
static constexpr size_t max_to_print = 10;
const size_t num_to_print = std::min(
list_contents.size(), max_to_print);
vlog(
cst_log.info,
"Labeled topic manifest download resulted in {} matching "
"manifests with prefix '{}', printing first {}",
list_contents.size(),
remote_label_str,
num_to_print);
size_t num_printed = 0;
for (const auto& item : list_contents) {
if (num_printed == num_to_print) {
break;
}
vlog(
cst_log.info,
"Match for hint '{}': {}",
remote_label_str,
item.key);
++num_printed;
}
co_return find_topic_manifest_outcome::multiple_matching_manifests;
}
// If there's exactly one, presume it's the one we care about. Since
Expand All @@ -98,21 +128,41 @@ topic_manifest_downloader::download_manifest(
// Regardless of the outcome (i.e. even not-found), return an error. If
// we had a list result but it was deleted, something is suspicious, so
// don't proceed.
vlog(
cst_log.error,
"Labeled topic manifest download result with path {}: {}",
labeled_manifest,
manifest_res);
co_return error_outcome::manifest_download_error;
}

// Then look for prefixed binary manifests. If we find one, return it:
// since they're newer than JSON manifests, they take precedence.
const remote_manifest_path prefixed_bin_path(
prefixed_topic_manifest_bin_path(topic_));
vlog(
cst_log.info,
"Labeled topic manifest download at {} resulted in no manifests, falling "
"back on hash-prefixed binary manifest",
labeled_manifest_filter);
auto bin_manifest_res = co_await remote_.download_manifest_bin(
bucket_, prefixed_bin_path, *manifest, retry_node);
if (bin_manifest_res == cloud_storage::download_result::success) {
co_return find_topic_manifest_outcome::success;
}
if (bin_manifest_res != cloud_storage::download_result::notfound) {
vlog(
cst_log.error,
"Prefixed topic manifest download result with path {}: {}",
prefixed_bin_path,
bin_manifest_res);
co_return error_outcome::manifest_download_error;
}
vlog(
cst_log.info,
"Prefixed topic manifest download at {} resulted in no manifests, "
"falling back on hash-prefixed JSON manifest",
prefixed_bin_path);

// Finally, look for prefixed json topic manifest.
const remote_manifest_path prefixed_json_path(
Expand All @@ -123,8 +173,17 @@ topic_manifest_downloader::download_manifest(
co_return find_topic_manifest_outcome::success;
}
if (json_manifest_res != cloud_storage::download_result::notfound) {
vlog(
cst_log.error,
"Prefixed topic manifest download result with path {}: {}",
prefixed_json_path,
json_manifest_res);
co_return error_outcome::manifest_download_error;
}
vlog(
cst_log.info,
"Prefixed topic manifest download at {} resulted in no manifests",
prefixed_json_path);
co_return find_topic_manifest_outcome::no_matching_manifest;
}

Expand All @@ -148,6 +207,12 @@ ss::future<result<list_outcome_t, error_outcome>> find_prefixed_manifest_paths(
retry,
cloud_storage_clients::object_key{fmt::format("{}/", hash_prefix)});
if (prefixed_list_res.has_error()) {
vlog(
cst_log.error,
"Finding prefixed topic manifest resulted in listing error under "
"prefix {}: {}",
hash_prefix,
prefixed_list_res.error());
co_return error_outcome::manifest_download_error;
}
co_return prefixed_list_res.value().contents;
Expand Down Expand Up @@ -183,6 +248,12 @@ topic_manifest_downloader::find_manifests(
cloud_storage_clients::object_key{
fmt::format("{}/", labeled_topic_manifests_root())});
if (labeled_list_res.has_error()) {
vlog(
cst_log.error,
"Finding topic manifest resulted in listing error with prefix "
"'{}': {}",
labeled_topic_manifests_root(),
labeled_list_res.error());
co_return error_outcome::manifest_download_error;
}
for (const auto& item : labeled_list_res.value().contents) {
Expand All @@ -200,15 +271,24 @@ topic_manifest_downloader::find_manifests(
remote, bucket, parent_retry, deadline, backoff, root));
}
auto prefixed_res = co_await ss::when_all_succeed(futs.begin(), futs.end());
bool has_prefix_list_error = false;
for (const auto& r : prefixed_res) {
if (r.has_error()) {
co_return error_outcome::manifest_download_error;
vlog(
cst_log.error,
"Finding topic manifest resulted in listing error for prefixed "
"roots: {}",
r.error());
has_prefix_list_error = true;
}
for (const auto& item : r.value()) {
auto tp_ns = tp_ns_from_prefixed_path(item.key);
maybe_add_topic(tp_ns);
}
}
if (has_prefix_list_error) {
co_return error_outcome::manifest_download_error;
}
// Use the manifest downloader to look for the filtered manifests.
chunked_vector<topic_manifest> m;
m.reserve(topics.size());
Expand All @@ -220,9 +300,19 @@ topic_manifest_downloader::find_manifests(
auto res = co_await dl.download_manifest(
parent_retry, deadline, backoff, &tm);
if (res.has_error()) {
vlog(
cst_log.error,
"Finding topic manifest for {} resulted in download error: {}",
tp,
res.error());
co_return res.error();
}
if (res.value() != find_topic_manifest_outcome::success) {
vlog(
cst_log.info,
"Finding topic manifest for {} resulted in download outcome: {}",
tp,
res.value());
co_return res.value();
}
m.push_back(std::move(tm));
Expand Down

0 comments on commit a94f2c5

Please sign in to comment.