Skip to content

Commit 1fc98f1

Browse files
authored
Merge pull request #173 from rabbitmq/gh_172
Fix bug that would delay offset listener notifications.
2 parents 36e055a + 69a7e2e commit 1fc98f1

File tree

3 files changed

+78
-52
lines changed

3 files changed

+78
-52
lines changed

src/osiris_log.erl

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,9 @@
3636
read_chunk_parsed/1,
3737
read_chunk_parsed/2,
3838
committed_offset/1,
39+
committed_chunk_id/1,
3940
set_committed_chunk_id/2,
41+
last_chunk_id/1,
4042
get_current_epoch/1,
4143
get_directory/1,
4244
get_name/1,
@@ -538,6 +540,7 @@ init(#{dir := Dir,
538540
case first_and_last_seginfos(Config) of
539541
none ->
540542
osiris_log_shared:set_first_chunk_id(Shared, DefaultNextOffset - 1),
543+
osiris_log_shared:set_last_chunk_id(Shared, DefaultNextOffset - 1),
541544
open_new_segment(#?MODULE{cfg = Cfg,
542545
mode =
543546
#write{type = WriterType,
@@ -607,6 +610,7 @@ init(#{dir := Dir,
607610
ok = file:truncate(SegFd),
608611
{ok, _} = file:position(IdxFd, ?IDX_HEADER_SIZE),
609612
osiris_log_shared:set_first_chunk_id(Shared, DefaultNextOffset - 1),
613+
osiris_log_shared:set_last_chunk_id(Shared, DefaultNextOffset - 1),
610614
#?MODULE{cfg = Cfg,
611615
mode =
612616
#write{type = WriterType,
@@ -1333,7 +1337,11 @@ last_user_chunk_id_in_index(NextPos, IdxFd) ->
13331337
end.
13341338

13351339
-spec committed_offset(state()) -> integer().
1336-
committed_offset(#?MODULE{cfg = #cfg{shared = Ref}}) ->
1340+
committed_offset(State) ->
1341+
committed_chunk_id(State).
1342+
1343+
-spec committed_chunk_id(state()) -> integer().
1344+
committed_chunk_id(#?MODULE{cfg = #cfg{shared = Ref}}) ->
13371345
osiris_log_shared:committed_chunk_id(Ref).
13381346

13391347
-spec set_committed_chunk_id(state(), offset()) -> ok.
@@ -1342,6 +1350,10 @@ set_committed_chunk_id(#?MODULE{mode = #write{},
13421350
when is_integer(ChunkId) ->
13431351
osiris_log_shared:set_committed_chunk_id(Ref, ChunkId).
13441352

1353+
-spec last_chunk_id(state()) -> integer().
1354+
last_chunk_id(#?MODULE{cfg = #cfg{shared = Ref}}) ->
1355+
osiris_log_shared:last_chunk_id(Ref).
1356+
13451357
-spec get_current_epoch(state()) -> non_neg_integer().
13461358
get_current_epoch(#?MODULE{mode = #write{current_epoch = Epoch}}) ->
13471359
Epoch.

src/osiris_replica.erl

Lines changed: 60 additions & 51 deletions
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@
5959
{cfg :: #cfg{},
6060
parse_state :: parse_state(),
6161
log :: osiris_log:state(),
62-
committed_offset = -1 :: -1 | osiris:offset(),
62+
committed_chunk_id = -1 :: -1 | osiris:offset(),
6363
offset_listeners = [] ::
6464
[{pid(), osiris:offset(), mfa() | undefined}]
6565
}).
@@ -364,7 +364,7 @@ handle_call(get_reader_context, _From,
364364
directory = Dir,
365365
reference = Ref,
366366
counter = CntRef},
367-
committed_offset = COffs,
367+
committed_chunk_id = COffs,
368368
log = Log} =
369369
State) ->
370370
Shared = osiris_log:get_shared(Log),
@@ -397,28 +397,43 @@ handle_call(Unknown, _From,
397397
%% {stop, Reason, State}
398398
%% @end
399399
%%--------------------------------------------------------------------
400-
handle_cast({committed_offset, Offs},
400+
handle_cast({committed_offset, CommittedChId},
401401
#?MODULE{cfg = #cfg{counter = Cnt},
402402
log = Log,
403-
committed_offset = Last} =
403+
committed_chunk_id = LastCommittedChId} =
404404
State) ->
405-
case Offs > Last of
405+
case CommittedChId > LastCommittedChId of
406406
true ->
407407
%% notify offset listeners
408-
counters:put(Cnt, ?C_COMMITTED_OFFSET, Offs),
409-
ok = osiris_log:set_committed_chunk_id(Log, Offs),
408+
counters:put(Cnt, ?C_COMMITTED_OFFSET, CommittedChId),
409+
ok = osiris_log:set_committed_chunk_id(Log, CommittedChId),
410410
{noreply,
411-
notify_offset_listeners(State#?MODULE{committed_offset = Offs})};
411+
notify_offset_listeners(
412+
State#?MODULE{committed_chunk_id = CommittedChId})};
412413
false ->
413414
State
414415
end;
415416
handle_cast({register_offset_listener, Pid, EvtFormatter, Offset},
416-
#?MODULE{offset_listeners = Listeners} = State0) ->
417-
State1 =
418-
State0#?MODULE{offset_listeners =
419-
[{Pid, Offset, EvtFormatter} | Listeners]},
420-
State = notify_offset_listeners(State1),
421-
{noreply, State};
417+
#?MODULE{cfg = #cfg{reference = Ref,
418+
event_formatter = DefaultFmt},
419+
log = Log,
420+
offset_listeners = Listeners} = State) ->
421+
Max = max_readable_chunk_id(Log),
422+
case Offset =< Max of
423+
true ->
424+
%% only evaluate the request, the rest will be evaluated
425+
%% when data is written or committed
426+
Evt = wrap_osiris_event(
427+
select_formatter(EvtFormatter, DefaultFmt),
428+
{osiris_offset, Ref, Max}),
429+
Pid ! Evt,
430+
{noreply, State};
431+
false ->
432+
%% queue the offset listener for later
433+
{noreply,
434+
State#?MODULE{offset_listeners = [{Pid, Offset, EvtFormatter} |
435+
Listeners]}}
436+
end;
422437
handle_cast(Msg, #?MODULE{cfg = #cfg{name = Name}} = State) ->
423438
?DEBUG_(Name, "osiris_replica unhandled cast ~w", [Msg]),
424439
{noreply, State}.
@@ -521,14 +536,14 @@ handle_info({'EXIT', Ref, Info},
521536
{stop, unexpected_exit, State}.
522537

523538
handle_incoming_data(Socket, Bin,
524-
#?MODULE{cfg =
525-
#cfg{socket = Socket,
526-
leader_pid = LeaderPid,
527-
transport = Transport,
528-
counter = Cnt},
529-
parse_state = ParseState0,
530-
log = Log0} =
531-
State0) ->
539+
#?MODULE{cfg =
540+
#cfg{socket = Socket,
541+
leader_pid = LeaderPid,
542+
transport = Transport,
543+
counter = Cnt},
544+
parse_state = ParseState0,
545+
log = Log0} =
546+
State0) ->
532547
counters:add(Cnt, ?C_PACKETS, 1),
533548
%% deliberately ignoring return value here as it would fail if the
534549
%% tcp connection has been closed and we still want to try to process
@@ -547,8 +562,8 @@ handle_incoming_data(Socket, Bin,
547562
undefined ->
548563
{noreply, State1};
549564
_ ->
550-
State = notify_offset_listeners(State1),
551565
ok = osiris_writer:ack(LeaderPid, OffsetTimestamp),
566+
State = notify_offset_listeners(State1),
552567
{noreply, State}
553568
end.
554569

@@ -594,7 +609,7 @@ format_status(#{state := #?MODULE{cfg = #cfg{name = Name,
594609
log = Log,
595610
parse_state = ParseState,
596611
offset_listeners = OffsetListeners,
597-
committed_offset = CommittedOffset}} = Status) ->
612+
committed_chunk_id = CommittedOffset}} = Status) ->
598613
maps:update(state,
599614
#{name => Name,
600615
external_reference => ExtRef,
@@ -670,34 +685,28 @@ parse_chunk(Bin, {FirstOffsetTs, IOData, RemSize}, Acc) ->
670685
{{FirstOffsetTs, [Bin | IOData], RemSize - byte_size(Bin)},
671686
lists:reverse(Acc)}.
672687

673-
notify_offset_listeners(#?MODULE{cfg =
674-
#cfg{reference = Ref,
675-
event_formatter = EvtFmt},
676-
committed_offset = COffs,
688+
notify_offset_listeners(#?MODULE{cfg = #cfg{reference = Ref,
689+
event_formatter = EvtFmt},
690+
committed_chunk_id = CommittedChId,
677691
log = Log,
678-
offset_listeners = L0} =
679-
State) ->
680-
case osiris_log:tail_info(Log) of
681-
{_NextOffs, {_, LastChId, _LastTs}} ->
682-
Max = min(COffs, LastChId),
683-
%% do not notify offset listeners if the committed offset isn't
684-
%% available locally yet
685-
{Notify, L} =
686-
lists:splitwith(fun({_Pid, O, _}) -> O =< Max end, L0),
687-
_ = [begin
688-
Evt =
689-
%% the per offset listener event formatter takes precedence of
690-
%% the process scoped one
691-
wrap_osiris_event(
692-
select_formatter(Fmt, EvtFmt),
693-
{osiris_offset, Ref, COffs}),
694-
P ! Evt
695-
end
696-
|| {P, _, Fmt} <- Notify],
697-
State#?MODULE{offset_listeners = L};
698-
_ ->
699-
State
700-
end.
692+
offset_listeners = L0} = State) ->
693+
Max = max_readable_chunk_id(Log),
694+
{Notify, L} =
695+
lists:partition(fun({_Pid, O, _}) -> O =< Max end, L0),
696+
_ = [begin
697+
Evt =
698+
%% the per offset listener event formatter takes precedence of
699+
%% the process scoped one
700+
wrap_osiris_event(
701+
select_formatter(Fmt, EvtFmt),
702+
{osiris_offset, Ref, CommittedChId}),
703+
P ! Evt
704+
end
705+
|| {P, _, Fmt} <- Notify],
706+
State#?MODULE{offset_listeners = L}.
707+
708+
max_readable_chunk_id(Log) ->
709+
min(osiris_log:committed_offset(Log), osiris_log:last_chunk_id(Log)).
701710

