diff --git a/deps/amqp10_client/src/amqp10_msg.erl b/deps/amqp10_client/src/amqp10_msg.erl index ac8b9f2a4ba9..d8e7013ca02b 100644 --- a/deps/amqp10_client/src/amqp10_msg.erl +++ b/deps/amqp10_client/src/amqp10_msg.erl @@ -265,23 +265,29 @@ body_bin(#amqp10_msg{body = #'v1_0.amqp_value'{} = Body}) -> %% A disposition will be notified to the sender by a message of the %% following stucture: %% {amqp10_disposition, {accepted | rejected, DeliveryTag}} --spec new(delivery_tag(), amqp10_body() | binary(), boolean()) -> amqp10_msg(). +-spec new(delivery_tag(), amqp10_body() | binary() | [amqp10_client_types:amqp10_msg_record()], boolean()) -> amqp10_msg(). new(DeliveryTag, Bin, Settled) when is_binary(Bin) -> Body = [#'v1_0.data'{content = Bin}], new(DeliveryTag, Body, Settled); new(DeliveryTag, Body, Settled) -> % TODO: constrain to amqp types - #amqp10_msg{ - transfer = #'v1_0.transfer'{ - delivery_tag = {binary, DeliveryTag}, - settled = Settled, - message_format = {uint, ?MESSAGE_FORMAT}}, - %% This lib is safe by default. - header = #'v1_0.header'{durable = true}, - body = Body}. + Transfer = #'v1_0.transfer'{ + delivery_tag = {binary, DeliveryTag}, + settled = Settled, + message_format = {uint, ?MESSAGE_FORMAT}}, + case is_amqp10_body(Body) orelse (not is_list(Body)) of + true -> + #amqp10_msg{ + transfer = Transfer, + %% This lib is safe by default. + header = #'v1_0.header'{durable = true}, + body = Body}; + false -> + from_amqp_records([Transfer | Body]) + end. %% @doc Create a new settled amqp10 message using the specified delivery tag %% and body. --spec new(delivery_tag(), amqp10_body() | binary()) -> amqp10_msg(). +-spec new(delivery_tag(), amqp10_body() | binary() | [amqp10_client_types:amqp10_msg_record()]) -> amqp10_msg(). new(DeliveryTag, Body) -> new(DeliveryTag, Body, false). @@ -462,3 +468,19 @@ uint(B) -> {uint, B}. has_value(undefined) -> false; has_value(_) -> true. + +is_amqp10_body(#'v1_0.amqp_value'{}) -> + true; +is_amqp10_body(List) when is_list(List) -> + lists:all(fun(#'v1_0.data'{}) -> + true; + (_) -> + false + end, List) orelse + lists:all(fun(#'v1_0.amqp_sequence'{}) -> + true; + (_) -> + false + end, List); +is_amqp10_body(_) -> + false. diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl index e9898665061a..40baa3521c6f 100644 --- a/deps/rabbit/src/rabbit_amqp_session.erl +++ b/deps/rabbit/src/rabbit_amqp_session.erl @@ -2457,7 +2457,6 @@ incoming_link_transfer( validate_message_size(PayloadSize, MaxMessageSize), rabbit_msg_size_metrics:observe(?PROTOCOL, PayloadSize), messages_received(Settled), - Mc0 = mc:init(mc_amqp, PayloadBin, #{}), case lookup_target(LinkExchange, LinkRKey, Mc0, Vhost, User, PermCache0) of {ok, X, RoutingKeys, Mc1, PermCache} -> diff --git a/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl index 53d2cad75ce9..7b3228450b6e 100644 --- a/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl +++ b/deps/rabbitmq_shovel/src/rabbit_amqp091_shovel.erl @@ -7,11 +7,10 @@ -module(rabbit_amqp091_shovel). --define(APP, rabbitmq_shovel). - -behaviour(rabbit_shovel_behaviour). -include_lib("amqp_client/include/amqp_client.hrl"). +-include_lib("rabbit/include/mc.hrl"). -include("rabbit_shovel.hrl"). -include_lib("kernel/include/logger.hrl"). @@ -34,7 +33,7 @@ ack/3, nack/3, status/1, - forward/4 + forward/3 ]). %% Function references should not be stored on the metadata store. @@ -57,7 +56,7 @@ parse(_Name, {source, Source}) -> CArgs = proplists:get_value(consumer_args, Source, []), #{module => ?MODULE, uris => proplists:get_value(uris, Source), - resource_decl => decl_fun({source, Source}), + resource_decl => rabbit_shovel_util:decl_fun(?MODULE, {source, Source}), queue => Queue, delete_after => proplists:get_value(delete_after, Source, never), prefetch_count => Prefetch, @@ -73,7 +72,7 @@ parse(Name, {destination, Dest}) -> PropsFun2 = add_timestamp_header_fun(ATH, PropsFun1), #{module => ?MODULE, uris => proplists:get_value(uris, Dest), - resource_decl => decl_fun({destination, Dest}), + resource_decl => rabbit_shovel_util:decl_fun(?MODULE, {destination, Dest}), props_fun => PropsFun2, fields_fun => PubFieldsFun, add_forward_headers => AFH, @@ -170,8 +169,8 @@ forward_pending(State) -> case pop_pending(State) of empty -> State; - {{Tag, Props, Payload}, S} -> - S2 = do_forward(Tag, Props, Payload, S), + {{Tag, Mc}, S} -> + S2 = do_forward(Tag, Mc, S), S3 = control_throttle(S2), case is_blocked(S3) of true -> @@ -184,91 +183,50 @@ forward_pending(State) -> end end. -forward(IncomingTag, Props, Payload, State) -> +forward(IncomingTag, Mc, State) -> case is_blocked(State) of true -> %% We are blocked by client-side flow-control and/or %% `connection.blocked` message from the destination %% broker. Simply cache the forward. - PendingEntry = {IncomingTag, Props, Payload}, + PendingEntry = {IncomingTag, Mc}, add_pending(PendingEntry, State); false -> - State1 = do_forward(IncomingTag, Props, Payload, State), + State1 = do_forward(IncomingTag, Mc, State), control_throttle(State1) end. -do_forward(IncomingTag, Props, Payload, +do_forward(IncomingTag, Mc0, State0 = #{dest := #{props_fun := {M, F, Args}, current := {_, _, DstUri}, fields_fun := {Mf, Ff, Argsf}}}) -> SrcUri = rabbit_shovel_behaviour:source_uri(State0), % do publish - Exchange = maps:get(exchange, Props, undefined), - RoutingKey = maps:get(routing_key, Props, undefined), + Exchange = mc:exchange(Mc0), + RoutingKey = case mc:routing_keys(Mc0) of + [RK | _] -> RK; + Any -> Any + end, Method = #'basic.publish'{exchange = Exchange, routing_key = RoutingKey}, Method1 = apply(Mf, Ff, Argsf ++ [SrcUri, DstUri, Method]), - Msg1 = #amqp_msg{props = apply(M, F, Args ++ [SrcUri, DstUri, props_from_map(Props)]), + Mc = mc:convert(mc_amqpl, Mc0), + {Props, Payload} = rabbit_basic_common:from_content(mc:protocol_state(Mc)), + Msg1 = #amqp_msg{props = apply(M, F, Args ++ [SrcUri, DstUri, Props]), payload = Payload}, publish(IncomingTag, Method1, Msg1, State0). -props_from_map(Map) -> - #'P_basic'{content_type = maps:get(content_type, Map, undefined), - content_encoding = maps:get(content_encoding, Map, undefined), - headers = maps:get(headers, Map, undefined), - delivery_mode = maps:get(delivery_mode, Map, undefined), - priority = maps:get(priority, Map, undefined), - correlation_id = maps:get(correlation_id, Map, undefined), - reply_to = maps:get(reply_to, Map, undefined), - expiration = maps:get(expiration, Map, undefined), - message_id = maps:get(message_id, Map, undefined), - timestamp = maps:get(timestamp, Map, undefined), - type = maps:get(type, Map, undefined), - user_id = maps:get(user_id, Map, undefined), - app_id = maps:get(app_id, Map, undefined), - cluster_id = maps:get(cluster_id, Map, undefined)}. - -map_from_props(#'P_basic'{content_type = Content_type, - content_encoding = Content_encoding, - headers = Headers, - delivery_mode = Delivery_mode, - priority = Priority, - correlation_id = Correlation_id, - reply_to = Reply_to, - expiration = Expiration, - message_id = Message_id, - timestamp = Timestamp, - type = Type, - user_id = User_id, - app_id = App_id, - cluster_id = Cluster_id}) -> - lists:foldl(fun({_K, undefined}, Acc) -> Acc; - ({K, V}, Acc) -> Acc#{K => V} - end, #{}, [{content_type, Content_type}, - {content_encoding, Content_encoding}, - {headers, Headers}, - {delivery_mode, Delivery_mode}, - {priority, Priority}, - {correlation_id, Correlation_id}, - {reply_to, Reply_to}, - {expiration, Expiration}, - {message_id, Message_id}, - {timestamp, Timestamp}, - {type, Type}, - {user_id, User_id}, - {app_id, App_id}, - {cluster_id, Cluster_id} - ]). - handle_source(#'basic.consume_ok'{}, State) -> State; handle_source({#'basic.deliver'{delivery_tag = Tag, exchange = Exchange, routing_key = RoutingKey}, #amqp_msg{props = Props0, payload = Payload}}, State) -> - Props = (map_from_props(Props0))#{exchange => Exchange, - routing_key => RoutingKey}, + Content = rabbit_basic_common:build_content(Props0, Payload), + Msg0 = mc:init(mc_amqpl, Content, #{}), + Msg1 = mc:set_annotation(?ANN_ROUTING_KEYS, [RoutingKey], Msg0), + Msg = mc:set_annotation(?ANN_EXCHANGE, Exchange, Msg1), % forward to destination - rabbit_shovel_behaviour:forward(Tag, Props, Payload, State); + rabbit_shovel_behaviour:forward(Tag, Msg, State); handle_source({'EXIT', Conn, Reason}, #{source := #{current := {Conn, _, _}}}) -> @@ -584,47 +542,6 @@ props_fun_timestamp_header({M, F, Args}, SrcUri, DestUri, Props) -> rabbit_shovel_util:add_timestamp_header( apply(M, F, Args ++ [SrcUri, DestUri, Props])). -parse_declaration({[], Acc}) -> - Acc; -parse_declaration({[{Method, Props} | Rest], Acc}) when is_list(Props) -> - FieldNames = try rabbit_framing_amqp_0_9_1:method_fieldnames(Method) - catch exit:Reason -> fail(Reason) - end, - case proplists:get_keys(Props) -- FieldNames of - [] -> ok; - UnknownFields -> fail({unknown_fields, Method, UnknownFields}) - end, - {Res, _Idx} = lists:foldl( - fun (K, {R, Idx}) -> - NewR = case proplists:get_value(K, Props) of - undefined -> R; - V -> setelement(Idx, R, V) - end, - {NewR, Idx + 1} - end, {rabbit_framing_amqp_0_9_1:method_record(Method), 2}, - FieldNames), - parse_declaration({Rest, [Res | Acc]}); -parse_declaration({[{Method, Props} | _Rest], _Acc}) -> - fail({expected_method_field_list, Method, Props}); -parse_declaration({[Method | Rest], Acc}) -> - parse_declaration({[{Method, []} | Rest], Acc}). - -decl_fun({source, Endpoint}) -> - case parse_declaration({proplists:get_value(declarations, Endpoint, []), []}) of - [] -> - case proplists:get_value(predeclared, application:get_env(?APP, topology, []), false) of - true -> case proplists:get_value(queue, Endpoint) of - <<>> -> fail({invalid_parameter_value, declarations, {require_non_empty}}); - Queue -> {?MODULE, check_fun, [Queue]} - end; - false -> {?MODULE, decl_fun, []} - end; - Decl -> {?MODULE, decl_fun, [Decl]} - end; -decl_fun({destination, Endpoint}) -> - Decl = parse_declaration({proplists:get_value(declarations, Endpoint, []), []}), - {?MODULE, decl_fun, [Decl]}. - decl_fun(Decl, _Conn, Ch) -> [begin amqp_channel:call(Ch, M) diff --git a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl index 84a4b7ea0b22..1ab04692c900 100644 --- a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl +++ b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl @@ -9,6 +9,7 @@ -behaviour(rabbit_shovel_behaviour). +-include_lib("rabbit/include/mc.hrl"). -include("rabbit_shovel.hrl"). -export([ @@ -30,7 +31,7 @@ ack/3, nack/3, status/1, - forward/4 + forward/3 ]). -import(rabbit_misc, [pget/2, pget/3]). @@ -121,7 +122,7 @@ connect(Name, SndSettleMode, Uri, Postfix, Addr, Map, AttachFun) -> {ok, Sess} = amqp10_client:begin_session(Conn), link(Conn), LinkName = begin - LinkName0 = gen_unique_name(Name, Postfix), + LinkName0 = rabbit_shovel_util:gen_unique_name(Name, Postfix), rabbit_data_coercion:to_binary(LinkName0) end, % needs to be sync, i.e. awaits the 'attach' event as @@ -150,9 +151,9 @@ init_source(State = #{source := #{current := #{link := Link}, init_dest(#{name := Name, shovel_type := Type, dest := #{add_forward_headers := true} = Dst} = State) -> - Props = #{<<"shovelled-by">> => rabbit_nodes:cluster_name(), - <<"shovel-type">> => rabbit_data_coercion:to_binary(Type), - <<"shovel-name">> => rabbit_data_coercion:to_binary(Name)}, + Props = #{<<"x-opt-shovelled-by">> => rabbit_nodes:cluster_name(), + <<"x-opt-shovel-type">> => rabbit_data_coercion:to_binary(Type), + <<"x-opt-shovel-name">> => rabbit_data_coercion:to_binary(Name)}, State#{dest => Dst#{cached_forward_headers => Props}}; init_dest(State) -> State. @@ -180,10 +181,12 @@ dest_endpoint(#{shovel_type := dynamic, -spec handle_source(Msg :: any(), state()) -> not_handled | state() | {stop, any()}. -handle_source({amqp10_msg, _LinkRef, Msg}, State) -> - Tag = amqp10_msg:delivery_id(Msg), - Payload = amqp10_msg:body_bin(Msg), - rabbit_shovel_behaviour:forward(Tag, #{}, Payload, State); +handle_source({amqp10_msg, _LinkRef, Msg0}, State) -> + Tag = amqp10_msg:delivery_id(Msg0), + [_ | Rest] = amqp10_msg:to_amqp_records(Msg0), + Bin = iolist_to_binary([amqp10_framing:encode_bin(D) || D <- Rest]), + Msg = mc:init(mc_amqp, Bin, #{}), + rabbit_shovel_behaviour:forward(Tag, Msg, State); handle_source({amqp10_event, {connection, Conn, opened}}, State = #{source := #{current := #{conn := Conn}}}) -> State; @@ -256,8 +259,8 @@ handle_dest({amqp10_event, {link, Link, credited}}, %% we have credit so can begin to forward State = State0#{dest => Dst#{link_state => credited, pending => []}}, - lists:foldl(fun ({A, B, C}, S) -> - forward(A, B, C, S) + lists:foldl(fun ({A, B}, S) -> + forward(A, B, S) end, State, lists:reverse(Pend)); handle_dest({amqp10_event, {link, Link, _Evt}}, State= #{dest := #{current := #{link := Link}}}) -> @@ -311,30 +314,29 @@ status(_) -> %% Destination not yet connected ignore. --spec forward(Tag :: tag(), Props :: #{atom() => any()}, - Payload :: binary(), state()) -> +-spec forward(Tag :: tag(), Mc :: mc:state(), state()) -> state() | {stop, any()}. -forward(_Tag, _Props, _Payload, +forward(_Tag, _Mc, #{source := #{remaining := 0}} = State) -> State; -forward(_Tag, _Props, _Payload, +forward(_Tag, _Mc, #{source := #{remaining_unacked := 0}} = State) -> State; -forward(Tag, Props, Payload, +forward(Tag, Mc, #{dest := #{current := #{link_state := attached}, pending := Pend0} = Dst} = State) -> %% simply cache the forward oo - Pend = [{Tag, Props, Payload} | Pend0], + Pend = [{Tag, Mc} | Pend0], State#{dest => Dst#{pending => {Pend}}}; -forward(Tag, Props, Payload, +forward(Tag, Msg0, #{dest := #{current := #{link := Link}, unacked := Unacked} = Dst, ack_mode := AckMode} = State) -> OutTag = rabbit_data_coercion:to_binary(Tag), - Msg0 = new_message(OutTag, Payload, State), - Msg = add_timestamp_header( - State, set_message_properties( - Props, add_forward_headers(State, Msg0))), + Msg1 = mc:protocol_state(mc:convert(mc_amqp, Msg0)), + Records = lists:flatten([amqp10_framing:decode_bin(iolist_to_binary(S)) || S <- Msg1]), + Msg2 = amqp10_msg:new(OutTag, Records, AckMode =/= on_confirm), + Msg = add_timestamp_header(State, add_forward_headers(State, Msg2)), case send_msg(Link, Msg) of ok -> rabbit_shovel_behaviour:decr_remaining_unacked( @@ -363,85 +365,11 @@ send_msg(Link, Msg) -> end end. -new_message(Tag, Payload, #{ack_mode := AckMode, - dest := #{properties := Props, - application_properties := AppProps, - message_annotations := MsgAnns}}) -> - Msg0 = amqp10_msg:new(Tag, Payload, AckMode =/= on_confirm), - Msg1 = amqp10_msg:set_properties(Props, Msg0), - Msg = amqp10_msg:set_message_annotations(MsgAnns, Msg1), - amqp10_msg:set_application_properties(AppProps, Msg). - add_timestamp_header(#{dest := #{add_timestamp_header := true}}, Msg) -> - P =#{creation_time => os:system_time(milli_seconds)}, - amqp10_msg:set_properties(P, Msg); + Anns = #{<<"x-opt-shovelled-timestamp">> => os:system_time(milli_seconds)}, + amqp10_msg:set_message_annotations(Anns, Msg); add_timestamp_header(_, Msg) -> Msg. -add_forward_headers(#{dest := #{cached_forward_headers := Props}}, Msg) -> - amqp10_msg:set_application_properties(Props, Msg); +add_forward_headers(#{dest := #{cached_forward_headers := Anns}}, Msg) -> + amqp10_msg:set_message_annotations(Anns, Msg); add_forward_headers(_, Msg) -> Msg. - -set_message_properties(Props, Msg) -> - %% this is effectively special handling properties from amqp 0.9.1 - maps:fold( - fun(content_type, Ct, M) -> - amqp10_msg:set_properties( - #{content_type => to_binary(Ct)}, M); - (content_encoding, Ct, M) -> - amqp10_msg:set_properties( - #{content_encoding => to_binary(Ct)}, M); - (delivery_mode, 2, M) -> - amqp10_msg:set_headers(#{durable => true}, M); - (delivery_mode, 1, M) -> - % by default the durable flag is false - M; - (priority, P, M) when is_integer(P) -> - amqp10_msg:set_headers(#{priority => P}, M); - (correlation_id, Ct, M) -> - amqp10_msg:set_properties(#{correlation_id => to_binary(Ct)}, M); - (reply_to, Ct, M) -> - amqp10_msg:set_properties(#{reply_to => to_binary(Ct)}, M); - (message_id, Ct, M) -> - amqp10_msg:set_properties(#{message_id => to_binary(Ct)}, M); - (timestamp, Ct, M) -> - amqp10_msg:set_properties(#{creation_time => Ct}, M); - (user_id, Ct, M) -> - amqp10_msg:set_properties(#{user_id => Ct}, M); - (headers, Headers0, M) when is_list(Headers0) -> - %% AMPQ 0.9.1 are added as applicatin properties - %% TODO: filter headers to make safe - Headers = lists:foldl( - fun ({K, _T, V}, Acc) -> - case is_amqp10_compat(V) of - true -> - Acc#{to_binary(K) => V}; - false -> - Acc - end - end, #{}, Headers0), - amqp10_msg:set_application_properties(Headers, M); - (Key, Value, M) -> - case is_amqp10_compat(Value) of - true -> - amqp10_msg:set_application_properties( - #{to_binary(Key) => Value}, M); - false -> - M - end - end, Msg, Props). - -gen_unique_name(Pre0, Post0) -> - Pre = to_binary(Pre0), - Post = to_binary(Post0), - Id = bin_to_hex(crypto:strong_rand_bytes(8)), - <
>/binary, Id/binary, <<"_">>/binary, Post/binary>>.
-
-bin_to_hex(Bin) ->
-    <<<= 10 -> N -10 + $a;
-           true  -> N + $0 end>>
-      || <> <= Bin>>.
-
-is_amqp10_compat(T) ->
-    is_binary(T) orelse
-    is_number(T) orelse
-    is_boolean(T).
diff --git a/deps/rabbitmq_shovel/src/rabbit_local_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_local_shovel.erl
new file mode 100644
index 000000000000..a8a1f7df1d53
--- /dev/null
+++ b/deps/rabbitmq_shovel/src/rabbit_local_shovel.erl
@@ -0,0 +1,746 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
+%%
+
+-module(rabbit_local_shovel).
+
+-behaviour(rabbit_shovel_behaviour).
+
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("amqp10_common/include/amqp10_types.hrl").
+-include_lib("kernel/include/logger.hrl").
+-include_lib("rabbit/include/mc.hrl").
+-include("rabbit_shovel.hrl").
+
+-export([
+         parse/2,
+         connect_source/1,
+         connect_dest/1,
+         init_source/1,
+         init_dest/1,
+         source_uri/1,
+         dest_uri/1,
+         source_protocol/1,
+         dest_protocol/1,
+         source_endpoint/1,
+         dest_endpoint/1,
+         close_dest/1,
+         close_source/1,
+         handle_source/2,
+         handle_dest/2,
+         ack/3,
+         nack/3,
+         forward/3,
+         status/1
+        ]).
+
+-export([
+         src_decl_exchange/4,
+         decl_queue/4,
+         dest_decl_queue/4,
+         check_queue/4,
+         dest_check_queue/4,
+         decl_fun/3,
+         check_fun/3
+        ]).
+
+-define(QUEUE, lqueue).
+%% "Note that, despite its name, the delivery-count is not a count but a
+%% sequence number initialized at an arbitrary point by the sender."
+%% See rabbit_amqp_session.erl
+-define(INITIAL_DELIVERY_COUNT, 16#ff_ff_ff_ff - 4).
+-define(DEFAULT_MAX_LINK_CREDIT, 170).
+
+-record(pending_ack, {
+                      delivery_tag,
+                      msg_id
+                     }).
+
+%% This is a significantly reduced version of its rabbit_amqp_session counterpart.
+%% Local shovels always use the maximum credit allowed.
+-record(credit_req, {
+  delivery_count :: sequence_no()
+}).
+
+parse(_Name, {source, Source}) ->
+    Queue = parse_parameter(queue, fun parse_binary/1,
+                            proplists:get_value(queue, Source)),
+    CArgs = proplists:get_value(consumer_args, Source, []),
+    #{module => ?MODULE,
+      uris => proplists:get_value(uris, Source),
+      resource_decl => rabbit_shovel_util:decl_fun(?MODULE, {source, Source}),
+      queue => Queue,
+      delete_after => proplists:get_value(delete_after, Source, never),
+      consumer_args => CArgs};
+parse(_Name, {destination, Dest}) ->
+    Exchange = parse_parameter(dest_exchange, fun parse_binary/1,
+                               proplists:get_value(dest_exchange, Dest, none)),
+    RK = parse_parameter(dest_exchange_key, fun parse_binary/1,
+                         proplists:get_value(dest_routing_key, Dest, none)),
+    #{module => ?MODULE,
+      uris => proplists:get_value(uris, Dest),
+      resource_decl  => rabbit_shovel_util:decl_fun(?MODULE, {destination, Dest}),
+      exchange => Exchange,
+      routing_key => RK,
+      add_forward_headers => proplists:get_value(add_forward_headers, Dest, false),
+      add_timestamp_header => proplists:get_value(add_timestamp_header, Dest, false)}.
+
+connect_source(State = #{source := Src = #{resource_decl := {M, F, MFArgs},
+                                           queue := QName0,
+                                           uris := [Uri | _]}}) ->
+    QState = rabbit_queue_type:init(),
+    {User, VHost} = get_user_vhost_from_amqp_param(Uri),
+    %% We handle the most recently declared queue to use anonymous functions
+    %% It's usually the channel that does it
+    MRDQ = apply(M, F, MFArgs ++ [VHost, User]),
+    QName = case QName0 of
+                <<>> -> MRDQ;
+                _ -> QName0
+            end,
+    State#{source => Src#{current => #{queue_states => QState,
+                                       next_tag => 1,
+                                       user => User,
+                                       vhost => VHost},
+                          queue => QName},
+           unacked_message_q => ?QUEUE:new()}.
+
+connect_dest(State = #{dest := Dest = #{resource_decl := {M, F, MFArgs},
+                                        uris := [Uri | _]
+                                       },
+                       ack_mode := AckMode}) ->
+    %% Shall we get the user from an URI or something else?
+    {User, VHost} = get_user_vhost_from_amqp_param(Uri),
+    apply(M, F, MFArgs ++ [VHost, User]),
+
+    QState = rabbit_queue_type:init(),
+    case AckMode of
+        on_confirm ->
+            State#{dest => Dest#{current => #{queue_states => QState,
+                                              delivery_id => 1,
+                                              vhost => VHost},
+                                 unacked => #{}}};
+        _ ->
+            State#{dest => Dest#{current => #{queue_states => QState,
+                                              vhost => VHost},
+                                 unacked => #{}}}
+    end.
+
+init_source(State = #{source := #{queue := QName0,
+                                  consumer_args := Args,
+                                  current := #{queue_states := QState0,
+                                               vhost := VHost} = Current} = Src,
+                      name := Name,
+                      ack_mode := AckMode}) ->
+    %% TODO put this shovel behind the rabbitmq_4.0.0 feature flag
+    Mode = case rabbit_feature_flags:is_enabled('rabbitmq_4.0.0') of
+               true ->
+                   {credited, ?INITIAL_DELIVERY_COUNT};
+               false ->
+                   {credited, credit_api_v1}
+           end,
+    MaxLinkCredit = max_link_credit(),
+    QName = rabbit_misc:r(VHost, queue, QName0),
+    CTag = consumer_tag(Name),
+    case rabbit_amqqueue:with(
+           QName,
+           fun(Q) ->
+                   SndSettled = case AckMode of
+                                    no_ack -> true;
+                                    on_publish -> false;
+                                    on_confirm -> false
+                               end,
+                   Spec = #{no_ack => SndSettled,
+                            channel_pid => self(),
+                            limiter_pid => none,
+                            limiter_active => false,
+                            mode => Mode,
+                            consumer_tag => CTag,
+                            exclusive_consume => false,
+                            args => Args,
+                            ok_msg => undefined,
+                            acting_user => ?SHOVEL_USER},
+                   case remaining(Q, State) of
+                       0 ->
+                          {0, {error, autodelete}};
+                       Remaining ->
+                           {Remaining, rabbit_queue_type:consume(Q, Spec, QState0)}
+                   end
+           end) of
+        {Remaining, {ok, QState1}} ->
+            {ok, QState, Actions} = rabbit_queue_type:credit(QName, CTag, ?INITIAL_DELIVERY_COUNT, MaxLinkCredit, false, QState1),
+            State2 = State#{source => Src#{current => Current#{queue_states => QState,
+                                                               consumer_tag => CTag},
+                                           remaining => Remaining,
+                                           remaining_unacked => Remaining,
+                                           delivery_count => ?INITIAL_DELIVERY_COUNT,
+                                           queue_delivery_count => ?INITIAL_DELIVERY_COUNT,
+                                           credit => MaxLinkCredit,
+                                           queue_credit => MaxLinkCredit,
+                                           at_least_one_credit_req_in_flight => true,
+                                           stashed_credit_req => none}},
+            handle_queue_actions(Actions, State2);
+        {0, {error, autodelete}} ->
+            exit({shutdown, autodelete});
+        {_Remaining, {error, Reason}} ->
+            ?LOG_ERROR(
+               "Shovel '~ts' in vhost '~ts' failed to consume: ~ts",
+               [Name, VHost, Reason]),
+            exit({shutdown, failed_to_consume_from_source});
+        {unlimited, {error, not_implemented, Reason, ReasonArgs}} ->
+            ?LOG_ERROR(
+              "Shovel '~ts' in vhost '~ts' failed to consume: ~ts",
+              [Name, VHost, io_lib:format(Reason, ReasonArgs)]),
+            exit({shutdown, failed_to_consume_from_source});
+        {error, not_found} ->
+            exit({shutdown, missing_source_queue})
+    end.
+
+init_dest(#{name := Name,
+            shovel_type := Type,
+            dest := #{add_forward_headers := AFH} = Dst} = State) ->
+    case AFH of
+        true ->
+            Props = #{<<"x-opt-shovelled-by">> => rabbit_nodes:cluster_name(),
+                      <<"x-opt-shovel-type">> => rabbit_data_coercion:to_binary(Type),
+                      <<"x-opt-shovel-name">> => rabbit_data_coercion:to_binary(Name)},
+            State#{dest => Dst#{cached_forward_headers => Props}};
+        false ->
+            State
+    end.
+
+source_uri(_State) ->
+    "".
+
+dest_uri(_State) ->
+    "".
+
+source_protocol(_State) ->
+    local.
+
+dest_protocol(_State) ->
+    local.
+
+source_endpoint(#{source := #{queue := Queue,
+                              exchange := SrcX,
+                              routing_key := SrcXKey}}) ->
+    [{src_exchange, SrcX},
+     {src_exchange_key, SrcXKey},
+     {src_queue, Queue}];
+source_endpoint(#{source := #{queue := Queue}}) ->
+    [{src_queue, Queue}];
+source_endpoint(_Config) ->
+    [].
+
+dest_endpoint(#{dest := #{exchange := SrcX,
+                          routing_key := SrcXKey}}) ->
+    [{dest_exchange, SrcX},
+     {dest_exchange_key, SrcXKey}];
+dest_endpoint(#{dest := #{queue := Queue}}) ->
+    [{dest_queue, Queue}];
+dest_endpoint(_Config) ->
+    [].
+      
+close_dest(_State) ->
+    ok.
+
+close_source(#{source := #{current := #{queue_states := QStates0,
+                                        consumer_tag := CTag,
+                                        user := User,
+                                        vhost := VHost},
+                           queue := QName0}}) ->
+    QName = rabbit_misc:r(VHost, queue, QName0),
+    case rabbit_amqqueue:with(
+           QName,
+           fun(Q) ->
+                   rabbit_queue_type:cancel(Q, #{consumer_tag => CTag,
+                                                 reason => remove,
+                                                 user => User#user.username}, QStates0)
+           end) of
+        {ok, _QStates} ->
+            ok;
+        {error, not_found} ->
+            ok;
+        {error, Reason} ->
+            ?LOG_WARNING("Local shovel failed to remove consumer ~tp: ~tp",
+                         [CTag, Reason]),
+            ok
+    end;
+close_source(_) ->
+    %% No consumer tag, no consumer to cancel
+    ok.
+
+handle_source({queue_event, #resource{name = Queue,
+                                      kind = queue,
+                                      virtual_host = VHost} = QRef, Evt},
+              #{source := Source = #{queue := Queue,
+                                     current := Current = #{queue_states := QueueStates0,
+                                                            vhost := VHost}}} = State0) ->
+    case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of
+        {ok, QState1, Actions} ->
+            State = State0#{source => Source#{current => Current#{queue_states => QState1}}},
+            handle_queue_actions(Actions, State);
+        {eol, Actions} ->
+            _ = handle_queue_actions(Actions, State0),
+            {stop, {inbound_link_or_channel_closure, queue_deleted}};
+        {protocol_error, _Type, Reason, ReasonArgs} ->
+            {stop, list_to_binary(io_lib:format(Reason, ReasonArgs))}
+    end;
+handle_source({{'DOWN', #resource{name = Queue,
+                                  kind = queue,
+                                  virtual_host = VHost}}, _, _, _, _}  ,
+              #{source := #{queue := Queue, current := #{vhost := VHost}}}) ->
+    {stop, {inbound_link_or_channel_closure, source_queue_down}};
+handle_source(_Msg, _State) ->
+    not_handled.
+
+handle_dest({queue_event, QRef, Evt},
+            #{ack_mode := on_confirm,
+              dest := Dest = #{current := Current = #{queue_states := QueueStates0}}} = State0) ->
+    case rabbit_queue_type:handle_event(QRef, Evt, QueueStates0) of
+        {ok, QState1, Actions} ->
+            State = State0#{dest => Dest#{current => Current#{queue_states => QState1}}},
+            handle_dest_queue_actions(Actions, State);
+        {eol, Actions} ->
+            _ = handle_dest_queue_actions(Actions, State0),
+            {stop, {outbound_link_or_channel_closure, queue_deleted}};
+        {protocol_error, _Type, Reason, ReasonArgs} ->
+            {stop, list_to_binary(io_lib:format(Reason, ReasonArgs))}
+    end;
+handle_dest({{'DOWN', #resource{name = Queue,
+                                kind = queue,
+                                virtual_host = VHost}}, _, _, _, _}  ,
+            #{dest := #{queue := Queue, current := #{vhost := VHost}}}) ->
+    {stop, {outbound_link_or_channel_closure, dest_queue_down}};
+handle_dest(_Msg, State) ->
+    State.
+
+ack(DeliveryTag, Multiple, State) ->
+    maybe_grant_or_stash_credit(settle(complete, DeliveryTag, Multiple, State)).
+
+nack(DeliveryTag, Multiple, State) ->
+    maybe_grant_or_stash_credit(settle(discard, DeliveryTag, Multiple, State)).
+
+forward(Tag, Msg0, #{dest := #{current := #{queue_states := QState} = Current,
+                               unacked := Unacked} = Dest,
+                     ack_mode := AckMode} = State0) ->
+    {Options, #{dest := #{current := Current1} = Dest1} = State} =
+        case AckMode of
+            on_confirm  ->
+                DeliveryId = maps:get(delivery_id, Current),
+                Opts = #{correlation => DeliveryId},
+                {Opts, State0#{dest => Dest#{current => Current#{delivery_id => DeliveryId + 1}}}};
+            _ ->
+                {#{}, State0}
+        end,
+    Msg = set_annotations(Msg0, Dest),
+    QNames = route(Msg, Dest),
+    Queues = rabbit_amqqueue:lookup_many(QNames),
+    case rabbit_queue_type:deliver(Queues, Msg, Options, QState) of
+        {ok, QState1, Actions} ->
+            State1 = State#{dest => Dest1#{current => Current1#{queue_states => QState1}}},
+            #{dest := Dst1} = State2 = rabbit_shovel_behaviour:incr_forwarded(State1),
+            State4 = rabbit_shovel_behaviour:decr_remaining_unacked(
+                       case AckMode of
+                           no_ack ->
+                               rabbit_shovel_behaviour:decr_remaining(1, State2);
+                           on_confirm ->
+                               Correlation = maps:get(correlation, Options),
+                               State2#{dest => Dst1#{unacked => Unacked#{Correlation => Tag}}};
+                           on_publish ->
+                               State3 = rabbit_shovel_behaviour:ack(Tag, false, State2),
+                               rabbit_shovel_behaviour:decr_remaining(1, State3)
+                       end),
+            handle_dest_queue_actions(Actions, State4);
+        {error, Reason} ->
+            exit({shutdown, Reason})
+    end.
+
+set_annotations(Msg, Dest) ->
+    add_routing(add_forward_headers(add_timestamp_header(Msg, Dest), Dest), Dest).
+
+add_timestamp_header(Msg, #{add_timestamp_header := true}) ->
+    mc:set_annotation(<<"x-opt-shovelled-timestamp">>, os:system_time(milli_seconds), Msg);
+add_timestamp_header(Msg, _) ->
+    Msg.
+
+add_forward_headers(Msg, #{cached_forward_headers := Props}) ->
+    maps:fold(fun(K, V, Acc) ->
+                      mc:set_annotation(K, V, Acc)
+              end, Msg, Props);
+add_forward_headers(Msg, _D) ->
+    Msg.
+
+add_routing(Msg0, Dest) ->
+    Msg = case maps:get(exchange, Dest, undefined) of
+              undefined -> Msg0;
+              Exchange -> mc:set_annotation(?ANN_EXCHANGE, Exchange, Msg0)
+          end,
+    case maps:get(routing_key, Dest, undefined) of
+        undefined -> Msg;
+        RK -> mc:set_annotation(?ANN_ROUTING_KEYS, [RK], Msg)
+    end.
+
+status(_) ->
+    running.
+
+%% Internal
+
+parse_parameter(_, _, none) ->
+    none;
+parse_parameter(Param, Fun, Value) ->
+    try
+        Fun(Value)
+    catch
+        _:{error, Err} ->
+            fail({invalid_parameter_value, Param, Err})
+    end.
+
+parse_binary(Binary) when is_binary(Binary) ->
+    Binary;
+parse_binary(NotABinary) ->
+    fail({require_binary, NotABinary}).
+
+consumer_tag(Name) ->
+    CTag0 = rabbit_shovel_util:gen_unique_name(Name, "receiver"),
+    rabbit_data_coercion:to_binary(CTag0).
+
+-spec fail(term()) -> no_return().
+fail(Reason) -> throw({error, Reason}).
+
+handle_queue_actions(Actions, State) ->
+    lists:foldl(
+      fun({deliver, _CTag, AckRequired, Msgs}, S0) ->
+              handle_deliver(AckRequired, Msgs, S0);
+         ({credit_reply, _, _, _, _, _} = Action, S0) ->
+              handle_credit_reply(Action, S0);
+         (_Action, S0) ->
+              S0
+      end, State, Actions).
+
+handle_deliver(AckRequired, Msgs, State) when is_list(Msgs) ->
+    maybe_grant_or_stash_credit(
+      lists:foldl(
+        fun({_QName, _QPid, MsgId, _Redelivered, Mc}, S0) ->
+                DeliveryTag = next_tag(S0),
+                S = record_pending(AckRequired, DeliveryTag, MsgId, increase_next_tag(S0)),
+                rabbit_shovel_behaviour:forward(DeliveryTag, Mc, S)
+        end, sent_delivery(State, length(Msgs)), Msgs)).
+
+next_tag(#{source := #{current := #{next_tag := DeliveryTag}}}) ->
+    DeliveryTag.
+
+increase_next_tag(#{source := Source = #{current := Current = #{next_tag := DeliveryTag}}} = State) ->
+    State#{source => Source#{current => Current#{next_tag => DeliveryTag + 1}}}.
+
+handle_dest_queue_actions(Actions, State) ->
+    lists:foldl(
+      fun({settled, _QName, MsgSeqNos}, S0) ->
+              confirm_to_inbound(fun(Tag, StateX) ->
+                                         rabbit_shovel_behaviour:ack(Tag, false, StateX)
+                                 end, MsgSeqNos, S0);
+         ({rejected, _QName, MsgSeqNos}, S0) ->
+              confirm_to_inbound(fun(Tag, StateX) ->
+                                         rabbit_shovel_behaviour:nack(Tag, false, StateX)
+                                 end, MsgSeqNos, S0);
+         %% TODO handle {block, QName}
+         (_Action, S0) ->
+              S0
+      end, State, Actions).
+
+record_pending(false, _DeliveryTag, _MsgId, State) ->
+    State;
+record_pending(true, DeliveryTag, MsgId, #{unacked_message_q := UAMQ0} = State) ->
+    UAMQ = ?QUEUE:in(#pending_ack{delivery_tag = DeliveryTag,
+                                  msg_id = MsgId}, UAMQ0),
+    State#{unacked_message_q => UAMQ}.
+
+remaining(_Q, #{source := #{delete_after := never}}) ->
+    unlimited;
+remaining(Q, #{source := #{delete_after := 'queue-length'}}) ->
+    [{messages, Count}] = rabbit_amqqueue:info(Q, [messages]),
+    Count;
+remaining(_Q, #{source := #{delete_after := Count}}) ->
+    Count.
+
+decl_fun(Decl, VHost, User) ->
+    lists:foldr(
+      fun(Method, MRDQ) -> %% keep track of most recently declared queue
+              Reply = rabbit_channel:handle_method(
+                        expand_shortcuts(Method, MRDQ),
+                        none, #{}, none, VHost, User),
+              case {Method, Reply} of
+                  {#'queue.declare'{}, {ok, QName, _, _}} ->
+                      QName#resource.name;
+                  _ ->
+                      MRDQ
+              end
+      end, <<>>, Decl).
+
+expand_shortcuts(#'queue.bind'   {queue = Q, routing_key = K} = M, MRDQ) ->
+    M#'queue.bind'   {queue       = expand_queue_name_shortcut(Q, MRDQ),
+                      routing_key = expand_routing_key_shortcut(Q, K, MRDQ)};
+expand_shortcuts(#'queue.unbind' {queue = Q, routing_key = K} = M, MRDQ) ->
+    M#'queue.unbind' {queue       = expand_queue_name_shortcut(Q, MRDQ),
+                      routing_key = expand_routing_key_shortcut(Q, K, MRDQ)};
+expand_shortcuts(M, _State) ->
+    M.
+
+expand_queue_name_shortcut(<<>>, <<>>) ->
+    exit({shutdown, {not_found, "no previously declared queue"}});
+expand_queue_name_shortcut(<<>>, MRDQ) ->
+    MRDQ;
+expand_queue_name_shortcut(QueueNameBin, _) ->
+    QueueNameBin.
+
+expand_routing_key_shortcut(<<>>, <<>>, <<>>) ->
+    exit({shutdown, {not_found, "no previously declared queue"}});
+expand_routing_key_shortcut(<<>>, <<>>, MRDQ) ->
+    MRDQ;
+expand_routing_key_shortcut(_QueueNameBin, RoutingKey, _) ->
+    RoutingKey.
+
+%% TODO A missing queue stops the shovel but because the error reason
+%% the failed status is not stored. Would not be it more useful to
+%% report it??? This is a rabbit_shovel_worker issues, last terminate
+%% clause
+check_fun(QName, VHost, User) ->
+    Method = #'queue.declare'{queue = QName,
+                              passive = true},
+    decl_fun([Method], VHost, User).
+
+src_decl_exchange(SrcX, SrcXKey, VHost, User) ->
+    Methods = [#'queue.bind'{routing_key = SrcXKey,
+                             exchange    = SrcX},
+               #'queue.declare'{exclusive = true}],
+    decl_fun(Methods, VHost, User).
+
+dest_decl_queue(none, _, _, _) ->
+    ok;
+dest_decl_queue(QName, QArgs, VHost, User) ->
+    decl_queue(QName, QArgs, VHost, User).
+
+decl_queue(QName, QArgs, VHost, User) ->
+    Args = rabbit_misc:to_amqp_table(QArgs),
+    Method = #'queue.declare'{queue = QName,
+                              durable = true,
+                              arguments = Args},
+    decl_fun([Method], VHost, User).
+
+dest_check_queue(none, _, _, _) ->
+    ok;
+dest_check_queue(QName, QArgs, VHost, User) ->
+    check_queue(QName, QArgs, VHost, User).
+
+check_queue(QName, _QArgs, VHost, User) ->
+    Method = #'queue.declare'{queue = QName,
+                              passive = true},
+    decl_fun([Method], VHost, User).
+
+get_user_vhost_from_amqp_param(Uri) ->
+    {ok, AmqpParam} = amqp_uri:parse(Uri),
+    {Username, Password, VHost} =
+        case AmqpParam of
+            #amqp_params_direct{username = U,
+                                password = P,
+                                virtual_host = V} ->
+                {U, P, V};
+            #amqp_params_network{username = U,
+                                 password = P,
+                                 virtual_host = V} ->
+                {U, P, V}
+        end,
+    case rabbit_access_control:check_user_login(Username, [{password, Password}]) of
+        {ok, User} ->
+            try
+                rabbit_access_control:check_vhost_access(User, VHost, undefined, #{}) of
+                ok ->
+                    {User, VHost}
+            catch
+                exit:#amqp_error{name = not_allowed} ->
+                    exit({shutdown, {access_refused, Username}})
+            end;
+        {refused, Username, _Msg, _Module} ->
+            ?LOG_ERROR("Local shovel user ~ts was refused access", [Username]),
+            exit({shutdown, {access_refused, Username}})
+    end.
+
+settle(Op, DeliveryTag, Multiple, #{unacked_message_q := UAMQ0,
+                             source := #{queue := Queue,
+                                         current := Current = #{queue_states := QState0,
+                                                                consumer_tag := CTag,
+                                                                vhost := VHost}} = Src} = State0) ->
+    {Acked, UAMQ} = collect_acks(UAMQ0, DeliveryTag, Multiple),
+    QRef = rabbit_misc:r(VHost, queue, Queue),
+    MsgIds = [Ack#pending_ack.msg_id || Ack <- Acked],
+    case rabbit_queue_type:settle(QRef, Op, CTag, MsgIds, QState0) of
+        {ok, QState1, Actions} ->
+            State = State0#{source => Src#{current => Current#{queue_states => QState1}},
+                            unacked_message_q => UAMQ},
+            handle_queue_actions(Actions, State);
+        {'protocol_error', Type, Reason, Args} ->
+            ?LOG_ERROR("Shovel failed to settle ~p acknowledgments with ~tp: ~tp",
+                       [Op, Type, io_lib:format(Reason, Args)]),
+            exit({shutdown, {ack_failed, Reason}})
+    end.
+
+%% From rabbit_channel
+%% Records a client-sent acknowledgement. Handles both single delivery acks
+%% and multi-acks.
+%%
+%% Returns a tuple of acknowledged pending acks and remaining pending acks.
+%% Sorts each group in the youngest-first order (descending by delivery tag).
+%% The special case for 0 comes from the AMQP 0-9-1 spec: if the multiple field is set to 1 (true),
+%% and the delivery tag is 0, this indicates acknowledgement of all outstanding messages (by a client).
+collect_acks(UAMQ, 0, true) ->
+    {lists:reverse(?QUEUE:to_list(UAMQ)), ?QUEUE:new()};
+collect_acks(UAMQ, DeliveryTag, Multiple) ->
+    collect_acks([], [], UAMQ, DeliveryTag, Multiple).
+
+collect_acks(AcknowledgedAcc, RemainingAcc, UAMQ, DeliveryTag, Multiple) ->
+    case ?QUEUE:out(UAMQ) of
+        {{value, UnackedMsg = #pending_ack{delivery_tag = CurrentDT}},
+         UAMQTail} ->
+            if CurrentDT == DeliveryTag ->
+                   {[UnackedMsg | AcknowledgedAcc],
+                    case RemainingAcc of
+                        [] -> UAMQTail;
+                        _  -> ?QUEUE:join(
+                                 ?QUEUE:from_list(lists:reverse(RemainingAcc)),
+                                 UAMQTail)
+                    end};
+               Multiple ->
+                    collect_acks([UnackedMsg | AcknowledgedAcc], RemainingAcc,
+                                 UAMQTail, DeliveryTag, Multiple);
+               true ->
+                    collect_acks(AcknowledgedAcc, [UnackedMsg | RemainingAcc],
+                                 UAMQTail, DeliveryTag, Multiple)
+            end;
+        {empty, UAMQTail} ->
+           {AcknowledgedAcc, UAMQTail}
+    end.
+
+route(_Msg, #{queue := Queue,
+              current := #{vhost := VHost}}) when Queue =/= none ->
+    QName = rabbit_misc:r(VHost, queue, Queue),
+    [QName];
+route(Msg, #{current := #{vhost := VHost}}) ->
+    ExchangeName = rabbit_misc:r(VHost, exchange, mc:exchange(Msg)),
+    Exchange = rabbit_exchange:lookup_or_die(ExchangeName),
+    rabbit_exchange:route(Exchange, Msg, #{return_binding_keys => true}).
+
+confirm_to_inbound(ConfirmFun, SeqNos, State)
+  when is_list(SeqNos) ->
+    lists:foldl(fun(Seq, State0) ->
+                        confirm_to_inbound(ConfirmFun, Seq, State0)
+                end, State, SeqNos);
+confirm_to_inbound(ConfirmFun, Seq,
+                   State0 = #{dest := #{unacked := Unacked} = Dst}) ->
+    #{Seq := InTag} = Unacked,
+    Unacked1 = maps:remove(Seq, Unacked),
+    State = rabbit_shovel_behaviour:decr_remaining(
+              1, State0#{dest => Dst#{unacked => Unacked1}}),
+    ConfirmFun(InTag, State).
+
+sent_delivery(#{source := #{delivery_count := DeliveryCount0,
+                            credit := Credit0,
+                            queue_delivery_count := QDeliveryCount0,
+                            queue_credit := QCredit0} = Src
+               } = State0, NumMsgs) ->
+    DeliveryCount = serial_number:add(DeliveryCount0, NumMsgs),
+    Credit = max(0, Credit0 - NumMsgs),
+    QDeliveryCount = serial_number:add(QDeliveryCount0, NumMsgs),
+    QCredit = max(0, QCredit0 - NumMsgs),
+    State0#{source => Src#{credit => Credit,
+                           delivery_count => DeliveryCount,
+                           queue_credit => QCredit,
+                           queue_delivery_count => QDeliveryCount
+                          }}.
+
+maybe_grant_or_stash_credit(#{source := #{queue := QName0,
+                                    credit := Credit,
+                                    delivery_count := DeliveryCount,
+                                    at_least_one_credit_req_in_flight := HaveCreditReqInFlight,
+                                    current := #{consumer_tag := CTag,
+                                                 vhost := VHost,
+                                                 queue_states := QState0} = Current
+                                   } = Src,
+                        dest := #{unacked := Unacked}} = State0) ->
+    MaxLinkCredit = max_link_credit(),
+    GrantLinkCredit = grant_link_credit(Credit, MaxLinkCredit, maps:size(Unacked)),
+    Src1 = case HaveCreditReqInFlight andalso GrantLinkCredit of
+               true ->
+                   Req = #credit_req {
+                            delivery_count = DeliveryCount
+                           },
+                   maps:put(stashed_credit_req, Req, Src);
+               false ->
+                   Src
+           end,
+    {ok, QState, Actions} = case (GrantLinkCredit and not HaveCreditReqInFlight) of
+                                true ->
+                                    QName = rabbit_misc:r(VHost, queue, QName0),
+                                    rabbit_queue_type:credit(
+                                      QName, CTag, DeliveryCount, MaxLinkCredit,
+                                      false, QState0);
+                                _ ->
+                                    {ok, QState0, []}
+                            end,
+    CreditReqInFlight = case GrantLinkCredit of
+                            true -> true;
+                            false -> HaveCreditReqInFlight
+                        end,
+    State = State0#{source => Src1#{current => Current#{queue_states => QState},
+                                    at_least_one_credit_req_in_flight => CreditReqInFlight
+                                   }},
+    handle_queue_actions(Actions, State).
+
+max_link_credit() ->
+    application:get_env(rabbit, max_link_credit, ?DEFAULT_MAX_LINK_CREDIT).
+
+grant_link_credit(Credit, MaxLinkCredit, NumUnconfirmed) ->
+    Credit =< MaxLinkCredit div 2 andalso
+    NumUnconfirmed < MaxLinkCredit.
+
+%% Drain is ignored because local shovels do not use it.
+handle_credit_reply({credit_reply, CTag, DeliveryCount, Credit, _Available, _Drain},
+                    #{source := #{credit := CCredit,
+                                  queue_delivery_count := QDeliveryCount,
+                                  stashed_credit_req := StashedCreditReq,
+                                  queue := QName0,
+                                  current := Current = #{queue_states := QState0,
+                                                         vhost := VHost}} = Src} = State0) ->
+    %% Assertion: Our (receiver) delivery-count should be always
+    %% in sync with the delivery-count of the sending queue.
+    QDeliveryCount = DeliveryCount,
+    case StashedCreditReq of
+        #credit_req{delivery_count = StashedDeliveryCount} ->
+          MaxLinkCredit = max_link_credit(),
+          QName = rabbit_misc:r(VHost, queue, QName0),
+          {ok, QState, Actions} = rabbit_queue_type:credit(QName, CTag, StashedDeliveryCount,
+            MaxLinkCredit, false, QState0),
+          State = State0#{source => Src#{queue_credit => MaxLinkCredit,
+            at_least_one_credit_req_in_flight => true,
+            stashed_credit_req => none,
+            current => Current#{queue_states => QState}}},
+          handle_queue_actions(Actions, State);
+        none when Credit =:= 0 andalso
+                  CCredit > 0 ->
+            MaxLinkCredit = max_link_credit(),
+            QName = rabbit_misc:r(VHost, queue, QName0),
+            {ok, QState, Actions} = rabbit_queue_type:credit(QName, CTag, DeliveryCount, MaxLinkCredit, false, QState0),
+            State = State0#{source => Src#{queue_credit => MaxLinkCredit,
+                                           at_least_one_credit_req_in_flight => true,
+                                           current => Current#{queue_states => QState}}},
+            handle_queue_actions(Actions, State);
+        none ->
+            %% Although we (the receiver) usually determine link credit, we set here
+            %% our link credit to what the queue says our link credit is (which is safer
+            %% in case credit requests got applied out of order in quorum queues).
+            %% This should be fine given that we asserted earlier that our delivery-count is
+            %% in sync with the delivery-count of the sending queue.
+            State0#{source => Src#{queue_credit => Credit,
+                                   at_least_one_credit_req_in_flight => false}}
+    end.
diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_behaviour.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_behaviour.erl
index 8f7a890d1698..929a34c5ccf1 100644
--- a/deps/rabbitmq_shovel/src/rabbit_shovel_behaviour.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_shovel_behaviour.erl
@@ -24,7 +24,7 @@
          dest_protocol/1,
          source_endpoint/1,
          dest_endpoint/1,
