Skip to content

Commit

Permalink
storage: rename delta() to offset_delta()
Browse files Browse the repository at this point in the history
Just delta() isn't necessarily clear what it is a delta of.
  • Loading branch information
andrwng committed Mar 21, 2024
1 parent a21b700 commit e5eb21f
Show file tree
Hide file tree
Showing 7 changed files with 11 additions and 10 deletions.
4 changes: 2 additions & 2 deletions src/v/archival/ntp_archiver_service.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1471,7 +1471,7 @@ ntp_archiver::do_schedule_single_upload(
auto base = upload.starting_offset;
auto log = _parent.log();
auto delta = base - model::offset_cast(log->from_log_offset(base));
auto delta_offset_next = log->delta(
auto delta_offset_next = log->offset_delta(
model::next_offset(upload.final_offset));

// The upload is successful only if the segment, and tx_range are
Expand Down Expand Up @@ -2927,7 +2927,7 @@ ss::future<bool> ntp_archiver::do_upload_local(
auto base = upload.starting_offset;
auto log = _parent.log();
auto delta = base - model::offset_cast(log->from_log_offset(base));
auto delta_offset_next = log->delta(
auto delta_offset_next = log->offset_delta(
model::next_offset(upload.final_offset));
auto archiver_term = _start_term;

Expand Down
2 changes: 1 addition & 1 deletion src/v/raft/consensus.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2462,7 +2462,7 @@ consensus::do_write_snapshot(model::offset last_included_index, iobuf&& data) {
.latest_configuration = *config,
.cluster_time = clock_type::time_point::min(),
.log_start_delta = offset_translator_delta(
_log->delta(model::next_offset(last_included_index))()),
_log->offset_delta(model::next_offset(last_included_index))()),
};

return details::persist_snapshot(
Expand Down
2 changes: 1 addition & 1 deletion src/v/raft/recovery_stm.cc
Original file line number Diff line number Diff line change
Expand Up @@ -490,7 +490,7 @@ recovery_stm::take_on_demand_snapshot(model::offset last_included_offset) {
.last_included_term = *term,
.latest_configuration = std::move(*cfg),
.log_start_delta = offset_translator_delta(
_ptr->log()->delta(model::next_offset(last_included_offset))),
_ptr->log()->offset_delta(model::next_offset(last_included_offset))),
};

co_await writer.write_metadata(reflection::to_iobuf(std::move(metadata)));
Expand Down
5 changes: 3 additions & 2 deletions src/v/raft/tests/raft_reconfiguration_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -145,14 +145,15 @@ void assert_offset_translator_state_is_consistent(
std::vector<int64_t> deltas;
for (model::offset o :
boost::irange<model::offset>(start_offset, dirty_offset)) {
deltas.push_back(first_raft->log()->delta(o));
deltas.push_back(first_raft->log()->offset_delta(o));
}

for (auto it = std::next(nodes.begin()); it != nodes.end(); ++it) {
auto idx = 0;
for (model::offset o :
boost::irange<model::offset>(start_offset, dirty_offset)) {
ASSERT_EQ(it->second->raft()->log()->delta(o), deltas[idx++]);
ASSERT_EQ(
it->second->raft()->log()->offset_delta(o), deltas[idx++]);
}
}
}
Expand Down
2 changes: 1 addition & 1 deletion src/v/storage/disk_log_impl.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1425,7 +1425,7 @@ bool disk_log_impl::is_new_log() const {
&& os.start_offset == not_initialized;
}

model::offset_delta disk_log_impl::delta(model::offset o) const {
model::offset_delta disk_log_impl::offset_delta(model::offset o) const {
return model::offset_delta{_offset_translator.state()->delta(o)};
}

Expand Down
2 changes: 1 addition & 1 deletion src/v/storage/disk_log_impl.h
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ class disk_log_impl final : public log {
return _offset_translator.state();
}
raft::offset_translator& offset_translator() { return _offset_translator; }
model::offset_delta delta(model::offset) const final;
model::offset_delta offset_delta(model::offset) const final;
model::offset from_log_offset(model::offset) const final;
model::offset to_log_offset(model::offset) const final;
model::offset find_last_term_start_offset() const final;
Expand Down
4 changes: 2 additions & 2 deletions src/v/storage/log.h
Original file line number Diff line number Diff line change
Expand Up @@ -99,15 +99,15 @@ class log {
virtual ss::future<std::optional<timequery_result>>
timequery(timequery_config) = 0;

// Prefer to use delta() or from/to_log_offset().
// Prefer to use offset_delta() or from/to_log_offset().
// TODO: remove direct access to the translator state and instead rely on
// the translation/delta interface.
virtual ss::lw_shared_ptr<const storage::offset_translator_state>
get_offset_translator_state() const = 0;

// Returns the offset delta for a given offset. This can be used for
// example to translate a Raft offset to a data offset.
virtual model::offset_delta delta(model::offset) const = 0;
virtual model::offset_delta offset_delta(model::offset) const = 0;

// Translate the given log offset into a data offset.
virtual model::offset from_log_offset(model::offset) const = 0;
Expand Down

0 comments on commit e5eb21f

Please sign in to comment.