Merge pull request #8453 from cloudamqp/cqv1_missing_del
Handle missing delivery marker in CQ v1 index
This commit is contained in:
		
						commit
						6d2e497382
					
				| 
						 | 
				
			
			@ -311,7 +311,7 @@ rabbitmq_integration_suite(
 | 
			
		|||
rabbitmq_integration_suite(
 | 
			
		||||
    name = "classic_queue_prop_SUITE",
 | 
			
		||||
    size = "large",
 | 
			
		||||
    shard_count = 3,
 | 
			
		||||
    shard_count = 6,
 | 
			
		||||
    sharding_method = "case",
 | 
			
		||||
    deps = [
 | 
			
		||||
        "@proper//:erlang_app",
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -952,6 +952,10 @@ action_to_entry(RelSeq, Action, JEntries) ->
 | 
			
		|||
        ({no_pub,    del, no_ack}) when Action == ack ->
 | 
			
		||||
            {set, {no_pub, del,    ack}};
 | 
			
		||||
        ({?PUB,      del, no_ack}) when Action == ack ->
 | 
			
		||||
            {reset, none};
 | 
			
		||||
        %% Special case, missing del
 | 
			
		||||
        %% See journal_minus_segment1/2
 | 
			
		||||
        ({?PUB,   no_del, no_ack}) when Action == ack ->
 | 
			
		||||
            {reset, none}
 | 
			
		||||
    end.
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -1342,6 +1346,11 @@ segment_plus_journal1({?PUB = Pub, no_del, no_ack}, {no_pub, del, no_ack}) ->
 | 
			
		|||
segment_plus_journal1({?PUB, no_del, no_ack},       {no_pub, del, ack}) ->
 | 
			
		||||
    {undefined, -1};
 | 
			
		||||
segment_plus_journal1({?PUB, del, no_ack},          {no_pub, no_del, ack}) ->
 | 
			
		||||
    {undefined, -1};
 | 
			
		||||
 | 
			
		||||
%% Special case, missing del
 | 
			
		||||
%% See journal_minus_segment1/2
 | 
			
		||||
segment_plus_journal1({?PUB, no_del, no_ack},          {no_pub, no_del, ack}) ->
 | 
			
		||||
    {undefined, -1}.
 | 
			
		||||
 | 
			
		||||
%% Remove from the journal entries for a segment, items that are
 | 
			
		||||
| 
						 | 
				
			
			@ -1413,6 +1422,16 @@ journal_minus_segment1({no_pub, no_del, ack},      {?PUB, del, no_ack}) ->
 | 
			
		|||
journal_minus_segment1({no_pub, no_del, ack},      {?PUB, del, ack}) ->
 | 
			
		||||
    {undefined, -1};
 | 
			
		||||
 | 
			
		||||
%% Just ack in journal, missing del
 | 
			
		||||
%% Since 3.10 message delivery is tracked per-queue, not per-message,
 | 
			
		||||
%% but to keep queue index v1 format messages are always marked as
 | 
			
		||||
%% delivered on publish. But for a message that was published before
 | 
			
		||||
%% 3.10 this is not the case and the delivery marker can be missing.
 | 
			
		||||
%% As a workaround we add the del marker because if a message is acked
 | 
			
		||||
%% it must have been delivered as well.
 | 
			
		||||
journal_minus_segment1({no_pub, no_del, ack},         {?PUB, no_del, no_ack}) ->
 | 
			
		||||
    {{no_pub, del, ack}, 0};
 | 
			
		||||
 | 
			
		||||
%% Deliver and ack in journal
 | 
			
		||||
journal_minus_segment1({no_pub, del, ack},         {?PUB, no_del, no_ack}) ->
 | 
			
		||||
    {keep, 0};
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
| 
						 | 
				
			
			@ -82,7 +82,10 @@ groups() ->
 | 
			
		|||
        classic_queue_v2
 | 
			
		||||
     ]},
 | 
			
		||||
     {classic_queue_regressions, [], [
 | 
			
		||||
        reg_v1_full_recover_only_journal
 | 
			
		||||
        reg_v1_full_recover_only_journal,
 | 
			
		||||
        reg_v1_no_del_jif,
 | 
			
		||||
        reg_v1_no_del_idx,
 | 
			
		||||
        reg_v1_no_del_idx_unclean
 | 
			
		||||
     ]}
 | 
			
		||||
    ].
 | 
			
		||||
 | 
			
		||||
