@@ -1006,7 +1006,7 @@ leader(enter, PreviousStateName, #raft_state{self = Self, log_view = View0} = St
10061006 % quorum in the new term by starting replication and clearing out
10071007 % any log mismatches on follower replicas.
10081008 {State4 , LogEntry } = get_log_entry (State3 , {make_ref (), noop }),
1009- {ok , View1 } = wa_raft_log :append (View0 , [LogEntry ], strict ),
1009+ {ok , View1 } = wa_raft_log :append (View0 , [LogEntry ]),
10101010 TermStartIndex = wa_raft_log :last_index (View1 ),
10111011 State5 = State4 # raft_state {log_view = View1 , first_current_term_log_index = TermStartIndex },
10121012
@@ -1308,7 +1308,7 @@ leader(
13081308 % soon as possible.
13091309 Op = {make_ref (), {config , NewConfig }},
13101310 {State1 , LogEntry } = get_log_entry (State0 , Op ),
1311- {ok , View1 } = wa_raft_log :append (View0 , [LogEntry ], strict ),
1311+ {ok , View1 } = wa_raft_log :append (View0 , [LogEntry ]),
13121312 LogIndex = wa_raft_log :last_index (View1 ),
13131313 ? RAFT_LOG_NOTICE (State1 , " appended configuration change from ~0p to ~0p at log index ~0p ." , [Config , NewConfig , LogIndex ]),
13141314 State2 = State1 # raft_state {log_view = View1 },
@@ -2637,7 +2637,7 @@ commit_pending(#raft_state{pending = [], pending_read = false} = Data) ->
26372637 Data ;
26382638commit_pending (# raft_state {log_view = View } = Data0 ) ->
26392639 {Entries , Data1 } = collect_pending (Data0 ),
2640- case wa_raft_log :append (View , Entries , relaxed ) of
2640+ case wa_raft_log :try_append (View , Entries ) of
26412641 {ok , NewView } ->
26422642 % We can clear pending read flag as we've successfully added at
26432643 % least one new log entry so the leader will proceed to replicate
@@ -2982,8 +2982,16 @@ append_entries(
29822982 % No conflicting log entries were found in the heartbeat, but the
29832983 % heartbeat does contain new log entries to be appended to the end
29842984 % of the log.
2985- {ok , View1 } = wa_raft_log :append (View0 , NewEntries ),
2986- {ok , true , PrevLogIndex + EntryCount , Data # raft_state {log_view = View1 }};
2985+ case wa_raft_log :try_append (View0 , NewEntries ) of
2986+ {ok , View1 } ->
2987+ {ok , true , PrevLogIndex + EntryCount , Data # raft_state {log_view = View1 }};
2988+ skipped ->
2989+ NewCount = length (NewEntries ),
2990+ Last = wa_raft_log :last_index (View0 ),
2991+ ? RAFT_LOG_WARNING (State , Data , " is not ready to append ~0p log entries in range ~0p to ~0p to log ending at ~0p ." ,
2992+ [NewCount , Last + 1 , Last + NewCount , Last ]),
2993+ {ok , false , Last , Data }
2994+ end ;
29872995 {conflict , ConflictIndex , [ConflictEntry | _ ]} when ConflictIndex =< CommitIndex ->
29882996 % A conflict is detected that would result in the truncation of a
29892997 % log entry that the local replica has committed. We cannot validly
@@ -3016,8 +3024,16 @@ append_entries(
30163024 false ->
30173025 % Otherwise, we can replace the truncated log
30183026 % entries with those from the current heartbeat.
3019- {ok , View2 } = wa_raft_log :append (View1 , NewEntries ),
3020- {ok , true , PrevLogIndex + EntryCount , Data # raft_state {log_view = View2 }}
3027+ case wa_raft_log :try_append (View1 , NewEntries ) of
3028+ {ok , View2 } ->
3029+ {ok , true , PrevLogIndex + EntryCount , Data # raft_state {log_view = View2 }};
3030+ skipped ->
3031+ NewCount = length (NewEntries ),
3032+ NewLast = wa_raft_log :last_index (View1 ),
3033+ ? RAFT_LOG_WARNING (State , Data , " is not ready to append ~0p log entries in range ~0p to ~0p to log ending at ~0p ." ,
3034+ [NewCount , NewLast + 1 , NewLast + NewCount , NewLast ]),
3035+ {ok , false , NewLast , Data }
3036+ end
30213037 end ;
30223038 {error , Reason } ->
30233039 ? RAFT_COUNT ({raft , State , 'heartbeat.truncate.error' }),
0 commit comments