702711
%% INTERNAL
703712

test/osiris_SUITE.erl

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -535,6 +535,7 @@ cluster_offset_listener(Config) ->
535535
replica_nodes => Replicas},
536536
{ok, #{leader_pid := Leader}} = osiris:start_cluster(Conf0),
537537
{ok, Log0} = osiris:init_reader(Leader, 0, {test, []}),
538+
0 = osiris_log:next_offset(Log0),
538539
osiris:register_offset_listener(Leader, 0),
539540
ok = osiris:write(Leader, undefined, 42, <<"mah-data">>),
540541
receive
@@ -572,6 +573,9 @@ replica_offset_listener(Config) ->
572573
fun() ->
573574
{ok, Log0} = osiris:init_reader(R, 0, {test, []}),
574575
osiris:register_offset_listener(R, 0),
576+
%% gh 172 - ensure a higher offset listener does not block
577+
%% a lower one
578+
osiris:register_offset_listener(R, 1),
575579
receive
576580
{osiris_offset, _Name, O} when O > -1 ->
577581
ct:pal("got offset ~w", [O]),
@@ -585,6 +589,7 @@ replica_offset_listener(Config) ->
585589
exit(osiris_offset_timeout)
586590
end
587591
end),
592+
timer:sleep(100),
588593
ok = osiris:write(Leader, undefined, 42, <<"mah-data">>),
589594

590595
receive

0 commit comments

Comments
 (0)