Skip to content

Commit 3f92d2a

Browse files
author
yawzhang
committed
1. fix handle_error and only trigger handle_error for timeout rreqs in handle_raft_event
2. include concept 'volatile' vs 'non-volatile' for log 3. update replay logic : add BLK_ALLOCATED and DATA_RECEIVED only when data linked and received
1 parent 6756b81 commit 3f92d2a

File tree

7 files changed

+72
-21
lines changed

7 files changed

+72
-21
lines changed

conanfile.py

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -9,7 +9,7 @@
99

1010
class HomestoreConan(ConanFile):
1111
name = "homestore"
12-
version = "6.5.26"
12+
version = "6.5.27"
1313

1414
homepage = "https://github.com/eBay/Homestore"
1515
description = "HomeStore Storage Engine"

src/lib/replication/log_store/repl_log_store.cpp

Lines changed: 14 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@ uint64_t ReplLogStore::append(nuraft::ptr< nuraft::log_entry >& entry) {
1717

1818
repl_req_ptr_t rreq = m_sm.localize_journal_entry_finish(*entry);
1919
ulong lsn = HomeRaftLogStore::append(entry);
20-
m_sm.link_lsn_to_req(rreq, int64_cast(lsn));
20+
m_sm.link_lsn_to_req(rreq, m_sm.to_volatile_lsn(lsn));
2121

2222
RD_LOGD("Raft Channel: Received append log entry rreq=[{}]", rreq->to_compact_string());
2323
return lsn;
@@ -32,7 +32,7 @@ void ReplLogStore::write_at(ulong index, nuraft::ptr< nuraft::log_entry >& entry
3232

3333
repl_req_ptr_t rreq = m_sm.localize_journal_entry_finish(*entry);
3434
HomeRaftLogStore::write_at(index, entry);
35-
m_sm.link_lsn_to_req(rreq, int64_cast(index));
35+
m_sm.link_lsn_to_req(rreq, m_sm.to_volatile_lsn(index));
3636
RD_LOGD("Raft Channel: Received write_at log entry rreq=[{}]", rreq->to_compact_string());
3737
}
3838

@@ -43,7 +43,7 @@ void ReplLogStore::end_of_append_batch(ulong start_lsn, ulong count) {
4343
auto reqs = sisl::VectorPool< repl_req_ptr_t >::alloc();
4444
auto proposer_reqs = sisl::VectorPool< repl_req_ptr_t >::alloc();
4545
for (int64_t lsn = int64_cast(start_lsn); lsn <= end_lsn; ++lsn) {
46-
auto rreq = m_sm.lsn_to_req(lsn);
46+
auto rreq = m_sm.lsn_to_req(m_sm.to_volatile_lsn(lsn));
4747
// Skip this call in proposer, since this method will synchronously flush the data, which is not required for
4848
// leader. Proposer will call the flush as part of commit after receiving quorum, upon which time, there is a
4949
// high possibility the log entry is already flushed. Skip it for rreq == nullptr which is the case for raft
@@ -93,6 +93,17 @@ void ReplLogStore::end_of_append_batch(ulong start_lsn, ulong count) {
9393
if (rreq) { rreq->add_state(repl_req_state_t::LOG_FLUSHED); }
9494
}
9595
}
96+
97+
// Convert volatile logs to non-volatile logs in state machine
98+
for (int64_t lsn = int64_cast(start_lsn); lsn <= end_lsn; ++lsn) {
99+
auto volatile_lsn = m_sm.to_volatile_lsn(lsn);
100+
auto rreq = m_sm.lsn_to_req(volatile_lsn);
101+
if (rreq != nullptr) {
102+
m_sm.link_lsn_to_req(rreq, lsn);
103+
m_sm.unlink_lsn_to_req(volatile_lsn, rreq);
104+
}
105+
}
106+
96107
sisl::VectorPool< repl_req_ptr_t >::free(reqs);
97108
sisl::VectorPool< repl_req_ptr_t >::free(proposer_reqs);
98109
}

src/lib/replication/repl_dev/common.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ raft_buf_ptr_t& repl_req_ctx::raft_journal_buf() { return std::get< raft_buf_ptr
105105
uint8_t* repl_req_ctx::raw_journal_buf() { return std::get< std::unique_ptr< uint8_t[] > >(m_journal_buf).get(); }
106106

107107
void repl_req_ctx::set_lsn(int64_t lsn) {
108-
DEBUG_ASSERT((m_lsn == -1) || (m_lsn == lsn),
108+
DEBUG_ASSERT((m_lsn == -1) || (m_lsn == lsn) || (m_lsn == -lsn),
109109
"Changing lsn for request={} on the fly can cause race condition, not expected. lsn {}, m_lsn {}",
110110
to_string(), lsn, m_lsn);
111111
m_lsn = lsn;

src/lib/replication/repl_dev/raft_repl_dev.cpp

Lines changed: 29 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -594,7 +594,8 @@ folly::Future< folly::Unit > RaftReplDev::notify_after_data_written(std::vector<
594594
});
595595
}
596596

597-
bool RaftReplDev::wait_for_data_receive(std::vector< repl_req_ptr_t > const& rreqs, uint64_t timeout_ms) {
597+
bool RaftReplDev::wait_for_data_receive(std::vector< repl_req_ptr_t > const& rreqs, uint64_t timeout_ms,
598+
std::vector< repl_req_ptr_t >* timeout_rreqs) {
598599
std::vector< folly::Future< folly::Unit > > futs;
599600
std::vector< repl_req_ptr_t > only_wait_reqs;
600601
only_wait_reqs.reserve(rreqs.size());
@@ -621,14 +622,23 @@ bool RaftReplDev::wait_for_data_receive(std::vector< repl_req_ptr_t > const& rre
621622

622623
// We are yet to support reactive fetch from remote.
623624
if (is_resync_mode()) {
624-
check_and_fetch_remote_data(std::move(only_wait_reqs));
625+
check_and_fetch_remote_data(only_wait_reqs);
625626
} else {
626-
m_repl_svc.add_to_fetch_queue(shared_from_this(), std::move(only_wait_reqs));
627+
m_repl_svc.add_to_fetch_queue(shared_from_this(), only_wait_reqs);
627628
}
628629

629630
// block waiting here until all the futs are ready (data channel filled in and promises are made);
630-
auto all_futs = folly::collectAllUnsafe(futs).wait(std::chrono::milliseconds(timeout_ms));
631-
return (all_futs.isReady());
631+
auto all_futs_ready = folly::collectAllUnsafe(futs).wait(std::chrono::milliseconds(timeout_ms)).isReady();
632+
if (!all_futs_ready && timeout_rreqs != nullptr) {
633+
timeout_rreqs->clear();
634+
for (size_t i{0}; i < futs.size(); ++i) {
635+
if (!futs[i].isReady()) {
636+
timeout_rreqs->emplace_back(only_wait_reqs[i]);
637+
}
638+
}
639+
all_futs_ready = timeout_rreqs->empty();
640+
}
641+
return all_futs_ready;
632642
}
633643

634644
void RaftReplDev::check_and_fetch_remote_data(std::vector< repl_req_ptr_t > rreqs) {
@@ -953,18 +963,22 @@ void RaftReplDev::handle_commit(repl_req_ptr_t rreq, bool recovery) {
953963

954964
void RaftReplDev::handle_error(repl_req_ptr_t const& rreq, ReplServiceError err) {
955965
if (err == ReplServiceError::OK) { return; }
966+
RD_LOGE("Raft Channel: Error in processing rreq=[{}] error={}", rreq->to_string(), err);
956967

957968
if (!rreq->add_state_if_not_already(repl_req_state_t::ERRORED)) {
958-
RD_LOGE("Raft Channel: Error in processing rreq=[{}] error={}", rreq->to_string(), err);
969+
RD_LOGE("Raft Channel: Error has been added for rreq=[{}] error={}", rreq->to_string(), err);
959970
return;
960971
}
961972

962973
// Remove from the map and thus its no longer accessible from applier_create_req
963974
m_repl_key_req_map.erase(rreq->rkey());
964975

965-
if (rreq->op_code() == journal_type_t::HS_DATA_INLINED) {
976+
// Ensure non-volatile lsn not exist because handle_error should not be called after append entries.
977+
HS_REL_ASSERT(m_state_machine->lsn_to_req(rreq->lsn()) == nullptr,
978+
"Unexpected: LSN={} is already ready to commit", rreq->lsn());
979+
980+
if (rreq->op_code() == journal_type_t::HS_DATA_LINKED) {
966981
// Free the blks which is allocated already
967-
RD_LOGE("Raft Channel: Error in processing rreq=[{}] error={}", rreq->to_string(), err);
968982
if (rreq->has_state(repl_req_state_t::BLK_ALLOCATED)) {
969983
auto blkid = rreq->local_blkid();
970984
data_service().async_free_blk(blkid).thenValue([blkid](auto&& err) {
@@ -1276,8 +1290,9 @@ std::pair< bool, nuraft::cb_func::ReturnCode > RaftReplDev::handle_raft_event(nu
12761290
}
12771291

12781292
// Wait till we receive the data from its originator for all the requests
1279-
if (!wait_for_data_receive(*reqs, HS_DYNAMIC_CONFIG(consensus.data_receive_timeout_ms))) {
1280-
for (auto const& rreq : *reqs) {
1293+
std::vector< repl_req_ptr_t > timeout_rreqs;
1294+
if (!wait_for_data_receive(*reqs, HS_DYNAMIC_CONFIG(consensus.data_receive_timeout_ms), &timeout_rreqs)) {
1295+
for (auto const& rreq : timeout_rreqs) {
12811296
handle_error(rreq, ReplServiceError::TIMEOUT);
12821297
}
12831298
ret = nuraft::cb_func::ReturnCode::ReturnNull;
@@ -1468,20 +1483,22 @@ void RaftReplDev::on_log_found(logstore_seq_num_t lsn, log_buffer buf, void* ctx
14681483
RD_DBG_ASSERT(happened, "rreq already exists for rkey={}", rkey.to_string());
14691484
uint32_t data_size{0u};
14701485

1486+
// If the data is linked and value_size is non-zero, it means blks have been allocated for data.
1487+
// Since the log is flushed after data is written, the data has already been received.
14711488
if ((jentry->code == journal_type_t::HS_DATA_LINKED) && (jentry->value_size > 0)) {
14721489
MultiBlkId entry_blkid;
14731490
entry_blkid.deserialize(entry_to_val(jentry), true /* copy */);
14741491
data_size = entry_blkid.blk_count() * get_blk_size();
14751492
rreq->set_local_blkid(entry_blkid);
1493+
rreq->add_state(repl_req_state_t::BLK_ALLOCATED);
1494+
rreq->add_state(repl_req_state_t::DATA_RECEIVED);
14761495
}
14771496

14781497
rreq->set_lsn(repl_lsn);
14791498
// keep lentry in scope for the lyfe cycle of the rreq
14801499
rreq->set_lentry(lentry);
14811500
rreq->init(rkey, jentry->code, false /* is_proposer */, entry_to_hdr(jentry), entry_to_key(jentry), data_size);
14821501
// we load the log from log device, implies log flushed. We only flush log after data is written to data device.
1483-
rreq->add_state(repl_req_state_t::BLK_ALLOCATED);
1484-
rreq->add_state(repl_req_state_t::DATA_RECEIVED);
14851502
rreq->add_state(repl_req_state_t::DATA_WRITTEN);
14861503
rreq->add_state(repl_req_state_t::LOG_RECEIVED);
14871504
rreq->add_state(repl_req_state_t::LOG_FLUSHED);

src/lib/replication/repl_dev/raft_repl_dev.h

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -279,8 +279,15 @@ class RaftReplDev : public ReplDev,
279279
void fetch_data_from_remote(std::vector< repl_req_ptr_t > rreqs);
280280
void handle_fetch_data_response(sisl::GenericClientResponse response, std::vector< repl_req_ptr_t > rreqs);
281281
bool is_resync_mode();
282+
283+
/**
284+
* \brief This method handles errors that occur during append entries or data receiving.
285+
* It should not be called after the append entries phase.
286+
*/
282287
void handle_error(repl_req_ptr_t const& rreq, ReplServiceError err);
283-
bool wait_for_data_receive(std::vector< repl_req_ptr_t > const& rreqs, uint64_t timeout_ms);
288+
289+
bool wait_for_data_receive(std::vector < repl_req_ptr_t > const &rreqs, uint64_t timeout_ms,
290+
std::vector < repl_req_ptr_t > *timeout_rreqs = nullptr);
284291
void on_log_found(logstore_seq_num_t lsn, log_buffer buf, void* ctx);
285292
void commit_blk(repl_req_ptr_t rreq);
286293
void replace_member(repl_req_ptr_t rreq);

src/lib/replication/repl_dev/raft_state_machine.cpp

Lines changed: 11 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -179,7 +179,7 @@ repl_req_ptr_t RaftStateMachine::localize_journal_entry_finish(nuraft::log_entry
179179
raft_buf_ptr_t RaftStateMachine::pre_commit_ext(nuraft::state_machine::ext_op_params const& params) {
180180
int64_t lsn = s_cast< int64_t >(params.log_idx);
181181

182-
repl_req_ptr_t rreq = lsn_to_req(lsn);
182+
repl_req_ptr_t rreq = lsn_to_req(to_volatile_lsn(lsn));
183183
RD_LOGD("Raft channel: Precommit rreq=[{}]", rreq->to_compact_string());
184184
m_rd.m_listener->on_pre_commit(rreq->lsn(), rreq->header(), rreq->key(), rreq);
185185

@@ -293,8 +293,11 @@ void RaftStateMachine::link_lsn_to_req(repl_req_ptr_t rreq, int64_t lsn) {
293293
rreq->add_state(repl_req_state_t::LOG_RECEIVED);
294294
// reset the rreq created_at time to now https://github.com/eBay/HomeStore/issues/506
295295
rreq->set_created_time();
296-
[[maybe_unused]] auto r = m_lsn_req_map.insert(lsn, std::move(rreq));
297-
RD_DBG_ASSERT_EQ(r.second, true, "lsn={} already in precommit list, exist_term={}", lsn, r.first->second->term());
296+
auto r = m_lsn_req_map.insert(lsn, std::move(rreq));
297+
if (!r.second) {
298+
RD_LOG(ERROR, "lsn={} already in precommit list, exist_term={}", lsn, r.first->second->term());
299+
// TODO: we need to think about the case where volatile is in the map already, is it safe to overwrite it?
300+
}
298301
}
299302

300303
repl_req_ptr_t RaftStateMachine::lsn_to_req(int64_t lsn) {
@@ -400,4 +403,9 @@ void RaftStateMachine::free_user_snp_ctx(void*& user_snp_ctx) { m_rd.m_listener-
400403

401404
std::string RaftStateMachine::rdev_name() const { return m_rd.rdev_name(); }
402405

406+
int64_t RaftStateMachine::to_volatile_lsn(ulong log_idx) {
407+
RELEASE_ASSERT(log_idx <= INT64_MAX, "lsn={} is greater than INT64_MAX", log_idx);
408+
return - int64_cast(log_idx);
409+
}
410+
403411
} // namespace homestore

src/lib/replication/repl_dev/raft_state_machine.h

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -145,6 +145,14 @@ class RaftStateMachine : public nuraft::state_machine {
145145

146146
static bool is_hs_snp_obj(uint64_t obj_id) { return (obj_id & snp_obj_id_type_app) == 0; }
147147

148+
// There are two types of logs in the m_lsn_req_map: volatile and non-volatile.
149+
// A log is considered volatile until it is flushed to the log store at the end of the append operation.
150+
// Before this flushing occurs, the log and its associated request can be treated as volatile.
151+
// Therefore, the pre_commit operation should utilize the volatile log.
152+
// Once the log is flushed to the log store, it converts to a non-volatile state.
153+
// At this point, the commit operation can use the non-volatile one.
154+
int64_t to_volatile_lsn(ulong log_idx);
155+
148156
private:
149157
void after_precommit_in_leader(const nuraft::raft_server::req_ext_cb_params& params);
150158
};

0 commit comments

Comments
 (0)