Skip to content

Commit

Permalink
Merge pull request redpanda-data#14037 from vbotbuildovich/backport-p…
Browse files Browse the repository at this point in the history
…r-13835-v23.2.x-540

[v23.2.x] archival: Audit meta in adjacent merger uploads
  • Loading branch information
Lazin authored Oct 12, 2023
2 parents 75b5c43 + bcf2fb2 commit cdacce9
Showing 1 changed file with 50 additions and 15 deletions.
65 changes: 50 additions & 15 deletions src/v/archival/ntp_archiver_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,40 @@ constexpr auto housekeeping_jit = 5ms;

namespace archival {

static bool segment_meta_matches_stats(
const cloud_storage::segment_meta& meta,
const cloud_storage::segment_record_stats& stats,
retry_chain_logger& ctxlog) {
// Validate segment content. The 'stats' is computed when
// the actual segment is scanned and represents the 'ground truth' about
// its content. The 'meta' is the expected segment metadata. We
// shouldn't replicate it if it doesn't match the 'stats'.
if (
meta.size_bytes != stats.size_bytes
|| meta.base_offset != stats.base_rp_offset
|| meta.committed_offset != stats.last_rp_offset
|| static_cast<size_t>(meta.delta_offset_end - meta.delta_offset)
!= stats.total_conf_records) {
vlog(
ctxlog.error,
"Metadata of the uploaded segment [size: {}, base: {}, last: {}, "
"begin delta: {}, end delta: {}] "
"doesn't match the segment [size: {}, base: {}, last: {}, total "
"config records: {}]",
meta.size_bytes,
meta.base_offset,
meta.committed_offset,
meta.delta_offset,
meta.delta_offset_end,
stats.size_bytes,
stats.base_rp_offset,
stats.last_rp_offset,
stats.total_conf_records);
return false;
}
return true;
}

ntp_archiver_upload_result::ntp_archiver_upload_result(
cloud_storage::upload_result r)
: _result(r) {}
Expand Down Expand Up @@ -1682,21 +1716,7 @@ ss::future<ntp_archiver::upload_group_result> ntp_archiver::wait_uploads(
if (
upload.upload_kind == segment_upload_kind::non_compacted
&& upload.meta.has_value()) {
if (
upload.meta->size_bytes != stats.size_bytes
|| upload.meta->base_offset != stats.base_rp_offset
|| upload.meta->committed_offset != stats.last_rp_offset) {
vlog(
_rtclog.error,
"Metadata of the uploaded segment [size: {}, base: {}, "
"last: {}] doesn't match the segment [size: {}, base: "
"{}, last: {}]",
upload.meta->size_bytes,
upload.meta->base_offset,
upload.meta->committed_offset,
stats.size_bytes,
stats.base_rp_offset,
stats.last_rp_offset);
if (!segment_meta_matches_stats(*upload.meta, stats, _rtclog)) {
break;
}
}
Expand Down Expand Up @@ -2883,6 +2903,21 @@ ss::future<bool> ntp_archiver::do_upload_local(
.metadata_size_hint = tx_size,
};

const bool checks_disabled
= config::shard_local_cfg()
.cloud_storage_disable_upload_consistency_checks.value();

if (!checks_disabled && upl_res.has_record_stats()) {
auto stats = upl_res.record_stats();
// Validate segment content. The 'stats' is computed when
// the actual segment is scanned and represents the 'ground truth' about
// its content. The 'meta' is the expected segment metadata. We
// shouldn't replicate it if it doesn't match the 'stats'.
if (!segment_meta_matches_stats(meta, stats, _rtclog)) {
co_return false;
}
}

auto deadline = ss::lowres_clock::now() + _conf->manifest_upload_timeout;
auto error = co_await _parent.archival_meta_stm()->add_segments(
{meta}, std::nullopt, deadline, _as);
Expand Down

0 comments on commit cdacce9

Please sign in to comment.