-         forward/4,
+         forward/3,
          ack/3,
          nack/3,
          status/1,
@@ -82,8 +82,7 @@
 
 -callback ack(Tag :: tag(), Multi :: boolean(), state()) -> state().
 -callback nack(Tag :: tag(), Multi :: boolean(), state()) -> state().
--callback forward(Tag :: tag(), Props :: #{atom() => any()},
-                  Payload :: binary(), state()) ->
+-callback forward(Tag :: tag(), Msg :: mc:state(), state()) ->
     state() | {stop, any()}.
 -callback status(state()) -> rabbit_shovel_status:shovel_status().
 
@@ -144,10 +143,10 @@ source_endpoint(#{source := #{module := Mod}} = State) ->
 dest_endpoint(#{dest := #{module := Mod}} = State) ->
     Mod:dest_endpoint(State).
 
--spec forward(tag(), #{atom() => any()}, binary(), state()) ->
+-spec forward(tag(), mc:state(), state()) ->
     state() | {stop, any()}.
-forward(Tag, Props, Payload, #{dest := #{module := Mod}} = State) ->
-    Mod:forward(Tag, Props, Payload, State).
+forward(Tag, Msg, #{dest := #{module := Mod}} = State) ->
+    Mod:forward(Tag, Msg, State).
 
 -spec ack(tag(), boolean(), state()) -> state().
 ack(Tag, Multi, #{source := #{module := Mod}} = State) ->
diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_config.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_config.erl
index c03a77c94f2f..0f640bc2c7cd 100644
--- a/deps/rabbitmq_shovel/src/rabbit_shovel_config.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_shovel_config.erl
@@ -13,7 +13,8 @@
 -include("rabbit_shovel.hrl").
 
 resolve_module(amqp091) -> rabbit_amqp091_shovel;
-resolve_module(amqp10) -> rabbit_amqp10_shovel.
+resolve_module(amqp10) -> rabbit_amqp10_shovel;
+resolve_module(local) -> rabbit_local_shovel.
 
 is_legacy(Config) ->
     not proplists:is_defined(source, Config).
diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl
index 2ae9ef6fe55b..06a7a223621e 100644
--- a/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_shovel_parameters.erl
@@ -86,13 +86,15 @@ internal_owner(Def) ->
 validate_src(Def) ->
     case protocols(Def)  of
         {amqp091, _} -> validate_amqp091_src(Def);
-        {amqp10, _} -> []
+        {amqp10, _} -> [];
+        {local, _} -> validate_local_src(Def)
     end.
 
 validate_dest(Def) ->
     case protocols(Def)  of
         {_, amqp091} -> validate_amqp091_dest(Def);
-        {_, amqp10} -> []
+        {_, amqp10} -> [];
+        {_, local} -> validate_local_dest(Def)
     end.
 
 validate_amqp091_src(Def) ->
@@ -108,6 +110,19 @@ validate_amqp091_src(Def) ->
              ok
      end].
 
+validate_local_src(Def) ->
+    [case pget2(<<"src-exchange">>, <<"src-queue">>, Def) of
+         zero -> {error, "Must specify 'src-exchange' or 'src-queue'", []};
+         one  -> ok;
+         both -> {error, "Cannot specify 'src-exchange' and 'src-queue'", []}
+     end,
+     case {pget(<<"src-delete-after">>, Def, pget(<<"delete-after">>, Def)), pget(<<"ack-mode">>, Def)} of
+         {N, <<"no-ack">>} when is_integer(N) ->
+             {error, "Cannot specify 'no-ack' and numerical 'delete-after'", []};
+         _ ->
+             ok
+     end].
+
 obfuscate_uris_in_definition(Def) ->
   SrcURIs  = get_uris(<<"src-uri">>, Def),
   ObfuscatedSrcURIsDef = pset(<<"src-uri">>, obfuscate_uris(SrcURIs), Def),
@@ -125,6 +140,13 @@ validate_amqp091_dest(Def) ->
          both -> {error, "Cannot specify 'dest-exchange' and 'dest-queue'", []}
      end].
 
+validate_local_dest(Def) ->
+    [case pget2(<<"dest-exchange">>, <<"dest-queue">>, Def) of
+         zero -> ok;
+         one  -> ok;
+         both -> {error, "Cannot specify 'dest-exchange' and 'dest-queue'", []}
+     end].
+
 shovel_validation() ->
     [{<<"internal">>, fun rabbit_parameter_validation:boolean/2, optional},
      {<<"internal_owner">>, fun validate_internal_owner/2, optional},
@@ -132,17 +154,29 @@ shovel_validation() ->
      {<<"ack-mode">>, rabbit_parameter_validation:enum(
                         ['no-ack', 'on-publish', 'on-confirm']), optional},
      {<<"src-protocol">>,
-      rabbit_parameter_validation:enum(['amqp10', 'amqp091']), optional},
+      rabbit_parameter_validation:enum(['amqp10', 'amqp091', 'local']), optional},
      {<<"dest-protocol">>,
-      rabbit_parameter_validation:enum(['amqp10', 'amqp091']), optional}
+      rabbit_parameter_validation:enum(['amqp10', 'amqp091', 'local']), optional}
     ].
 
 src_validation(Def, User) ->
     case protocols(Def)  of
         {amqp091, _} -> amqp091_src_validation(Def, User);
-        {amqp10, _} -> amqp10_src_validation(Def, User)
+        {amqp10, _} -> amqp10_src_validation(Def, User);
+        {local, _} -> local_src_validation(Def, User)
     end.
 
+local_src_validation(_Def, User) ->
+    [
+     {<<"src-uri">>, validate_uri_fun(User), mandatory},
+     {<<"src-exchange">>,     fun rabbit_parameter_validation:binary/2, optional},
+     {<<"src-exchange-key">>, fun rabbit_parameter_validation:binary/2, optional},
+     {<<"src-queue">>, fun rabbit_parameter_validation:binary/2, optional},
+     {<<"src-queue-args">>,   fun validate_queue_args/2, optional},
+     {<<"src-consumer-args">>, fun validate_consumer_args/2, optional},
+     {<<"src-delete-after">>, fun validate_delete_after/2, optional},
+     {<<"src-predeclared">>,  fun rabbit_parameter_validation:boolean/2, optional}
+    ].
 
 amqp10_src_validation(_Def, User) ->
     [
@@ -173,7 +207,8 @@ dest_validation(Def0, User) ->
     Def = rabbit_data_coercion:to_proplist(Def0),
     case protocols(Def)  of
         {_, amqp091} -> amqp091_dest_validation(Def, User);
-        {_, amqp10} -> amqp10_dest_validation(Def, User)
+        {_, amqp10} -> amqp10_dest_validation(Def, User);
+        {_, local} -> local_dest_validation(Def, User)
     end.
 
 amqp10_dest_validation(_Def, User) ->
@@ -181,9 +216,16 @@ amqp10_dest_validation(_Def, User) ->
      {<<"dest-address">>, fun rabbit_parameter_validation:binary/2, mandatory},
      {<<"dest-add-forward-headers">>, fun rabbit_parameter_validation:boolean/2, optional},
      {<<"dest-add-timestamp-header">>, fun rabbit_parameter_validation:boolean/2, optional},
+     %% The bare message should be inmutable in the AMQP network.
+     %% Before RabbitMQ 4.2, we allowed to set application properties, message
+     %% annotations and any property. This is wrong.
+     %% From 4.2, the only message modification allowed is the optional
+     %% addition of forward headers and shovelled timestamp inside message
+     %% annotations.
+     %% To avoid breaking existing deployments, the following configuration
+     %% keys are still accepted but will be ignored.
      {<<"dest-application-properties">>, fun validate_amqp10_map/2, optional},
      {<<"dest-message-annotations">>, fun validate_amqp10_map/2, optional},
-     % TODO: restrict to allowed fields
      {<<"dest-properties">>, fun validate_amqp10_map/2, optional}
     ].
 
@@ -202,6 +244,17 @@ amqp091_dest_validation(_Def, User) ->
      {<<"dest-predeclared">>,  fun rabbit_parameter_validation:boolean/2, optional}
     ].
 
+local_dest_validation(_Def, User) ->
+    [{<<"dest-uri">>,        validate_uri_fun(User), mandatory},
+     {<<"dest-exchange">>,   fun rabbit_parameter_validation:binary/2,optional},
+     {<<"dest-exchange-key">>,fun rabbit_parameter_validation:binary/2,optional},
+     {<<"dest-queue">>,      fun rabbit_parameter_validation:amqp091_queue_name/2,optional},
+     {<<"dest-queue-args">>, fun validate_queue_args/2, optional},
+     {<<"dest-add-forward-headers">>, fun rabbit_parameter_validation:boolean/2,optional},
+     {<<"dest-add-timestamp-header">>, fun rabbit_parameter_validation:boolean/2,optional},
+     {<<"dest-predeclared">>,  fun rabbit_parameter_validation:boolean/2, optional}
+    ].
+
 validate_uri_fun(User) ->
     fun (Name, Term) -> validate_uri(Name, Term, User) end.
 
@@ -315,7 +368,8 @@ parse({VHost, Name}, ClusterName, Def) ->
 parse_source(Def) ->
     case protocols(Def) of
         {amqp10, _} -> parse_amqp10_source(Def);
-        {amqp091, _} -> parse_amqp091_source(Def)
+        {amqp091, _} -> parse_amqp091_source(Def);
+        {local, _} -> parse_local_source(Def)
     end.
 
 parse_dest(VHostName, ClusterName, Def, SourceHeaders) ->
@@ -323,7 +377,9 @@ parse_dest(VHostName, ClusterName, Def, SourceHeaders) ->
         {_, amqp10} ->
             parse_amqp10_dest(VHostName, ClusterName, Def, SourceHeaders);
         {_, amqp091} ->
-            parse_amqp091_dest(VHostName, ClusterName, Def, SourceHeaders)
+            parse_amqp091_dest(VHostName, ClusterName, Def, SourceHeaders);
+        {_, local} ->
+            parse_local_dest(VHostName, ClusterName, Def, SourceHeaders)
     end.
 
 parse_amqp10_dest({_VHost, _Name}, _ClusterName, Def, SourceHeaders) ->
@@ -400,6 +456,35 @@ parse_amqp091_dest({VHost, Name}, ClusterName, Def, SourceHeaders) ->
                                                     AddTimestampHeader]}
                 }, Details).
 
+parse_local_dest({_VHost, _Name}, _ClusterName, Def, _SourceHeaders) ->
+    Mod       = rabbit_local_shovel,
+    DestURIs  = deobfuscated_uris(<<"dest-uri">>,      Def),
+    DestX     = pget(<<"dest-exchange">>,     Def, none),
+    DestXKey  = pget(<<"dest-exchange-key">>, Def, none),
+    DestQ     = pget(<<"dest-queue">>,        Def, none),
+    DestQArgs = pget(<<"dest-queue-args">>,   Def, #{}),
+    GlobalPredeclared = proplists:get_value(predeclared, application:get_env(?APP, topology, []), false),
+    Predeclared = pget(<<"dest-predeclared">>, Def, GlobalPredeclared),
+    DestDeclFun = case Predeclared of
+        true -> {Mod, dest_check_queue, [DestQ, DestQArgs]};
+        false -> {Mod, dest_decl_queue, [DestQ, DestQArgs]}
+    end,
+
+    AddHeaders = pget(<<"dest-add-forward-headers">>, Def, false),
+    AddTimestampHeader = pget(<<"dest-add-timestamp-header">>, Def, false),
+    %% Details are only used for status report in rabbitmqctl, as vhost is not
+    %% available to query the runtime parameters.
+    Details = maps:from_list([{K, V} || {K, V} <- [{exchange, DestX},
+                                                   {routing_key, DestXKey},
+                                                   {queue, DestQ}],
+                                        V =/= none]),
+    maps:merge(#{module => rabbit_local_shovel,
+                 uris => DestURIs,
+                 resource_decl => DestDeclFun,
+                 add_forward_headers => AddHeaders,
+                 add_timestamp_header => AddTimestampHeader
+                }, Details).
+
 fields_fun(X, Key, _SrcURI, _DestURI, P0) ->
     P1 = case X of
              none -> P0;
