Unverified Commit 8d7eff58 authored by Ozan Tezcan's avatar Ozan Tezcan Committed by GitHub
Browse files

Snapshot RPC (#68)

Added snapshot RPC to support large state machines.
parent 5ff31968
......@@ -17,11 +17,12 @@ typedef enum {
RAFT_ERR_ONE_VOTING_CHANGE_ONLY=-3,
RAFT_ERR_SHUTDOWN=-4,
RAFT_ERR_NOMEM=-5,
RAFT_ERR_NEEDS_SNAPSHOT=-6,
RAFT_ERR_SNAPSHOT_IN_PROGRESS=-7,
RAFT_ERR_SNAPSHOT_ALREADY_LOADED=-8,
RAFT_ERR_INVALID_NODEID=-9,
RAFT_ERR_LEADER_TRANSFER_IN_PROGRESS=-10,
RAFT_ERR_SNAPSHOT_IN_PROGRESS=-6,
RAFT_ERR_SNAPSHOT_ALREADY_LOADED=-7,
RAFT_ERR_INVALID_NODEID=-8,
RAFT_ERR_LEADER_TRANSFER_IN_PROGRESS=-9,
RAFT_ERR_DONE=-10,
RAFT_ERR_STALE_TERM=-11,
RAFT_ERR_LAST=-100,
} raft_error_e;
......@@ -120,6 +121,21 @@ typedef struct raft_entry
* applied to the FSM. */
typedef raft_entry_t msg_entry_t;
typedef struct
{
/** chunk offset */
raft_size_t offset;
/** Chunk data pointer */
void *data;
/** Chunk len */
raft_size_t len;
/** 1 if this is the last chunk */
int last_chunk;
} raft_snapshot_chunk_t;
/** Entry message response.
* Indicates to client if entry was committed or not. */
typedef struct
......@@ -175,6 +191,47 @@ typedef struct
int vote_granted;
} msg_requestvote_response_t;
typedef struct
{
/** currentTerm, for follower to update itself */
raft_term_t term;
/** used to identify the sender node. Useful when this message is received
* from the nodes that are not part of the configuration yet. */
raft_node_id_t leader_id;
/** id, to make it possible to associate responses with requests. */
raft_msg_id_t msg_id;
/** last included index of the snapshot */
raft_index_t snapshot_index;
/** last included term of the snapshot */
raft_term_t snapshot_term;
/** snapshot chunk **/
raft_snapshot_chunk_t chunk;
} msg_snapshot_t;
typedef struct
{
/** the msg_id this response refers to */
raft_msg_id_t msg_id;
/** currentTerm, to force other leader to step down */
raft_term_t term;
/** indicates last acknowledged snapshot offset by the follower */
raft_size_t offset;
/** 1 if request is accepted */
int success;
/** 1 if this is a response to the final chunk */
int last_chunk;
} msg_snapshot_response_t;
/** Appendentries message.
* This message is used to tell nodes if it's safe to apply entries to the FSM.
* Can be sent without any entries as a keep alive message.
......@@ -266,22 +323,112 @@ typedef int (
msg_appendentries_t* msg
);
/**
* Log compaction
* Callback for telling the user to send a snapshot.
*
/** Callback for sending snapshot messages.
* @param[in] raft Raft server making this callback
* @param[in] user_data User data that is passed from Raft server
* @param[in] node Node's ID that needs a snapshot sent to
* @param[in] msg Snapshot msg
**/
typedef int (
*func_send_snapshot_f
) (
raft_server_t* raft,
void *user_data,
raft_node_t* node
raft_node_t* node,
msg_snapshot_t* msg
);
/** Callback for loading the received snapshot. User should load snapshot using
* raft_begin_load_snapshot() and raft_end_load_snapshot();
* e.g
*
* int loadsnapshot_callback()
* {
* // User loads the received snapshot
* int rc = loadSnapshotData();
* if (rc != 0) {
* return rc;
* }
*
* rc = raft_begin_load_snapshot(raft, snapshot_term, snapshot_index);
* if (rc != 0) {
* return -1;
* }
*
* // User should configure nodes using configuration data in the snapshot
* // e.g Using raft_add_node(), raft_node_set_voting() etc.
* configureNodesFromSnapshot();
*
* raft_end_load_snapshot(raft);
* return 0;
* }
*
* @param[in] raft Raft server making this callback
* @param[in] user_data User data that is passed from Raft server
* @param[in] snapshot_index Received snapshot index
* @param[in] snapshot_term Received snapshot term
* @return 0 on success */
typedef int (
*func_load_snapshot_f
) (
raft_server_t* raft,
void *user_data,
raft_index_t snapshot_index,
raft_term_t snapshot_term
);
/** Callback to get a chunk from the snapshot file. This chunk will be sent
* to the follower.
*
* 'chunk' struct fields should be filled with the appropriate data.
* To apply backpressure, return RAFT_ERR_DONE.
* @param[in] raft Raft server making this callback
* @param[in] user_data User data that is passed from Raft server
* @param[in] node Chunk will be sent to this node
* @param[in] offset Snapshot offset we request
* @param[in] chunk Snapshot chunk
* @return 0 on success */
typedef int (
*func_get_snapshot_chunk_f
) (
raft_server_t* raft,
void *user_data,
raft_node_t* node,
raft_size_t offset,
raft_snapshot_chunk_t* chunk
);
/** Callback to store a snapshot chunk. This chunk is received from the leader.
* @param[in] raft Raft server making this callback
* @param[in] user_data User data that is passed from Raft server
* @param[in] snapshot_index Last index of the received snapshot
* @param[in] offset Offset of the chunk we received
* @param[in] chunk Snapshot chunk
* @return 0 on success */
typedef int (
*func_store_snapshot_chunk_f
) (
raft_server_t* raft,
void *user_data,
raft_index_t snapshot_index,
raft_size_t offset,
raft_snapshot_chunk_t* chunk
);
/** Callback to clear incoming snapshot file. This might be called to clean up
* a partial snapshot file. e.g While we are still receiving snapshot, leader
* takes another snapshot and starts to send it.
*
* @param[in] raft Raft server making this callback
* @param[in] user_data User data that is passed from Raft server
* @return 0 on success */
typedef int (
*func_clear_snapshot_f
) (
raft_server_t* raft,
void *user_data
);
/** Callback for detecting when non-voting nodes have obtained enough logs.
* This triggers only when there are no pending configuration changes.
* @param[in] raft The Raft server making this callback
......@@ -464,9 +611,23 @@ typedef struct
/** Callback for sending appendentries messages */
func_send_appendentries_f send_appendentries;
/** Callback for notifying user that a node needs a snapshot sent */
/** Callback for sending snapshot messages */
func_send_snapshot_f send_snapshot;
/** Callback for loading snapshot. This will be called when we complete
* receiving snapshot from the leader */
func_load_snapshot_f load_snapshot;
/** Callback to get a chunk of the snapshot file */
func_get_snapshot_chunk_f get_snapshot_chunk;
/** Callback to store a chunk of the snapshot */
func_store_snapshot_chunk_f store_snapshot_chunk;
/** Callback to dismiss temporary file which is used for incoming
* snapshot chunks */
func_clear_snapshot_f clear_snapshot;
/** Callback for finite state machine application
* Return 0 on success.
* Return RAFT_ERR_SHUTDOWN if you want the server to shutdown. */
......@@ -777,7 +938,6 @@ int raft_periodic(raft_server_t* me, int msec_elapsed);
* @param[out] r The resulting response
* @return
* 0 on success
* RAFT_ERR_NEEDS_SNAPSHOT
* */
int raft_recv_appendentries(raft_server_t* me,
raft_node_t* node,
......@@ -795,6 +955,28 @@ int raft_recv_appendentries_response(raft_server_t* me,
raft_node_t* node,
msg_appendentries_response_t* r);
/** Receive a snapshot message.
* @param[in] node The node who sent us this message
* @param[in] req The snapshot message
* @param[out] resp The resulting response
* @return
* 0 on success */
int raft_recv_snapshot(raft_server_t* me_,
raft_node_t* node,
msg_snapshot_t *req,
msg_snapshot_response_t *resp);
/** Receive a response from a snapshot message we sent.
* @param[in] node The node who sent us this message
* @param[in] r The snapshot response message
* @return
* 0 on success;
* -1 on error;
* RAFT_ERR_NOT_LEADER server is not the leader */
int raft_recv_snapshot_response(raft_server_t* me_,
raft_node_t* node,
msg_snapshot_response_t *r);
/** Receive a requestvote message.
* @param[in] node The node who sent us this message
* @param[in] vr The requestvote message
......@@ -1315,4 +1497,6 @@ raft_node_id_t raft_get_transfer_leader(raft_server_t* me_);
/* cause this server to force an election on its next raft_periodic function call */
void raft_set_timeout_now(raft_server_t* me_);
raft_index_t raft_get_num_snapshottable_logs(raft_server_t* me_);
#endif /* RAFT_H_ */
......@@ -99,11 +99,15 @@ typedef struct {
raft_index_t snapshot_last_idx;
raft_term_t snapshot_last_term;
/* Previous index/term values stored during snapshot,
* which are restored if the operation is cancelled.
*/
raft_index_t saved_snapshot_last_idx;
raft_term_t saved_snapshot_last_term;
/* Next index/term values stored during snapshot */
raft_index_t next_snapshot_last_idx;
raft_term_t next_snapshot_last_term;
/* Last included index of the incoming snapshot */
raft_index_t snapshot_recv_idx;
/* Current offset of the incoming snapshot */
raft_size_t snapshot_recv_offset;
/* Read requests that await a network round trip to confirm
* we're still the leader.
......@@ -166,8 +170,6 @@ int raft_is_single_node_voting_cluster(raft_server_t *me_);
int raft_votes_is_majority(int nnodes, int nvotes);
raft_index_t raft_get_num_snapshottable_logs(raft_server_t* me_);
void raft_node_set_last_ack(raft_node_t* me_, raft_msg_id_t msgid, raft_term_t term);
raft_msg_id_t raft_node_get_last_acked_msgid(raft_node_t* me_);
......@@ -188,4 +190,8 @@ raft_msg_id_t raft_get_msg_id(raft_server_t* me_);
/* attempt to abort the leadership transfer */
void raft_reset_transfer_leader(raft_server_t* me_, int timed_out);
raft_size_t raft_node_get_snapshot_offset(raft_node_t *me_);
void raft_node_set_snapshot_offset(raft_node_t *me_, raft_size_t snapshot_offset);
#endif /* RAFT_PRIVATE_H_ */
......@@ -20,6 +20,11 @@ typedef long int raft_term_t;
*/
typedef long int raft_index_t;
/**
* Size type. This should be at least 64 bits.
*/
typedef unsigned long long raft_size_t;
/**
* Unique node identifier.
*/
......
......@@ -33,6 +33,9 @@ typedef struct
raft_node_id_t id;
/* Next snapshot offset to send to this node */
raft_size_t snapshot_offset;
/* last AE heartbeat response received */
raft_term_t last_acked_term;
raft_msg_id_t last_acked_msgid;
......@@ -221,4 +224,16 @@ raft_msg_id_t raft_node_get_max_seen_msg_id(raft_node_t *me_)
{
raft_node_private_t* me = (raft_node_private_t*)me_;
return me->max_seen_msgid;
}
\ No newline at end of file
}
raft_size_t raft_node_get_snapshot_offset(raft_node_t *me_)
{
raft_node_private_t* me = (raft_node_private_t*)me_;
return me->snapshot_offset;
}
void raft_node_set_snapshot_offset(raft_node_t *me_, raft_size_t snapshot_offset)
{
raft_node_private_t* me = (raft_node_private_t*)me_;
me->snapshot_offset = snapshot_offset;
}
......@@ -92,6 +92,20 @@ void raft_update_quorum_meta(raft_server_t* me_, raft_msg_id_t id)
me->last_acked_msg_id = id;
}
int raft_clear_incoming_snapshot(raft_server_t* me_, raft_index_t new_idx)
{
int e = 0;
raft_server_private_t* me = (raft_server_private_t*)me_;
if (me->snapshot_recv_idx != 0)
e = me->cb.clear_snapshot(me_, me->udata);
me->snapshot_recv_idx = new_idx;
me->snapshot_recv_offset = 0;
return e;
}
raft_server_t* raft_new_with_log(const raft_log_impl_t *log_impl, void *log_arg)
{
raft_server_private_t* me = raft_calloc(1, sizeof(raft_server_private_t));
......@@ -373,6 +387,22 @@ int raft_election_start(raft_server_t* me_)
return raft_become_precandidate(me_);
}
void raft_accept_leader(raft_server_t* me_, raft_node_id_t leader)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
if (!raft_is_follower(me_)) {
raft_become_follower(me_);
}
if (me->leader_id != leader) {
raft_clear_incoming_snapshot(me_, 0);
}
me->timeout_elapsed = 0;
me->leader_id = leader;
}
int raft_become_leader(raft_server_t* me_)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
......@@ -400,6 +430,7 @@ int raft_become_leader(raft_server_t* me_)
raft_set_state(me_, RAFT_STATE_LEADER);
raft_update_quorum_meta(me_, me->msg_id);
raft_clear_incoming_snapshot(me_, 0);
me->timeout_elapsed = 0;
raft_reset_transfer_leader(me_, 0);
......@@ -414,6 +445,7 @@ int raft_become_leader(raft_server_t* me_)
if (me->node == node)
continue;
raft_node_set_snapshot_offset(node, 0);
raft_node_set_next_idx(node, next_idx);
raft_node_set_match_idx(node, 0);
raft_send_appendentries(me_, node);
......@@ -492,6 +524,7 @@ void raft_become_follower(raft_server_t* me_)
raft_set_state(me_, RAFT_STATE_FOLLOWER);
raft_randomize_election_timeout(me_);
raft_clear_incoming_snapshot(me_, 0);
me->timeout_elapsed = 0;
me->leader_id = RAFT_NODE_ID_NONE;
}
......@@ -753,6 +786,31 @@ int raft_recv_appendentries_response(raft_server_t* me_,
return 0;
}
static int raft_receive_term(raft_server_t* me_, raft_term_t term)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
int e;
if (raft_is_candidate(me_) && me->current_term == term)
{
raft_become_follower(me_);
}
else if (me->current_term < term)
{
e = raft_set_current_term(me_, term);
if (0 != e)
return e;
raft_become_follower(me_);
}
else if (term < me->current_term)
{
return RAFT_ERR_STALE_TERM;
}
return 0;
}
int raft_recv_appendentries(
raft_server_t* me_,
raft_node_t* node,
......@@ -774,26 +832,18 @@ int raft_recv_appendentries(
}
r->msg_id = ae->msg_id;
r->success = 0;
if (raft_is_candidate(me_) && me->current_term == ae->term)
{
raft_become_follower(me_);
}
else if (me->current_term < ae->term)
{
e = raft_set_current_term(me_, ae->term);
if (0 != e)
goto out;
raft_become_follower(me_);
}
else if (ae->term < me->current_term)
{
/* 1. Reply false if term < currentTerm (§5.1) */
raft_log_node(me_, ae->leader_id,
"AE term %ld is less than current term %ld",
ae->term, me->current_term);
e = raft_receive_term(me_, ae->term);
if (e != 0) {
if (e == RAFT_ERR_STALE_TERM) {
/* 1. Reply false if term < currentTerm (§5.1) */
raft_log_node(me_, ae->leader_id,
"AE term %ld is less than current term %ld", ae->term,
me->current_term);
e = 0;
}
goto out;
}
......@@ -802,9 +852,8 @@ int raft_recv_appendentries(
}
/* update current leader because ae->term is up to date */
me->leader_id = ae->leader_id;
raft_accept_leader(me_, ae->leader_id);
raft_reset_transfer_leader(me_, 0);
me->timeout_elapsed = 0;
/* Not the first appendentries we've received */
/* NOTE: the log starts at 1 */
......@@ -1275,6 +1324,194 @@ raft_entry_t** raft_get_entries_from_idx(raft_server_t* me_, raft_index_t idx, i
return e;
}
int raft_send_snapshot(raft_server_t* me_, raft_node_t* node)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
if (!me->cb.send_snapshot)
return 0;
while (1) {
raft_size_t offset = raft_node_get_snapshot_offset(node);
msg_snapshot_t msg = {
.leader_id = raft_get_nodeid(me_),
.snapshot_index = me->snapshot_last_idx,
.snapshot_term = me->snapshot_last_term,
.term = me->current_term,
.msg_id = ++me->msg_id,
.chunk.offset = offset
};
raft_snapshot_chunk_t *chunk = &msg.chunk;
int e = me->cb.get_snapshot_chunk(me_, me->udata, node, offset, chunk);
if (e != 0) {
return (e != RAFT_ERR_DONE) ? e : 0;
}
e = me->cb.send_snapshot(me_, me->udata, node, &msg);
if (e != 0) {
return e;
}
if (chunk->last_chunk) {
raft_node_set_snapshot_offset(node, 0);
raft_node_set_next_idx(node, me->snapshot_last_idx + 1);
return 0;
}
raft_node_set_snapshot_offset(node, offset + chunk->len);
}
}
int raft_recv_snapshot(raft_server_t* me_,
raft_node_t* node,
msg_snapshot_t *req,
msg_snapshot_response_t *resp)
{
int e = 0;
raft_server_private_t* me = (raft_server_private_t*)me_;
raft_log_node(me_, raft_node_get_id(node),
"recv snapshot: ci:%lu comi:%lu t:%lu li:%d mi:%lu si:%lu st:%lu o:%llu, lc:%d, len:%llu",
raft_get_current_idx(me_),
raft_get_commit_idx(me_),
req->term,
req->leader_id,
req->msg_id,
req->snapshot_index,
req->snapshot_term,
req->chunk.offset,
req->chunk.last_chunk,
req->chunk.len);
resp->msg_id = req->msg_id;
resp->last_chunk = req->chunk.last_chunk;
resp->offset = 0;
resp->success = 0;
e = raft_receive_term(me_, req->term);
if (e != 0) {
if (e == RAFT_ERR_STALE_TERM) {
raft_log_node(me_, req->leader_id,
"Snapshot req term %ld is less than current term %ld",
req->term, me->current_term);
e = 0;
}
goto out;
}
if (node != NULL) {
raft_node_update_max_seen_msg_id(node, req->msg_id);
}
raft_accept_leader(me_, req->leader_id);
raft_reset_transfer_leader(me_, 0);
/** If we already have this snapshot, inform the leader. */
if (req->snapshot_index <= me->snapshot_last_idx) {
/** Set response as if it is the last chunk to tell leader that we have
* the snapshot */
resp->last_chunk = 1;
goto success;
}
/** In case leader takes another snapshot, it may start sending a more
* recent snapshot. In that case, we dismiss existing snapshot file. */
if (me->snapshot_recv_idx != req->snapshot_index) {
e = raft_clear_incoming_snapshot(me_, req->snapshot_index);
if (e != 0) {
goto out;
}
}
/** Reject message if this is not our current offset. */
if (me->snapshot_recv_offset != req->chunk.offset) {
resp->offset = me->snapshot_recv_offset;
goto out;
}
e = me->cb.store_snapshot_chunk(me_, me->udata, req->snapshot_index,
req->chunk.offset, &req->chunk);
if (e != 0) {
goto out;
}
me->snapshot_recv_offset = req->chunk.offset + req->chunk.len;
if (req->chunk.last_chunk) {
e = me->cb.load_snapshot(me_, me->udata,
req->snapshot_index, req->snapshot_term);
if (e != 0) {
goto out;
}
}
success:
resp->offset = req->chunk.len + req->chunk.offset;
resp->success = 1;
out:
resp->term = me->current_term;
return e;
}
int raft_recv_snapshot_response(raft_server_t* me_,
raft_node_t* node,
msg_snapshot_response_t *r)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
raft_log_node(me_, raft_node_get_id(node),
"recv snapshot response: ci:%lu comi:%lu mi:%lu t:%ld o:%llu s:%d lc:%d",
raft_get_current_idx(me_),
raft_get_commit_idx(me_),
r->msg_id,
r->term,
r->offset,
r->success,
r->last_chunk);
if (!raft_is_leader(me_))
return RAFT_ERR_NOT_LEADER;
if (raft_node_get_last_acked_msgid(node) > r->msg_id) {
return 0;
}
if (me->current_term < r->term)
{
int e = raft_set_current_term(me_, r->term);
if (0 != e)
return e;
raft_become_follower(me_);
return 0;
}
else if (me->current_term != r->term)
{
return 0;
}
raft_node_set_last_ack(node, r->msg_id, r->term);
if (!r->success) {
raft_node_set_snapshot_offset(node, r->offset);
}
if (r->success && r->last_chunk) {
raft_node_set_snapshot_offset(node, 0);
raft_node_set_next_idx(node, max(me->snapshot_last_idx + 1,
raft_node_get_next_idx(node)));
}
/* Send snapshot or appendentries depending on next idx */
return raft_send_appendentries(me_, node);
}
int raft_send_appendentries(raft_server_t* me_, raft_node_t* node)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
......@@ -1291,10 +1528,7 @@ int raft_send_appendentries(raft_server_t* me_, raft_node_t* node)
/* figure out if the client needs a snapshot sent */
if (me->snapshot_last_idx > 0 && next_idx <= me->snapshot_last_idx)
{
if (me->cb.send_snapshot)
me->cb.send_snapshot(me_, me->udata, node);
return RAFT_ERR_NEEDS_SNAPSHOT;
return raft_send_snapshot(me_, node);
}
if (!me->cb.send_appendentries)
......@@ -1521,8 +1755,9 @@ int raft_begin_snapshot(raft_server_t *me_, int flags)
assert(raft_get_commit_idx(me_) == raft_get_last_applied_idx(me_));
raft_set_snapshot_metadata(me_, ety_term, snapshot_target);
me->snapshot_in_progress = 1;
me->next_snapshot_last_idx = snapshot_target;
me->next_snapshot_last_term = ety_term;
me->snapshot_flags = flags;
raft_log(me_,
......@@ -1538,12 +1773,6 @@ int raft_cancel_snapshot(raft_server_t *me_)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
if (!me->snapshot_in_progress)
return -1;
me->snapshot_last_idx = me->saved_snapshot_last_idx;
me->snapshot_last_term = me->saved_snapshot_last_term;
me->snapshot_in_progress = 0;
return 0;
......@@ -1553,11 +1782,11 @@ int raft_end_snapshot(raft_server_t *me_)
{
raft_server_private_t* me = (raft_server_private_t*)me_;
if (!me->snapshot_in_progress || me->snapshot_last_idx == 0)
if (!me->snapshot_in_progress)
return -1;
// TODO: What is the purpose of this assert? Looks wrong.
// assert(raft_get_num_snapshottable_logs(me_) != 0);
me->snapshot_last_idx = me->next_snapshot_last_idx;
me->snapshot_last_term = me->next_snapshot_last_term;
/* If needed, remove compacted logs */
int e = me->log_impl->poll(me->log, me->snapshot_last_idx + 1);
......@@ -1582,13 +1811,13 @@ int raft_end_snapshot(raft_server_t *me_)
if (me->node == node || !raft_node_is_active(node))
continue;
raft_node_set_snapshot_offset(node, 0);
raft_index_t next_idx = raft_node_get_next_idx(node);
/* figure out if the client needs a snapshot sent */
if (me->snapshot_last_idx > 0 && next_idx <= me->snapshot_last_idx)
{
if (me->cb.send_snapshot)
me->cb.send_snapshot(me_, me->udata, node);
raft_send_snapshot(me_, node);
}
}
......@@ -1616,7 +1845,7 @@ int raft_begin_load_snapshot(
if (last_included_index < raft_get_current_idx(me_))
return -1;
if (last_included_term == me->snapshot_last_term && last_included_index == me->snapshot_last_idx)
if (last_included_index <= me->snapshot_last_idx)
return RAFT_ERR_SNAPSHOT_ALREADY_LOADED;
if (me->current_term < last_included_term) {
......@@ -1633,7 +1862,8 @@ int raft_begin_load_snapshot(
raft_set_commit_idx(me_, last_included_index);
me->last_applied_idx = last_included_index;
raft_set_snapshot_metadata(me_, last_included_term, me->last_applied_idx);
me->next_snapshot_last_term = last_included_term;
me->next_snapshot_last_idx = last_included_index;
/* remove all nodes but self */
int i, my_node_by_idx = 0;
......@@ -1665,6 +1895,9 @@ int raft_end_load_snapshot(raft_server_t *me_)
raft_server_private_t* me = (raft_server_private_t*)me_;
int i;
me->snapshot_last_idx = me->next_snapshot_last_idx;
me->snapshot_last_term = me->next_snapshot_last_term;
/* Set nodes' voting status as committed */
for (i = 0; i < me->num_nodes; i++)
{
......
......@@ -270,8 +270,8 @@ void raft_set_snapshot_metadata(raft_server_t *me_, raft_term_t term, raft_index
{
raft_server_private_t* me = (raft_server_private_t*)me_;
me->last_applied_idx = idx;
me->saved_snapshot_last_term = me->snapshot_last_term;
me->saved_snapshot_last_idx = me->snapshot_last_idx;
me->next_snapshot_last_term = me->snapshot_last_term;
me->next_snapshot_last_idx = me->snapshot_last_idx;
me->snapshot_last_term = term;
me->snapshot_last_idx = idx;
}
......
......@@ -69,7 +69,8 @@ static int __raft_send_appendentries_capture(raft_server_t* raft,
static int __raft_send_snapshot_increment(raft_server_t* raft,
void* udata,
raft_node_t* node)
raft_node_t* node,
msg_snapshot_t *msg)
{
int *counter = udata;
......@@ -77,6 +78,114 @@ static int __raft_send_snapshot_increment(raft_server_t* raft,
return 0;
}
static int __raft_get_snapshot_chunk(raft_server_t* raft,
void *user_data,
raft_node_t* node,
raft_size_t offset,
raft_snapshot_chunk_t* chunk)
{
if (offset > 0) {
return RAFT_ERR_DONE;
}
chunk->data = "test";
chunk->len = strlen("test");
chunk->last_chunk = 1;
return 0;
}
static int __raft_store_snapshot_chunk(raft_server_t* raft,
void *user_data,
raft_index_t snapshot_index,
raft_size_t offset,
raft_snapshot_chunk_t* chunk)
{
return 0;
}
static int __raft_clear_snapshot(raft_server_t* raft,
void *user_data)
{
return 0;
}
struct test_data
{
int send;
int get_chunk;
int store_chunk;
int clear;
int load;
};
static int test_send_snapshot_increment(raft_server_t* raft,
void* udata,
raft_node_t* node,
msg_snapshot_t *msg)
{
struct test_data *t = udata;
t->send++;
return 0;
}
static int test_get_snapshot_chunk(raft_server_t* raft,
void *user_data,
raft_node_t* node,
raft_size_t offset,
raft_snapshot_chunk_t* chunk)
{
struct test_data *t = user_data;
t->get_chunk++;
if (offset > 0) {
return RAFT_ERR_DONE;
}
chunk->data = "test";
chunk->len = strlen("test");
chunk->last_chunk = 1;
return 0;
}
static int test_store_snapshot_chunk(raft_server_t* raft,
void *user_data,
raft_index_t snapshot_index,
raft_size_t offset,
raft_snapshot_chunk_t* chunk)
{
struct test_data *t = user_data;
t->store_chunk++;
return 0;
}
static int test_clear_snapshot(raft_server_t* raft,
void *user_data)
{
struct test_data *t = user_data;
t->clear++;
return 0;
}
static int test_load_snapshot(raft_server_t* raft,
void *user_data,
raft_index_t snapshot_index,
raft_term_t snapshot_term)
{
struct test_data *t = user_data;
t->load++;
raft_begin_load_snapshot(raft, snapshot_term, snapshot_index);
raft_end_load_snapshot(raft);
return 0;
}
/* static raft_cbs_t generic_funcs = { */
/* .persist_term = __raft_persist_term, */
/* .persist_vote = __raft_persist_vote, */
......@@ -490,7 +599,10 @@ void TestRaft_follower_load_from_snapshot_fails_if_log_is_newer(CuTest * tc)
void TestRaft_leader_sends_snapshot_when_node_next_index_was_compacted(CuTest* tc)
{
raft_cbs_t funcs = {
.send_snapshot = __raft_send_snapshot_increment
.send_snapshot = __raft_send_snapshot_increment,
.clear_snapshot = __raft_clear_snapshot,
.store_snapshot_chunk = __raft_store_snapshot_chunk,
.get_snapshot_chunk = __raft_get_snapshot_chunk
};
int increment = 0;
......@@ -524,7 +636,7 @@ void TestRaft_leader_sends_snapshot_when_node_next_index_was_compacted(CuTest* t
/* verify snapshot is sent */
int rc = raft_send_appendentries(r, node);
CuAssertIntEquals(tc, RAFT_ERR_NEEDS_SNAPSHOT, rc);
CuAssertIntEquals(tc, 0, rc);
CuAssertIntEquals(tc, 1, increment);
/* update callbacks, verify correct appendreq is sent after the snapshot */
......@@ -732,7 +844,10 @@ void TestRaft_leader_sends_snapshot_if_log_was_compacted(CuTest* tc)
{
raft_cbs_t funcs = {
.send_snapshot = __raft_send_snapshot_increment,
.send_appendentries = __raft_send_appendentries
.send_appendentries = __raft_send_appendentries,
.clear_snapshot = __raft_clear_snapshot,
.store_snapshot_chunk = __raft_store_snapshot_chunk,
.get_snapshot_chunk = __raft_get_snapshot_chunk
};
int send_snapshot_count = 0;
......@@ -784,6 +899,138 @@ void TestRaft_leader_sends_snapshot_if_log_was_compacted(CuTest* tc)
CuAssertIntEquals(tc, 1, send_snapshot_count);
}
void TestRaft_clear_snapshot_on_leader_change(CuTest * tc)
{
raft_cbs_t funcs = {
.send_snapshot = test_send_snapshot_increment,
.send_appendentries = __raft_send_appendentries,
.clear_snapshot = test_clear_snapshot,
.load_snapshot = test_load_snapshot,
.store_snapshot_chunk = test_store_snapshot_chunk,
.get_snapshot_chunk = test_get_snapshot_chunk
};
struct test_data data = {0};
void *r = raft_new();
raft_set_callbacks(r, &funcs, &data);
raft_add_node(r, NULL, 1, 1);
msg_snapshot_t msg = {
.leader_id = 2,
.snapshot_index = 1,
.snapshot_term = 1,
.msg_id = 1,
.chunk.data = "tmp",
.chunk.len = strlen("tmp"),
.chunk.last_chunk = 0,
};
msg_snapshot_response_t resp = {0};
raft_recv_snapshot(r, NULL, &msg, &resp);
CuAssertIntEquals(tc, 1, data.store_chunk);
msg.msg_id = 2;
msg.leader_id = 3;
raft_recv_snapshot(r, NULL, &msg, &resp);
CuAssertIntEquals(tc, 1, data.clear);
CuAssertIntEquals(tc, 2, data.store_chunk);
}
void TestRaft_reject_wrong_offset(CuTest * tc)
{
raft_cbs_t funcs = {
.send_snapshot = test_send_snapshot_increment,
.send_appendentries = __raft_send_appendentries,
.clear_snapshot = test_clear_snapshot,
.load_snapshot = test_load_snapshot,
.store_snapshot_chunk = test_store_snapshot_chunk,
.get_snapshot_chunk = test_get_snapshot_chunk
};
struct test_data data = {0};
void *r = raft_new();
raft_set_callbacks(r, &funcs, &data);
raft_add_node(r, NULL, 1, 1);
msg_snapshot_t msg = {
.leader_id = 2,
.snapshot_index = 1,
.snapshot_term = 1,
.msg_id = 1,
.chunk.data = "tmp",
.chunk.offset = 0,
.chunk.len = 50,
.chunk.last_chunk = 0,
};
msg_snapshot_response_t resp = {0};
raft_recv_snapshot(r, NULL, &msg, &resp);
CuAssertIntEquals(tc, 1, data.store_chunk);
msg.chunk.offset = 80;
raft_recv_snapshot(r, NULL, &msg, &resp);
CuAssertIntEquals(tc, 0, resp.success);
CuAssertIntEquals(tc, 50, resp.offset);
}
void TestRaft_set_last_chunk_on_duplicate(CuTest * tc)
{
raft_cbs_t funcs = {
.send_snapshot = test_send_snapshot_increment,
.send_appendentries = __raft_send_appendentries,
.clear_snapshot = test_clear_snapshot,
.load_snapshot = test_load_snapshot,
.store_snapshot_chunk = test_store_snapshot_chunk,
.get_snapshot_chunk = test_get_snapshot_chunk
};
struct test_data data = {0};
void *r = raft_new();
raft_set_callbacks(r, &funcs, &data);
raft_add_node(r, NULL, 1, 1);
msg_snapshot_t msg = {
.term = 1,
.leader_id = 2,
.snapshot_index = 5,
.snapshot_term = 1,
.msg_id = 1,
.chunk.data = "tmp",
.chunk.offset = 0,
.chunk.len = 50,
.chunk.last_chunk = 1,
};
msg_snapshot_response_t resp = {0};
raft_recv_snapshot(r, NULL, &msg, &resp);
CuAssertIntEquals(tc, 1, data.store_chunk);
CuAssertIntEquals(tc, 1, data.load);
msg_snapshot_t msg2 = {
.term = 1,
.leader_id = 2,
.snapshot_index = 4,
.snapshot_term = 1,
.msg_id = 1,
.chunk.offset = 0,
.chunk.len = 50,
.chunk.last_chunk = 0,
};
raft_recv_snapshot(r, NULL, &msg2, &resp);
CuAssertIntEquals(tc, 1, resp.success);
CuAssertIntEquals(tc, 1, resp.last_chunk);
}
int main(void)
{
CuString *output = CuStringNew();
......@@ -808,6 +1055,9 @@ int main(void)
SUITE_ADD_TEST(suite, TestRaft_leader_sends_appendentries_with_correct_prev_log_idx_when_snapshotted);
SUITE_ADD_TEST(suite, TestRaft_cancel_snapshot_restores_state);
SUITE_ADD_TEST(suite, TestRaft_leader_sends_snapshot_if_log_was_compacted);
SUITE_ADD_TEST(suite, TestRaft_clear_snapshot_on_leader_change);
SUITE_ADD_TEST(suite, TestRaft_reject_wrong_offset);
SUITE_ADD_TEST(suite, TestRaft_set_last_chunk_on_duplicate);
CuSuiteRun(suite);
CuSuiteDetails(suite, output);
......
......@@ -77,6 +77,7 @@ NODE_CONNECTING = 1
NODE_CONNECTED = 2
NODE_DISCONNECTING = 3
SNAPSHOT_SIZE = 41 * 1023
class ServerDoesNotExist(Exception):
pass
......@@ -125,9 +126,11 @@ def err2str(err):
lib.RAFT_ERR_ONE_VOTING_CHANGE_ONLY: 'RAFT_ERR_ONE_VOTING_CHANGE_ONLY',
lib.RAFT_ERR_SHUTDOWN: 'RAFT_ERR_SHUTDOWN',
lib.RAFT_ERR_NOMEM: 'RAFT_ERR_NOMEM',
lib.RAFT_ERR_NEEDS_SNAPSHOT: 'RAFT_ERR_NEEDS_SNAPSHOT',
lib.RAFT_ERR_SNAPSHOT_IN_PROGRESS: 'RAFT_ERR_SNAPSHOT_IN_PROGRESS',
lib.RAFT_ERR_SNAPSHOT_ALREADY_LOADED: 'RAFT_ERR_SNAPSHOT_ALREADY_LOADED',
lib.RAFT_INVALID_NODEID: 'RAFT_INVALID_NODEID',
lib.RAFT_ERR_LEADER_TRANSFER_IN_PROGRESS: 'RAFT_ERR_LEADER_TRANSFER_IN_PROGRESS',
lib.RAFT_ERR_DONE: 'RAFT_ERR_DONE',
lib.RAFT_ERR_LAST: 'RAFT_ERR_LAST',
}[err]
......@@ -177,8 +180,45 @@ def raft_send_appendentries(raft, udata, node, msg):
return 0
def raft_send_snapshot(raft, udata, node):
return ffi.from_handle(udata).send_snapshot(node)
def raft_send_snapshot(raft, udata, node, msg):
server = ffi.from_handle(udata)
dst_server = ffi.from_handle(lib.raft_node_get_udata(node))
server.network.enqueue_msg(msg, server, dst_server)
return 0
def raft_load_snapshot(raft, udata, index, term):
return ffi.from_handle(udata).load_snapshot(index, term)
def raft_clear_snapshot(raft, udata):
return ffi.from_handle(udata).clear_snapshot()
def raft_get_snapshot_chunk(raft, udata, node, offset, chunk):
server = ffi.from_handle(udata)
chunk.len = min(4096, len(server.snapshot_buf) - offset)
chunk.last_chunk = offset + chunk.len == len(server.snapshot_buf)
chunk.data = ffi.from_buffer("char*", server.snapshot_buf, 0) + offset
target_node = ffi.from_handle(lib.raft_node_get_udata(node))
# If receiver node has more than 8 messages to be consumed, apply some
# backpressure by returning RAFT_ERR_DONE
count = 0
for message in target_node.network.messages:
if message.sendee == target_node:
count += 1
if count > 8 or chunk.len == 0:
return lib.RAFT_ERR_DONE
return 0
def raft_store_snapshot_chunk(raft, udata, index, offset, chunk):
buf = ffi.buffer(chunk.data, chunk.len)
return ffi.from_handle(udata).store_snapshot(offset, buf)
def raft_applylog(raft, udata, ety, idx):
......@@ -498,8 +538,6 @@ class Network(object):
logger.error(msg.sendee.debug_log())
self.diagnostic_info()
sys.exit(1)
elif lib.RAFT_ERR_NEEDS_SNAPSHOT == e:
pass # TODO: pretend as if snapshot works
else:
self.enqueue_msg(response, msg.sendee, msg.sendor)
......@@ -507,6 +545,16 @@ class Network(object):
node = lib.raft_get_node(msg.sendee.raft, msg.sendor.id)
lib.raft_recv_appendentries_response(msg.sendee.raft, node, msg.data)
elif msg_type == 'msg_snapshot_t *':
response = ffi.new('msg_snapshot_response_t *')
node = lib.raft_get_node(msg.sendee.raft, msg.sendor.id)
lib.raft_recv_snapshot(msg.sendee.raft, node, msg.data, response)
self.enqueue_msg(response, msg.sendee, msg.sendor)
elif msg_type == 'msg_snapshot_response_t *':
node = lib.raft_get_node(msg.sendee.raft, msg.sendor.id)
lib.raft_recv_snapshot_response(msg.sendee.raft, node, msg.data)
elif msg_type == 'msg_requestvote_t *':
response = ffi.new('msg_requestvote_response_t*')
node = lib.raft_get_node(msg.sendee.raft, msg.sendor.id)
......@@ -770,6 +818,10 @@ class RaftServer(object):
cbs.send_requestvote = self.raft_send_requestvote
cbs.send_appendentries = self.raft_send_appendentries
cbs.send_snapshot = self.raft_send_snapshot
cbs.load_snapshot = self.raft_load_snapshot
cbs.clear_snapshot = self.raft_clear_snapshot
cbs.get_snapshot_chunk = self.raft_get_snapshot_chunk
cbs.store_snapshot_chunk = self.raft_store_snapshot_chunk
cbs.applylog = self.raft_applylog
cbs.persist_vote = self.raft_persist_vote
cbs.persist_term = self.raft_persist_term
......@@ -790,6 +842,14 @@ class RaftServer(object):
self.fsm_dict = {}
self.fsm_log = []
self.random = random.Random(self.id)
# Dummy snapshot file for this node
self.snapshot_buf = bytearray(SNAPSHOT_SIZE)
# Save incoming snapshot chunks to this buffer
self.snapshot_recv_buf = bytearray(SNAPSHOT_SIZE)
def __str__(self):
return '<Server: {0}>'.format(self.id)
......@@ -824,6 +884,9 @@ class RaftServer(object):
assert(lib.raft_snapshot_is_in_progress(self.raft))
# Generate a dummy snapshot
self.snapshot_buf = bytearray(self.random.randbytes(SNAPSHOT_SIZE))
e = lib.raft_end_snapshot(self.raft)
assert(e == 0)
if e != 0:
......@@ -868,7 +931,11 @@ class RaftServer(object):
def load_callbacks(self):
self.raft_send_requestvote = ffi.callback("int(raft_server_t*, void*, raft_node_t*, msg_requestvote_t*)", raft_send_requestvote)
self.raft_send_appendentries = ffi.callback("int(raft_server_t*, void*, raft_node_t*, msg_appendentries_t*)", raft_send_appendentries)
self.raft_send_snapshot = ffi.callback("int(raft_server_t*, void* , raft_node_t*)", raft_send_snapshot)
self.raft_send_snapshot = ffi.callback("int(raft_server_t*, void* , raft_node_t*, msg_snapshot_t*)", raft_send_snapshot)
self.raft_load_snapshot = ffi.callback("int(raft_server_t*, void*, raft_index_t, raft_term_t)", raft_load_snapshot)
self.raft_clear_snapshot = ffi.callback("int(raft_server_t*, void*)", raft_clear_snapshot)
self.raft_get_snapshot_chunk = ffi.callback("int(raft_server_t*, void*, raft_node_t*, raft_size_t offset, raft_snapshot_chunk_t*)", raft_get_snapshot_chunk)
self.raft_store_snapshot_chunk = ffi.callback("int(raft_server_t*, void*, raft_index_t index, raft_size_t offset, raft_snapshot_chunk_t*)", raft_store_snapshot_chunk)
self.raft_applylog = ffi.callback("int(raft_server_t*, void*, raft_entry_t*, raft_index_t)", raft_applylog)
self.raft_persist_vote = ffi.callback("int(raft_server_t*, void*, raft_node_id_t)", raft_persist_vote)
self.raft_persist_term = ffi.callback("int(raft_server_t*, void*, raft_term_t, raft_node_id_t)", raft_persist_term)
......@@ -993,13 +1060,31 @@ class RaftServer(object):
self.snapshot.members.append(
SnapshotMember(id, lib.raft_node_is_voting_committed(n)))
def load_snapshot(self, snapshot, other):
def clear_snapshot(self):
self.snapshot_recv_buf = bytearray(SNAPSHOT_SIZE)
return 0
def store_snapshot(self, offset, data):
for i in range(0, len(data)):
self.snapshot_recv_buf[offset + i] = bytes(data)[i]
return 0
def load_snapshot(self, index, term):
logger.debug('{} loading snapshot'.format(self))
e = lib.raft_begin_load_snapshot(
self.raft,
snapshot.last_term,
snapshot.last_idx,
)
leader = find_leader()
leader_snapshot = leader.snapshot_buf
# Copy received snapshot as our snapshot and clear the temp buf
self.snapshot_buf[:] = self.snapshot_recv_buf
self.snapshot_recv_buf = bytearray(SNAPSHOT_SIZE)
# Validate received snapshot against the leader's snapshot
for i in range(len(self.snapshot_buf)):
assert self.snapshot_buf[i] == leader_snapshot[i]
e = lib.raft_begin_load_snapshot(self.raft, term, index)
logger.debug(f"return value from raft_begin_load_snapshot = {e}")
if e == -1:
return 0
......@@ -1010,17 +1095,11 @@ class RaftServer(object):
else:
assert False
# Send appendentries response for this snapshot
response = ffi.new('msg_appendentries_response_t*')
response.success = 1
response.current_idx = snapshot.last_idx
response.term = lib.raft_get_current_term(self.raft)
self.network.enqueue_msg(response, self, other)
snapshot_info = leader.snapshot
node_id = lib.raft_get_nodeid(self.raft)
# set membership configuration according to snapshot
for member in snapshot.members:
for member in snapshot_info.members:
if -1 == member.id:
continue
......@@ -1056,37 +1135,17 @@ class RaftServer(object):
assert(lib.raft_get_log_count(self.raft) == 0)
self.do_membership_snapshot()
self.snapshot.image = dict(snapshot.image)
self.snapshot.last_term = snapshot.last_term
self.snapshot.last_idx = snapshot.last_idx
self.snapshot.image = dict(snapshot_info.image)
self.snapshot.last_term = snapshot_info.last_term
self.snapshot.last_idx = snapshot_info.last_idx
assert(lib.raft_get_my_node(self.raft))
# assert(sv->snapshot_fsm);
self.fsm_dict = dict(snapshot.image)
self.fsm_dict = dict(snapshot_info.image)
# logger.warning('{} loaded snapshot t:{} idx:{}'.format(
# self, snapshot.last_term, snapshot.last_idx))
def send_snapshot(self, node):
assert not lib.raft_snapshot_is_in_progress(self.raft)
# FIXME: Why would this happen?
if not hasattr(self, 'snapshot'):
return 0
node_sv = ffi.from_handle(lib.raft_node_get_udata(node))
# TODO: Why would this happen?
# seems odd that we would send something to a node that didn't exist
if not node_sv:
return 0
# NOTE:
# In a real server we would have to send the snapshot file to the
# other node. Here we have the convenience of the transfer being
# "immediate".
node_sv.load_snapshot(self.snapshot, self)
return 0
def persist_vote(self, voted_for):
......
Markdown is supported
0% or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment