Skip to content

Commit

Permalink
Merge pull request #499 from rabbitmq/log-tweaks
Browse files Browse the repository at this point in the history
Improve some multi line log messages.
  • Loading branch information
kjnilsson authored Jan 16, 2025
2 parents d333e83 + 0a5307f commit cc54084
Show file tree
Hide file tree
Showing 6 changed files with 24 additions and 23 deletions.
8 changes: 4 additions & 4 deletions src/ra.erl
Original file line number Diff line number Diff line change
Expand Up @@ -444,7 +444,7 @@ start_cluster(System, [#{cluster_name := ClusterName} | _] = ServerConfigs,
end, ServerConfigs),
case Started of
[] ->
?ERR("ra: failed to form a new cluster ~w.~n "
?ERR("ra: failed to form a new cluster ~w. "
"No servers were successfully started.",
[ClusterName]),
{error, cluster_not_formed};
Expand All @@ -460,15 +460,15 @@ start_cluster(System, [#{cluster_name := ClusterName} | _] = ServerConfigs,
case members(TriggeredId,
length(ServerConfigs) * Timeout) of
{ok, _, Leader} ->
?INFO("ra: started cluster ~ts with ~b servers~n"
"~b servers failed to start: ~w~nLeader: ~w",
?INFO("ra: started cluster ~ts with ~b servers. "
"~b servers failed to start: ~w. Leader: ~w",
[ClusterName, length(ServerConfigs),
length(NotStarted), NotStartedIds,
Leader]),
% we have a functioning cluster
{ok, StartedIds, NotStartedIds};
Err ->
?WARN("ra: failed to form new cluster ~w.~n "
?WARN("ra: failed to form new cluster ~w. "
"Error: ~w", [ClusterName, Err]),
_ = [force_delete_server(System, N) || N <- StartedIds],
% we do not have a functioning cluster
Expand Down
2 changes: 1 addition & 1 deletion src/ra_log.erl
Original file line number Diff line number Diff line change
Expand Up @@ -1109,7 +1109,7 @@ delete_everything(#?MODULE{cfg = #cfg{uid = UId,
catch
_:_ = Err ->
?WARN("ra_log:delete_everything/1 failed to delete "
"directory ~ts~n Error: ~p", [Dir, Err])
"directory ~ts. Error: ~p", [Dir, Err])
end,
ok.

Expand Down
26 changes: 14 additions & 12 deletions src/ra_server.erl
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
%% @hidden
-module(ra_server).

-include_lib("stdlib/include/assert.hrl").
-include("ra.hrl").
-include("ra_server.hrl").

Expand Down Expand Up @@ -1160,6 +1161,7 @@ handle_follower(#append_entries_rpc{term = Term,
State00 = #{cfg := #cfg{log_id = LogId,
id = Id} = Cfg,
log := Log00,
commit_index := CommitIndex,
current_term := CurTerm})
when Term >= CurTerm ->
ok = incr_counter(Cfg, ?C_RA_SRV_AER_RECEIVED_FOLLOWER, 1),
Expand All @@ -1175,13 +1177,14 @@ handle_follower(#append_entries_rpc{term = Term,
case Entries of
[] ->
ok = incr_counter(Cfg, ?C_RA_SRV_AER_RECEIVED_FOLLOWER_EMPTY, 1),
LastIdx = ra_log:last_index_term(Log1),
{LastIdx, _} = ra_log:last_index_term(Log1),
Log2 = case Entries0 of
[] when element(1, LastIdx) > PLIdx ->
[] when LastIdx > PLIdx ->
%% if no entries were sent we need to reset
%% last index to match the leader
?DEBUG("~ts: resetting last index to ~b",
[LogId, PLIdx]),
?DEBUG("~ts: resetting last index to ~b from ~b in term ~b",
[LogId, PLIdx, LastIdx, Term]),
?assertNot(PLIdx =< CommitIndex),
{ok, L} = ra_log:set_last_index(PLIdx, Log1),
L;
_ ->
Expand Down Expand Up @@ -1218,7 +1221,6 @@ handle_follower(#append_entries_rpc{term = Term,
%% alternative where the WAL writes the last index, term
%% it wrote for each UID into an ETS table and query
%% this.
% Log = ra_log:reset_to_last_known_written(Log1),
Log = Log1,
{await_condition,
State1#{log => Log,
Expand Down Expand Up @@ -1249,7 +1251,7 @@ handle_follower(#append_entries_rpc{term = Term,
%% NB: this is the commit index before update
LastApplied = maps:get(last_applied, State00),
?INFO("~ts: term mismatch - follower had entry at ~b with term ~b "
"but not with term ~b~n"
"but not with term ~b. "
"Asking leader ~w to resend from ~b",
[LogId, PLIdx, OtherTerm, PLTerm, LeaderId, LastApplied + 1]),
% This situation arises when a minority leader replicates entries
Expand Down Expand Up @@ -1346,16 +1348,16 @@ handle_follower(#request_vote_rpc{term = Term, candidate_id = Cand,
LastIdxTerm = last_idx_term(State1),
case is_candidate_log_up_to_date(LLIdx, LLTerm, LastIdxTerm) of
true ->
?INFO("~ts: granting vote for ~w with last indexterm ~w"
?INFO("~ts: granting vote for ~w with last {index, term} ~w"
" for term ~b previous term was ~b",
[LogId, Cand, {LLIdx, LLTerm}, Term, CurTerm]),
Reply = #request_vote_result{term = Term, vote_granted = true},
State = update_term_and_voted_for(Term, Cand, State1),
{follower, State, [{reply, Reply}]};
false ->
?INFO("~ts: declining vote for ~w for term ~b,"
" candidate last log index term was: ~w~n"
" last log entry idxterm seen was: ~w",
" candidate last log {index, term} was: ~w "
" last log entry {index, term} is: ~w",
[LogId, Cand, Term, {LLIdx, LLTerm}, {LastIdxTerm}]),
Reply = #request_vote_result{term = Term, vote_granted = false},
{follower, update_term(Term, State1), [{reply, Reply}]}
Expand Down Expand Up @@ -2399,7 +2401,7 @@ process_pre_vote(FsmState, #pre_vote_rpc{term = Term, candidate_id = Cand,
TheirMacVer =< OurMacVer) ->
?DEBUG("~ts: granting pre-vote for ~w"
" machine version (their:ours:effective) ~b:~b:~b"
" with last indexterm ~w"
" with last {index, term} ~w"
" for term ~b previous term ~b",
[log_id(State0), Cand, TheirMacVer, OurMacVer, EffMacVer,
{LLIdx, LLTerm}, Term, CurTerm]),
Expand All @@ -2413,8 +2415,8 @@ process_pre_vote(FsmState, #pre_vote_rpc{term = Term, candidate_id = Cand,
start_election_timeout]};
false ->
?DEBUG("~ts: declining pre-vote for ~w for term ~b,"
" candidate last log index term was: ~w~n"
"Last log entry idxterm seen was: ~w",
" candidate last log {index, term} was: ~w "
"last log entry {index, term} seen is: ~w",
[log_id(State0), Cand, Term, {LLIdx, LLTerm}, LastIdxTerm]),
case FsmState of
follower ->
Expand Down
6 changes: 3 additions & 3 deletions src/ra_server_sup_sup.erl
Original file line number Diff line number Diff line change
Expand Up @@ -171,7 +171,7 @@ delete_server_rpc(System, RaName) ->
#{data_dir := _SysDir,
names := #{log_meta := Meta,
server_sup := SrvSup} = Names} ->
?INFO("Deleting server ~w and its data directory.~n",
?INFO("Deleting server ~w and its data directory.",
[RaName]),
%% TODO: better handle and report errors
%% UId could be `undefined' here
Expand Down Expand Up @@ -201,7 +201,7 @@ delete_data_directory(Directory) ->
ok
catch
_:_ = Err ->
?WARN("ra: delete_server/1 failed to delete directory ~ts~n"
?WARN("ra: delete_server/1 failed to delete directory ~ts. "
"Error: ~p", [Directory, Err]),
error
end
Expand All @@ -218,7 +218,7 @@ delete_data_directory(Directory) ->
remove_all(System) when is_atom(System) ->
#{names := #{server_sup := Sup}} = ra_system:fetch(System),
_ = [begin
?DEBUG("ra: terminating child ~w in system ~ts~n", [Pid, System]),
?DEBUG("ra: terminating child ~w in system ~ts", [Pid, System]),
supervisor:terminate_child(Sup, Pid)
end
|| {_, Pid, _, _} <- supervisor:which_children(Sup)],
Expand Down
2 changes: 1 addition & 1 deletion src/ra_system_recover.erl
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ init([System]) ->
ok ->
ok
catch C:E:S ->
?ERROR("~s: ~s encountered during server recovery ~p~n "
?ERROR("~s: ~s encountered during server recovery ~p. "
"stack ~p",
[?MODULE, C, E, S]),
ok
Expand Down
3 changes: 1 addition & 2 deletions test/ra_server_SUITE.erl
Original file line number Diff line number Diff line change
Expand Up @@ -533,7 +533,7 @@ follower_aer_5(_Config) ->
%% next index
Init = empty_state(3, n2),
AER1 = #append_entries_rpc{term = 1, leader_id = N1, prev_log_index = 0,
prev_log_term = 0, leader_commit = 10,
prev_log_term = 0, leader_commit = 2,
entries = [
entry(1, 1, one),
entry(2, 1, two),
Expand All @@ -555,7 +555,6 @@ follower_aer_5(_Config) ->
?assertMatch(#append_entries_reply{next_index = 4,
last_term = 1,
last_index = 3}, M),
% ct:pal("Effects ~p~n State: ~p", [Effects, State1]),
ok.


Expand Down

0 comments on commit cc54084

Please sign in to comment.