@@ -489,6 +574,46 @@ parse_amqp091_source(Def) ->
                   consumer_args => SrcCArgs
                  }, Details), DestHeaders}.
 
+parse_local_source(Def) ->
+    %% TODO add exchange source back
+    Mod      = rabbit_local_shovel,
+    SrcURIs  = deobfuscated_uris(<<"src-uri">>, Def),
+    SrcX     = pget(<<"src-exchange">>,Def, none),
+    SrcXKey  = pget(<<"src-exchange-key">>, Def, <<>>),
+    SrcQ     = pget(<<"src-queue">>, Def, none),
+    SrcQArgs = pget(<<"src-queue-args">>,   Def, #{}),
+    SrcCArgs = rabbit_misc:to_amqp_table(pget(<<"src-consumer-args">>, Def, [])),
+    GlobalPredeclared = proplists:get_value(predeclared, application:get_env(?APP, topology, []), false),
+    Predeclared = pget(<<"src-predeclared">>, Def, GlobalPredeclared),
+    {SrcDeclFun, Queue, DestHeaders} =
+    case SrcQ of
+        none -> {{Mod, src_decl_exchange, [SrcX, SrcXKey]}, <<>>,
+                 [{<<"src-exchange">>,     SrcX},
+                  {<<"src-exchange-key">>, SrcXKey}]};
+        _ -> case Predeclared of
+                false ->
+                    {{Mod, decl_queue, [SrcQ, SrcQArgs]},
+                        SrcQ, [{<<"src-queue">>, SrcQ}]};
+                true ->
+                    {{Mod, check_queue, [SrcQ, SrcQArgs]},
+                        SrcQ, [{<<"src-queue">>, SrcQ}]}
+            end
+    end,
+    DeleteAfter = pget(<<"src-delete-after">>, Def,
+                       pget(<<"delete-after">>, Def, <<"never">>)),
+    %% Details are only used for status report in rabbitmqctl, as vhost is not
+    %% available to query the runtime parameters.
+    Details = maps:from_list([{K, V} || {K, V} <- [{exchange, SrcX},
+                                                   {routing_key, SrcXKey}],
+                                        V =/= none]),
+    {maps:merge(#{module => Mod,
+                  uris => SrcURIs,
+                  resource_decl => SrcDeclFun,
+                  queue => Queue,
+                  delete_after => opt_b2a(DeleteAfter),
+                  consumer_args => SrcCArgs
+                 }, Details), DestHeaders}.
+
 src_decl_exchange(SrcX, SrcXKey, _Conn, Ch) ->
     Ms = [#'queue.declare'{exclusive = true},
           #'queue.bind'{routing_key = SrcXKey,
diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_util.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_util.erl
index e33ec1058804..5541041093b2 100644
--- a/deps/rabbitmq_shovel/src/rabbit_shovel_util.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_shovel_util.erl
@@ -11,12 +11,15 @@
          add_timestamp_header/1,
          delete_shovel/3,
          restart_shovel/2,
-         get_shovel_parameter/1]).
+         get_shovel_parameter/1,
+         gen_unique_name/2,
+         decl_fun/2]).
 
 -include_lib("rabbit_common/include/rabbit_framing.hrl").
 -include_lib("rabbit_common/include/rabbit.hrl").
 -include_lib("kernel/include/logger.hrl").
 
+-define(APP, rabbitmq_shovel).
 -define(ROUTING_HEADER, <<"x-shovelled">>).
 -define(TIMESTAMP_HEADER, <<"x-shovelled-timestamp">>).
 
@@ -99,3 +102,58 @@ get_shovel_parameter({VHost, ShovelName}) ->
     rabbit_runtime_parameters:lookup(VHost, <<"shovel">>, ShovelName);
 get_shovel_parameter(ShovelName) ->
     rabbit_runtime_parameters:lookup(<<"/">>, <<"shovel">>, ShovelName).
+
+gen_unique_name(Pre0, Post0) ->
+    Pre = rabbit_data_coercion:to_binary(Pre0),
+    Post = rabbit_data_coercion:to_binary(Post0),
+    Id = bin_to_hex(crypto:strong_rand_bytes(8)),
+    <
>/binary, Id/binary, <<"_">>/binary, Post/binary>>.
+
+bin_to_hex(Bin) ->
+    <<<= 10 -> N -10 + $a;
+           true  -> N + $0 end>>
+      || <> <= Bin>>.
+
+decl_fun(Mod, {source, Endpoint}) ->
+    case parse_declaration({proplists:get_value(declarations, Endpoint, []), []}) of
+        [] ->
+            case proplists:get_value(predeclared, application:get_env(?APP, topology, []), false) of
+                true -> case proplists:get_value(queue, Endpoint) of
+                            <<>> -> fail({invalid_parameter_value, declarations, {require_non_empty}});
+                            Queue -> {Mod, check_fun, [Queue]}
+                        end;
+                false -> {Mod, decl_fun, []}
+            end;
+        Decl -> {Mod, decl_fun, [Decl]}
+    end;
+decl_fun(Mod, {destination, Endpoint}) ->
+    Decl = parse_declaration({proplists:get_value(declarations, Endpoint, []), []}),
+    {Mod, decl_fun, [Decl]}.
+
+parse_declaration({[], Acc}) ->
+    Acc;
+parse_declaration({[{Method, Props} | Rest], Acc}) when is_list(Props) ->
+    FieldNames = try rabbit_framing_amqp_0_9_1:method_fieldnames(Method)
+                 catch exit:Reason -> fail(Reason)
+                 end,
+    case proplists:get_keys(Props) -- FieldNames of
+        []            -> ok;
+        UnknownFields -> fail({unknown_fields, Method, UnknownFields})
+    end,
+    {Res, _Idx} = lists:foldl(
+                    fun (K, {R, Idx}) ->
+                            NewR = case proplists:get_value(K, Props) of
+                                       undefined -> R;
+                                       V         -> setelement(Idx, R, V)
+                                   end,
+                            {NewR, Idx + 1}
+                    end, {rabbit_framing_amqp_0_9_1:method_record(Method), 2},
+                    FieldNames),
+    parse_declaration({Rest, [Res | Acc]});
+parse_declaration({[{Method, Props} | _Rest], _Acc}) ->
+    fail({expected_method_field_list, Method, Props});
+parse_declaration({[Method | Rest], Acc}) ->
+    parse_declaration({[{Method, []} | Rest], Acc}).
+
+-spec fail(term()) -> no_return().
+fail(Reason) -> throw({error, Reason}).
diff --git a/deps/rabbitmq_shovel/src/rabbit_shovel_worker.erl b/deps/rabbitmq_shovel/src/rabbit_shovel_worker.erl
index 368bb60ec622..df7bda6191e3 100644
--- a/deps/rabbitmq_shovel/src/rabbit_shovel_worker.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_shovel_worker.erl
@@ -52,6 +52,9 @@ init([Type, Name, Config0]) ->
                      Config0;
                 dynamic ->
                     ClusterName = rabbit_nodes:cluster_name(),
+                     %% TODO It could handle errors while parsing
+                     %% (i.e. missing predeclared queues) and stop nicely
+                     %% without long stacktraces
                     {ok, Conf} = rabbit_shovel_parameters:parse(Name,
                                                                 ClusterName,
                                                                 Config0),
