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 ;
415416handle_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 ;
422437handle_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
523538handle_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
0 commit comments