| 
						 | 
				
			
			@ -1122,6 +1125,244 @@ do_reg_v1_full_recover_only_journal(Config) ->
 | 
			
		|||
 | 
			
		||||
    Res15 = cmd_restart_vhost_clean(St14),
 | 
			
		||||
    true = postcondition(St14, {call, undefined, cmd_restart_vhost_clean, [St14]}, Res15),
 | 
			
		||||
    _ = next_state(St14, Res15, {call, undefined, cmd_restart_vhost_clean, [St14]}),
 | 
			
		||||
    St15 = next_state(St14, Res15, {call, undefined, cmd_restart_vhost_clean, [St14]}),
 | 
			
		||||
 | 
			
		||||
    cmd_teardown_queue(St15),
 | 
			
		||||
 | 
			
		||||
    true.
 | 
			
		||||
 | 
			
		||||
%% The following reg_v1_no_del_* cases test when a classic queue has a
 | 
			
		||||
%% published message before an upgrade to 3.10. In that case there is
 | 
			
		||||
%% no delivery marker in the v1 queue index.
 | 
			
		||||
 | 
			
		||||
%% After upgrade to 3.10 there is a published message in the journal file.
 | 
			
		||||
%% Consuming and acknowledging the message should work fine.
 | 
			
		||||
reg_v1_no_del_jif(Config) ->
 | 
			
		||||
    try
 | 
			
		||||
        true = rabbit_ct_broker_helpers:rpc(
 | 
			
		||||
                 Config, 0, ?MODULE, do_reg_v1_no_del_jif, [Config])
 | 
			
		||||
    catch exit:{exception, Reason} ->
 | 
			
		||||
            exit(Reason)
 | 
			
		||||
    end.
 | 
			
		||||
 | 
			
		||||