@@ -103,10 +106,14 @@ handle_cast(init_shovel, State = #state{config = Config}) ->
                [human_readable_name(maps:get(name, Config2))]),
     State1 = State#state{config = Config2},
     ok = report_running(State1),
-    {noreply, State1}.
+    {noreply, State1};
+handle_cast(Msg, State) ->
+    handle_msg(Msg, State).
 
+handle_info(Msg, State) ->
+    handle_msg(Msg, State).
 
-handle_info(Msg, State = #state{config = Config, name = Name}) ->
+handle_msg(Msg, State = #state{config = Config, name = Name}) ->
     case rabbit_shovel_behaviour:handle_source(Msg, Config) of
         not_handled ->
             case rabbit_shovel_behaviour:handle_dest(Msg, Config) of
diff --git a/deps/rabbitmq_shovel/test/amqp10_SUITE.erl b/deps/rabbitmq_shovel/test/amqp10_SUITE.erl
index 937d37037cd3..bb8a118a8f28 100644
--- a/deps/rabbitmq_shovel/test/amqp10_SUITE.erl
+++ b/deps/rabbitmq_shovel/test/amqp10_SUITE.erl
@@ -8,6 +8,7 @@
 -module(amqp10_SUITE).
 
 -include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
 -include_lib("amqp_client/include/amqp_client.hrl").
 
 -compile(export_all).
@@ -119,21 +120,29 @@ amqp10_destination(Config, AckMode) ->
     receive
         {amqp10_msg, Receiver, InMsg} ->
             [<<42>>] = amqp10_msg:body(InMsg),
-            #{content_type := ?UNSHOVELLED,
-              content_encoding := ?UNSHOVELLED,
-              correlation_id := ?UNSHOVELLED,
-              user_id := <<"guest">>,
-              message_id := ?UNSHOVELLED,
-              reply_to := ?UNSHOVELLED
-              %% timestamp gets overwritten
-              % creation_time := Timestamp
-             } = amqp10_msg:properties(InMsg),
-            #{<<"routing_key">> := ?TO_SHOVEL,
-              <<"type">> := ?UNSHOVELLED,
-              <<"header1">> := 1,
-              <<"header2">> := <<"h2">>
-             } = amqp10_msg:application_properties(InMsg),
-            #{durable := true} = amqp10_msg:headers(InMsg),
+            Ts = Timestamp * 1000,
+            ?assertMatch(
+               #{content_type := ?UNSHOVELLED,
+                 content_encoding := ?UNSHOVELLED,
+                 correlation_id := ?UNSHOVELLED,
+                 user_id := <<"guest">>,
+                 message_id := ?UNSHOVELLED,
+                 reply_to := ?UNSHOVELLED,
+                 %% Message timestamp is no longer overwritten
+                 creation_time := Ts},
+               amqp10_msg:properties(InMsg)),
+            ?assertMatch(
+               #{<<"header1">> := 1,
+                 <<"header2">> := <<"h2">>},
+               amqp10_msg:application_properties(InMsg)),
+            ?assertMatch(
+               #{<<"x-basic-type">> := ?UNSHOVELLED,
+                 <<"x-opt-shovel-type">> := <<"static">>,
+                 <<"x-opt-shovel-name">> := <<"test_shovel">>,
+                 <<"x-opt-shovelled-by">> := _,
+                 <<"x-opt-shovelled-timestamp">> := _},
+               amqp10_msg:message_annotations(InMsg)),
+            ?assertMatch(#{durable := true}, amqp10_msg:headers(InMsg)),
             ok
     after ?TIMEOUT ->
               throw(timeout_waiting_for_deliver1)
diff --git a/deps/rabbitmq_shovel/test/amqp10_dynamic_SUITE.erl b/deps/rabbitmq_shovel/test/amqp10_dynamic_SUITE.erl
index 639045c76ae7..8a47a16feac6 100644
--- a/deps/rabbitmq_shovel/test/amqp10_dynamic_SUITE.erl
+++ b/deps/rabbitmq_shovel/test/amqp10_dynamic_SUITE.erl
@@ -28,7 +28,8 @@ groups() ->
           autodelete_amqp091_dest_on_publish,
           simple_amqp10_dest,
           simple_amqp10_src,
-          amqp091_to_amqp10_with_dead_lettering
+          amqp091_to_amqp10_with_dead_lettering,
+          amqp10_to_amqp091_application_properties
         ]},
       {with_map_config, [], [
           simple,
@@ -154,16 +155,19 @@ test_amqp10_destination(Config, Src, Dest, Sess, Protocol, ProtocolSrc) ->
                                   end}]),
     Msg = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>),
     AppProps = amqp10_msg:application_properties(Msg),
-
-    ?assertMatch((#{user_id := <<"guest">>, creation_time := _}),
-                 (amqp10_msg:properties(Msg))),
-    ?assertMatch((#{<<"shovel-name">> := <<"test">>,
-                    <<"shovel-type">> := <<"dynamic">>, <<"shovelled-by">> := _,
-                    <<"app-prop-key">> := <<"app-prop-value">>}),
-                 (AppProps)),
+    Anns = amqp10_msg:message_annotations(Msg),
+    %% We no longer add/override properties, application properties or
+    %% message annotations. Just the forward headers and timestamp as
+    %% message annotations. The AMQP 1.0 message is inmutable
+    ?assertNot(maps:is_key(user_id, amqp10_msg:properties(Msg))),
+    ?assertNot(maps:is_key(<<"app-prop-key">>, AppProps)),
     ?assertEqual(undefined, maps:get(<<"delivery_mode">>, AppProps, undefined)),
-    ?assertMatch((#{<<"x-message-ann-key">> := <<"message-ann-value">>}),
-                 (amqp10_msg:message_annotations(Msg))).
+    ?assertNot(maps:is_key(<<"x-message-ann-key">>, Anns)),
+    ?assertMatch(#{<<"x-opt-shovel-name">> := <<"test">>,
+                   <<"x-opt-shovel-type">> := <<"dynamic">>,
+                   <<"x-opt-shovelled-by">> := _,
+                   <<"x-opt-shovelled-timestamp">> := _
+                  }, Anns).
 
 simple_amqp10_src(Config) ->
     MapConfig = ?config(map_config, Config),
@@ -193,6 +197,42 @@ simple_amqp10_src(Config) ->
               ok
       end).
 
+amqp10_to_amqp091_application_properties(Config) ->
+    MapConfig = ?config(map_config, Config),
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+              shovel_test_utils:set_param(
+                Config,
+                <<"test">>, [{<<"src-protocol">>, <<"amqp10">>},
+                             {<<"src-address">>,  Src},
+                             {<<"dest-protocol">>, <<"amqp091">>},
+                             {<<"dest-queue">>, Dest},
+                             {<<"add-forward-headers">>, true},
+                             {<<"dest-add-timestamp-header">>, true},
+                             {<<"publish-properties">>,
+                                case MapConfig of
+                                    true -> #{<<"cluster_id">> => <<"x">>};
+                                    _    -> [{<<"cluster_id">>, <<"x">>}]
+                                end}
+                            ]),
+
+              MsgSent = amqp10_msg:set_application_properties(
+                      #{<<"key">> => <<"value">>},
+                      amqp10_msg:set_headers(
+                        #{durable => true},
+                        amqp10_msg:new(<<"tag1">>, <<"hello">>, false))),
+
+              Msg = publish_expect_msg(Sess, Src, Dest, MsgSent),
+              AppProps = amqp10_msg:application_properties(Msg),
+              ct:pal("MSG ~p", [Msg]),
+
+              ?assertMatch(#{<<"key">> := <<"value">>},
+                           AppProps),
+              ok
+      end).
+
 change_definition(Config) ->
     Src = ?config(srcq, Config),
     Dest = ?config(destq, Config),
@@ -257,8 +297,8 @@ autodelete_do(Config, {AckMode, After, ExpSrc, ExpDest}) ->
                            {<<"ack-mode">>,     AckMode}
                           ]),
             await_autodelete(Config, <<"test">>),
-            expect_count(Session, Dest, <<"hello">>, ExpDest),
-            expect_count(Session, Src, <<"hello">>, ExpSrc)
+            expect_count(Session, Dest, ExpDest),
+            expect_count(Session, Src, ExpSrc)
     end.
 
 autodelete_amqp091_src(Config, {AckMode, After, ExpSrc, ExpDest}) ->
@@ -277,8 +317,8 @@ autodelete_amqp091_src(Config, {AckMode, After, ExpSrc, ExpDest}) ->
                            {<<"ack-mode">>, AckMode}
                           ]),
             await_autodelete(Config, <<"test">>),
-            expect_count(Session, Dest, <<"hello">>, ExpDest),
-            expect_count(Session, Src, <<"hello">>, ExpSrc)
+            expect_count(Session, Dest, ExpDest),
+            expect_count(Session, Src, ExpSrc)
     end.
 
 autodelete_amqp091_dest(Config, {AckMode, After, ExpSrc, ExpDest}) ->
@@ -297,8 +337,8 @@ autodelete_amqp091_dest(Config, {AckMode, After, ExpSrc, ExpDest}) ->
                            {<<"ack-mode">>, AckMode}
                           ]),
             await_autodelete(Config, <<"test">>),
-            expect_count(Session, Dest, <<"hello">>, ExpDest),
-            expect_count(Session, Src, <<"hello">>, ExpSrc)
+            expect_count(Session, Dest, ExpDest),
+            expect_count(Session, Src, ExpSrc)
     end.
 
 %%----------------------------------------------------------------------------
@@ -323,6 +363,15 @@ publish(Sender, Tag, Payload) when is_binary(Payload) ->
               exit(publish_disposition_not_received)
     end.
 
+publish(Sender, Msg) ->
+    ok = amqp10_client:send_msg(Sender, Msg),
+    Tag = amqp10_msg:delivery_tag(Msg),
+    receive
+        {amqp10_disposition, {accepted, Tag}} -> ok
+    after 3000 ->
+            exit(publish_disposition_not_received)
+    end.
+
 publish_expect(Session, Source, Dest, Tag, Payload) ->
     LinkName = <<"dynamic-sender-", Dest/binary>>,
     {ok, Sender} = amqp10_client:attach_sender_link(Session, LinkName, Source,
@@ -330,7 +379,16 @@ publish_expect(Session, Source, Dest, Tag, Payload) ->
     ok = await_amqp10_event(link, Sender, attached),
     publish(Sender, Tag, Payload),
     amqp10_client:detach_link(Sender),
-    expect_one(Session, Dest, Payload).
+    expect_one(Session, Dest).
+
+publish_expect_msg(Session, Source, Dest, Msg) ->
+    LinkName = <<"dynamic-sender-", Dest/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, LinkName, Source,
+                                                    unsettled, unsettled_state),
+    ok = await_amqp10_event(link, Sender, attached),
+    publish(Sender, Msg),
+    amqp10_client:detach_link(Sender),
+    expect_one(Session, Dest).
 
 await_amqp10_event(On, Ref, Evt) ->
     receive
@@ -339,17 +397,17 @@ await_amqp10_event(On, Ref, Evt) ->
           exit({amqp10_event_timeout, On, Ref, Evt})
     end.
 
-expect_one(Session, Dest, Payload) ->
+expect_one(Session, Dest) ->
     LinkName = <<"dynamic-receiver-", Dest/binary>>,
     {ok, Receiver} = amqp10_client:attach_receiver_link(Session, LinkName,
                                                         Dest, settled,
                                                         unsettled_state),
     ok = amqp10_client:flow_link_credit(Receiver, 1, never),
-    Msg = expect(Receiver, Payload),
+    Msg = expect(Receiver),
     amqp10_client:detach_link(Receiver),
     Msg.
 
-expect(Receiver, _Payload) ->
+expect(Receiver) ->
     receive
         {amqp10_msg, Receiver, InMsg} ->
             InMsg
@@ -379,7 +437,7 @@ publish_count(Session, Address, Payload, Count) ->
      end || I <- lists:seq(1, Count)],
      amqp10_client:detach_link(Sender).
 
-expect_count(Session, Address, Payload, Count) ->
+expect_count(Session, Address, Count) ->
     {ok, Receiver} = amqp10_client:attach_receiver_link(Session,
                                                         <<"dynamic-receiver",
                                                           Address/binary>>,
@@ -387,7 +445,7 @@ expect_count(Session, Address, Payload, Count) ->
                                                         unsettled_state),
     ok = amqp10_client:flow_link_credit(Receiver, Count, never),
     [begin
-         expect(Receiver, Payload)
+         expect(Receiver)
      end || _ <- lists:seq(1, Count)],
     expect_empty(Session, Address),
     amqp10_client:detach_link(Receiver).
diff --git a/deps/rabbitmq_shovel/test/amqp10_shovel_SUITE.erl b/deps/rabbitmq_shovel/test/amqp10_shovel_SUITE.erl
index 9550c1b74309..834813fe6aea 100644
--- a/deps/rabbitmq_shovel/test/amqp10_shovel_SUITE.erl
+++ b/deps/rabbitmq_shovel/test/amqp10_shovel_SUITE.erl
@@ -62,8 +62,8 @@ end_per_testcase(_TestCase, _Config) ->
 amqp_encoded_data_list(_Config) ->
     meck:new(rabbit_shovel_behaviour, [passthrough]),
     meck:expect(rabbit_shovel_behaviour, forward,
-                fun (_, _, Pay, S) ->
-                        ?assert(erlang:is_binary(Pay)),
+                fun (_, Msg, S) ->
+                        ?assert(mc:is(Msg)),
                         S
                 end),
     %% fake some shovel state
@@ -83,8 +83,8 @@ amqp_encoded_data_list(_Config) ->
 amqp_encoded_amqp_value(_Config) ->
     meck:new(rabbit_shovel_behaviour, [passthrough]),
     meck:expect(rabbit_shovel_behaviour, forward,
-                fun (_, _, Pay, S) ->
-                        ?assert(erlang:is_binary(Pay)),
+                fun (_, Msg, S) ->
+                        ?assert(mc:is(Msg)),
                         S
                 end),
     %% fake some shovel state
