Skip to content

Commit

Permalink
Receive pending acks after assignments_revoked is invoked (#351)
Browse files Browse the repository at this point in the history
We are building a multi-process data processing pipeline
with Brod and we are using the assignments_revoked callback
to flush all messages being processed. However, if any message
is acknowledged during the assignments_revoked step, those
acks won't be committed, they will just sit in the group coordinator
inbox and eventually be processed too late.

This commit makes it so we retrieve any pending ack from the
inbox before we commit them.
  • Loading branch information
josevalim authored and k32 committed Oct 29, 2019
1 parent ee4027a commit 6bc90ef
Show file tree
Hide file tree
Showing 3 changed files with 176 additions and 24 deletions.
1 change: 1 addition & 0 deletions scripts/setup-test-env.sh
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ create_topic "brod-client-SUITE-topic"
create_topic "brod_consumer_SUITE"
create_topic "brod_producer_SUITE" 2
create_topic "brod_topic_subscriber_SUITE" 3 2
create_topic "brod-group-coordinator" 3 2
create_topic "brod-group-subscriber-1" 3 2
create_topic "brod-group-subscriber-2" 3 2
create_topic "brod-group-subscriber-3" 3 2
Expand Down
61 changes: 37 additions & 24 deletions src/brod_group_coordinator.erl
Original file line number Diff line number Diff line change
Expand Up @@ -287,14 +287,7 @@ init({Client, GroupId, Topics, Config, CbModule, MemberPid}) ->
{ok, State}.

handle_info({ack, GenerationId, Topic, Partition, Offset}, State) ->
case GenerationId < State#state.generationId of
true ->
%% Ignore stale acks
{noreply, State};
false ->
{ok, NewState} = handle_ack(State, Topic, Partition, Offset),
{noreply, NewState}
end;
{noreply, handle_ack(State, GenerationId, Topic, Partition, Offset)};
handle_info(?LO_CMD_COMMIT_OFFSETS, #state{is_in_group = true} = State) ->
{ok, NewState} =
try
Expand Down Expand Up @@ -434,29 +427,36 @@ stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds
%% 1. unsubscribe all currently assigned partitions
ok = MemberModule:assignments_revoked(MemberPid),

%% 2. try to commit current offsets before re-joinning the group.
%% 2. some brod_group_member implementations may wait for messages
%% to finish processing when assignments_revoked is called.
%% The acknowledments of those messages would then be sitting
%% in our inbox. So we do an explicit pass to collect all pending
%% acks so they are included in the best-effort commit below.
State1 = receive_pending_acks(State0),

%% 3. try to commit current offsets before re-joinning the group.
%% try only on the first re-join attempt
%% do not try if it was illegal generation exception received
%% because it will fail on the same exception again
State1 =
State2 =
case AttemptNo =:= 0 andalso
Reason =/= ?illegal_generation of
true ->
{ok, #state{} = State1_} = try_commit_offsets(State0),
State1_;
{ok, #state{} = State2_} = try_commit_offsets(State1),
State2_;
false ->
State0
State1
end,
State2 = State1#state{is_in_group = false},
State3 = State2#state{is_in_group = false},

%$ 3. Clean up state based on the last failure reason
State = maybe_reset_member_id(State2, Reason),
%$ 4. Clean up state based on the last failure reason
State = maybe_reset_member_id(State3, Reason),

%% 4. ensure we have a connection to the (maybe new) group coordinator
%% 5. ensure we have a connection to the (maybe new) group coordinator
F1 = fun discover_coordinator/1,
%% 5. join group
%% 6. join group
F2 = fun join_group/1,
%% 6. sync assignemnts
%% 7. sync assignemnts
F3 = fun sync_group/1,

RetryFun =
Expand All @@ -474,6 +474,16 @@ stabilize(#state{ rejoin_delay_seconds = RejoinDelaySeconds
end,
do_stabilize([F1, F2, F3], RetryFun, State).

-spec receive_pending_acks(state()) -> state().
receive_pending_acks(State) ->
receive
{ack, GenerationId, Topic, Partition, Offset} ->
NewState = handle_ack(State, GenerationId, Topic, Partition, Offset),
receive_pending_acks(NewState)
after
0 -> State
end.

do_stabilize([], _RetryFun, State) ->
{ok, State};
do_stabilize([F | Rest], RetryFun, State) ->
Expand Down Expand Up @@ -579,13 +589,16 @@ sync_group(#state{ groupId = GroupId
[format_assignments(TopicAssignments)]),
start_offset_commit_timer(NewState).

-spec handle_ack(state(), brod:topic(), brod:partition(), brod:offset()) ->
{ok, state()}.
handle_ack(#state{ acked_offsets = AckedOffsets
} = State, Topic, Partition, Offset) ->
-spec handle_ack(state(), brod:group_generation_id(), brod:topic(),
brod:partition(), brod:offset()) -> state().
handle_ack(State, GenerationId, _Topic, _Partition, _Offset)
when GenerationId < State#state.generationId ->
State;
handle_ack(#state{acked_offsets = AckedOffsets} = State,
_GenerationId, Topic, Partition, Offset) ->
NewAckedOffsets =
merge_acked_offsets(AckedOffsets, [{{Topic, Partition}, Offset}]),
{ok, State#state{acked_offsets = NewAckedOffsets}}.
State#state{acked_offsets = NewAckedOffsets}.

%% Add new offsets to be acked into the acked offsets collection.
-spec merge_acked_offsets(Offsets, Offsets) -> Offsets when
Expand Down
138 changes: 138 additions & 0 deletions test/brod_group_coordinator_SUITE.erl
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
%%%
%%% Copyright (c) 2015-2019 Klarna Bank AB (publ)
%%%
%%% Licensed under the Apache License, Version 2.0 (the "License");
%%% you may not use this file except in compliance with the License.
%%% You may obtain a copy of the License at
%%%
%%% http://www.apache.org/licenses/LICENSE-2.0
%%%
%%% Unless required by applicable law or agreed to in writing, software
%%% distributed under the License is distributed on an "AS IS" BASIS,
%%% WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
%%% See the License for the specific language governing permissions and
%%% limitations under the License.
%%%

%% @private
-module(brod_group_coordinator_SUITE).
-define(CLIENT_ID, ?MODULE).
-define(OTHER_CLIENT_ID, other_coordinator_id).
-define(TOPIC, <<"brod-group-coordinator">>).
-define(GROUP, <<"brod-group-coordinator">>).
-define(PARTITION, 0).

%% Test framework
-export([ init_per_suite/1
, end_per_suite/1
, common_init_per_testcase/2
, common_end_per_testcase/2
, suite/0
]).

%% brod coordinator callbacks
-export([ assignments_revoked/1
, assignments_received/4
]).

%% Test cases
-export([ t_acks_during_revoke/1 ]).

-define(assert_receive(Pattern, Return),
receive
Pattern -> Return
after
30000 -> ct:fail(erlang:process_info(self(), messages))
end).

-include_lib("snabbkaffe/include/ct_boilerplate.hrl").
-include("brod.hrl").

%%%_* ct callbacks =============================================================

suite() -> [{timetrap, {seconds, 60}}].

init_per_suite(Config) -> Config.
end_per_suite(_Config) -> ok.

common_init_per_testcase(_Case, Config) ->
{ok, _} = application:ensure_all_started(brod),
BootstrapHosts = [{"localhost", 9092}],
ClientConfig = client_config(),
ok = brod:start_client(BootstrapHosts, ?CLIENT_ID, ClientConfig),
ok = brod:start_client(BootstrapHosts, ?OTHER_CLIENT_ID, ClientConfig),
ok = brod:start_producer(?CLIENT_ID, ?TOPIC, _ProducerConfig = []),
Config.

common_end_per_testcase(_Case, Config) when is_list(Config) ->
ok = brod:stop_client(?CLIENT_ID),
ok = brod:stop_client(?OTHER_CLIENT_ID),
ok = application:stop(brod).

client_config() ->
case os:getenv("KAFKA_VERSION") of
"0.9" ++ _ -> [{query_api_versions, false}];
_ -> []
end.

%%%_* Group coordinator callbacks ==============================================

assignments_revoked({Pid, Count}) ->
Pid ! {assignments_revoked, Count},
receive continue -> ok end,
ok.

assignments_received({Pid, Count}, _MemberId, GenerationId, TopicAssignments) ->
Pid ! {assignments_received, Count, GenerationId, TopicAssignments},
ok.

%%%_* Test functions ===========================================================

t_acks_during_revoke(Config) when is_list(Config) ->
{ok, GroupCoordinator1Pid} =
brod_group_coordinator:start_link(?CLIENT_ID, ?GROUP, [?TOPIC],
_Config = [], ?MODULE, {self(), 1}),

?assert_receive({assignments_revoked, 1}, ok),
GroupCoordinator1Pid ! continue,
GenerationId = ?assert_receive({assignments_received, 1, GId, _}, GId),

{ok, Offset} =
brod:produce_sync_offset(?CLIENT_ID, ?TOPIC, ?PARTITION, <<>>, <<1, 2, 3>>),

{ok, {_, [_]}} = brod:fetch(?CLIENT_ID, ?TOPIC, ?PARTITION, Offset),

{ok, GroupCoordinator2Pid} =
brod_group_coordinator:start_link(?OTHER_CLIENT_ID, ?GROUP, [?TOPIC],
_Config = [], ?MODULE, {self(), 2}),

%% Allow new partition to be started
?assert_receive({assignments_revoked, 2}, ok),
GroupCoordinator2Pid ! continue,

%% We only ack when we are inside assignments_revoked
?assert_receive({assignments_revoked, 1}, ok),
brod_group_coordinator:ack(GroupCoordinator1Pid, GenerationId,
?TOPIC, ?PARTITION, Offset),
GroupCoordinator1Pid ! continue,

TopicAssignments1 = ?assert_receive({assignments_received, 1, _, TA1}, TA1),
TopicAssignments2 = ?assert_receive({assignments_received, 2, _, TA2}, TA2),
Assignments = TopicAssignments1 ++ TopicAssignments2,

%% The assigment needs to start at the chosen offset.
?assertMatch( [ok]
, [ok || #brod_received_assignment{
partition=?PARTITION,
begin_offset=BeginOffset
} <- Assignments,
BeginOffset == Offset + 1]
),

ok.

%%%_* Emacs ====================================================================
%%% Local Variables:
%%% allout-layout: t
%%% erlang-indent-level: 2
%%% End:

0 comments on commit 6bc90ef

Please sign in to comment.