do_reg_v1_no_del_jif(Config) ->
 | 
			
		||||
    St0 = #cq{name=prop_classic_queue_v1, version=1,
 | 
			
		||||
              config=minimal_config(Config)},
 | 
			
		||||
 | 
			
		||||
    Res1 = cmd_setup_queue(St0),
 | 
			
		||||
    St3 = St0#cq{amq=Res1},
 | 
			
		||||
 | 
			
		||||
    {St4, Ch} = cmd(cmd_channel_open, St3, []),
 | 
			
		||||
 | 
			
		||||
    %% Simulate pre-3.10.0 behaviour by making deliver a noop
 | 
			
		||||
    ok = meck:new(rabbit_queue_index, [passthrough]),
 | 
			
		||||
    ok = meck:expect(rabbit_queue_index, deliver, fun(_, State) -> State end),
 | 
			
		||||
 | 
			
		||||
    {St5, _Res5} = cmd(cmd_channel_publish, St4, [Ch, 4, _Persistent = 2, _NotMandatory = false, _NoExpiration = undefined]),
 | 
			
		||||
 | 
			
		||||
    %% Enforce syncing journal to disk
 | 
			
		||||
    %% (Not strictly necessary as vhost restart also triggers a sync)
 | 
			
		||||
    %% At this point there should be a publish entry in the journal and no segment files
 | 
			
		||||
    rabbit_amqqueue:pid_of(St5#cq.amq) ! timeout,
 | 
			
		||||
 | 
			
		||||
    {SyncTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, sync, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for sync took ~p ms", [SyncTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    %% Simulate RabbitMQ version upgrade by a clean vhost restart
 | 
			
		||||
    %% (also reset delivery to normal operation)
 | 
			
		||||
    ok = meck:delete(rabbit_queue_index, deliver, 2),
 | 
			
		||||
    {St10, _} = cmd(cmd_restart_vhost_clean, St5, []),
 | 
			
		||||
 | 
			
		||||
    meck:reset(rabbit_queue_index),
 | 
			
		||||
 | 
			
		||||
    %% Consume the message and acknowledge it
 | 
			
		||||
    %% The queue index should not crash when finding a pub+ack but no_del in the journal
 | 
			
		||||
    %% (It used to crash in `action_to_entry/3' with a case_clause)
 | 
			
		||||
    {St6, _Tag} = cmd(cmd_channel_consume, St10, [Ch]),
 | 
			
		||||
    receive SomeMsg -> self() ! SomeMsg
 | 
			
		||||
    after 5000 -> ct:fail(no_message_consumed)
 | 
			
		||||
    end,
 | 
			
		||||
    {St7, _Msg = #amqp_msg{}} = cmd(cmd_channel_receive_and_ack, St6, [Ch]),
 | 
			
		||||
 | 
			
		||||
    %% enforce syncing journal to disk
 | 
			
		||||
    rabbit_amqqueue:pid_of(St7#cq.amq) ! timeout,
 | 
			
		||||
 | 
			
		||||
    {SyncTime2, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, sync, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for sync took ~p ms", [SyncTime2 div 1000]),
 | 
			
		||||
 | 
			
		||||
    validate_and_teaddown(St7).
 | 
			
		||||
 | 
			
		||||
%% After upgrade to 3.10 there is a published message in a segment file.
 | 
			
		||||
%% Consuming and acknowledging the message inserts an ack entry in the journal file.
 | 
			
		||||
%% A subsequent restart (of the queue/vhost/node) should work fine.
 | 
			
		||||
reg_v1_no_del_idx(Config) ->
 | 
			
		||||
    try
 | 
			
		||||
        true = rabbit_ct_broker_helpers:rpc(
 | 
			
		||||
                 Config, 0, ?MODULE, do_reg_v1_no_del_idx, [Config])
 | 
			
		||||
    catch exit:{exception, Reason} ->
 | 
			
		||||
            exit(Reason)
 | 
			
		||||
    end.
 | 
			
		||||
 | 
			
		||||
do_reg_v1_no_del_idx(Config) ->
 | 
			
		||||
    St0 = #cq{name=prop_classic_queue_v1, version=1,
 | 
			
		||||
              config=minimal_config(Config)},
 | 
			
		||||
 | 
			
		||||
    Res1 = cmd_setup_queue(St0),
 | 
			
		||||
    St3 = St0#cq{amq=Res1},
 | 
			
		||||
 | 
			
		||||
    {St4, Ch} = cmd(cmd_channel_open, St3, []),
 | 
			
		||||
 | 
			
		||||
    %% Simulate pre-3.10.0 behaviour by making deliver a noop
 | 
			
		||||
    ok = meck:new(rabbit_queue_index, [passthrough]),
 | 
			
		||||
    ok = meck:expect(rabbit_queue_index, deliver, fun(_, State) -> State end),
 | 
			
		||||
 | 
			
		||||
    ok = meck:new(rabbit_variable_queue, [passthrough]),
 | 
			
		||||
 | 
			
		||||
    {St5, _Res5} = cmd(cmd_channel_publish, St4, [Ch, 4, _Persistent = 2, _NotMandatory = false, _NoExpiration = undefined]),
 | 
			
		||||
 | 
			
		||||
    %% Wait for the queue process to get hibernated
 | 
			
		||||
    %% handle_pre_hibernate syncs and flushes the journal
 | 
			
		||||
    %% At this point there should be a publish entry in the segment file and an empty journal
 | 
			
		||||
    {Time, ok} = timer:tc(fun() -> meck:wait(rabbit_variable_queue, handle_pre_hibernate, '_', 10000) end),
 | 
			
		||||
    ct:pal("wait for hibernate took ~p ms", [Time div 1000]),
 | 
			
		||||
    ok = meck:unload(rabbit_variable_queue),
 | 
			
		||||
 | 
			
		||||
    %% Simulate RabbitMQ version upgrade by a clean vhost restart
 | 
			
		||||
    %% (also reset delivery to normal operation)
 | 
			
		||||
    ok = meck:delete(rabbit_queue_index, deliver, 2),
 | 
			
		||||
    {St10, _} = cmd(cmd_restart_vhost_clean, St5, []),
 | 
			
		||||
 | 
			
		||||
    %% Consume the message and acknowledge it
 | 
			
		||||
    {St6, _Tag} = cmd(cmd_channel_consume, St10, [Ch]),
 | 
			
		||||
    receive SomeMsg -> self() ! SomeMsg
 | 
			
		||||
    after 5000 -> ct:fail(no_message_consumed)
 | 
			
		||||
    end,
 | 
			
		||||
    {St7, _Msg = #amqp_msg{}} = cmd(cmd_channel_receive_and_ack, St6, [Ch]),
 | 
			
		||||
 | 
			
		||||
    meck:reset(rabbit_queue_index),
 | 
			
		||||
 | 
			
		||||
    %% enforce syncing journal to disk
 | 
			
		||||
    %% At this point there should be a publish entry in the segment file and an ack in the journal
 | 
			
		||||
    rabbit_amqqueue:pid_of(St7#cq.amq) ! timeout,
 | 
			
		||||
    {SyncTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, sync, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for sync took ~p ms", [SyncTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    meck:reset(rabbit_queue_index),
 | 
			
		||||
 | 
			
		||||
    %% Another clean vhost restart
 | 
			
		||||
    %% The queue index should not crash when finding a pub in a
 | 
			
		||||
    %% segment, an ack in the journal, but no_del
 | 
			
		||||
    %% (It used to crash in `segment_plus_journal1/2' with a function_clause)
 | 
			
		||||
    catch cmd(cmd_restart_vhost_clean, St7, []),
 | 
			
		||||
 | 
			
		||||
    {ReadTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, read, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for queue read took ~p ms", [ReadTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    validate_and_teaddown(St7).
 | 
			
		||||
 | 
			
		||||
%% After upgrade to 3.10 there is a published message in a segment file.
 | 
			
		||||
%% Consuming and acknowledging the message inserts an ack entry in the journal file.
 | 
			
		||||
%% The recovery after a subsequent unclean shutdown (of the queue/vhost/node) should work fine.
 | 
			
		||||
reg_v1_no_del_idx_unclean(Config) ->
 | 
			
		||||
    try
 | 
			
		||||
        true = rabbit_ct_broker_helpers:rpc(
 | 
			
		||||
                 Config, 0, ?MODULE, do_reg_v1_no_del_idx_unclean, [Config])
 | 
			
		||||
    catch exit:{exception, Reason} ->
 | 
			
		||||
            exit(Reason)
 | 
			
		||||
    end.
 | 
			
		||||
 | 
			
		||||
do_reg_v1_no_del_idx_unclean(Config) ->
 | 
			
		||||
    St0 = #cq{name=prop_classic_queue_v1, version=1,
 | 
			
		||||
              config=minimal_config(Config)},
 | 
			
		||||
 | 
			
		||||
    Res1 = cmd_setup_queue(St0),
 | 
			
		||||
    St3 = St0#cq{amq=Res1},
 | 
			
		||||
 | 
			
		||||
    {St4, Ch} = cmd(cmd_channel_open, St3, []),
 | 
			
		||||
 | 
			
		||||
    %% Simulate pre-3.10.0 behaviour by making deliver a noop
 | 
			
		||||
    ok = meck:new(rabbit_queue_index, [passthrough]),
 | 
			
		||||
    ok = meck:expect(rabbit_queue_index, deliver, fun(_, State) -> State end),
 | 
			
		||||
 | 
			
		||||
    ok = meck:new(rabbit_variable_queue, [passthrough]),
 | 
			
		||||
 | 
			
		||||
    {St5, _Res5} = cmd(cmd_channel_publish, St4, [Ch, 4, _Persistent = 2, _NotMandatory = false, _NoExpiration = undefined]),
 | 
			
		||||
 | 
			
		||||
    %% Wait for the queue process to get hibernated
 | 
			
		||||
    %% handle_pre_hibernate syncs and flushes the journal
 | 
			
		||||
    %% At this point there should be a publish entry in the segment file and an empty journal
 | 
			
		||||
    {Time, ok} = timer:tc(fun() -> meck:wait(rabbit_variable_queue, handle_pre_hibernate, '_', 10000) end),
 | 
			
		||||
    ct:pal("wait for hibernate took ~p ms", [Time div 1000]),
 | 
			
		||||
    ok = meck:unload(rabbit_variable_queue),
 | 
			
		||||
 | 
			
		||||
    %% Simulate RabbitMQ version upgrade by a clean vhost restart
 | 
			
		||||
    %% (also reset delivery to normal operation)
 | 
			
		||||
    ok = meck:delete(rabbit_queue_index, deliver, 2),
 | 
			
		||||
    {St10, _} = cmd(cmd_restart_vhost_clean, St5, []),
 | 
			
		||||
 | 
			
		||||
    %% Consume the message and acknowledge it
 | 
			
		||||
    {St6, _Tag} = cmd(cmd_channel_consume, St10, [Ch]),
 | 
			
		||||
    receive SomeMsg -> self() ! SomeMsg
 | 
			
		||||
    after 5000 -> ct:fail(no_message_consumed)
 | 
			
		||||
    end,
 | 
			
		||||
    meck:reset(rabbit_queue_index),
 | 
			
		||||
    {St7, _Msg = #amqp_msg{}} = cmd(cmd_channel_receive_and_ack, St6, [Ch]),
 | 
			
		||||
 | 
			
		||||
    %% (need to ensure that the queue processed the ack before triggering the sync)
 | 
			
		||||
    {AckTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, ack, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for ack took ~p ms", [AckTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    %% enforce syncing journal to disk
 | 
			
		||||
    %% At this point there should be a publish entry in the segment file and an ack in the journal
 | 
			
		||||
    rabbit_amqqueue:pid_of(St7#cq.amq) ! timeout,
 | 
			
		||||
    {SyncTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, sync, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for sync took ~p ms", [SyncTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    meck:reset(rabbit_queue_index),
 | 
			
		||||
 | 
			
		||||
    %% Recovery after unclean queue shutdown
 | 
			
		||||
    %% The queue index should not crash when finding a pub in a
 | 
			
		||||
    %% segment, an ack in the journal, but no_del
 | 
			
		||||
    %% (It used to crash in `journal_minus_segment1/2' with a function_clause)
 | 
			
		||||
    {St20, _} = cmd(cmd_restart_queue_dirty, St7, []),
 | 
			
		||||
 | 
			
		||||
    {RecoverTime, ok} = timer:tc(fun() -> meck:wait(rabbit_queue_index, recover, '_', 1000) end),
 | 
			
		||||
    ct:pal("wait for queue recover took ~p ms", [RecoverTime div 1000]),
 | 
			
		||||
 | 
			
		||||
    validate_and_teaddown(St20).
 | 
			
		||||
 | 
			
		||||
cmd(CmdName, StIn, ExtraArgs) ->
 | 
			
		||||
    Res0 = apply(?MODULE, CmdName, [StIn | ExtraArgs]),
 | 
			
		||||
    true = postcondition(StIn, {call, undefined, CmdName, [StIn | ExtraArgs]}, Res0),
 | 
			
		||||
    StOut = next_state(StIn, Res0, {call, undefined, CmdName, [StIn | ExtraArgs]}),
 | 
			
		||||
    {StOut, Res0}.
 | 
			
		||||
 | 
			
		||||
validate_and_teaddown(St) ->
 | 
			
		||||
    try
 | 
			
		||||
        case meck:validate(rabbit_queue_index) of
 | 
			
		||||
            true ->
 | 
			
		||||
                true;
 | 
			
		||||
            false ->
 | 
			
		||||
                FailedCalls =
 | 
			
		||||
                    [Hist || Hist = {_CallerPid, _MFA, _Class, _Reason, _ST}
 | 
			
		||||
                                 <- meck:history(rabbit_queue_index)],
 | 
			
		||||
                ct:pal("Failed call(s) to rabbit_queue_index:~n~p", [FailedCalls]),
 | 
			
		||||
 | 
			
		||||
                {_, _, _, _, [{_M, F, _A, _Loc}|_]} = hd(FailedCalls),
 | 
			
		||||
                ct:fail({queue_index_crashed, F})
 | 
			
		||||
        end
 | 
			
		||||
    after
 | 
			
		||||
        ok = meck:unload(rabbit_queue_index),
 | 
			
		||||
        safe_teardown_queue(St)
 | 
			
		||||
    end.
 | 
			
		||||
 | 
			
		||||
safe_teardown_queue(St) ->
 | 
			
		||||
    try cmd_teardown_queue(St)
 | 
			
		||||
    catch _:_ ->
 | 
			
		||||
            %% It is possible that asking a queue process in cyclic
 | 
			
		||||
            %% crashing to stop fails.
 | 
			
		||||
            VHostDir = rabbit_vhost:msg_store_dir_path(<<"/">>),
 | 
			
		||||
            [ok = file:delete(QIFile)
 | 
			
		||||
             || QIFile <- filelib:wildcard(filename:join(VHostDir, "queues/*/*"))],
 | 
			
		||||
            cmd_teardown_queue(St)
 | 
			
		||||
    end.
 | 
			
		||||
| 
						 | 
				
			
			
 | 
			
		|||
		Loading…
	
		Reference in New Issue