diff --git a/deps/rabbitmq_shovel/test/local_SUITE.erl b/deps/rabbitmq_shovel/test/local_SUITE.erl
new file mode 100644
index 000000000000..f12c53f83512
--- /dev/null
+++ b/deps/rabbitmq_shovel/test/local_SUITE.erl
@@ -0,0 +1,510 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
+%%
+
+-module(local_SUITE).
+
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
+
+-compile(export_all).
+
+-define(EXCHANGE,    <<"test_exchange">>).
+-define(TO_SHOVEL,   <<"to_the_shovel">>).
+-define(FROM_SHOVEL, <<"from_the_shovel">>).
+-define(UNSHOVELLED, <<"unshovelled">>).
+-define(SHOVELLED,   <<"shovelled">>).
+-define(TIMEOUT,     1000).
+
+all() ->
+    [
+      {group, tests}
+    ].
+
+groups() ->
+    [
+      {tests, [], [
+          local_destination_no_ack,
+          local_destination_on_publish,
+          local_destination_on_confirm,
+          local_destination_forward_headers_amqp10,
+          local_destination_forward_headers_amqp091,
+          local_destination_no_forward_headers_amqp10,
+          local_destination_timestamp_header_amqp10,
+          local_destination_timestamp_header_amqp091,
+          local_destination_no_timestamp_header_amqp10,
+          local_source_no_ack,
+          local_source_on_publish,
+          local_source_on_confirm,
+          local_source_anonymous_queue
+        ]}
+    ].
+
+%% -------------------------------------------------------------------
+%% Testsuite setup/teardown.
+%% -------------------------------------------------------------------
+
+init_per_suite(Config) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    Config1 = rabbit_ct_helpers:set_config(Config, [
+        {rmq_nodename_suffix, ?MODULE}
+      ]),
+    rabbit_ct_helpers:run_setup_steps(Config1,
+      rabbit_ct_broker_helpers:setup_steps() ++
+      rabbit_ct_client_helpers:setup_steps() ++
+      [fun stop_shovel_plugin/1]).
+
+end_per_suite(Config) ->
+    application:stop(amqp10_client),
+    rabbit_ct_helpers:run_teardown_steps(Config,
+      rabbit_ct_client_helpers:teardown_steps() ++
+      rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_group(_, Config) ->
+    Config.
+
+end_per_group(_, Config) ->
+    Config.
+
+init_per_testcase(Testcase, Config) ->
+    rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config) ->
+    rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queues, [[<<"source-queue">>, <<"dest-queue">>]]),
+    rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+stop_shovel_plugin(Config) ->
+    ok = rabbit_ct_broker_helpers:rpc(Config, 0,
+                                      application, stop, [rabbitmq_shovel]),
+    Config.
+
+%% -------------------------------------------------------------------
+%% Testcases.
+%% -------------------------------------------------------------------
+
+%% TODO test errors when queue has not been predeclared, it just crashes
+%% with a case_clause right now
+
+local_destination_no_ack(Config) ->
+    local_destination(Config, no_ack).
+
+local_destination_on_publish(Config) ->
+    local_destination(Config, on_publish).
+
+local_destination_on_confirm(Config) ->
+    local_destination(Config, on_confirm).
+
+local_destination(Config, AckMode) ->
+    TargetQ =  <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, AckMode, []),
+    {Conn, Receiver} = attach_receiver(Config, TargetQ),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+    Timestamp = erlang:system_time(millisecond),
+    Msg = #amqp_msg{payload = <<42>>,
+                    props = #'P_basic'{delivery_mode = 2,
+                                       headers = [{<<"header1">>, long, 1},
+                                                  {<<"header2">>, longstr, <<"h2">>}],
+                                       content_encoding = ?UNSHOVELLED,
+                                       content_type = ?UNSHOVELLED,
+                                       correlation_id = ?UNSHOVELLED,
+                                       %% needs to be guest here
+                                       user_id = <<"guest">>,
+                                       message_id = ?UNSHOVELLED,
+                                       reply_to = ?UNSHOVELLED,
+                                       timestamp = Timestamp,
+                                       type = ?UNSHOVELLED
+                                      }},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            [<<42>>] = amqp10_msg:body(InMsg),
+            #{content_type := ?UNSHOVELLED,
+              content_encoding := ?UNSHOVELLED,
+              correlation_id := ?UNSHOVELLED,
+              user_id := <<"guest">>,
+              message_id := ?UNSHOVELLED,
+              reply_to := ?UNSHOVELLED
+             } = amqp10_msg:properties(InMsg),
+            #{<<"header1">> := 1,
+              <<"header2">> := <<"h2">>
+             } = amqp10_msg:application_properties(InMsg),
+            #{<<"x-basic-type">> := ?UNSHOVELLED
+             } = amqp10_msg:message_annotations(InMsg),
+            #{durable := true} = amqp10_msg:headers(InMsg),
+            ok
+    after ?TIMEOUT ->
+              throw(timeout_waiting_for_deliver1)
+    end,
+
+
+    ?awaitMatch([[_, <<"1">>, <<"0">>],
+                 [<<"dest-queue">>, <<"1">>, <<"0">>]],
+                lists:sort(
+                  rabbit_ct_broker_helpers:rabbitmqctl_list(
+                    Config, 0,
+                    ["list_queues", "name", "consumers", "messages", "--no-table-headers"])),
+                30000),
+
+    [{test_shovel, static, {running, _Info}, _Metrics, _Time}] =
+        rabbit_ct_broker_helpers:rpc(Config, 0,
+          rabbit_shovel_status, status, []),
+    detach_receiver(Conn, Receiver),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_forward_headers_amqp10(Config) ->
+    TargetQ = <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_forward_headers, true}]),
+    {Conn, Receiver} = attach_receiver(Config, TargetQ),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            ?assertMatch(#{<<"x-opt-shovelled-by">> := _,
+                           <<"x-opt-shovel-type">> := <<"static">>,
+                           <<"x-opt-shovel-name">> := <<"test_shovel">>},
+                         amqp10_msg:message_annotations(InMsg))
+    after ?TIMEOUT ->
+              throw(timeout_waiting_for_deliver1)
+    end,
+
+    detach_receiver(Conn, Receiver),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_forward_headers_amqp091(Config) ->
+    %% Check that we can consume with 0.9.1 or 1.0 and no properties are
+    %% lost in translation
+    TargetQ = <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_forward_headers, true}]),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+    CTag = consume(Chan, TargetQ, true),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {#'basic.deliver'{consumer_tag = CTag},
+         #amqp_msg{props = #'P_basic'{headers = Headers}}} ->
+            ?assertMatch([{<<"x-opt-shovel-name">>, longstr, <<"test_shovel">>},
+                          {<<"x-opt-shovel-type">>, longstr, <<"static">>},
+                          {<<"x-opt-shovelled-by">>, longstr, _}],
+                         lists:sort(Headers))
+    after ?TIMEOUT -> throw(timeout_waiting_for_deliver1)
+    end,
+
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_no_forward_headers_amqp10(Config) ->
+    TargetQ =  <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_forward_headers, false}]),
+    {Conn, Receiver} = attach_receiver(Config, TargetQ),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            Anns = amqp10_msg:message_annotations(InMsg),
+            ?assertNot(maps:is_key(<<"x-opt-shovelled-by">>, Anns)),
+            ?assertNot(maps:is_key(<<"x-opt-shovel-type">>, Anns)),
+            ?assertNot(maps:is_key(<<"x-opt-shovel-name">>, Anns)),
+            ok
+    after ?TIMEOUT ->
+              throw(timeout_waiting_for_deliver1)
+    end,
+
+    detach_receiver(Conn, Receiver),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_timestamp_header_amqp10(Config) ->
+    TargetQ = <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_timestamp_header, true}]),
+    {Conn, Receiver} = attach_receiver(Config, TargetQ),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            ?assertMatch(#{<<"x-opt-shovelled-timestamp">> := _},
+                         amqp10_msg:message_annotations(InMsg))
+    after ?TIMEOUT ->
+              throw(timeout_waiting_for_deliver1)
+    end,
+
+    detach_receiver(Conn, Receiver),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_timestamp_header_amqp091(Config) ->
+    TargetQ = <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_timestamp_header, true}]),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+    CTag = consume(Chan, TargetQ, true),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {#'basic.deliver'{consumer_tag = CTag},
+         #amqp_msg{props = #'P_basic'{headers = Headers}}} ->
+            ?assertMatch([{<<"x-opt-shovelled-timestamp">>, long, _}],
+                         Headers)
+    after ?TIMEOUT -> throw(timeout_waiting_for_deliver1)
+    end,
+
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_destination_no_timestamp_header_amqp10(Config) ->
+    TargetQ =  <<"dest-queue">>,
+    ok = setup_local_destination_shovel(Config, TargetQ, on_publish,
+                                        [{add_timestamp_header, false}]),
+    {Conn, Receiver} = attach_receiver(Config, TargetQ),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+
+    Msg = #amqp_msg{props = #'P_basic'{}},
+    publish(Chan, Msg, ?EXCHANGE, ?TO_SHOVEL),
+
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            Anns = amqp10_msg:message_annotations(InMsg),
+            ?assertNot(maps:is_key(<<"x-opt-shovelled-timestamp">>, Anns))
+    after ?TIMEOUT ->
+              throw(timeout_waiting_for_deliver1)
+    end,
+
+    detach_receiver(Conn, Receiver),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_source_no_ack(Config) ->
+    local_source(Config, no_ack).
+
+local_source_on_publish(Config) ->
+    local_source(Config, on_publish).
+
+local_source_on_confirm(Config) ->
+    local_source(Config, on_confirm).
+
+local_source(Config, AckMode) ->
+    SourceQ =  <<"source-queue">>,
+    DestQ =  <<"dest-queue">>,
+    ok = setup_local_source_shovel(Config, SourceQ, DestQ, AckMode),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+    CTag = consume(Chan, DestQ, AckMode =:= no_ack),
+    Msg = #amqp_msg{payload = <<42>>,
+                    props = #'P_basic'{delivery_mode = 2,
+                                       content_type = ?UNSHOVELLED}},
+    % publish to source
+    publish(Chan, Msg, <<>>, SourceQ),
+
+    receive
+        {#'basic.deliver'{consumer_tag = CTag, delivery_tag = AckTag},
+         #amqp_msg{payload = <<42>>,
+                   props = #'P_basic'{headers = [{<<"x-shovelled">>, _, _},
+                                                 {<<"x-shovelled-timestamp">>,
+                                                  long, _}]}}} ->
+            case AckMode of
+                no_ack -> ok;
+                _      -> ok = amqp_channel:call(
+                                 Chan, #'basic.ack'{delivery_tag = AckTag})
+            end
+    after ?TIMEOUT -> throw(timeout_waiting_for_deliver1)
+    end,
+
+    QueuesAndConsumers = lists:sort([[<<"source-queue">>,<<"1">>,<<"0">>],
+                                     [<<"dest-queue">>,<<"1">>,<<"0">>]]),
+    ?awaitMatch(QueuesAndConsumers,
+                lists:sort(
+                  rabbit_ct_broker_helpers:rabbitmqctl_list(
+                    Config, 0,
+                    ["list_queues", "name", "consumers", "messages", "--no-table-headers"])),
+                30000),
+
+    [{test_shovel, static, {running, _Info}, _Metrics, _Time}] =
+        rabbit_ct_broker_helpers:rpc(Config, 0,
+          rabbit_shovel_status, status, []),
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+local_source_anonymous_queue(Config) ->
+    DestQ =  <<"dest-queue">>,
+    ok = setup_local_server_named_shovel(Config, DestQ, no_ack),
+    Chan = rabbit_ct_client_helpers:open_channel(Config, 0),
+    CTag = consume(Chan, DestQ, true),
+    Msg = #amqp_msg{payload = <<42>>,
+                    props = #'P_basic'{delivery_mode = 2,
+                                       content_type = ?UNSHOVELLED}},
+    % publish to source
+    publish(Chan, Msg, <<"amq.fanout">>, <<>>),
+
+    receive
+        {#'basic.deliver'{consumer_tag = CTag},
+         #amqp_msg{payload = <<42>>,
+                   props = #'P_basic'{}}} ->
+            ok
+    after ?TIMEOUT -> throw(timeout_waiting_for_deliver1)
+    end,
+
+    rabbit_ct_client_helpers:close_channel(Chan).
+
+%%
+%% Internal
+%%
+setup_local_source_shovel(Config, SourceQueue, DestQueue, AckMode) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    Shovel = [{test_shovel,
+               [{source,
+                 [{uris, [rabbit_misc:format("amqp://~ts:~b/%2f?heartbeat=5",
+                                             [Hostname, Port])]},
+                  {protocol, local},
+                  {queue, SourceQueue},
+                  {declarations,
+                   [{'queue.declare', [{queue, SourceQueue}, auto_delete]}]}
+                  ]
+                },
+                {destination,
+                 [{uris, [rabbit_misc:format("amqp://~ts:~b/%2f?heartbeat=5",
+                                             [Hostname, Port])]},
+                  {declarations,
+                   [{'queue.declare', [{queue, DestQueue}, auto_delete]}]},
+                  {publish_fields, [{exchange, <<>>},
+                                    {routing_key, DestQueue}]},
+                  {publish_properties, [{delivery_mode, 2},
+                                        {content_type,  ?SHOVELLED}]},
+                  {add_forward_headers, true},
+                  {add_timestamp_header, true}]},
+                {queue, <<>>},
+                {ack_mode, AckMode}
+               ]}],
+    ok = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, setup_shovel,
+                                      [Shovel]).
+
+setup_local_destination_shovel(Config, Queue, AckMode, Dest) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    Shovel = [{test_shovel,
+               [{source,
+                 [{uris, [rabbit_misc:format("amqp://~ts:~b/%2f?heartbeat=5",
+                                             [Hostname, Port])]},
+                  {declarations,
+                   [{'queue.declare', [exclusive, auto_delete]},
+                    {'exchange.declare', [{exchange, ?EXCHANGE}, auto_delete]},
+                    {'queue.bind', [{queue, <<>>}, {exchange, ?EXCHANGE},
+                                    {routing_key, ?TO_SHOVEL}]}]},
+                  {queue, <<>>}]},
+                {destination,
+                 [{protocol, local},
+                  {declarations,
+                   [{'queue.declare', [{queue, Queue}, auto_delete]}]},
+                  {uris, [rabbit_misc:format("amqp://~ts:~b",
+                                             [Hostname, Port])]},
+                  {dest_exchange, <<>>},
+                  {dest_routing_key, Queue}] ++ Dest
+                },
+                {ack_mode, AckMode}]}],
+    ok = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, setup_shovel,
+                                      [Shovel]).
+
+setup_local_server_named_shovel(Config, DestQueue, AckMode) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    Shovel = [{test_shovel,
+               [{source,
+                 [{uris, [rabbit_misc:format("amqp://~ts:~b/%2f?heartbeat=5",
+                                             [Hostname, Port])]},
+                  {protocol, local},
+                  {queue, <<>>},
+                  {declarations,
+                   ['queue.declare',
+                    {'queue.bind', [
+                                    {exchange, <<"amq.fanout">>},
+                                    {queue,    <<>>}
+                                   ]}]}
+                 ]
+                },
+                {destination,
+                 [{protocol, local},
+                  {declarations,
+                   [{'queue.declare', [{queue, DestQueue}, auto_delete]}]},
+                  {uris, [rabbit_misc:format("amqp://~ts:~b",
+                                             [Hostname, Port])]},
+                  {dest_exchange, <<>>},
+                  {dest_routing_key, DestQueue}]},
+                {ack_mode, AckMode}
+               ]}],
+    ok = rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, setup_shovel,
+                                      [Shovel]).
+
+setup_shovel(ShovelConfig) ->
+    _ = application:stop(rabbitmq_shovel),
+    application:set_env(rabbitmq_shovel, shovels, ShovelConfig, infinity),
+    ok = application:start(rabbitmq_shovel),
+    await_running_shovel(test_shovel).
+
+await_running_shovel(Name) ->
+    case [N || {N, _, {running, _}, _, _}
+                      <- rabbit_shovel_status:status(),
+                         N =:= Name] of
+        [_] -> ok;
+        _   -> timer:sleep(100),
+               await_running_shovel(Name)
+    end.
+
+consume(Chan, Queue, NoAck) ->
+    #'basic.consume_ok'{consumer_tag = CTag} =
+        amqp_channel:subscribe(Chan, #'basic.consume'{queue = Queue,
+                                                      no_ack = NoAck,
+                                                      exclusive = false},
+                               self()),
+    receive
+        #'basic.consume_ok'{consumer_tag = CTag} -> ok
+    after ?TIMEOUT -> throw(timeout_waiting_for_consume_ok)
+    end,
+    CTag.
+
+publish(Chan, Msg, Exchange, RoutingKey) ->
+    ok = amqp_channel:call(Chan, #'basic.publish'{exchange = Exchange,
+                                                  routing_key = RoutingKey},
+                           Msg).
+
+delete_queues(Qs) when is_list(Qs) ->
+    (catch lists:foreach(fun delete_testcase_queue/1, Qs)).
+
+delete_testcase_queue(Name) ->
+    QName = rabbit_misc:r(<<"/">>, queue, Name),
+    case rabbit_amqqueue:lookup(QName) of
+        {ok, Q} ->
+            {ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>);
+        _ ->
+            ok
+    end.
+
+attach_receiver(Config, TargetQ) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    {ok, Conn} = amqp10_client:open_connection(Hostname, Port),
+    {ok, Sess} = amqp10_client:begin_session(Conn),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Sess,
+                                                        <<"amqp-destination-receiver">>,
+                                                        TargetQ, settled, unsettled_state),
+    ok = amqp10_client:flow_link_credit(Receiver, 5, never),
+    {Conn, Receiver}.
+
+detach_receiver(Conn, Receiver) ->
+    amqp10_client:detach_link(Receiver),
+    amqp10_client:close_connection(Conn).
diff --git a/deps/rabbitmq_shovel/test/local_dynamic_SUITE.erl b/deps/rabbitmq_shovel/test/local_dynamic_SUITE.erl
new file mode 100644
index 000000000000..d6242d818574
--- /dev/null
+++ b/deps/rabbitmq_shovel/test/local_dynamic_SUITE.erl
@@ -0,0 +1,1199 @@
+%% This Source Code Form is subject to the terms of the Mozilla Public
+%% License, v. 2.0. If a copy of the MPL was not distributed with this
+%% file, You can obtain one at https://mozilla.org/MPL/2.0/.
+%%
+%% Copyright (c) 2007-2025 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
+%%
+
+-module(local_dynamic_SUITE).
+
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("rabbitmq_ct_helpers/include/rabbit_assert.hrl").
+
+-compile(export_all).
+
+-import(shovel_test_utils, [await_amqp10_event/3, await_credit/1]).
+
+-define(PARAM, <<"test">>).
+
+all() ->
+    [
+      {group, tests}
+    ].
+
+groups() ->
+    [
+     {tests, [], [
+                  local_to_local_opt_headers,
+                  local_to_local_queue_dest,
+                  local_to_local_original_dest,
+                  local_to_local_exchange_dest,
+                  local_to_local_missing_exchange_dest,
+                  local_to_local_predeclared_src,
+                  local_to_local_predeclared_quorum_src,
+                  local_to_local_predeclared_stream_first_offset_src,
+                  local_to_local_predeclared_stream_last_offset_src,
+                  local_to_local_missing_predeclared_src,
+                  local_to_local_exchange_src,
+                  local_to_local_queue_args_src,
+                  local_to_local_queue_args_dest,
+                  local_to_local_predeclared_dest,
+                  local_to_local_predeclared_quorum_dest,
+                  local_to_local_missing_predeclared_dest,
+                  local_to_local_queue_status,
+                  local_to_local_exchange_status,
+                  local_to_local_queue_and_exchange_src_fails,
+                  local_to_local_queue_and_exchange_dest_fails,
+                  local_to_local_delete_after_never,
+                  local_to_local_delete_after_queue_length,
+                  local_to_local_delete_after_queue_length_zero,
+                  local_to_local_delete_after_number,
+                  local_to_local_no_ack,
+                  local_to_local_quorum_no_ack,
+                  local_to_local_stream_no_ack,
+                  local_to_local_on_publish,
+                  local_to_local_quorum_on_publish,
+                  local_to_local_stream_on_publish,
+                  local_to_local_on_confirm,
+                  local_to_local_quorum_on_confirm,
+                  local_to_local_stream_on_confirm,
+                  local_to_local_reject_publish,
+                  local_to_amqp091,
+                  local_to_amqp10,
+                  amqp091_to_local,
+                  amqp10_to_local,
+                  local_to_local_delete_src_queue,
+                  local_to_local_delete_dest_queue,
+                  local_to_local_vhost_access,
+                  local_to_local_user_access,
+                  local_to_local_credit_flow_on_confirm,
+                  local_to_local_credit_flow_on_publish,
+                  local_to_local_credit_flow_no_ack,
+                  local_to_local_quorum_credit_flow_on_confirm,
+                  local_to_local_quorum_credit_flow_on_publish,
+                  local_to_local_quorum_credit_flow_no_ack,
+                  local_to_local_stream_credit_flow_on_confirm,
+                  local_to_local_stream_credit_flow_on_publish,
+                  local_to_local_stream_credit_flow_no_ack
+                 ]}
+    ].
+
+%% -------------------------------------------------------------------
+%% Testsuite setup/teardown.
+%% -------------------------------------------------------------------
+
+init_per_suite(Config0) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    Config1 = rabbit_ct_helpers:set_config(Config0, [
+        {rmq_nodename_suffix, ?MODULE},
+      {ignored_crashes, [
+          "server_initiated_close,404",
+          "writer,send_failed,closed",
+          "source_queue_down",
+          "dest_queue_down"
+        ]}
+      ]),
+    rabbit_ct_helpers:run_setup_steps(Config1,
+      rabbit_ct_broker_helpers:setup_steps() ++
+      rabbit_ct_client_helpers:setup_steps()).
+
+end_per_suite(Config) ->
+    application:stop(amqp10_client),
+    rabbit_ct_helpers:run_teardown_steps(Config,
+      rabbit_ct_client_helpers:teardown_steps() ++
+      rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_group(_, Config) ->
+    [Node] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+    ok = rabbit_ct_broker_helpers:enable_feature_flag(
+           Config, [Node], 'rabbitmq_4.0.0'),
+    Config.
+
+end_per_group(_, Config) ->
+    Config.
+
+init_per_testcase(Testcase, Config0) ->
+    SrcQ = list_to_binary(atom_to_list(Testcase) ++ "_src"),
+    DestQ = list_to_binary(atom_to_list(Testcase) ++ "_dest"),
+    DestQ2 = list_to_binary(atom_to_list(Testcase) ++ "_dest2"),
+    VHost = list_to_binary(atom_to_list(Testcase) ++ "_vhost"),
+    Config = [{srcq, SrcQ}, {destq, DestQ}, {destq2, DestQ2},
+              {alt_vhost, VHost} | Config0],
+
+    rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config) ->
+    shovel_test_utils:clear_param(Config, ?PARAM),
+    rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_all_queues, []),
+    _ = rabbit_ct_broker_helpers:delete_vhost(Config, ?config(alt_vhost, Config)),
+    rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+%% -------------------------------------------------------------------
+%% Testcases.
+%% -------------------------------------------------------------------
+
+local_to_local_opt_headers(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"dest-add-forward-headers">>, true},
+                                           {<<"dest-add-timestamp-header">>, true}
+                                          ]),
+              Msg = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>),
+              ?assertMatch(#{<<"x-opt-shovel-name">> := ?PARAM,
+                             <<"x-opt-shovel-type">> := <<"dynamic">>,
+                             <<"x-opt-shovelled-by">> := _,
+                             <<"x-opt-shovelled-timestamp">> := _},
+                           amqp10_msg:message_annotations(Msg))
+      end).
+
+local_to_local_queue_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_original_dest(Config) ->
+    %% Publish with the original routing keys, but use a different vhost
+    %% to avoid a loop (this is a single-node test).
+    Src = ?config(srcq, Config),
+    Dest = Src,
+    AltVHost = ?config(alt_vhost, Config),
+    ok = rabbit_ct_broker_helpers:add_vhost(Config, AltVHost),
+    ok = rabbit_ct_broker_helpers:set_full_permissions(Config, <<"guest">>, AltVHost),
+    declare_queue(Config, AltVHost, Dest),
+    with_session(
+      Config,
+      fun (Sess) ->
+              SrcUri = shovel_test_utils:make_uri(Config, 0, <<"%2F">>),
+              DestUri = shovel_test_utils:make_uri(Config, 0, AltVHost),
+              ok = rabbit_ct_broker_helpers:rpc(
+                     Config, 0, rabbit_runtime_parameters, set,
+                     [<<"/">>, <<"shovel">>, ?PARAM, [{<<"src-uri">>,  SrcUri},
+                                                      {<<"dest-uri">>, [DestUri]},
+                                                      {<<"src-protocol">>, <<"local">>},
+                                                      {<<"src-queue">>, Src},
+                                                      {<<"dest-protocol">>, <<"local">>}],
+                      none]),
+              shovel_test_utils:await_shovel(Config, 0, ?PARAM),
+              _ = publish(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end),
+    with_session(Config, AltVHost,
+                 fun (Sess) ->
+                         expect_one(Sess, Dest)
+                 end).
+
+local_to_local_exchange_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    AltExchange = <<"alt-exchange">>,
+    RoutingKey = <<"funky-routing-key">>,
+    declare_exchange(Config, <<"/">>, AltExchange),
+    declare_and_bind_queue(Config, <<"/">>, AltExchange, Dest, RoutingKey),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-exchange">>, AltExchange},
+                                           {<<"dest-exchange-key">>, RoutingKey}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_missing_exchange_dest(Config) ->
+    Src = ?config(srcq, Config),
+    AltExchange = <<"alt-exchange">>,
+    RoutingKey = <<"funky-routing-key">>,
+    %% If the destination exchange doesn't exist, it succeeds to start
+    %% the shovel. Just messages will not be routed
+    shovel_test_utils:set_param(Config, ?PARAM,
+                                [{<<"src-protocol">>, <<"local">>},
+                                 {<<"src-queue">>, Src},
+                                 {<<"dest-protocol">>, <<"local">>},
+                                 {<<"dest-exchange">>, AltExchange},
+                                 {<<"dest-exchange-key">>, RoutingKey}
+                                ]).
+
+local_to_local_predeclared_src(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_predeclared_quorum_src(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_predeclared_stream_first_offset_src(Config) ->
+    %% TODO test this in static
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+              publish_many(Sess, Src, Dest, <<"tag1">>, 20),
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-consumer-args">>,  #{<<"x-stream-offset">> => <<"first">>}},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              expect_many(Sess, Dest, 20),
+              expect_none(Sess, Dest),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_predeclared_stream_last_offset_src(Config) ->
+    %% TODO test this in static
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+              publish_many(Sess, Src, Dest, <<"tag1">>, 20),
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-consumer-args">>,  #{<<"x-stream-offset">> => <<"last">>}},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              %% Deliver last
+              expect_many(Sess, Dest, 1),
+              expect_none(Sess, Dest),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_missing_predeclared_src(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    shovel_test_utils:set_param_nowait(Config, ?PARAM,
+                                       [{<<"src-protocol">>, <<"local">>},
+                                        {<<"src-queue">>, Src},
+                                        {<<"src-predeclared">>, true},
+                                        {<<"dest-protocol">>, <<"local">>},
+                                        {<<"dest-queue">>, Dest}
+                                       ]),
+    shovel_test_utils:await_no_shovel(Config, ?PARAM),
+    %% The shovel parameter is only deleted when 'delete-after'
+    %% is used. In any other failure, the shovel should
+    %% remain and try to restart
+    ?assertNotMatch(
+       not_found,
+       rabbit_ct_broker_helpers:rpc(
+         Config, 0, rabbit_runtime_parameters, lookup,
+         [<<"/">>, <<"shovel">>, ?PARAM])).
+
+local_to_local_exchange_src(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-exchange">>, <<"amq.direct">>},
+                                           {<<"src-exchange-key">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              Target = <<"/exchange/amq.direct/", Src/binary>>,
+              _ = publish_expect(Sess, Target, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_queue_args_src(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    shovel_test_utils:set_param(Config, ?PARAM,
+                                [{<<"src-protocol">>, <<"local">>},
+                                 {<<"src-queue">>, Src},
+                                 {<<"src-queue-args">>, #{<<"x-queue-type">> => <<"quorum">>}},
+                                 {<<"dest-protocol">>, <<"local">>},
+                                 {<<"dest-queue">>, Dest}
+                                ]),
+    Expected = lists:sort([[Src, <<"quorum">>], [Dest, <<"classic">>]]),
+    ?assertMatch(Expected,
+                 lists:sort(rabbit_ct_broker_helpers:rabbitmqctl_list(
+                              Config, 0,
+                              ["list_queues", "name", "type", "--no-table-headers"]))).
+
+local_to_local_queue_args_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    shovel_test_utils:set_param(Config, ?PARAM,
+                                [{<<"src-protocol">>, <<"local">>},
+                                 {<<"src-queue">>, Src},
+                                 {<<"dest-protocol">>, <<"local">>},
+                                 {<<"dest-queue">>, Dest},
+                                 {<<"dest-queue-args">>, #{<<"x-queue-type">> => <<"quorum">>}}
+                                ]),
+    Expected = lists:sort([[Dest, <<"quorum">>], [Src, <<"classic">>]]),
+    ?assertMatch(Expected,
+                 lists:sort(rabbit_ct_broker_helpers:rabbitmqctl_list(
+                              Config, 0,
+                              ["list_queues", "name", "type", "--no-table-headers"]))).
+
+local_to_local_predeclared_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Dest),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_predeclared_quorum_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Dest, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_missing_predeclared_dest(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    shovel_test_utils:set_param_nowait(
+      Config, ?PARAM, [{<<"src-protocol">>, <<"local">>},
+                       {<<"src-queue">>, Src},
+                       {<<"dest-predeclared">>, true},
+                       {<<"dest-protocol">>, <<"local">>},
+                       {<<"dest-queue">>, Dest}
+                      ]),
+    shovel_test_utils:await_no_shovel(Config, ?PARAM),
+    %% The shovel parameter is only deleted when 'delete-after'
+    %% is used. In any other failure, the shovel should
+    %% remain and try to restart
+    ?assertNotMatch(
+       not_found,
+       rabbit_ct_broker_helpers:rpc(
+         Config, 0, rabbit_runtime_parameters, lookup,
+         [<<"/">>, <<"shovel">>, ?PARAM])).
+
+local_to_local_queue_status(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    shovel_test_utils:set_param(Config, ?PARAM,
+                                [{<<"src-protocol">>, <<"local">>},
+                                 {<<"src-queue">>, Src},
+                                 {<<"dest-protocol">>, <<"local">>},
+                                 {<<"dest-queue">>, Dest}
+                                ]),
+    Status = rabbit_ct_broker_helpers:rpc(Config, 0,
+                                          rabbit_shovel_status, status, []),
+    ?assertMatch([{_, dynamic, {running, _}, _, _}], Status),
+    [{_, dynamic, {running, Info}, _, _}] = Status,
+    ?assertMatch(<<"local">>, proplists:get_value(src_protocol, Info)),
+    ?assertMatch(<<"local">>, proplists:get_value(dest_protocol, Info)),
+    ?assertMatch(Src, proplists:get_value(src_queue, Info)),
+    ?assertMatch(Dest, proplists:get_value(dest_queue, Info)),
+    ok.
+
+local_to_local_exchange_status(Config) ->
+    DefExchange = <<"amq.direct">>,
+    RK1 = <<"carrots">>,
+    AltExchange = <<"amq.fanout">>,
+    RK2 = <<"bunnies">>,
+    shovel_test_utils:set_param(Config, ?PARAM,
+                                [{<<"src-protocol">>, <<"local">>},
+                                 {<<"src-exchange">>, DefExchange},
+                                 {<<"src-exchange-key">>, RK1},
+                                 {<<"dest-protocol">>, <<"local">>},
+                                 {<<"dest-exchange">>, AltExchange},
+                                 {<<"dest-exchange-key">>, RK2}
+                                ]),
+    Status = rabbit_ct_broker_helpers:rpc(Config, 0,
+                                          rabbit_shovel_status, status, []),
+    ?assertMatch([{_, dynamic, {running, _}, _, _}], Status),
+    [{_, dynamic, {running, Info}, _, _}] = Status,
+    ?assertMatch(<<"local">>, proplists:get_value(src_protocol, Info)),
+    ?assertMatch(<<"local">>, proplists:get_value(dest_protocol, Info)),
+    ?assertMatch(match, re:run(proplists:get_value(src_queue, Info),
+                               "amq\.gen.*", [{capture, none}])),
+    ?assertMatch(DefExchange, proplists:get_value(src_exchange, Info)),
+    ?assertMatch(RK1, proplists:get_value(src_exchange_key, Info)),
+    ?assertMatch(AltExchange, proplists:get_value(dest_exchange, Info)),
+    ?assertMatch(RK2, proplists:get_value(dest_exchange_key, Info)),
+    ok.
+
+local_to_local_queue_and_exchange_src_fails(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    %% Setting both queue and exchange for source fails
+    try
+        shovel_test_utils:set_param(Config, ?PARAM,
+                                    [{<<"src-protocol">>, <<"local">>},
+                                     {<<"src-queue">>, Src},
+                                     {<<"src-exchange">>, <<"amq.direct">>},
+                                     {<<"src-exchange-key">>, <<"bunnies">>},
+                                     {<<"dest-protocol">>, <<"local">>},
+                                     {<<"dest-queue">>, Dest}
+                                    ]),
+        throw(unexpected_success)
+    catch
+        _:{badmatch, {error_string, Reason}} ->
+            ?assertMatch(match, re:run(Reason, "Validation failed", [{capture, none}]))
+    end.
+
+local_to_local_queue_and_exchange_dest_fails(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    %% Setting both queue and exchange for dest fails
+    try
+        shovel_test_utils:set_param(Config, ?PARAM,
+                                    [{<<"src-protocol">>, <<"local">>},
+                                     {<<"src-queue">>, Src},
+                                     {<<"dest-protocol">>, <<"local">>},
+                                     {<<"dest-queue">>, Dest},
+                                     {<<"dest-exchange">>, <<"amq.direct">>},
+                                     {<<"dest-exchange-key">>, <<"bunnies">>}
+                                    ]),
+        throw(unexpected_success)
+    catch
+        _:{badmatch, {error_string, Reason}} ->
+            ?assertMatch(match, re:run(Reason, "Validation failed", [{capture, none}]))
+    end.
+
+local_to_local_delete_after_never(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 20),
+              expect_many(Sess, Dest, 20)
+      end).
+
+local_to_local_delete_after_queue_length_zero(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src),
+    shovel_test_utils:set_param_nowait(Config, ?PARAM,
+                                       [{<<"src-protocol">>, <<"local">>},
+                                        {<<"src-predeclared">>, true},
+                                        {<<"src-queue">>, Src},
+                                        {<<"src-delete-after">>, <<"queue-length">>},
+                                        {<<"dest-protocol">>, <<"local">>},
+                                        {<<"dest-queue">>, Dest}
+                                       ]),
+    shovel_test_utils:await_no_shovel(Config, ?PARAM),
+    %% The shovel parameter is only deleted when 'delete-after'
+    %% is used. In any other failure, the shovel should
+    %% remain and try to restart
+    ?assertMatch(not_found, rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_runtime_parameters, lookup, [<<"/">>, <<"shovel">>, ?PARAM])).
+
+local_to_local_delete_after_queue_length(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src),
+    with_session(Config,
+      fun (Sess) ->
+              publish_many(Sess, Src, Dest, <<"tag1">>, 18),
+              shovel_test_utils:set_param_nowait(Config, ?PARAM,
+                                                 [{<<"src-protocol">>, <<"local">>},
+                                                  {<<"src-predeclared">>, true},
+                                                  {<<"src-queue">>, Src},
+                                                  {<<"src-delete-after">>, <<"queue-length">>},
+                                                  {<<"dest-protocol">>, <<"local">>},
+                                                  {<<"dest-queue">>, Dest}
+                                                 ]),
+              %% The shovel parameter is only deleted when 'delete-after'
+              %% is used. In any other failure, the shovel should
+              %% remain and try to restart
+              expect_many(Sess, Dest, 18),
+              ?awaitMatch(not_found, rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_runtime_parameters, lookup, [<<"/">>, <<"shovel">>, ?PARAM]), 30_000),
+              ?awaitMatch([],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30_000),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 5),
+              expect_none(Sess, Dest)
+      end).
+
+local_to_local_delete_after_number(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+              publish_many(Sess, Src, Dest, <<"tag1">>, 5),
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-delete-after">>, 10},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              expect_many(Sess, Dest, 5),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 10),
+              expect_many(Sess, Dest, 5),
+              ?assertMatch(not_found, rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_runtime_parameters, lookup, [<<"/">>, <<"shovel">>, ?PARAM])),
+              expect_none(Sess, Dest)
+      end).
+
+local_to_local_no_ack(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"no-ack">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_quorum_no_ack(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"no-ack">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_stream_no_ack(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    declare_queue(Config, <<"/">>, Dest, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"no-ack">>}
+                                          ]),
+              Receiver = subscribe(Sess, Dest),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 10),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 10}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              _ = expect(Receiver, 10, []),
+              amqp10_client:detach_link(Receiver)
+      end).
+
+local_to_local_on_confirm(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-confirm">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_quorum_on_confirm(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-confirm">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_stream_on_confirm(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-confirm">>}
+                                          ]),
+              Receiver = subscribe(Sess, Dest),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 10),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 10}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              _ = expect(Receiver, 10, []),
+              amqp10_client:detach_link(Receiver)
+      end).
+
+local_to_local_on_publish(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-publish">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_quorum_on_publish(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-publish">>}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_stream_on_publish(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-publish">>}
+                                          ]),
+              Receiver = subscribe(Sess, Dest),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 10),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 10}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              _ = expect(Receiver, 10, []),
+              amqp10_client:detach_link(Receiver)
+      end).
+
+local_to_local_reject_publish(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    declare_queue(Config, <<"/">>, Dest, [{<<"x-max-length">>, long, 1},
+                                          {<<"x-overflow">>, longstr, <<"reject-publish">>}
+                                         ]),
+    with_session(
+      Config,
+      fun (Sess) ->
+              publish_many(Sess, Src, Dest, <<"tag1">>, 5),
+              shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, <<"on-confirm">>}
+                                          ]),
+              expect_many(Sess, Dest, 1),
+              expect_none(Sess, Dest)
+      end).
+
+local_to_amqp091(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"amqp091">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_amqp10(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"amqp10">>},
+                                           {<<"dest-address">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+amqp091_to_local(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"amqp091">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+amqp10_to_local(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"amqp10">>},
+                                           {<<"src-address">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>)
+      end).
+
+local_to_local_delete_src_queue(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 1}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queue,
+                                           [Src, <<"/">>]),
+              ?awaitMatch([{_Name, dynamic, {terminated,source_queue_down}, _, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000)
+      end).
+
+local_to_local_delete_dest_queue(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest}
+                                          ]),
+              _ = publish_expect(Sess, Src, Dest, <<"tag1">>, <<"hello">>),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 1}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_queue,
+                                           [Dest, <<"/">>]),
+              ?awaitMatch([{_Name, dynamic, {terminated, dest_queue_down}, _, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000)
+      end).
+
+local_to_local_vhost_access(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    AltVHost = ?config(alt_vhost, Config),
+    ok = rabbit_ct_broker_helpers:add_vhost(Config, AltVHost),
+    Uri = shovel_test_utils:make_uri(Config, 0, AltVHost),
+    ok = rabbit_ct_broker_helpers:rpc(
+           Config, 0, rabbit_runtime_parameters, set,
+           [<<"/">>, <<"shovel">>, ?PARAM, [{<<"src-uri">>,  Uri},
+                                            {<<"dest-uri">>, [Uri]},
+                                            {<<"src-protocol">>, <<"local">>},
+                                            {<<"src-queue">>, Src},
+                                            {<<"dest-protocol">>, <<"local">>},
+                                            {<<"dest-queue">>, Dest}],
+            none]),
+    shovel_test_utils:await_no_shovel(Config, ?PARAM).
+
+local_to_local_user_access(Config) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    Uri = shovel_test_utils:make_uri(
+            Config, 0, <<"guest">>, <<"forgotmypassword">>, <<"%2F">>),
+    ok = rabbit_ct_broker_helpers:rpc(
+           Config, 0, rabbit_runtime_parameters, set,
+           [<<"/">>, <<"shovel">>, ?PARAM, [{<<"src-uri">>,  Uri},
+                                            {<<"dest-uri">>, [Uri]},
+                                            {<<"src-protocol">>, <<"local">>},
+                                            {<<"src-queue">>, Src},
+                                            {<<"dest-protocol">>, <<"local">>},
+                                            {<<"dest-queue">>, Dest}],
+            none]),
+    shovel_test_utils:await_no_shovel(Config, ?PARAM).
+
+local_to_local_credit_flow_on_confirm(Config) ->
+    local_to_local_credit_flow(Config, <<"on-confirm">>).
+
+local_to_local_credit_flow_on_publish(Config) ->
+    local_to_local_credit_flow(Config, <<"on-publish">>).
+
+local_to_local_credit_flow_no_ack(Config) ->
+    local_to_local_credit_flow(Config, <<"no-ack">>).
+
+local_to_local_credit_flow(Config, AckMode) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"ack-mode">>, AckMode}
+                                          ]),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 1000),
+              expect_many(Sess, Dest, 1000)
+      end).
+
+local_to_local_quorum_credit_flow_on_confirm(Config) ->
+    local_to_local_quorum_credit_flow(Config, <<"on-confirm">>).
+
+local_to_local_quorum_credit_flow_on_publish(Config) ->
+    local_to_local_quorum_credit_flow(Config, <<"on-publish">>).
+
+local_to_local_quorum_credit_flow_no_ack(Config) ->
+    local_to_local_quorum_credit_flow(Config, <<"no-ack">>).
+
+local_to_local_quorum_credit_flow(Config, AckMode) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"quorum">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"ack-mode">>, AckMode}
+                                          ]),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 1000),
+              expect_many(Sess, Dest, 1000)
+      end).
+
+local_to_local_stream_credit_flow_on_confirm(Config) ->
+    local_to_local_stream_credit_flow(Config, <<"on-confirm">>).
+
+local_to_local_stream_credit_flow_on_publish(Config) ->
+    local_to_local_stream_credit_flow(Config, <<"on-publish">>).
+
+local_to_local_stream_credit_flow_no_ack(Config) ->
+    local_to_local_stream_credit_flow(Config, <<"no-ack">>).
+
+local_to_local_stream_credit_flow(Config, AckMode) ->
+    Src = ?config(srcq, Config),
+    Dest = ?config(destq, Config),
+    VHost = <<"/">>,
+    declare_queue(Config, VHost, Src, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    declare_queue(Config, VHost, Dest, [{<<"x-queue-type">>, longstr, <<"stream">>}]),
+    with_session(Config,
+      fun (Sess) ->
+             shovel_test_utils:set_param(Config, ?PARAM,
+                                          [{<<"src-protocol">>, <<"local">>},
+                                           {<<"src-queue">>, Src},
+                                           {<<"src-predeclared">>, true},
+                                           {<<"dest-protocol">>, <<"local">>},
+                                           {<<"dest-queue">>, Dest},
+                                           {<<"dest-predeclared">>, true},
+                                           {<<"ack-mode">>, AckMode}
+                                          ]),
+
+              Receiver = subscribe(Sess, Dest),
+              publish_many(Sess, Src, Dest, <<"tag1">>, 1000),
+              ?awaitMatch([{_Name, dynamic, {running, _}, #{forwarded := 1000}, _}],
+                          rabbit_ct_broker_helpers:rpc(Config, 0,
+                                                       rabbit_shovel_status, status, []),
+                          30000),
+              _ = expect(Receiver, 1000, []),
+              amqp10_client:detach_link(Receiver)
+      end).
+
+
+%%----------------------------------------------------------------------------
+with_session(Config, Fun) ->
+    with_session(Config, <<"/">>, Fun).
+
+with_session(Config, VHost, Fun) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    Cfg = #{address => Hostname,
+            port => Port,
+            sasl => {plain, <<"guest">>, <<"guest">>},
+            hostname => <<"vhost:", VHost/binary>>},
+    {ok, Conn} = amqp10_client:open_connection(Cfg),
+    {ok, Sess} = amqp10_client:begin_session(Conn),
+    Fun(Sess),
+    amqp10_client:close_connection(Conn),
+    ok.
+
+publish(Sender, Tag, Payload) when is_binary(Payload) ->
+    Headers = #{durable => true},
+    Msg = amqp10_msg:set_headers(Headers,
+                                 amqp10_msg:new(Tag, Payload, false)),
+    %% N.B.: this function does not attach a link and does not
+    %%       need to use await_credit/1
+    ok = amqp10_client:send_msg(Sender, Msg),
+    receive
+        {amqp10_disposition, {accepted, Tag}} -> ok
+    after 3000 ->
+              exit(publish_disposition_not_received)
+    end.
+
+publish(Session, Source, Dest, Tag, Payloads) ->
+    LinkName = <<"dynamic-sender-", Dest/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, LinkName, Source,
+                                                    unsettled, unsettled_state),
+    ok = await_amqp10_event(link, Sender, attached),
+    ok = await_credit(Sender),
+    case is_list(Payloads) of
+        true ->
+            [publish(Sender, Tag, Payload) || Payload <- Payloads];
+        false ->
+            publish(Sender, Tag, Payloads)
+    end,
+    amqp10_client:detach_link(Sender).
+
+publish_expect(Session, Source, Dest, Tag, Payload) ->
+    publish(Session, Source, Dest, Tag, Payload),
+    expect_one(Session, Dest).
+
+publish_many(Session, Source, Dest, Tag, N) ->
+    Payloads = [integer_to_binary(Payload) || Payload <- lists:seq(1, N)],
+    publish(Session, Source, Dest, Tag, Payloads).
+
+expect_one(Session, Dest) ->
+    LinkName = <<"dynamic-receiver-", Dest/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, LinkName,
+                                                        Dest, settled,
+                                                        unsettled_state),
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    Msg = expect(Receiver),
+    amqp10_client:detach_link(Receiver),
+    Msg.
+
+expect_none(Session, Dest) ->
+    LinkName = <<"dynamic-receiver-", Dest/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, LinkName,
+                                                        Dest, settled,
+                                                        unsettled_state),
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    receive
+        {amqp10_msg, Receiver, _} ->
+            throw(unexpected_msg)
+    after 4000 ->
+            ok
+    end,
+    amqp10_client:detach_link(Receiver).
+
+subscribe(Session, Dest) ->
+    LinkName = <<"dynamic-receiver-", Dest/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, LinkName,
+                                                        Dest, settled,
+                                                        unsettled_state),
+    ok = amqp10_client:flow_link_credit(Receiver, 10, 1),
+    Receiver.
+
+expect_many(Session, Dest, N) ->
+    LinkName = <<"dynamic-receiver-", Dest/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, LinkName,
+                                                        Dest, settled,
+                                                        unsettled_state),
+    ok = amqp10_client:flow_link_credit(Receiver, 10, 1),
+    Msgs = expect(Receiver, N, []),
+    amqp10_client:detach_link(Receiver),
+    Msgs.
+
+expect(_, 0, Acc) ->
+    Acc;
+expect(Receiver, N, Acc) ->
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            expect(Receiver, N - 1, [amqp10_msg:body(InMsg) | Acc])
+    after 4000 ->
+            throw({timeout_in_expect_waiting_for_delivery, N, Acc})
+    end.
+
+expect(Receiver) ->
+    receive
+        {amqp10_msg, Receiver, InMsg} ->
+            InMsg
+    after 4000 ->
+            throw(timeout_in_expect_waiting_for_delivery)
+    end.
+
+declare_queue(Config, VHost, QName) ->
+    declare_queue(Config, VHost, QName, []).
+
+declare_queue(Config, VHost, QName, Args) ->
+    Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost),
+    {ok, Ch} = amqp_connection:open_channel(Conn),
+    ?assertEqual(
+       {'queue.declare_ok', QName, 0, 0},
+       amqp_channel:call(
+         Ch, #'queue.declare'{queue = QName, durable = true, arguments = Args})),
+    rabbit_ct_client_helpers:close_channel(Ch),
+    rabbit_ct_client_helpers:close_connection(Conn).
+
+declare_and_bind_queue(Config, VHost, Exchange, QName, RoutingKey) ->
+    Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost),
+    {ok, Ch} = amqp_connection:open_channel(Conn),
+    ?assertEqual(
+       {'queue.declare_ok', QName, 0, 0},
+       amqp_channel:call(
+         Ch, #'queue.declare'{queue = QName, durable = true,
+                              arguments = [{<<"x-queue-type">>, longstr, <<"classic">>}]})),
+    ?assertMatch(
+       #'queue.bind_ok'{},
+       amqp_channel:call(Ch, #'queue.bind'{
+                                queue = QName,
+                                exchange = Exchange,
+                                routing_key = RoutingKey
+                               })),
+    rabbit_ct_client_helpers:close_channel(Ch),
+    rabbit_ct_client_helpers:close_connection(Conn).
+
+declare_exchange(Config, VHost, Exchange) ->
+    Conn = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0, VHost),
+    {ok, Ch} = amqp_connection:open_channel(Conn),
+    ?assertMatch(
+       #'exchange.declare_ok'{},
+       amqp_channel:call(Ch, #'exchange.declare'{exchange = Exchange})),
+    rabbit_ct_client_helpers:close_channel(Ch),
+    rabbit_ct_client_helpers:close_connection(Conn).
+
+delete_all_queues() ->
+    Queues = rabbit_amqqueue:list(),
+    lists:foreach(
+      fun(Q) ->
+              {ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>)
+      end, Queues).
+
+delete_queue(Name, VHost) ->
+    QName = rabbit_misc:r(VHost, queue, Name),
+    case rabbit_amqqueue:lookup(QName) of
+        {ok, Q} ->
+            {ok, _} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>);
+        _ ->
+            ok
+    end.
diff --git a/deps/rabbitmq_shovel/test/shovel_test_utils.erl b/deps/rabbitmq_shovel/test/shovel_test_utils.erl
index b3593c4d9984..e0e26b570725 100644
--- a/deps/rabbitmq_shovel/test/shovel_test_utils.erl
+++ b/deps/rabbitmq_shovel/test/shovel_test_utils.erl
@@ -13,7 +13,10 @@
          shovels_from_status/0, shovels_from_status/1,
          get_shovel_status/2, get_shovel_status/3,
          restart_shovel/2,
-         await/1, await/2, clear_param/2, clear_param/3, make_uri/2]).
+         await/1, await/2, await_amqp10_event/3, await_credit/1,
+         clear_param/2, clear_param/3, make_uri/2,
+         make_uri/3, make_uri/5,
+         await_shovel1/4, await_no_shovel/2]).
 
 make_uri(Config, Node) ->
     Hostname = ?config(rmq_hostname, Config),
@@ -21,6 +24,18 @@ make_uri(Config, Node) ->
     list_to_binary(lists:flatten(io_lib:format("amqp://~ts:~b",
                                                [Hostname, Port]))).
 
+make_uri(Config, Node, VHost) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_amqp),
+    list_to_binary(lists:flatten(io_lib:format("amqp://~ts:~b/~ts",
+                                               [Hostname, Port, VHost]))).
+
+make_uri(Config, Node, User, Password, VHost) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_amqp),
+    list_to_binary(lists:flatten(io_lib:format("amqp://~ts:~ts@~ts:~b/~ts",
+                                               [User, Password, Hostname, Port, VHost]))).
+
 set_param(Config, Name, Value) ->
     set_param_nowait(Config, 0, 0, Name, Value),
     await_shovel(Config, 0, Name).
@@ -53,13 +68,51 @@ await_shovel(Config, Node, Name, ExpectedState) ->
     rabbit_ct_broker_helpers:rpc(Config, Node,
       ?MODULE, await_shovel1, [Config, Name, ExpectedState]).
 
-await_shovel1(_Config, Name, ExpectedState) ->
+await_shovel1(Config, Name, ExpectedState) ->
+    await_shovel1(Config, Name, ExpectedState, 30_000).
+
+await_shovel1(_Config, Name, ExpectedState, Timeout) ->
     Ret = await(fun() ->
                   Status = shovels_from_status(ExpectedState),
                   lists:member(Name, Status)
-          end, 30_000),
+          end, Timeout),
     Ret.
 
+await_no_shovel(Config, Name) ->
+    try
+        rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, await_shovel1,
+                                     [Config, Name, running, 10_000]),
+        throw(unexpected_success)
+    catch
+        _:{exception, {await_timeout, false}, _} ->
+            ok
+    end.
+
+flush(Prefix) ->
+  receive
+    Msg ->
+      ct:log("~p flushed: ~p~n", [Prefix, Msg]),
+      flush(Prefix)
+  after 1 ->
+    ok
+  end.
+
+await_credit(Sender) ->
+  receive
+    {amqp10_event, {link, Sender, credited}} ->
+      ok
+  after 5_000 ->
+      flush("await_credit timed out"),
+      ct:fail(credited_timeout)
+  end.
+
+await_amqp10_event(On, Ref, Evt) ->
+    receive
+        {amqp10_event, {On, Ref, Evt}} -> ok
+    after 5_000 ->
+        exit({amqp10_event_timeout, On, Ref, Evt})
+    end.
+
 shovels_from_status() ->
     shovels_from_status(running).
 
diff --git a/deps/rabbitmq_shovel_management/priv/www/js/shovel.js b/deps/rabbitmq_shovel_management/priv/www/js/shovel.js
index ee46e8562ff2..496c7876eb00 100644
--- a/deps/rabbitmq_shovel_management/priv/www/js/shovel.js
+++ b/deps/rabbitmq_shovel_management/priv/www/js/shovel.js
@@ -42,17 +42,27 @@ dispatcher_add(function(sammy) {
             //remove fields not required by the selected protocol
             if (this.params['src-protocol'] == 'amqp10') {
                 remove_params_with(this, 'amqp091-src');
+                remove_params_with(this, 'local-src');
+            } else if (this.params['src-protocol'] == 'amqp091') {
+                remove_params_with(this, 'amqp10-src');
+                remove_params_with(this, 'local-src');
             } else {
                 remove_params_with(this, 'amqp10-src');
+                remove_params_with(this, 'amqp091-src');
             }
             if (this.params['dest-protocol'] == 'amqp10') {
                 remove_params_with(this, 'amqp091-dest');
+                remove_params_with(this, 'local-dest');
+            } else if (this.params['dest-protocol'] == 'amqp091'){
+                remove_params_with(this, 'amqp10-dest');
+                remove_params_with(this, 'local-dest');
             } else {
+                remove_params_with(this, 'amqp091-dest');
                 remove_params_with(this, 'amqp10-dest');
             }
 
             var trimProtoPrefix = function (x) {
-                if(x.startsWith('amqp10-') || x.startsWith('amqp091-')) {
+                if(x.startsWith('amqp10-') || x.startsWith('amqp091-') || x.startsWith('local-')) {
                     return x.substr(x.indexOf('-') + 1, x.length);
                 }
                 return x;
diff --git a/deps/rabbitmq_shovel_management/priv/www/js/tmpl/dynamic-shovels.ejs b/deps/rabbitmq_shovel_management/priv/www/js/tmpl/dynamic-shovels.ejs
index 979bd420bf6f..2e8e1fb81c31 100644
--- a/deps/rabbitmq_shovel_management/priv/www/js/tmpl/dynamic-shovels.ejs
+++ b/deps/rabbitmq_shovel_management/priv/www/js/tmpl/dynamic-shovels.ejs
@@ -95,6 +95,7 @@
             
             
+            
           
         
         
@@ -214,6 +264,7 @@
             
             
+            
           
         
         
diff --git a/deps/rabbitmq_shovel_management/test/http_SUITE.erl b/deps/rabbitmq_shovel_management/test/http_SUITE.erl
index 12bf2345d4da..28cee8bdcd4c 100644
--- a/deps/rabbitmq_shovel_management/test/http_SUITE.erl
+++ b/deps/rabbitmq_shovel_management/test/http_SUITE.erl
@@ -32,6 +32,7 @@ groups() ->
                   start_and_get_a_dynamic_amqp091_shovel_with_publish_properties,
                   start_and_get_a_dynamic_amqp091_shovel_with_missing_publish_properties,
                   start_and_get_a_dynamic_amqp091_shovel_with_empty_publish_properties,
+                  start_and_get_a_dynamic_local_shovel,
                   create_and_delete_a_dynamic_shovel_that_successfully_connects,
                   create_and_delete_a_dynamic_shovel_that_fails_to_connect
                  ]},
@@ -212,6 +213,20 @@ start_and_get_a_dynamic_amqp091_shovel_with_empty_publish_properties(Config) ->
 
     ok.
 
+start_and_get_a_dynamic_local_shovel(Config) ->
+    remove_all_dynamic_shovels(Config, <<"/">>),
+    Name = rabbit_data_coercion:to_binary(?FUNCTION_NAME),
+    ID = {<<"/">>, Name},
+    await_shovel_removed(Config, ID),
+
+    declare_local_shovel(Config, Name),
+    await_shovel_startup(Config, ID),
+    Sh = get_shovel(Config, Name),
+    ?assertEqual(Name, maps:get(name, Sh)),
+    delete_shovel(Config, Name),
+
+    ok.
+
 start_static_shovels(Config) ->
     http_put(Config, "/users/admin",
 	     #{password => <<"admin">>, tags => <<"administrator">>}, ?CREATED),
@@ -455,6 +470,22 @@ declare_amqp091_shovel_with_publish_properties(Config, Name, Props) ->
             }
         }, ?CREATED).
 
+declare_local_shovel(Config, Name) ->
+    Port = integer_to_binary(
+        rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp)),
+    http_put(Config, io_lib:format("/parameters/shovel/%2f/~ts", [Name]),
+        #{
+            value => #{
+                <<"src-protocol">> => <<"local">>,
+                <<"src-uri">> => <<"amqp://localhost:", Port/binary>>,
+                <<"src-queue">>  => <<"local.src.test">>,
+                <<"src-delete-after">> => <<"never">>,
+                <<"dest-protocol">> => <<"local">>,
+                <<"dest-uri">> => <<"amqp://localhost:", Port/binary>>,
+                <<"dest-queue">> => <<"local.dest.test">>
+            }
+        }, ?CREATED).
+
 await_shovel_startup(Config, Name) ->
     await_shovel_startup(Config, Name, 10_000).
 
@@ -480,4 +511,4 @@ does_shovel_exist(Config, Name) ->
     case lookup_shovel_status(Config, Name) of
         not_found -> false;
         _Found    -> true
-    end.
\ No newline at end of file
+    end.