From a25b48b25a8f1bcfa50058f6dd2fb99c5eb134ad Mon Sep 17 00:00:00 2001 From: David Ansari Date: Fri, 21 Jul 2023 10:29:07 +0000 Subject: [PATCH 01/16] Support AMQP 1.0 natively MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What Similar to Native MQTT in #5895, this commits implements Native AMQP 1.0. By "native", we mean do not proxy via AMQP 0.9.1 anymore. ## Why Native AMQP 1.0 comes with the following major benefits: 1. Similar to Native MQTT, this commit provides better throughput, latency, scalability, and resource usage for AMQP 1.0. See https://blog.rabbitmq.com/posts/2023/03/native-mqtt for native MQTT improvements. See further below for some benchmarks. 2. Since AMQP 1.0 is not limited anymore by the AMQP 0.9.1 protocol, this commit allows implementing more AMQP 1.0 features in the future. Some features are already implemented in this commit (see next section). 3. Simpler, better understandable, and more maintainable code. Native AMQP 1.0 as implemented in this commit has the following major benefits compared to AMQP 0.9.1: 4. Memory and disk alarms will only stop accepting incoming TRANSFER frames. New connections can still be created to consume from RabbitMQ to empty queues. 5. Due to 4. no need anymore for separate connections for publishers and consumers as we currently recommended for AMQP 0.9.1. which potentially halves the number of physical TCP connections. 6. When a single connection sends to multiple target queues, a single slow target queue won't block the entire connection. Publisher can still send data quickly to all other target queues. 7. A publisher can request whether it wants publisher confirmation on a per-message basis. In AMQP 0.9.1 publisher confirms are configured per channel only. 8. Consumers can change their "prefetch count" dynamically which isn't possible in our AMQP 0.9.1 implementation. See #10174 9. AMQP 1.0 is an extensible protocol This commit also fixes dozens of bugs present in the AMQP 1.0 plugin in RabbitMQ 3.x - most of which cannot be backported due to the complexity and limitations of the old 3.x implementation. This commit contains breaking changes and is therefore targeted for RabbitMQ 4.0. ## Implementation details 1. Breaking change: With Native AMQP, the behaviour of ``` Convert AMQP 0.9.1 message headers to application properties for an AMQP 1.0 consumer amqp1_0.convert_amqp091_headers_to_app_props = false | true (default false) Convert AMQP 1.0 Application Properties to AMQP 0.9.1 headers amqp1_0.convert_app_props_to_amqp091_headers = false | true (default false) ``` will break because we always convert according to the message container conversions. For example, AMQP 0.9.1 x-headers will go into message-annotations instead of application properties. Also, `false` won’t be respected since we always convert the headers with message containers. 2. Remove rabbit_queue_collector rabbit_queue_collector is responsible for synchronously deleting exclusive queues. Since the AMQP 1.0 plugin never creates exclusive queues, rabbit_queue_collector doesn't need to be started in the first place. This will save 1 Erlang process per AMQP 1.0 connection. 3. 7 processes per connection + 1 process per session in this commit instead of 7 processes per connection + 15 processes per session in 3.x Supervision hierarchy got re-designed. 4. Use 1 writer process per AMQP 1.0 connection AMQP 0.9.1 uses a separate rabbit_writer Erlang process per AMQP 0.9.1 channel. Prior to this commit, AMQP 1.0 used a separate rabbit_amqp1_0_writer process per AMQP 1.0 session. Advantage of single writer proc per session (prior to this commit): * High parallelism for serialising packets if multiple sessions within a connection write heavily at the same time. This commit uses a single writer process per AMQP 1.0 connection that is shared across all AMQP 1.0 sessions. Advantages of single writer proc per connection (this commit): * Lower memory usage with hundreds of thousands of AMQP 1.0 sessions * Less TCP and IP header overhead given that the single writer process can accumulate across all sessions bytes before flushing the socket. In other words, this commit decides that a reader / writer process pair per AMQP 1.0 connection is good enough for bi-directional TRANSFER flows. Having a writer per session is too heavy. We still ensure high throughput by having separate reader, writer, and session processes. 5. Transform rabbit_amqp1_0_writer into gen_server Why: Prior to this commit, when clicking on the AMQP 1.0 writer process in observer, the process crashed. Instead of handling all these debug messages of the sys module, it's better to implement a gen_server. There is no advantage of using a special OTP process over gen_server for the AMQP 1.0 writer. gen_server also provides cleaner format status output. How: Message callbacks return a timeout of 0. After all messages in the inbox are processed, the timeout message is handled by flushing any pending bytes. 6. Remove stats timer from writer AMQP 1.0 connections haven't emitted any stats previously. 7. When there are contiguous queue confirmations in the session process mailbox, batch them. When the confirmations are sent to the publisher, a single DISPOSITION frame is sent for contiguously confirmed delivery IDs. This approach should be good enough. However it's sub optimal in scenarios where contiguous delivery IDs that need confirmations are rare, for example: * There are multiple links in the session with different sender settlement modes and sender publishes across these links interleaved. * sender settlement mode is mixed and sender publishes interleaved settled and unsettled TRANSFERs. 8. Introduce credit API v2 Why: The AMQP 0.9.1 credit extension which is to be removed in 4.0 was poorly designed since basic.credit is a synchronous call into the queue process blocking the entire AMQP 1.0 session process. How: Change the interactions between queue clients and queue server implementations: * Clients only request a credit reply if the FLOW's `echo` field is set * Include all link flow control state held by the queue process into a new credit_reply queue event: * `available` after the queue sends any deliveries * `link-credit` after the queue sends any deliveries * `drain` which allows us to combine the old queue events send_credit_reply and send_drained into a single new queue event credit_reply. * Include the consumer tag into the credit_reply queue event such that the AMQP 1.0 session process can process any credit replies asynchronously. Link flow control state `delivery-count` also moves to the queue processes. The new interactions are hidden behind feature flag credit_api_v2 to allow for rolling upgrades from 3.13 to 4.0. 9. Use serial number arithmetic in quorum queues and session process. 10. Completely bypass the rabbit_limiter module for AMQP 1.0 flow control. The goal is to eventually remove the rabbit_limiter module in 4.0 since AMQP 0.9.1 global QoS will be unsupported in 4.0. This commit lifts the AMQP 1.0 link flow control logic out of rabbit_limiter into rabbit_queue_consumers. 11. Fix credit bug for streams: AMQP 1.0 settlements shouldn't top up link credit, only FLOW frames should top up link credit. 12. Allow sender settle mode unsettled for streams since AMQP 1.0 acknowledgements to streams are no-ops (currently). 13. Fix AMQP 1.0 client bugs Auto renewing credits should not be related to settling TRANSFERs. Remove field link_credit_unsettled as it was wrong and confusing. Prior to this commit auto renewal did not work when the sender uses sender settlement mode settled. 14. Fix AMQP 1.0 client bugs The wrong outdated Link was passed to function auto_flow/2 15. Use osiris chunk iterator Only hold messages of uncompressed sub batches in memory if consumer doesn't have sufficient credits. Compressed sub batches are skipped for non Stream protocol consumers. 16. Fix incoming link flow control Always use confirms between AMQP 1.0 queue clients and queue servers. As already done internally by rabbit_fifo_client and rabbit_stream_queue, use confirms for classic queues as well. 17. Include link handle into correlation when publishing messages to target queues such that session process can correlate confirms from target queues to incoming links. 18. Only grant more credits to publishers if publisher hasn't sufficient credits anymore and there are not too many unconfirmed messages on the link. 19. Completely ignore `block` and `unblock` queue actions and RabbitMQ credit flow between classic queue process and session process. 20. Link flow control is independent between links. A client can refer to a queue or to an exchange with multiple dynamically added target queues. Multiple incoming links can also fan in to the same queue. However the link topology looks like, this commit ensures that each link is only granted more credits if that link isn't overloaded. 21. A connection or a session can send to many different queues. In AMQP 0.9.1, a single slow queue will lead to the entire channel, and then entire connection being blocked. This commit makes sure that a single slow queue from one link won't slow down sending on other links. For example, having link A sending to a local classic queue and link B sending to 5 replica quorum queue, link B will naturally grant credits slower than link A. So, despite the quorum queue being slower in confirming messages, the same AMQP 1.0 connection and session can still pump data very fast into the classic queue. 22. If cluster wide memory or disk alarm occurs. Each session sends a FLOW with incoming-window to 0 to sending client. If sending clients don’t obey, force disconnect the client. If cluster wide memory alarm clears: Each session resumes with a FLOW defaulting to initial incoming-window. 23. All operations apart of publishing TRANSFERS to RabbitMQ can continue during cluster wide alarms, specifically, attaching consumers and consuming, i.e. emptying queues. There is no need for separate AMQP 1.0 connections for publishers and consumers as recommended in our AMQP 0.9.1 implementation. 24. Flow control summary: * If queue becomes bottleneck, that’s solved by slowing down individual sending links (AMQP 1.0 link flow control). * If session becomes bottleneck (more unlikely), that’s solved by AMQP 1.0 session flow control. * If connection becomes bottleneck, it naturally won’t read fast enough from the socket causing TCP backpressure being applied. Nowhere will RabbitMQ internal credit based flow control (i.e. module credit_flow) be used on the incoming AMQP 1.0 message path. 25. Register AMQP sessions Prefer local-only pg over our custom pg_local implementation as pg is a better process group implementation than pg_local. pg_local was identified as bottleneck in tests where many MQTT clients were disconnected at once. 26. Start a local-only pg when Rabbit boots: > A scope can be kept local-only by using a scope name that is unique cluster-wide, e.g. the node name: > pg:start_link(node()). Register AMQP 1.0 connections and sessions with pg. In future we should remove pg_local and instead use the new local-only pg for all registered processes such as AMQP 0.9.1 connections and channels. 27. Requeue messages if link detached Although the spec allows to settle delivery IDs on detached links, RabbitMQ does not respect the 'closed' field of the DETACH frame and therefore handles every DETACH frame as closed. Since the link is closed, we expect every outstanding delivery to be requeued. In addition to consumer cancellation, detaching a link therefore causes in flight deliveries to be requeued. Note that this behaviour is different from merely consumer cancellation in AMQP 0.9.1: "After a consumer is cancelled there will be no future deliveries dispatched to it. Note that there can still be "in flight" deliveries dispatched previously. Cancelling a consumer will neither discard nor requeue them." [https://www.rabbitmq.com/consumers.html#unsubscribing] An AMQP receiver can first drain, and then detach to prevent "in flight" deliveries 28. Init AMQP session with BEGIN frame Similar to how there can't be an MQTT processor without a CONNECT frame, there can't be an AMQP session without a BEGIN frame. This allows having strict dialyzer types for session flow control fields (i.e. not allowing 'undefined'). 29. Move serial_number to AMQP 1.0 common lib such that it can be used by both AMQP 1.0 server and client 30. Fix AMQP client to do serial number arithmetic. 31. AMQP client: Differentiate between delivery-id and transfer-id for better understandability. 32. Fix link flow control in classic queues This commit fixes ``` java -jar target/perf-test.jar -ad false -f persistent -u cq -c 3000 -C 1000000 -y 0 ``` followed by ``` ./omq -x 0 amqp -T /queue/cq -D 1000000 --amqp-consumer-credits 2 ``` Prior to this commit, (and on RabbitMQ 3.x) the consuming would halt after around 8 - 10,000 messages. The bug was that in flight messages from classic queue process to session process were not taken into account when topping up credit to the classic queue process. Fixes #2597 The solution to this bug (and a much cleaner design anyway independent of this bug) is that queues should hold all link flow control state including the delivery-count. Hence, when credit API v2 is used the delivery-count will be held by the classic queue process, quorum queue process, and stream queue client instead of managing the delivery-count in the session. 33. The double level crediting between (a) session process and rabbit_fifo_client, and (b) rabbit_fifo_client and rabbit_fifo was removed. Therefore, instead of managing 3 separate delivery-counts (i. session, ii. rabbit_fifo_client, iii. rabbit_fifo), only 1 delivery-count is used in rabbit_fifo. This is a big simplification. 34. This commit fixes quorum queues without bumping the machine version nor introducing new rabbit_fifo commands. Whether credit API v2 is used is solely determined at link attachment time depending on whether feature flag credit_api_v2 is enabled. Even when that feature flag will be enabled later on, this link will keep using credit API v1 until detached (or the node is shut down). Eventually, after feature flag credit_api_v2 has been enabled and a subsequent rolling upgrade, all links will use credit API v2. This approach is safe and simple. The 2 alternatives to move delivery-count from the session process to the queue processes would have been: i. Explicit feature flag credit_api_v2 migration function * Can use a gen_server:call and only finish migration once all delivery-counts were migrated. Cons: * Extra new message format just for migration is required. * Risky as migration will fail if a target queue doesn’t reply. ii. Session always includes DeliveryCountSnd when crediting to the queue: Cons: * 2 delivery counts will be hold simultaneously in session proc and queue proc; could be solved by deleting the session proc’s delivery-count for credit-reply * What happens if the receiver doesn’t provide credit for a very long time? Is that a problem? 35. Support stream filtering in AMQP 1.0 (by @acogoluegnes) Use the x-stream-filter-value message annotation to carry the filter value in a published message. Use the rabbitmq:stream-filter and rabbitmq:stream-match-unfiltered filters when creating a receiver that wants to filter out messages from a stream. 36. Remove credit extension from AMQP 0.9.1 client 37. Support maintenance mode closing AMQP 1.0 connections. 38. Remove AMQP 0.9.1 client dependency from AMQP 1.0 implementation. 39. Move AMQP 1.0 plugin to the core. AMQP 1.0 is enabled by default. The old rabbitmq_amqp1_0 plugin will be kept as a no-op plugin to prevent deployment tools from failing that execute: ``` rabbitmq-plugins enable rabbitmq_amqp1_0 rabbitmq-plugins disable rabbitmq_amqp1_0 ``` 40. Breaking change: Remove CLI command `rabbitmqctl list_amqp10_connections`. Instead, list both AMQP 0.9.1 and AMQP 1.0 connections in `list_connections`: ``` rabbitmqctl list_connections protocol Listing connections ... protocol {1, 0} {0,9,1} ``` ## Benchmarks ### Throughput & Latency Setup: * Single node Ubuntu 22.04 * Erlang 26.1.1 Start RabbitMQ: ``` make run-broker PLUGINS="rabbitmq_management rabbitmq_amqp1_0" FULL=1 RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS="+S 3" ``` Predeclare durable classic queue cq1, durable quorum queue qq1, durable stream queue sq1. Start client: https://github.com/ssorj/quiver https://hub.docker.com/r/ssorj/quiver/tags (digest 453a2aceda64) ``` docker run -it --rm --add-host host.docker.internal:host-gateway ssorj/quiver:latest bash-5.1# quiver --version quiver 0.4.0-SNAPSHOT ``` 1. Classic queue ``` quiver //host.docker.internal//amq/queue/cq1 --durable --count 1m --duration 10m --body-size 12 --credit 1000 ``` This commit: ``` Count ............................................. 1,000,000 messages Duration ............................................... 73.8 seconds Sender rate .......................................... 13,548 messages/s Receiver rate ........................................ 13,547 messages/s End-to-end rate ...................................... 13,547 messages/s Latencies by percentile: 0% ........ 0 ms 90.00% ........ 9 ms 25% ........ 2 ms 99.00% ....... 14 ms 50% ........ 4 ms 99.90% ....... 17 ms 100% ....... 26 ms 99.99% ....... 24 ms ``` RabbitMQ 3.x (main branch as of 30 January 2024): ``` ---------------------- Sender ----------------------- --------------------- Receiver ---------------------- -------- Time [s] Count [m] Rate [m/s] CPU [%] RSS [M] Time [s] Count [m] Rate [m/s] CPU [%] RSS [M] Lat [ms] ----------------------------------------------------- ----------------------------------------------------- -------- 2.1 130,814 65,342 6 73.6 2.1 3,217 1,607 0 8.0 511 4.1 163,580 16,367 2 74.1 4.1 3,217 0 0 8.0 0 6.1 229,114 32,767 3 74.1 6.1 3,217 0 0 8.0 0 8.1 261,880 16,367 2 74.1 8.1 67,874 32,296 8 8.2 7,662 10.1 294,646 16,367 2 74.1 10.1 67,874 0 0 8.2 0 12.1 360,180 32,734 3 74.1 12.1 67,874 0 0 8.2 0 14.1 392,946 16,367 3 74.1 14.1 68,604 365 0 8.2 12,147 16.1 458,480 32,734 3 74.1 16.1 68,604 0 0 8.2 0 18.1 491,246 16,367 2 74.1 18.1 68,604 0 0 8.2 0 20.1 556,780 32,767 4 74.1 20.1 68,604 0 0 8.2 0 22.1 589,546 16,375 2 74.1 22.1 68,604 0 0 8.2 0 receiver timed out 24.1 622,312 16,367 2 74.1 24.1 68,604 0 0 8.2 0 quiver: error: PlanoProcessError: Command 'quiver-arrow receive //host.docker.internal//amq/queue/cq1 --impl qpid-proton-c --duration 10m --count 1m --rate 0 --body-size 12 --credit 1000 --transaction-size 0 --timeout 10 --durable --output /tmp/quiver-otujr23y' returned non-zero exit status 1. Traceback (most recent call last): File "/usr/local/lib/quiver/python/quiver/pair.py", line 144, in run _plano.wait(receiver, check=True) File "/usr/local/lib/quiver/python/plano/main.py", line 1243, in wait raise PlanoProcessError(proc) plano.main.PlanoProcessError: Command 'quiver-arrow receive //host.docker.internal//amq/queue/cq1 --impl qpid-proton-c --duration 10m --count 1m --rate 0 --body-size 12 --credit 1000 --transaction-size 0 --timeout 10 --durable --output /tmp/quiver-otujr23y' returned non-zero exit status 1. ``` 2. Quorum queue: ``` quiver //host.docker.internal//amq/queue/qq1 --durable --count 1m --duration 10m --body-size 12 --credit 1000 ``` This commit: ``` Count ............................................. 1,000,000 messages Duration .............................................. 101.4 seconds Sender rate ........................................... 9,867 messages/s Receiver rate ......................................... 9,868 messages/s End-to-end rate ....................................... 9,865 messages/s Latencies by percentile: 0% ....... 11 ms 90.00% ....... 23 ms 25% ....... 15 ms 99.00% ....... 28 ms 50% ....... 18 ms 99.90% ....... 33 ms 100% ....... 49 ms 99.99% ....... 47 ms ``` RabbitMQ 3.x: ``` ---------------------- Sender ----------------------- --------------------- Receiver ---------------------- -------- Time [s] Count [m] Rate [m/s] CPU [%] RSS [M] Time [s] Count [m] Rate [m/s] CPU [%] RSS [M] Lat [ms] ----------------------------------------------------- ----------------------------------------------------- -------- 2.1 130,814 65,342 9 69.9 2.1 18,430 9,206 5 7.6 1,221 4.1 163,580 16,375 5 70.2 4.1 18,867 218 0 7.6 2,168 6.1 229,114 32,767 6 70.2 6.1 18,867 0 0 7.6 0 8.1 294,648 32,734 7 70.2 8.1 18,867 0 0 7.6 0 10.1 360,182 32,734 6 70.2 10.1 18,867 0 0 7.6 0 12.1 425,716 32,767 6 70.2 12.1 18,867 0 0 7.6 0 receiver timed out 14.1 458,482 16,367 5 70.2 14.1 18,867 0 0 7.6 0 quiver: error: PlanoProcessError: Command 'quiver-arrow receive //host.docker.internal//amq/queue/qq1 --impl qpid-proton-c --duration 10m --count 1m --rate 0 --body-size 12 --credit 1000 --transaction-size 0 --timeout 10 --durable --output /tmp/quiver-b1gcup43' returned non-zero exit status 1. Traceback (most recent call last): File "/usr/local/lib/quiver/python/quiver/pair.py", line 144, in run _plano.wait(receiver, check=True) File "/usr/local/lib/quiver/python/plano/main.py", line 1243, in wait raise PlanoProcessError(proc) plano.main.PlanoProcessError: Command 'quiver-arrow receive //host.docker.internal//amq/queue/qq1 --impl qpid-proton-c --duration 10m --count 1m --rate 0 --body-size 12 --credit 1000 --transaction-size 0 --timeout 10 --durable --output /tmp/quiver-b1gcup43' returned non-zero exit status 1. ``` 3. Stream: ``` quiver-arrow send //host.docker.internal//amq/queue/sq1 --durable --count 1m -d 10m --summary --verbose ``` This commit: ``` Count ............................................. 1,000,000 messages Duration ................................................ 8.7 seconds Message rate ........................................ 115,154 messages/s ``` RabbitMQ 3.x: ``` Count ............................................. 1,000,000 messages Duration ............................................... 21.2 seconds Message rate ......................................... 47,232 messages/s ``` ### Memory usage Start RabbitMQ: ``` ERL_MAX_PORTS=3000000 RABBITMQ_SERVER_ADDITIONAL_ERL_ARGS="+P 3000000 +S 6" make run-broker PLUGINS="rabbitmq_amqp1_0" FULL=1 RABBITMQ_CONFIG_FILE="rabbitmq.conf" ``` ``` /bin/cat rabbitmq.conf tcp_listen_options.sndbuf = 2048 tcp_listen_options.recbuf = 2048 vm_memory_high_watermark.relative = 0.95 vm_memory_high_watermark_paging_ratio = 0.95 loopback_users = none ``` Create 50k connections with 2 sessions per connection, i.e. 100k session in total: ```go package main import ( "context" "log" "time" "github.com/Azure/go-amqp" ) func main() { for i := 0; i < 50000; i++ { conn, err := amqp.Dial(context.TODO(), "amqp://nuc", &amqp.ConnOptions{SASLType: amqp.SASLTypeAnonymous()}) if err != nil { log.Fatal("dialing AMQP server:", err) } _, err = conn.NewSession(context.TODO(), nil) if err != nil { log.Fatal("creating AMQP session:", err) } _, err = conn.NewSession(context.TODO(), nil) if err != nil { log.Fatal("creating AMQP session:", err) } } log.Println("opened all connections") time.Sleep(5 * time.Hour) } ``` This commit: ``` erlang:memory(). [{total,4586376480}, {processes,4025898504}, {processes_used,4025871040}, {system,560477976}, {atom,1048841}, {atom_used,1042841}, {binary,233228608}, {code,21449982}, {ets,108560464}] erlang:system_info(process_count). 450289 ``` 7 procs per connection + 1 proc per session. (7 + 2*1) * 50,000 = 450,000 procs RabbitMQ 3.x: ``` erlang:memory(). [{total,15168232704}, {processes,14044779256}, {processes_used,14044755120}, {system,1123453448}, {atom,1057033}, {atom_used,1052587}, {binary,236381264}, {code,21790238}, {ets,391423744}] erlang:system_info(process_count). 1850309 ``` 7 procs per connection + 15 per session (7 + 2*15) * 50,000 = 1,850,000 procs 50k connections + 100k session require with this commit: 4.5 GB in RabbitMQ 3.x: 15 GB ## Future work 1. More efficient parser and serializer 2. TODO in mc_amqp: Do not store the parsed message on disk. 3. Implement both AMQP HTTP extension and AMQP management extension to allow AMQP clients to create RabbitMQ objects (queues, exchanges, ...). --- Makefile | 2 - deps/amqp10_client/BUILD.bazel | 1 - deps/amqp10_client/Makefile | 16 +- deps/amqp10_client/README.md | 14 +- deps/amqp10_client/app.bzl | 3 - deps/amqp10_client/src/amqp10_client.erl | 45 +- deps/amqp10_client/src/amqp10_client.hrl | 2 - deps/amqp10_client/src/amqp10_client_app.erl | 20 +- .../src/amqp10_client_connection.erl | 82 +- .../src/amqp10_client_connection_sup.erl | 46 +- .../src/amqp10_client_connections_sup.erl | 38 - .../src/amqp10_client_session.erl | 742 ++-- .../src/amqp10_client_sessions_sup.erl | 29 +- deps/amqp10_client/src/amqp10_client_sup.erl | 26 +- .../amqp10_client/src/amqp10_client_types.erl | 17 +- deps/amqp10_client/src/amqp10_msg.erl | 50 +- deps/amqp10_client/test/system_SUITE.erl | 126 +- deps/amqp10_common/BUILD.bazel | 5 + deps/amqp10_common/app.bzl | 13 +- deps/amqp10_common/include/amqp10_types.hrl | 12 + .../src/amqp10_binary_parser.erl | 7 +- deps/amqp10_common/src/serial_number.erl | 118 + .../test/serial_number_SUITE.erl | 124 + deps/amqp_client/src/amqp_channel.erl | 5 - .../src/amqp_selective_consumer.erl | 8 +- deps/amqp_client/src/rabbit_routing_util.erl | 71 +- deps/amqp_client/test/system_SUITE.erl | 6 +- deps/amqp_client/test/unit_SUITE.erl | 4 +- deps/rabbit/.gitignore | 3 + deps/rabbit/BUILD.bazel | 55 + deps/rabbit/Makefile | 6 +- deps/rabbit/app.bzl | 77 +- deps/rabbit/docs/rabbitmq.conf.example | 4 - deps/rabbit/include/rabbit_amqp.hrl | 74 + .../rabbit/include/rabbit_global_counters.hrl | 1 - deps/rabbit/priv/schema/rabbit.schema | 27 + deps/rabbit/src/mc.erl | 10 + deps/rabbit/src/mc_amqp.erl | 60 +- deps/rabbit/src/mc_amqpl.erl | 3 +- deps/rabbit/src/mc_compat.erl | 11 + deps/rabbit/src/rabbit.erl | 27 +- deps/rabbit/src/rabbit_access_control.erl | 28 +- deps/rabbit/src/rabbit_alarm.erl | 2 +- .../src/rabbit_amqp1_0.erl | 33 +- deps/rabbit/src/rabbit_amqp_reader.erl | 1042 +++++ deps/rabbit/src/rabbit_amqp_session.erl | 2325 +++++++++++ deps/rabbit/src/rabbit_amqp_session_sup.erl | 39 + deps/rabbit/src/rabbit_amqp_util.erl | 19 + deps/rabbit/src/rabbit_amqp_writer.erl | 218 + deps/rabbit/src/rabbit_amqqueue.erl | 25 +- deps/rabbit/src/rabbit_amqqueue_process.erl | 139 +- deps/rabbit/src/rabbit_autoheal.erl | 2 +- deps/rabbit/src/rabbit_channel.erl | 201 +- deps/rabbit/src/rabbit_classic_queue.erl | 94 +- deps/rabbit/src/rabbit_confirms.erl | 40 +- .../src/rabbit_connection_helper_sup.erl | 16 +- deps/rabbit/src/rabbit_connection_sup.erl | 46 +- deps/rabbit/src/rabbit_core_ff.erl | 8 + deps/rabbit/src/rabbit_fifo.erl | 210 +- deps/rabbit/src/rabbit_fifo.hrl | 24 +- deps/rabbit/src/rabbit_fifo_client.erl | 158 +- deps/rabbit/src/rabbit_global_counters.erl | 17 +- deps/rabbit/src/rabbit_limiter.erl | 61 +- deps/rabbit/src/rabbit_networking.erl | 35 +- deps/rabbit/src/rabbit_queue_consumers.erl | 275 +- deps/rabbit/src/rabbit_queue_type.erl | 85 +- deps/rabbit/src/rabbit_quorum_queue.erl | 88 +- deps/rabbit/src/rabbit_reader.erl | 197 +- deps/rabbit/src/rabbit_stream_queue.erl | 459 ++- deps/rabbit/src/rabbit_trace.erl | 75 +- deps/rabbit/test/amqp_auth_SUITE.erl | 621 +++ deps/rabbit/test/amqp_client_SUITE.erl | 3654 +++++++++++++++++ deps/rabbit/test/amqp_credit_api_v2_SUITE.erl | 221 + .../test/amqp_proxy_protocol_SUITE.erl} | 112 +- .../test/amqp_system_SUITE.erl} | 115 +- .../console/Program.cs | 0 .../amqp_system_SUITE_data}/console/README.md | 0 .../console/standalone.csproj | 0 .../fsharp-tests/Program.fs | 19 +- .../fsharp-tests/fsharp-tests.fsproj | 4 +- .../fsharp-tests/global.json | 0 .../java-tests/.gitignore | 0 .../.mvn/wrapper/MavenWrapperDownloader.java | 0 .../java-tests/.mvn/wrapper/maven-wrapper.jar | Bin .../.mvn/wrapper/maven-wrapper.properties | 0 .../amqp_system_SUITE_data}/java-tests/mvnw | 0 .../java-tests/mvnw.cmd | 0 .../java-tests/pom.xml | 0 .../amqp1_0/tests/jms/RoundTripTest.java | 0 .../src/test/resources/logback-test.xml | 0 deps/rabbit/test/event_recorder.erl | 71 + deps/rabbit/test/message_size_limit_SUITE.erl | 34 +- deps/rabbit/test/rabbit_confirms_SUITE.erl | 11 +- deps/rabbit/test/rabbit_fifo_SUITE.erl | 177 +- deps/rabbit/test/rabbit_fifo_int_SUITE.erl | 97 +- .../rabbit/test/rabbit_stream_queue_SUITE.erl | 76 +- .../test/single_active_consumer_SUITE.erl | 51 +- .../rabbit/test/unit_access_control_SUITE.erl | 42 +- deps/rabbit_common/app.bzl | 3 + deps/rabbit_common/include/rabbit.hrl | 4 +- .../rabbit_common/src/rabbit_core_metrics.erl | 16 +- .../src/rabbit_data_coercion.erl | 4 +- deps/rabbit_common/src/rabbit_net.erl | 4 +- .../src/rabbit_routing_parser.erl | 76 + deps/rabbit_common/src/rabbit_types.erl | 2 + deps/rabbit_common/src/rabbit_writer.erl | 5 +- deps/rabbitmq_amqp1_0/.gitignore | 20 +- deps/rabbitmq_amqp1_0/BUILD.bazel | 107 +- deps/rabbitmq_amqp1_0/CODE_OF_CONDUCT.md | 1 - deps/rabbitmq_amqp1_0/CONTRIBUTING.md | 1 - deps/rabbitmq_amqp1_0/LICENSE | 3 - deps/rabbitmq_amqp1_0/LICENSE-MPL-RabbitMQ | 373 -- deps/rabbitmq_amqp1_0/Makefile | 38 +- deps/rabbitmq_amqp1_0/README.md | 240 +- deps/rabbitmq_amqp1_0/app.bzl | 165 +- .../include/rabbit_amqp1_0.hrl | 52 - .../priv/schema/rabbitmq_amqp1_0.schema | 37 - ....Commands.ListAmqp10ConnectionsCommand.erl | 80 - .../src/rabbit_amqp1_0_channel.erl | 61 - .../src/rabbit_amqp1_0_incoming_link.erl | 246 -- .../src/rabbit_amqp1_0_link_util.erl | 64 - .../src/rabbit_amqp1_0_message.erl | 371 -- .../src/rabbit_amqp1_0_outgoing_link.erl | 278 -- .../src/rabbit_amqp1_0_reader.erl | 815 ---- .../src/rabbit_amqp1_0_session.erl | 420 -- .../src/rabbit_amqp1_0_session_process.erl | 467 --- .../src/rabbit_amqp1_0_session_sup.erl | 104 - .../src/rabbit_amqp1_0_session_sup_sup.erl | 38 - .../src/rabbit_amqp1_0_util.erl | 73 - .../src/rabbit_amqp1_0_writer.erl | 258 -- .../src/rabbitmq_amqp1_0_noop.erl | 1 + .../test/amqp10_client_SUITE.erl | 624 --- deps/rabbitmq_amqp1_0/test/command_SUITE.erl | 163 - .../test/config_schema_SUITE.erl | 54 - .../rabbitmq_amqp1_0.snippets | 17 - deps/rabbitmq_amqp1_0/test/unit_SUITE.erl | 39 - deps/rabbitmq_cli/test/test_helper.exs | 6 +- .../src/rabbit_ct_broker_helpers.erl | 21 +- .../test/rabbit_federation_test_util.erl | 2 +- deps/rabbitmq_management/BUILD.bazel | 3 + .../test/authnz-msg-protocols/enabled_plugins | 2 +- .../selenium/test/basic-auth/enabled_plugins | 2 +- .../selenium/test/mgt-only/enabled_plugins | 2 +- .../selenium/test/oauth/enabled_plugins | 2 +- .../test/rabbit_mgmt_http_SUITE.erl | 182 +- .../src/rabbit_mgmt_format.erl | 2 + deps/rabbitmq_mqtt/BUILD.bazel | 1 - deps/rabbitmq_mqtt/Makefile | 2 +- deps/rabbitmq_mqtt/app.bzl | 2 +- .../src/rabbit_mqtt_confirms.erl | 8 +- .../src/rabbit_mqtt_processor.erl | 13 +- .../src/rabbit_mqtt_qos0_queue.erl | 8 +- deps/rabbitmq_mqtt/test/auth_SUITE.erl | 6 +- .../test/protocol_interop_SUITE.erl | 51 +- deps/rabbitmq_mqtt/test/reader_SUITE.erl | 19 +- deps/rabbitmq_mqtt/test/shared_SUITE.erl | 6 +- deps/rabbitmq_shovel/BUILD.bazel | 1 - deps/rabbitmq_shovel/Makefile | 2 +- .../src/rabbit_amqp10_shovel.erl | 21 +- deps/rabbitmq_shovel_management/BUILD.bazel | 1 - deps/rabbitmq_shovel_management/Makefile | 2 +- .../src/rabbit_stomp_client_sup.erl | 6 +- .../src/rabbit_stomp_processor.erl | 10 +- deps/rabbitmq_stream/BUILD.bazel | 11 +- deps/rabbitmq_stream/Makefile | 2 +- deps/rabbitmq_stream/app.bzl | 9 + .../src/rabbit_stream_reader.hrl | 2 + .../test/protocol_interop_SUITE.erl | 412 ++ deps/rabbitmq_stream_management/BUILD.bazel | 1 - moduleindex.yaml | 24 +- plugins.mk | 1 + 171 files changed, 12073 insertions(+), 7352 deletions(-) delete mode 100644 deps/amqp10_client/src/amqp10_client_connections_sup.erl create mode 100644 deps/amqp10_common/include/amqp10_types.hrl create mode 100644 deps/amqp10_common/src/serial_number.erl create mode 100644 deps/amqp10_common/test/serial_number_SUITE.erl create mode 100644 deps/rabbit/include/rabbit_amqp.hrl rename deps/{rabbitmq_amqp1_0 => rabbit}/src/rabbit_amqp1_0.erl (60%) create mode 100644 deps/rabbit/src/rabbit_amqp_reader.erl create mode 100644 deps/rabbit/src/rabbit_amqp_session.erl create mode 100644 deps/rabbit/src/rabbit_amqp_session_sup.erl create mode 100644 deps/rabbit/src/rabbit_amqp_util.erl create mode 100644 deps/rabbit/src/rabbit_amqp_writer.erl create mode 100644 deps/rabbit/test/amqp_auth_SUITE.erl create mode 100644 deps/rabbit/test/amqp_client_SUITE.erl create mode 100644 deps/rabbit/test/amqp_credit_api_v2_SUITE.erl rename deps/{rabbitmq_amqp1_0/test/proxy_protocol_SUITE.erl => rabbit/test/amqp_proxy_protocol_SUITE.erl} (63%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE.erl => rabbit/test/amqp_system_SUITE.erl} (71%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/console/Program.cs (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/console/README.md (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/console/standalone.csproj (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/fsharp-tests/Program.fs (96%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/fsharp-tests/fsharp-tests.fsproj (83%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/fsharp-tests/global.json (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/.gitignore (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/.mvn/wrapper/MavenWrapperDownloader.java (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/.mvn/wrapper/maven-wrapper.jar (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/.mvn/wrapper/maven-wrapper.properties (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/mvnw (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/mvnw.cmd (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/pom.xml (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/src/test/java/com/rabbitmq/amqp1_0/tests/jms/RoundTripTest.java (100%) rename deps/{rabbitmq_amqp1_0/test/system_SUITE_data => rabbit/test/amqp_system_SUITE_data}/java-tests/src/test/resources/logback-test.xml (100%) create mode 100644 deps/rabbit/test/event_recorder.erl create mode 100644 deps/rabbit_common/src/rabbit_routing_parser.erl delete mode 120000 deps/rabbitmq_amqp1_0/CODE_OF_CONDUCT.md delete mode 120000 deps/rabbitmq_amqp1_0/CONTRIBUTING.md delete mode 100644 deps/rabbitmq_amqp1_0/LICENSE delete mode 100644 deps/rabbitmq_amqp1_0/LICENSE-MPL-RabbitMQ delete mode 100644 deps/rabbitmq_amqp1_0/include/rabbit_amqp1_0.hrl delete mode 100644 deps/rabbitmq_amqp1_0/priv/schema/rabbitmq_amqp1_0.schema delete mode 100644 deps/rabbitmq_amqp1_0/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_channel.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_incoming_link.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_link_util.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_message.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_outgoing_link.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_process.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup_sup.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_util.erl delete mode 100644 deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_writer.erl create mode 100644 deps/rabbitmq_amqp1_0/src/rabbitmq_amqp1_0_noop.erl delete mode 100644 deps/rabbitmq_amqp1_0/test/amqp10_client_SUITE.erl delete mode 100644 deps/rabbitmq_amqp1_0/test/command_SUITE.erl delete mode 100644 deps/rabbitmq_amqp1_0/test/config_schema_SUITE.erl delete mode 100644 deps/rabbitmq_amqp1_0/test/config_schema_SUITE_data/rabbitmq_amqp1_0.snippets delete mode 100644 deps/rabbitmq_amqp1_0/test/unit_SUITE.erl create mode 100644 deps/rabbitmq_stream/test/protocol_interop_SUITE.erl diff --git a/Makefile b/Makefile index 3b1af65e8a67..52dda43ba126 100644 --- a/Makefile +++ b/Makefile @@ -176,8 +176,6 @@ RSYNC_FLAGS += -a $(RSYNC_V) \ --exclude '/cowboy/doc/' \ --exclude '/cowboy/examples/' \ --exclude '/rabbit/escript/' \ - --exclude '/rabbitmq_amqp1_0/test/swiftmq/build/'\ - --exclude '/rabbitmq_amqp1_0/test/swiftmq/swiftmq*'\ --exclude '/rabbitmq_cli/escript/' \ --exclude '/rabbitmq_mqtt/test/build/' \ --exclude '/rabbitmq_mqtt/test/test_client/'\ diff --git a/deps/amqp10_client/BUILD.bazel b/deps/amqp10_client/BUILD.bazel index 71106162a4c4..6d865915a811 100644 --- a/deps/amqp10_client/BUILD.bazel +++ b/deps/amqp10_client/BUILD.bazel @@ -100,7 +100,6 @@ dialyze( ) broker_for_integration_suites( - extra_plugins = ["//deps/rabbitmq_amqp1_0:erlang_app"], ) TEST_DEPS = [ diff --git a/deps/amqp10_client/Makefile b/deps/amqp10_client/Makefile index 20a223a2a997..7dd0f4c657a5 100644 --- a/deps/amqp10_client/Makefile +++ b/deps/amqp10_client/Makefile @@ -30,7 +30,7 @@ PACKAGES_DIR ?= $(abspath PACKAGES) BUILD_DEPS = rabbit_common elvis_mk DEPS = amqp10_common credentials_obfuscation -TEST_DEPS = rabbit rabbitmq_amqp1_0 rabbitmq_ct_helpers +TEST_DEPS = rabbit rabbitmq_ct_helpers LOCAL_DEPS = ssl inets crypto public_key DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-test.mk @@ -51,20 +51,6 @@ include erlang.mk HEX_TARBALL_FILES += rabbitmq-components.mk \ git-revisions.txt -# -------------------------------------------------------------------- -# Compiler flags. -# -------------------------------------------------------------------- - -# gen_fsm is deprecated starting from Erlang 20, but we want to support -# Erlang 19 as well. - -ERTS_VER := $(shell erl -version 2>&1 | sed -E 's/.* version //') -ERLANG_20_ERTS_VER := 9.0 - -ifeq ($(call compare_version,$(ERTS_VER),$(ERLANG_20_ERTS_VER),>=),true) -ERLC_OPTS += -Dnowarn_deprecated_gen_fsm -endif - # Dialyze the tests. DIALYZER_OPTS += --src -r test diff --git a/deps/amqp10_client/README.md b/deps/amqp10_client/README.md index 9ad1e803483d..8e7337169eae 100644 --- a/deps/amqp10_client/README.md +++ b/deps/amqp10_client/README.md @@ -2,16 +2,16 @@ This is an [Erlang client for the AMQP 1.0](https://www.amqp.org/resources/specifications) protocol. -It's primary purpose is to be used in RabbitMQ related projects but it is a -generic client that was tested with at least 4 implementations of AMQP 1.0. +Its primary purpose is to be used in RabbitMQ related projects but it is a +generic client that was tested with at least 3 implementations of AMQP 1.0. If you are looking for an Erlang client for [AMQP 0-9-1](https://www.rabbitmq.com/tutorials/amqp-concepts.html) — a completely different -protocol despite the name — [consider this one](https://github.com/rabbitmq/rabbitmq-erlang-client). +protocol despite the name — [consider this one](../amqp_client). ## Project Maturity and Status This client is used in the cross-protocol version of the RabbitMQ Shovel plugin. It is not 100% -feature complete but moderately mature and was tested against at least three AMQP 1.0 servers: +feature complete but moderately mature and was tested against at least 3 AMQP 1.0 servers: RabbitMQ, Azure ServiceBus, ActiveMQ. This client library is not officially supported by VMware at this time. @@ -80,8 +80,8 @@ after 2000 -> exit(credited_timeout) end. -%% create a new message using a delivery-tag, body and indicate -%% it's settlement status (true meaning no disposition confirmation +%% Create a new message using a delivery-tag, body and indicate +%% its settlement status (true meaning no disposition confirmation %% will be sent by the receiver). OutMsg = amqp10_msg:new(<<"my-tag">>, <<"my-body">>, true), ok = amqp10_client:send_msg(Sender, OutMsg), @@ -112,7 +112,7 @@ after the `Open` frame has been successfully written to the socket rather than waiting until the remote end returns with their `Open` frame. The client will notify the caller of various internal/async events using `amqp10_event` messages. In the example above when the remote replies with their `Open` frame -a message is sent of the following forma: +a message is sent of the following form: ``` {amqp10_event, {connection, ConnectionPid, opened}} diff --git a/deps/amqp10_client/app.bzl b/deps/amqp10_client/app.bzl index 2278cb326178..8fcdad73cf9d 100644 --- a/deps/amqp10_client/app.bzl +++ b/deps/amqp10_client/app.bzl @@ -13,7 +13,6 @@ def all_beam_files(name = "all_beam_files"): "src/amqp10_client_app.erl", "src/amqp10_client_connection.erl", "src/amqp10_client_connection_sup.erl", - "src/amqp10_client_connections_sup.erl", "src/amqp10_client_frame_reader.erl", "src/amqp10_client_session.erl", "src/amqp10_client_sessions_sup.erl", @@ -42,7 +41,6 @@ def all_test_beam_files(name = "all_test_beam_files"): "src/amqp10_client_app.erl", "src/amqp10_client_connection.erl", "src/amqp10_client_connection_sup.erl", - "src/amqp10_client_connections_sup.erl", "src/amqp10_client_frame_reader.erl", "src/amqp10_client_session.erl", "src/amqp10_client_sessions_sup.erl", @@ -77,7 +75,6 @@ def all_srcs(name = "all_srcs"): "src/amqp10_client_app.erl", "src/amqp10_client_connection.erl", "src/amqp10_client_connection_sup.erl", - "src/amqp10_client_connections_sup.erl", "src/amqp10_client_frame_reader.erl", "src/amqp10_client_session.erl", "src/amqp10_client_sessions_sup.erl", diff --git a/deps/amqp10_client/src/amqp10_client.erl b/deps/amqp10_client/src/amqp10_client.erl index 9dba52d332e4..c9939bc263e4 100644 --- a/deps/amqp10_client/src/amqp10_client.erl +++ b/deps/amqp10_client/src/amqp10_client.erl @@ -35,7 +35,7 @@ settle_msg/3, flow_link_credit/3, flow_link_credit/4, - echo/1, + stop_receiver_link/1, link_handle/1, get_msg/1, get_msg/2, @@ -55,7 +55,7 @@ -type attach_role() :: amqp10_client_session:attach_role(). -type attach_args() :: amqp10_client_session:attach_args(). -type filter() :: amqp10_client_session:filter(). --type properties() :: amqp10_client_session:properties(). +-type properties() :: amqp10_client_types:properties(). -type connection_config() :: amqp10_client_connection:connection_config(). @@ -109,10 +109,10 @@ open_connection(ConnectionConfig0) -> notify_when_closed => NotifyWhenClosed }, Sasl = maps:get(sasl, ConnectionConfig1), - ConnectionConfig2 = ConnectionConfig1#{sasl => amqp10_client_connection:encrypt_sasl(Sasl)}, - amqp10_client_connection:open(ConnectionConfig2). + ConnectionConfig = ConnectionConfig1#{sasl => amqp10_client_connection:encrypt_sasl(Sasl)}, + amqp10_client_connection:open(ConnectionConfig). -%% @doc Opens a connection using a connection_config map +%% @doc Closes a connection. %% This is asynchronous and will notify completion to the caller using %% an amqp10_event of the following format: %% {amqp10_event, {connection, ConnectionPid, {closed, Why}}} @@ -271,9 +271,8 @@ attach_receiver_link(Session, Name, Source, SettleMode, Durability, Filter) -> %% This is asynchronous and will notify completion of the attach request to the %% caller using an amqp10_event of the following format: %% {amqp10_event, {link, LinkRef, attached | {detached, Why}}} --spec attach_receiver_link(pid(), binary(), binary(), - snd_settle_mode(), terminus_durability(), filter(), - properties()) -> +-spec attach_receiver_link(pid(), binary(), binary(), snd_settle_mode(), + terminus_durability(), filter(), properties()) -> {ok, link_ref()}. attach_receiver_link(Session, Name, Source, SettleMode, Durability, Filter, Properties) when is_pid(Session) andalso @@ -307,43 +306,45 @@ detach_link(#link_ref{link_handle = Handle, session = Session}) -> amqp10_client_session:detach(Session, Handle). %% @doc Grant credit to a sender. -%% The amqp10_client will automatically grant more credit to the sender when +%% The amqp10_client will automatically grant Credit to the sender when %% the remaining link credit falls below the value of RenewWhenBelow. -%% If RenewWhenBelow is 'never' the client will never grant new credit. Instead +%% If RenewWhenBelow is 'never' the client will never grant more credit. Instead %% the caller will be notified when the link_credit reaches 0 with an %% amqp10_event of the following format: %% {amqp10_event, {link, LinkRef, credit_exhausted}} -spec flow_link_credit(link_ref(), Credit :: non_neg_integer(), - RenewWhenBelow :: never | non_neg_integer()) -> ok. + RenewWhenBelow :: never | pos_integer()) -> ok. flow_link_credit(Ref, Credit, RenewWhenBelow) -> flow_link_credit(Ref, Credit, RenewWhenBelow, false). -spec flow_link_credit(link_ref(), Credit :: non_neg_integer(), - RenewWhenBelow :: never | non_neg_integer(), + RenewWhenBelow :: never | pos_integer(), Drain :: boolean()) -> ok. flow_link_credit(#link_ref{role = receiver, session = Session, link_handle = Handle}, - Credit, RenewWhenBelow, Drain) -> + Credit, RenewWhenBelow, Drain) + when RenewWhenBelow =:= never orelse + is_integer(RenewWhenBelow) andalso + RenewWhenBelow > 0 andalso + RenewWhenBelow =< Credit -> Flow = #'v1_0.flow'{link_credit = {uint, Credit}, drain = Drain}, ok = amqp10_client_session:flow(Session, Handle, Flow, RenewWhenBelow). -%% @doc Request that the sender's flow state is echoed back -%% This may be used to determine when the Link has finally quiesced. -%% see §2.6.10 of the spec -echo(#link_ref{role = receiver, session = Session, - link_handle = Handle}) -> +%% @doc Stop a receiving link. +%% See AMQP 1.0 spec §2.6.10. +stop_receiver_link(#link_ref{role = receiver, + session = Session, + link_handle = Handle}) -> Flow = #'v1_0.flow'{link_credit = {uint, 0}, echo = true}, - ok = amqp10_client_session:flow(Session, Handle, Flow, 0). + ok = amqp10_client_session:flow(Session, Handle, Flow, never). %%% messages %% @doc Send a message on a the link referred to be the 'LinkRef'. -%% Returns ok for "async" transfers when messages are sent with settled=true -%% else it returns the delivery state from the disposition -spec send_msg(link_ref(), amqp10_msg:amqp10_msg()) -> - ok | {error, insufficient_credit | link_not_found | half_attached}. + ok | amqp10_client_session:transfer_error(). send_msg(#link_ref{role = sender, session = Session, link_handle = Handle}, Msg0) -> Msg = amqp10_msg:set_handle(Handle, Msg0), diff --git a/deps/amqp10_client/src/amqp10_client.hrl b/deps/amqp10_client/src/amqp10_client.hrl index c36e0b72a5de..137e82552199 100644 --- a/deps/amqp10_client/src/amqp10_client.hrl +++ b/deps/amqp10_client/src/amqp10_client.hrl @@ -7,8 +7,6 @@ -define(AMQP_PROTOCOL_HEADER, <<"AMQP", 0, 1, 0, 0>>). -define(SASL_PROTOCOL_HEADER, <<"AMQP", 3, 1, 0, 0>>). --define(MIN_MAX_FRAME_SIZE, 512). --define(MAX_MAX_FRAME_SIZE, 1024 * 1024). -define(FRAME_HEADER_SIZE, 8). -define(TIMEOUT, 5000). diff --git a/deps/amqp10_client/src/amqp10_client_app.erl b/deps/amqp10_client/src/amqp10_client_app.erl index 25e2d92a52e5..6edce5d199e5 100644 --- a/deps/amqp10_client/src/amqp10_client_app.erl +++ b/deps/amqp10_client/src/amqp10_client_app.erl @@ -9,30 +9,12 @@ -behaviour(application). -%% Application callbacks +%% application callbacks -export([start/2, stop/1]). --type start_type() :: ( - normal | - {takeover, Node :: node()} | - {failover, Node :: node()} - ). --type state() :: term(). - -%%==================================================================== -%% API -%%==================================================================== - --spec start(StartType :: start_type(), StartArgs :: term()) -> - {ok, Pid :: pid()} | {ok, Pid :: pid(), State :: state()} | {error, Reason :: term()}. start(_Type, _Args) -> amqp10_client_sup:start_link(). --spec stop(State :: state()) -> ok. stop(_State) -> ok. - -%%==================================================================== -%% Internal functions -%%==================================================================== diff --git a/deps/amqp10_client/src/amqp10_client_connection.erl b/deps/amqp10_client/src/amqp10_client_connection.erl index 4ce4a4e7a6cc..b3e0f1c450a9 100644 --- a/deps/amqp10_client/src/amqp10_client_connection.erl +++ b/deps/amqp10_client/src/amqp10_client_connection.erl @@ -11,21 +11,13 @@ -include("amqp10_client.hrl"). -include_lib("amqp10_common/include/amqp10_framing.hrl"). +-include_lib("amqp10_common/include/amqp10_types.hrl"). --ifdef(nowarn_deprecated_gen_fsm). --compile({nowarn_deprecated_function, - [{gen_fsm, reply, 2}, - {gen_fsm, send_all_state_event, 2}, - {gen_fsm, send_event, 2}, - {gen_fsm, start_link, 3}, - {gen_fsm, sync_send_all_state_event, 2}]}). --endif. - -%% Public API. +%% public API -export([open/1, close/2]). -%% Private API. +%% private API -export([start_link/2, socket_ready/2, protocol_header_received/5, @@ -34,13 +26,14 @@ encrypt_sasl/1, decrypt_sasl/1]). -%% gen_fsm callbacks. +%% gen_statem callbacks -export([init/1, callback_mode/0, terminate/3, code_change/4]). -%% gen_fsm state callbacks. +%% gen_statem state callbacks +%% see figure 2.23 -export([expecting_socket/3, sasl_hdr_sent/3, sasl_hdr_rcvds/3, @@ -71,8 +64,10 @@ notify => pid() | none, % the pid to send connection events to notify_when_opened => pid() | none, notify_when_closed => pid() | none, - max_frame_size => non_neg_integer(), % TODO: constrain to large than 512 - outgoing_max_frame_size => non_neg_integer() | undefined, + %% incoming maximum frame size set by our client application + max_frame_size => pos_integer(), % TODO: constrain to large than 512 + %% outgoing maximum frame size set by AMQP peer in OPEN performative + outgoing_max_frame_size => pos_integer() | undefined, idle_time_out => milliseconds(), % set to a negative value to allow a sender to "overshoot" the flow % control by this margin @@ -80,9 +75,7 @@ %% These credentials_obfuscation-wrapped values have the type of %% decrypted_sasl/0 sasl => encrypted_sasl() | decrypted_sasl(), - notify => pid(), - notify_when_opened => pid() | none, - notify_when_closed => pid() | none + properties => amqp10_client_types:properties() }. -record(state, @@ -167,13 +160,13 @@ protocol_header_received(Pid, Protocol, Maj, Min, Rev) -> -spec begin_session(pid()) -> supervisor:startchild_ret(). begin_session(Pid) -> - gen_statem:call(Pid, begin_session, {dirty_timeout, ?TIMEOUT}). + gen_statem:call(Pid, begin_session, ?TIMEOUT). heartbeat(Pid) -> gen_statem:cast(Pid, heartbeat). %% ------------------------------------------------------------------- -%% gen_fsm callbacks. +%% gen_statem callbacks. %% ------------------------------------------------------------------- callback_mode() -> [state_functions]. @@ -259,7 +252,7 @@ hdr_sent({call, From}, begin_session, State1 = State#state{pending_session_reqs = [From | PendingSessionReqs]}, {keep_state, State1}. -open_sent(_EvtType, #'v1_0.open'{max_frame_size = MFSz, +open_sent(_EvtType, #'v1_0.open'{max_frame_size = MaybeMaxFrameSize, idle_time_out = Timeout}, #state{pending_session_reqs = PendingSessionReqs, config = Config} = State0) -> @@ -271,8 +264,14 @@ open_sent(_EvtType, #'v1_0.open'{max_frame_size = MFSz, heartbeat_timer = Tmr}; _ -> State0 end, - State1 = State#state{config = - Config#{outgoing_max_frame_size => unpack(MFSz)}}, + MaxFrameSize = case unpack(MaybeMaxFrameSize) of + undefined -> + %% default as per 2.7.1 + ?UINT_MAX; + Bytes when is_integer(Bytes) -> + Bytes + end, + State1 = State#state{config = Config#{outgoing_max_frame_size => MaxFrameSize}}, State2 = lists:foldr( fun(From, S0) -> {Ret, S2} = handle_begin_session(From, S0), @@ -403,32 +402,32 @@ handle_begin_session({FromPid, _Ref}, end, {Ret, State1}. -send_open(#state{socket = Socket, config = Config}) -> +send_open(#state{socket = Socket, config = Config0}) -> {ok, Product} = application:get_key(description), {ok, Version} = application:get_key(vsn), Platform = "Erlang/OTP " ++ erlang:system_info(otp_release), - Props = {map, [{{symbol, <<"product">>}, - {utf8, list_to_binary(Product)}}, - {{symbol, <<"version">>}, - {utf8, list_to_binary(Version)}}, - {{symbol, <<"platform">>}, - {utf8, list_to_binary(Platform)}} - ]}, + Props0 = #{<<"product">> => {utf8, list_to_binary(Product)}, + <<"version">> => {utf8, list_to_binary(Version)}, + <<"platform">> => {utf8, list_to_binary(Platform)}}, + Config = maps:update_with(properties, + fun(Val) -> maps:merge(Props0, Val) end, + Props0, + Config0), + Props = amqp10_client_types:make_properties(Config), ContainerId = maps:get(container_id, Config, generate_container_id()), IdleTimeOut = maps:get(idle_time_out, Config, 0), + IncomingMaxFrameSize = maps:get(max_frame_size, Config), Open0 = #'v1_0.open'{container_id = {utf8, ContainerId}, channel_max = {ushort, 100}, idle_time_out = {uint, IdleTimeOut}, - properties = Props}, - Open1 = case Config of - #{max_frame_size := MFSz} -> - Open0#'v1_0.open'{max_frame_size = {uint, MFSz}}; - _ -> Open0 - end, + properties = Props, + max_frame_size = {uint, IncomingMaxFrameSize} + }, Open = case Config of #{hostname := Hostname} -> - Open1#'v1_0.open'{hostname = {utf8, Hostname}}; - _ -> Open1 + Open0#'v1_0.open'{hostname = {utf8, Hostname}}; + _ -> + Open0 end, Encoded = amqp10_framing:encode_bin(Open), Frame = amqp10_binary_generator:build_frame(0, Encoded), @@ -508,7 +507,7 @@ unpack(V) -> amqp10_client_types:unpack(V). -spec generate_container_id() -> binary(). generate_container_id() -> - Pre = list_to_binary(atom_to_list(node())), + Pre = atom_to_binary(node()), Id = bin_to_hex(crypto:strong_rand_bytes(8)), <
>/binary, Id/binary>>.
 
@@ -552,4 +551,5 @@ decrypted_sasl_to_bin(none) -> <<"ANONYMOUS">>.
 config_defaults() ->
     #{sasl => none,
       transfer_limit_margin => 0,
-      max_frame_size => ?MAX_MAX_FRAME_SIZE}.
+      %% 1 MB
+      max_frame_size => 1_048_576}.
diff --git a/deps/amqp10_client/src/amqp10_client_connection_sup.erl b/deps/amqp10_client/src/amqp10_client_connection_sup.erl
index c8a82174d3de..1049e08bac4f 100644
--- a/deps/amqp10_client/src/amqp10_client_connection_sup.erl
+++ b/deps/amqp10_client/src/amqp10_client_connection_sup.erl
@@ -8,35 +8,31 @@
 
 -behaviour(supervisor).
 
-%% Private API.
+%% API
 -export([start_link/1]).
 
-%% Supervisor callbacks.
+%% Supervisor callbacks
 -export([init/1]).
 
--define(CHILD(Id, Mod, Type, Args), {Id, {Mod, start_link, Args},
-                                     transient, 5000, Type, [Mod]}).
-
-%% -------------------------------------------------------------------
-%% Private API.
-%% -------------------------------------------------------------------
-
--spec start_link(amqp10_client_connection:connection_config()) ->
-    {ok, pid()} | ignore | {error, any()}.
 start_link(Config) ->
     supervisor:start_link(?MODULE, [Config]).
 
-%% -------------------------------------------------------------------
-%% Supervisor callbacks.
-%% -------------------------------------------------------------------
-
-init(Args) ->
-    ReaderSpec = ?CHILD(reader, amqp10_client_frame_reader,
-                        worker, [self() | Args]),
-    ConnectionSpec = ?CHILD(connection, amqp10_client_connection,
-                            worker, [self() | Args]),
-    SessionsSupSpec = ?CHILD(sessions, amqp10_client_sessions_sup,
-                             supervisor, []),
-    {ok, {{one_for_all, 0, 1}, [ConnectionSpec,
-                                ReaderSpec,
-                                SessionsSupSpec]}}.
+init(Args0) ->
+    SupFlags = #{strategy => one_for_all,
+                 intensity => 0,
+                 period => 1},
+    Fun = start_link,
+    Args = [self() | Args0],
+    ConnectionSpec = #{id => connection,
+                       start => {amqp10_client_connection, Fun, Args},
+                       restart => transient},
+    ReaderSpec = #{id => reader,
+                   start => {amqp10_client_frame_reader, Fun, Args},
+                   restart => transient},
+    SessionsSupSpec = #{id => sessions,
+                        start => {amqp10_client_sessions_sup, Fun, []},
+                        restart => transient,
+                        type => supervisor},
+    {ok, {SupFlags, [ConnectionSpec,
+                     ReaderSpec,
+                     SessionsSupSpec]}}.
diff --git a/deps/amqp10_client/src/amqp10_client_connections_sup.erl b/deps/amqp10_client/src/amqp10_client_connections_sup.erl
deleted file mode 100644
index e12c969e69b7..000000000000
--- a/deps/amqp10_client/src/amqp10_client_connections_sup.erl
+++ /dev/null
@@ -1,38 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
--module(amqp10_client_connections_sup).
-
--behaviour(supervisor).
-
-%% Private API.
--export([start_link/0,
-         stop_child/1]).
-
-%% Supervisor callbacks.
--export([init/1]).
-
--define(CHILD(Id, Mod, Type, Args), {Id, {Mod, start_link, Args},
-                                     temporary, infinity, Type, [Mod]}).
-
-%% -------------------------------------------------------------------
-%% Private API.
-%% -------------------------------------------------------------------
-
-stop_child(Pid) ->
-    supervisor:terminate_child({local, ?MODULE}, Pid).
-
-start_link() ->
-    supervisor:start_link({local, ?MODULE}, ?MODULE, []).
-
-%% -------------------------------------------------------------------
-%% Supervisor callbacks.
-%% -------------------------------------------------------------------
-
-init([]) ->
-    Template = ?CHILD(connection_sup, amqp10_client_connection_sup,
-                      supervisor, []),
-    {ok, {{simple_one_for_one, 0, 1}, [Template]}}.
diff --git a/deps/amqp10_client/src/amqp10_client_session.erl b/deps/amqp10_client/src/amqp10_client_session.erl
index 93f654676602..e728f4f5ce05 100644
--- a/deps/amqp10_client/src/amqp10_client_session.erl
+++ b/deps/amqp10_client/src/amqp10_client_session.erl
@@ -10,6 +10,7 @@
 
 -include("amqp10_client.hrl").
 -include_lib("amqp10_common/include/amqp10_framing.hrl").
+-include_lib("amqp10_common/include/amqp10_types.hrl").
 
 %% Public API.
 -export(['begin'/1,
@@ -23,7 +24,7 @@
          disposition/6
         ]).
 
-%% Private API.
+%% Private API
 -export([start_link/4,
          socket_ready/2
         ]).
@@ -33,10 +34,12 @@
          init/1,
          terminate/3,
          code_change/4,
-         callback_mode/0
+         callback_mode/0,
+         format_status/1
         ]).
 
-%% gen_statem state callbacks.
+%% gen_statem state callbacks
+%% see figure 2.30
 -export([
          unmapped/3,
          begin_sent/3,
@@ -44,21 +47,28 @@
          end_sent/3
         ]).
 
+-import(serial_number,
+        [add/2,
+         diff/2]).
+
 -define(MAX_SESSION_WINDOW_SIZE, 65535).
--define(DEFAULT_MAX_HANDLE, 16#ffffffff).
 -define(DEFAULT_TIMEOUT, 5000).
--define(INITIAL_OUTGOING_ID, 0).
--define(INITIAL_DELIVERY_COUNT, 0).
-
-% -type from() :: {pid(), term()}.
+-define(UINT_OUTGOING_WINDOW, {uint, ?UINT_MAX}).
+-define(INITIAL_OUTGOING_DELIVERY_ID, ?UINT_MAX).
+%% "The next-outgoing-id MAY be initialized to an arbitrary value" [2.5.6]
+-define(INITIAL_OUTGOING_TRANSFER_ID, ?UINT_MAX - 1).
+%% "Note that, despite its name, the delivery-count is not a count but a
+%% sequence number initialized at an arbitrary point by the sender." [2.6.7]
+-define(INITIAL_DELIVERY_COUNT, ?UINT_MAX - 2).
 
--type transfer_id() :: non_neg_integer().
--type link_handle() :: non_neg_integer().
 -type link_name() :: binary().
 -type link_address() :: binary().
 -type link_role() :: sender | receiver.
--type link_source() :: link_address() | undefined.
 -type link_target() :: {pid, pid()} | binary() | undefined.
+%% "The locally chosen handle is referred to as the output handle." [2.6.2]
+-type output_handle() :: link_handle().
+%% "The remotely chosen handle is referred to as the input handle." [2.6.2]
+-type input_handle() :: link_handle().
 
 -type snd_settle_mode() :: unsettled | settled | mixed.
 -type rcv_settle_mode() :: first | second.
@@ -74,16 +84,25 @@
 
 % http://www.amqp.org/specification/1.0/filters
 -type filter() :: #{binary() => binary() | map() | list(binary())}.
--type properties() :: #{binary() => tuple()}.
+-type max_message_size() :: undefined | non_neg_integer().
 
 -type attach_args() :: #{name => binary(),
                          role => attach_role(),
                          snd_settle_mode => snd_settle_mode(),
                          rcv_settle_mode => rcv_settle_mode(),
                          filter => filter(),
-                         properties => properties()
+                         properties => amqp10_client_types:properties(),
+                         max_message_size => max_message_size(),
+                         handle => output_handle()
                         }.
 
+-type transfer_error() :: {error,
+                           insufficient_credit |
+                           remote_incoming_window_exceeded |
+                           message_size_exceeded |
+                           link_not_found |
+                           half_attached}.
+
 -type link_ref() :: #link_ref{}.
 
 -export_type([snd_settle_mode/0,
@@ -93,56 +112,53 @@
               attach_role/0,
               target_def/0,
               source_def/0,
-              properties/0,
-              filter/0]).
+              filter/0,
+              max_message_size/0,
+              transfer_error/0]).
 
 -record(link,
         {name :: link_name(),
          ref :: link_ref(),
          state = detached :: detached | attach_sent | attached | detach_sent,
          notify :: pid(),
-         output_handle :: link_handle(),
-         input_handle :: link_handle() | undefined,
+         output_handle :: output_handle(),
+         input_handle :: input_handle() | undefined,
          role :: link_role(),
-         source :: link_source(),
          target :: link_target(),
-         delivery_count = 0 :: non_neg_integer(),
+         max_message_size :: non_neg_integer() | Unlimited :: undefined,
+         delivery_count :: sequence_no() | undefined,
          link_credit = 0 :: non_neg_integer(),
-         link_credit_unsettled = 0 :: non_neg_integer(),
          available = 0 :: non_neg_integer(),
          drain = false :: boolean(),
          partial_transfers :: undefined | {#'v1_0.transfer'{}, [binary()]},
-         auto_flow :: never | {auto, non_neg_integer(), non_neg_integer()}
+         auto_flow :: never | {auto, RenewWhenBelow :: pos_integer(), Credit :: pos_integer()}
          }).
 
 -record(state,
         {channel :: pos_integer(),
          remote_channel :: pos_integer() | undefined,
-         next_incoming_id = 0 :: transfer_id(),
+
+         %% session flow control, see section 2.5.6
+         next_incoming_id :: transfer_number() | undefined,
          incoming_window = ?MAX_SESSION_WINDOW_SIZE :: non_neg_integer(),
-         next_outgoing_id = ?INITIAL_OUTGOING_ID + 1 :: transfer_id(),
-         outgoing_window = ?MAX_SESSION_WINDOW_SIZE  :: non_neg_integer(),
+         next_outgoing_id = ?INITIAL_OUTGOING_TRANSFER_ID :: transfer_number(),
          remote_incoming_window = 0 :: non_neg_integer(),
          remote_outgoing_window = 0 :: non_neg_integer(),
+
          reader :: pid(),
          socket :: amqp10_client_connection:amqp10_socket() | undefined,
-         links = #{} :: #{link_handle() => #link{}},
-         % maps link name to outgoing link handle
-         link_index = #{} :: #{link_name() => link_handle()},
-         % maps incoming handle to outgoing
-         link_handle_index = #{} :: #{link_handle() => link_handle()},
-         next_link_handle = 0 :: link_handle(),
-         early_attach_requests = [] :: [term()],
-         connection_config = #{} :: amqp10_client_connection:connection_config(),
-         % the unsettled map needs to go in the session state as a disposition
-         % can reference transfers for many different links
-         unsettled = #{} :: #{transfer_id() => {amqp10_msg:delivery_tag(),
-                                                any()}}, %TODO: refine as FsmRef
-         incoming_unsettled = #{} :: #{transfer_id() => link_handle()},
+         links = #{} :: #{output_handle() => #link{}},
+         link_index = #{} :: #{link_name() => output_handle()},
+         link_handle_index = #{} :: #{input_handle() => output_handle()},
+         next_link_handle = 0 :: output_handle(),
+         early_attach_requests :: [term()],
+         connection_config :: amqp10_client_connection:connection_config(),
+         outgoing_delivery_id = ?INITIAL_OUTGOING_DELIVERY_ID :: delivery_number(),
+         outgoing_unsettled = #{} :: #{delivery_number() => {amqp10_msg:delivery_tag(), Notify :: pid()}},
+         incoming_unsettled = #{} :: #{delivery_number() => output_handle()},
          notify :: pid()
         }).
 
-
 %% -------------------------------------------------------------------
 %% Public API.
 %% -------------------------------------------------------------------
@@ -173,27 +189,26 @@ begin_sync(Connection, Timeout) ->
 
 -spec attach(pid(), attach_args()) -> {ok, link_ref()}.
 attach(Session, Args) ->
-    gen_statem:call(Session, {attach, Args}, {dirty_timeout, ?TIMEOUT}).
+    gen_statem:call(Session, {attach, Args}, ?TIMEOUT).
 
--spec detach(pid(), link_handle()) -> ok | {error, link_not_found | half_attached}.
+-spec detach(pid(), output_handle()) -> ok | {error, link_not_found | half_attached}.
 detach(Session, Handle) ->
-    gen_statem:call(Session, {detach, Handle}, {dirty_timeout, ?TIMEOUT}).
+    gen_statem:call(Session, {detach, Handle}, ?TIMEOUT).
 
 -spec transfer(pid(), amqp10_msg:amqp10_msg(), timeout()) ->
-    ok | {error, insufficient_credit | link_not_found | half_attached}.
+    ok | transfer_error().
 transfer(Session, Amqp10Msg, Timeout) ->
     [Transfer | Records] = amqp10_msg:to_amqp_records(Amqp10Msg),
-    gen_statem:call(Session, {transfer, Transfer, Records},
-                    {dirty_timeout, Timeout}).
+    gen_statem:call(Session, {transfer, Transfer, Records}, Timeout).
 
-flow(Session, Handle, Flow, RenewAfter) ->
-    gen_statem:cast(Session, {flow, Handle, Flow, RenewAfter}).
+flow(Session, Handle, Flow, RenewWhenBelow) ->
+    gen_statem:cast(Session, {flow_link, Handle, Flow, RenewWhenBelow}).
 
--spec disposition(pid(), link_role(), transfer_id(), transfer_id(), boolean(),
+-spec disposition(pid(), link_role(), delivery_number(), delivery_number(), boolean(),
                   amqp10_client_types:delivery_state()) -> ok.
 disposition(Session, Role, First, Last, Settled, DeliveryState) ->
     gen_statem:call(Session, {disposition, Role, First, Last, Settled,
-                              DeliveryState}, {dirty_timeout, ?TIMEOUT}).
+                              DeliveryState}, ?TIMEOUT).
 
 
 
@@ -217,8 +232,11 @@ callback_mode() -> [state_functions].
 init([FromPid, Channel, Reader, ConnConfig]) ->
     process_flag(trap_exit, true),
     amqp10_client_frame_reader:register_session(Reader, self(), Channel),
-    State = #state{notify = FromPid, channel = Channel, reader = Reader,
-                   connection_config = ConnConfig},
+    State = #state{notify = FromPid,
+                   channel = Channel,
+                   reader = Reader,
+                   connection_config = ConnConfig,
+                   early_attach_requests = []},
     {ok, unmapped, State}.
 
 unmapped(cast, {socket_ready, Socket}, State) ->
@@ -258,22 +276,29 @@ mapped(cast, 'end', State) ->
     %% We send the first end frame and wait for the reply.
     send_end(State),
     {next_state, end_sent, State};
-mapped(cast, {flow, OutHandle, Flow0, RenewAfter}, State0) ->
-    State = send_flow(fun send/2, OutHandle, Flow0, RenewAfter, State0),
-    {next_state, mapped, State};
+mapped(cast, {flow_link, OutHandle, Flow0, RenewWhenBelow}, State0) ->
+    State = send_flow_link(fun send/2, OutHandle, Flow0, RenewWhenBelow, State0),
+    {keep_state, State};
+mapped(cast, {flow_session, Flow0 = #'v1_0.flow'{incoming_window = {uint, IncomingWindow}}},
+       #state{next_incoming_id = NII,
+              next_outgoing_id = NOI} = State) ->
+    Flow = Flow0#'v1_0.flow'{
+                   next_incoming_id = maybe_uint(NII),
+                   next_outgoing_id = uint(NOI),
+                   outgoing_window = ?UINT_OUTGOING_WINDOW},
+    ok = send(Flow, State),
+    {keep_state, State#state{incoming_window = IncomingWindow}};
 mapped(cast, #'v1_0.end'{error = Err}, State) ->
     %% We receive the first end frame, reply and terminate.
     _ = send_end(State),
     % TODO: send notifications for links?
-    Reason = case Err of
-                 undefined -> normal;
-                 _ -> Err
-             end,
+    Reason = reason(Err),
     ok = notify_session_ended(State, Reason),
     {stop, normal, State};
 mapped(cast, #'v1_0.attach'{name = {utf8, Name},
                             initial_delivery_count = IDC,
-                            handle = {uint, InHandle}},
+                            handle = {uint, InHandle},
+                            max_message_size = MaybeMaxMessageSize},
        #state{links = Links, link_index = LinkIndex,
               link_handle_index = LHI} = State0) ->
 
@@ -281,33 +306,41 @@ mapped(cast, #'v1_0.attach'{name = {utf8, Name},
     #{OutHandle := Link0} = Links,
     ok = notify_link_attached(Link0),
 
-    DeliveryCount = case Link0 of
-                        #link{role = sender, delivery_count = DC} -> DC;
-                        _ -> unpack(IDC)
-                    end,
-    Link = Link0#link{input_handle = InHandle, state = attached,
-                      delivery_count = DeliveryCount},
+    {DeliveryCount, MaxMessageSize} =
+    case Link0 of
+        #link{role = sender,
+              delivery_count = DC} ->
+            MSS = case MaybeMaxMessageSize of
+                      {ulong, S} when S > 0 -> S;
+                      _ -> undefined
+                  end,
+            {DC, MSS};
+        #link{role = receiver,
+              max_message_size = MSS} ->
+            {unpack(IDC), MSS}
+    end,
+    Link = Link0#link{state = attached,
+                      input_handle = InHandle,
+                      delivery_count = DeliveryCount,
+                      max_message_size = MaxMessageSize},
     State = State0#state{links = Links#{OutHandle => Link},
                          link_index = maps:remove(Name, LinkIndex),
                          link_handle_index = LHI#{InHandle => OutHandle}},
-    {next_state, mapped, State};
+    {keep_state, State};
 mapped(cast, #'v1_0.detach'{handle = {uint, InHandle},
                             error = Err},
-        #state{links = Links, link_handle_index = LHI} = State0) ->
+       #state{links = Links, link_handle_index = LHI} = State0) ->
     with_link(InHandle, State0,
               fun (#link{output_handle = OutHandle} = Link, State) ->
-                      Reason = case Err of
-                                   undefined -> normal;
-                                   Err -> Err
-                               end,
+                      Reason = reason(Err),
                       ok = notify_link_detached(Link, Reason),
-                      {next_state, mapped,
+                      {keep_state,
                        State#state{links = maps:remove(OutHandle, Links),
                                    link_handle_index = maps:remove(InHandle, LHI)}}
               end);
 mapped(cast, #'v1_0.flow'{handle = undefined} = Flow, State0) ->
     State = handle_session_flow(Flow, State0),
-    {next_state, mapped, State};
+    {keep_state, State};
 mapped(cast, #'v1_0.flow'{handle = {uint, InHandle}} = Flow,
        #state{links = Links} = State0) ->
 
@@ -319,12 +352,12 @@ mapped(cast, #'v1_0.flow'{handle = {uint, InHandle}} = Flow,
      % TODO: handle `send_flow` return tag
     {ok, Link} = handle_link_flow(Flow, Link0),
     ok = maybe_notify_link_credit(Link0, Link),
-    Links1 = Links#{OutHandle => Link},
+    Links1 = Links#{OutHandle := Link},
     State1 = State#state{links = Links1},
-    {next_state, mapped, State1};
+    {keep_state, State1};
 mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
-                         more = true} = Transfer, Payload},
-                         #state{links = Links} = State0) ->
+                               more = true} = Transfer, Payload},
+       #state{links = Links} = State0) ->
 
     {ok, #link{output_handle = OutHandle} = Link} =
         find_link_by_input_handle(InHandle, State0),
@@ -333,18 +366,18 @@ mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
 
     State = book_partial_transfer_received(
               State0#state{links = Links#{OutHandle => Link1}}),
-    {next_state, mapped, State};
+    {keep_state, State};
 mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
-                         delivery_id = MaybeDeliveryId,
-                         settled = Settled} = Transfer0, Payload0},
-                         #state{incoming_unsettled = Unsettled0} = State0) ->
+                               delivery_id = MaybeDeliveryId,
+                               settled = Settled} = Transfer0, Payload0},
+       #state{incoming_unsettled = Unsettled0} = State0) ->
 
     {ok, #link{target = {pid, TargetPid},
                output_handle = OutHandle,
                ref = LinkRef} = Link0} =
         find_link_by_input_handle(InHandle, State0),
 
-    {Transfer, Payload, Link} = complete_partial_transfer(Transfer0, Payload0, Link0),
+    {Transfer, Payload, Link1} = complete_partial_transfer(Transfer0, Payload0, Link0),
     Msg = decode_as_msg(Transfer, Payload),
 
     % stash the DeliveryId - not sure for what yet
@@ -359,23 +392,22 @@ mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
     % notify when credit is exhausted (link_credit = 0)
     % detach the Link with a transfer-limit-exceeded error code if further
     % transfers are received
-    case book_transfer_received(Settled,
-                                State0#state{incoming_unsettled = Unsettled},
-                                Link) of
-        {ok, State} ->
+    State1 = State0#state{incoming_unsettled = Unsettled},
+    case book_transfer_received(State1, Link1) of
+        {ok, Link2, State2} ->
             % deliver
             TargetPid ! {amqp10_msg, LinkRef, Msg},
-            State1 = auto_flow(Link, State),
-            {next_state, mapped, State1};
-        {credit_exhausted, State} ->
+            State = auto_flow(Link2, State2),
+            {keep_state, State};
+        {credit_exhausted, Link2, State} ->
             TargetPid ! {amqp10_msg, LinkRef, Msg},
-            ok = notify_link(Link, credit_exhausted),
-            {next_state, mapped, State};
-        {transfer_limit_exceeded, State} ->
-            logger:warning("transfer_limit_exceeded for link ~tp", [Link]),
-            Link1 = detach_with_error_cond(Link, State,
+            notify_credit_exhausted(Link2),
+            {keep_state, State};
+        {transfer_limit_exceeded, Link2, State} ->
+            logger:warning("transfer_limit_exceeded for link ~tp", [Link2]),
+            Link = detach_with_error_cond(Link2, State,
                                            ?V_1_0_LINK_ERROR_TRANSFER_LIMIT_EXCEEDED),
-            {next_state, mapped, update_link(Link1, State)}
+            {keep_state, update_link(Link, State)}
     end;
 
 
@@ -386,126 +418,122 @@ mapped(cast, #'v1_0.disposition'{role = true,
                                  first = {uint, First},
                                  last = Last0,
                                  state = DeliveryState},
-       #state{unsettled = Unsettled0} = State) ->
+       #state{outgoing_unsettled = Unsettled0} = State) ->
     Last = case Last0 of
                undefined -> First;
                {uint, L} -> L
            end,
     % TODO: no good if the range becomes very large!! refactor
-    Unsettled =
-        lists:foldl(fun(Id, Acc) ->
-                            case Acc of
-                                #{Id := {DeliveryTag, Receiver}} ->
-                                    %% TODO: currently all modified delivery states
-                                    %% will be translated to the old, `modified` atom.
-                                    %% At some point we should translate into the
-                                    %% full {modified, bool, bool, map) tuple.
-                                    S = translate_delivery_state(DeliveryState),
-                                    ok = notify_disposition(Receiver,
-                                                            {S, DeliveryTag}),
-                                    maps:remove(Id, Acc);
-                                _ -> Acc
-                            end
-                    end, Unsettled0, lists:seq(First, Last)),
-
-    {next_state, mapped, State#state{unsettled = Unsettled}};
+    Unsettled = serial_number:foldl(
+                  fun(Id, Acc0) ->
+                          case maps:take(Id, Acc0) of
+                              {{DeliveryTag, Pid}, Acc} ->
+                                  %% TODO: currently all modified delivery states
+                                  %% will be translated to the old, `modified` atom.
+                                  %% At some point we should translate into the
+                                  %% full {modified, bool, bool, map) tuple.
+                                  S = translate_delivery_state(DeliveryState),
+                                  ok = notify_disposition(Pid, {S, DeliveryTag}),
+                                  Acc;
+                              error ->
+                                  Acc0
+                          end
+                  end, Unsettled0, First, Last),
+
+    {keep_state, State#state{outgoing_unsettled = Unsettled}};
 mapped(cast, Frame, State) ->
     logger:warning("Unhandled session frame ~tp in state ~tp",
                              [Frame, State]),
-    {next_state, mapped, State};
+    {keep_state, State};
+mapped({call, From},
+       {transfer, _Transfer, _Parts},
+       #state{remote_incoming_window = Window})
+  when Window =< 0 ->
+    {keep_state_and_data, {reply, From, {error, remote_incoming_window_exceeded}}};
 mapped({call, From},
+       {transfer, _Transfer, _Parts},
+       #state{remote_incoming_window = Window})
+  when Window =< 0 ->
+    {keep_state_and_data, {reply, From, {error, remote_incoming_window_exceeded}}};
+mapped({call, From = {Pid, _}},
        {transfer, #'v1_0.transfer'{handle = {uint, OutHandle},
                                    delivery_tag = {binary, DeliveryTag},
                                    settled = false} = Transfer0, Parts},
-       #state{next_outgoing_id = NOI, links = Links,
-              unsettled = Unsettled} = State) ->
+       #state{outgoing_delivery_id = DeliveryId, links = Links,
+              outgoing_unsettled = Unsettled} = State) ->
     case Links of
         #{OutHandle := #link{input_handle = undefined}} ->
-            {keep_state, State, [{reply, From, {error, half_attached}}]};
+            {keep_state_and_data, {reply, From, {error, half_attached}}};
         #{OutHandle := #link{link_credit = LC}} when LC =< 0 ->
-            {keep_state, State, [{reply, From, {error, insufficient_credit}}]};
-        #{OutHandle := Link} ->
-            Transfer = Transfer0#'v1_0.transfer'{delivery_id = uint(NOI),
-                                                 resume = false},
-            {ok, NumFrames} = send_transfer(Transfer, Parts, State),
-            State1 = State#state{unsettled = Unsettled#{NOI => {DeliveryTag, From}}},
-            {keep_state, book_transfer_send(NumFrames, Link, State1),
-             [{reply, From, ok}]};
+            {keep_state_and_data, {reply, From, {error, insufficient_credit}}};
+        #{OutHandle := Link = #link{max_message_size = MaxMessageSize}} ->
+            Transfer = Transfer0#'v1_0.transfer'{delivery_id = uint(DeliveryId)},
+            case send_transfer(Transfer, Parts, MaxMessageSize, State) of
+                {ok, NumFrames} ->
+                    State1 = State#state{outgoing_unsettled = Unsettled#{DeliveryId => {DeliveryTag, Pid}}},
+                    {keep_state, book_transfer_send(NumFrames, Link, State1), {reply, From, ok}};
+                Error ->
+                    {keep_state_and_data, {reply, From, Error}}
+            end;
         _ ->
-            {keep_state, State, [{reply, From, {error, link_not_found}}]}
+            {keep_state_and_data, {reply, From, {error, link_not_found}}}
 
     end;
 mapped({call, From},
        {transfer, #'v1_0.transfer'{handle = {uint, OutHandle}} = Transfer0,
-        Parts}, #state{next_outgoing_id = NOI,
+        Parts}, #state{outgoing_delivery_id = DeliveryId,
                        links = Links} = State) ->
     case Links of
         #{OutHandle := #link{input_handle = undefined}} ->
-            {keep_state_and_data, [{reply, From, {error, half_attached}}]};
+            {keep_state_and_data, {reply, From, {error, half_attached}}};
         #{OutHandle := #link{link_credit = LC}} when LC =< 0 ->
-            {keep_state_and_data, [{reply, From, {error, insufficient_credit}}]};
-        #{OutHandle := Link} ->
-            Transfer = Transfer0#'v1_0.transfer'{delivery_id = uint(NOI)},
-            {ok, NumFrames} = send_transfer(Transfer, Parts, State),
-            % TODO look into if erlang will correctly wrap integers during
-            % binary conversion.
-            {keep_state, book_transfer_send(NumFrames, Link, State),
-             [{reply, From, ok}]};
+            {keep_state_and_data, {reply, From, {error, insufficient_credit}}};
+        #{OutHandle := Link = #link{max_message_size = MaxMessageSize}} ->
+            Transfer = Transfer0#'v1_0.transfer'{delivery_id = uint(DeliveryId)},
+            case send_transfer(Transfer, Parts, MaxMessageSize, State) of
+                {ok, NumFrames} ->
+                    {keep_state, book_transfer_send(NumFrames, Link, State), {reply, From, ok}};
+                Error ->
+                    {keep_state_and_data, {reply, From, Error}}
+            end;
         _ ->
-            {keep_state, [{reply, From, {error, link_not_found}}]}
+            {keep_state_and_data, {reply, From, {error, link_not_found}}}
     end;
 
 mapped({call, From},
        {disposition, Role, First, Last, Settled0, DeliveryState},
-       #state{incoming_unsettled = Unsettled0,
-              links = Links0} = State0) ->
-    Disposition =
-    begin
-        DS = translate_delivery_state(DeliveryState),
-        #'v1_0.disposition'{role = translate_role(Role),
-                            first = {uint, First},
-                            last = {uint, Last},
-                            settled = Settled0,
-                            state = DS}
-    end,
-
-    Ks = lists:seq(First, Last),
-    Settled = maps:values(maps:with(Ks, Unsettled0)),
-    Links = lists:foldl(fun (H, Acc) ->
-                                #{H := #link{link_credit_unsettled = LCU} = L} = Acc,
-                                Acc#{H => L#link{link_credit_unsettled = LCU-1}}
-                        end, Links0, Settled),
-    Unsettled = maps:without(Ks, Unsettled0),
-    State = lists:foldl(fun(H, S) ->
-                                #{H := L} = Links,
-                                auto_flow(L, S)
-                        end,
-                        State0#state{incoming_unsettled = Unsettled,
-                                     links = Links},
-                        lists:usort(Settled)),
-
+       #state{incoming_unsettled = Unsettled0} = State0) ->
+    Unsettled = serial_number:foldl(fun maps:remove/2, Unsettled0, First, Last),
+    State = State0#state{incoming_unsettled = Unsettled},
+    Disposition = #'v1_0.disposition'{
+                     role = translate_role(Role),
+                     first = {uint, First},
+                     last = {uint, Last},
+                     settled = Settled0,
+                     state = translate_delivery_state(DeliveryState)},
     Res = send(Disposition, State),
-
-    {keep_state, State, [{reply, From, Res}]};
+    {keep_state, State, {reply, From, Res}};
 
 mapped({call, From}, {attach, Attach}, State) ->
     {State1, LinkRef} = send_attach(fun send/2, Attach, From, State),
-    {keep_state, State1, [{reply, From, {ok, LinkRef}}]};
+    {keep_state, State1, {reply, From, {ok, LinkRef}}};
 
 mapped({call, From}, Msg, State) ->
     {Reply, State1} = send_detach(fun send/2, Msg, From, State),
-    {keep_state, State1, [{reply, From, Reply}]};
+    {keep_state, State1, {reply, From, Reply}};
 
 mapped(_EvtType, Msg, _State) ->
     logger:warning("amqp10_session: unhandled msg in mapped state ~W",
-                          [Msg, 10]),
+                   [Msg, 10]),
     keep_state_and_data.
 
-end_sent(_EvtType, #'v1_0.end'{}, State) ->
+end_sent(_EvtType, #'v1_0.end'{error = Err}, State) ->
+    Reason = reason(Err),
+    ok = notify_session_ended(State, Reason),
     {stop, normal, State};
-end_sent(_EvtType, _Frame, State) ->
+end_sent(_EvtType, _Frame, _State) ->
     % just drop frames here
-    {next_state, end_sent, State}.
+    keep_state_and_data.
 
 terminate(Reason, _StateName, #state{channel = Channel,
                                      remote_channel = RemoteChannel,
@@ -526,11 +554,10 @@ code_change(_OldVsn, StateName, State, _Extra) ->
 
 send_begin(#state{socket = Socket,
                   next_outgoing_id = NextOutId,
-                  incoming_window = InWin,
-                  outgoing_window = OutWin} = State) ->
+                  incoming_window = InWin} = State) ->
     Begin = #'v1_0.begin'{next_outgoing_id = uint(NextOutId),
                           incoming_window = uint(InWin),
-                          outgoing_window = uint(OutWin) },
+                          outgoing_window = ?UINT_OUTGOING_WINDOW},
     Frame = encode_frame(Begin, State),
     socket_send(Socket, Frame).
 
@@ -551,57 +578,56 @@ send(Record, #state{socket = Socket} = State) ->
     Frame = encode_frame(Record, State),
     socket_send(Socket, Frame).
 
-send_transfer(Transfer0, Parts0, #state{socket = Socket, channel = Channel,
-                                        connection_config = Config}) ->
-    OutMaxFrameSize = case Config of
-                          #{outgoing_max_frame_size := undefined} ->
-                              ?MAX_MAX_FRAME_SIZE;
-                          #{outgoing_max_frame_size := Sz} -> Sz;
-                          _ -> ?MAX_MAX_FRAME_SIZE
-                      end,
+send_transfer(Transfer0, Parts0, MaxMessageSize, #state{socket = Socket,
+                                                        channel = Channel,
+                                                        connection_config = Config}) ->
+    OutMaxFrameSize = maps:get(outgoing_max_frame_size, Config),
     Transfer = amqp10_framing:encode_bin(Transfer0),
-    TSize = iolist_size(Transfer),
+    TransferSize = iolist_size(Transfer),
     Parts = [amqp10_framing:encode_bin(P) || P <- Parts0],
     PartsBin = iolist_to_binary(Parts),
+    if is_integer(MaxMessageSize) andalso
+       MaxMessageSize > 0 andalso
+       byte_size(PartsBin) > MaxMessageSize ->
+           {error, message_size_exceeded};
+       true ->
+           % TODO: this does not take the extended header into account
+           % see: 2.3
+           MaxPayloadSize = OutMaxFrameSize - TransferSize - ?FRAME_HEADER_SIZE,
+           Frames = build_frames(Channel, Transfer0, PartsBin, MaxPayloadSize, []),
+           ok = socket_send(Socket, Frames),
+           {ok, length(Frames)}
+    end.
 
-    % TODO: this does not take the extended header into account
-    % see: 2.3
-    MaxPayloadSize = OutMaxFrameSize - TSize - ?FRAME_HEADER_SIZE,
-
-    Frames = build_frames(Channel, Transfer0, PartsBin, MaxPayloadSize, []),
-    ok = socket_send(Socket, Frames),
-    {ok, length(Frames)}.
-
-send_flow(Send, OutHandle,
-          #'v1_0.flow'{link_credit = {uint, Credit}} = Flow0, RenewAfter,
-          #state{links = Links,
-                 next_incoming_id = NII,
-                 next_outgoing_id = NOI,
-                 outgoing_window = OutWin,
-                 incoming_window = InWin} = State) ->
-    AutoFlow = case RenewAfter of
+send_flow_link(Send, OutHandle,
+               #'v1_0.flow'{link_credit = {uint, Credit}} = Flow0, RenewWhenBelow,
+               #state{links = Links,
+                      next_incoming_id = NII,
+                      next_outgoing_id = NOI,
+                      incoming_window = InWin} = State) ->
+    AutoFlow = case RenewWhenBelow of
                    never -> never;
                    Limit -> {auto, Limit, Credit}
                end,
     #{OutHandle := #link{output_handle = H,
                          role = receiver,
                          delivery_count = DeliveryCount,
-                         available = Available,
-                         link_credit_unsettled = LCU} = Link} = Links,
-    Flow = Flow0#'v1_0.flow'{handle = uint(H),
-                             link_credit = uint(Credit),
-                             next_incoming_id = uint(NII),
-                             next_outgoing_id = uint(NOI),
-                             outgoing_window = uint(OutWin),
-                             incoming_window = uint(InWin),
-                             delivery_count = uint(DeliveryCount),
-                             available = uint(Available)},
+                         available = Available} = Link} = Links,
+    Flow = Flow0#'v1_0.flow'{
+                   handle = uint(H),
+                   %% "This value MUST be set if the peer has received the begin
+                   %% frame for the session, and MUST NOT be set if it has not." [2.7.4]
+                   next_incoming_id = maybe_uint(NII),
+                   next_outgoing_id = uint(NOI),
+                   outgoing_window = ?UINT_OUTGOING_WINDOW,
+                   incoming_window = uint(InWin),
+                   %% "In the event that the receiving link endpoint has not yet seen the
+                   %% initial attach frame from the sender this field MUST NOT be set." [2.7.4]
+                   delivery_count = maybe_uint(DeliveryCount),
+                   available = uint(Available)},
     ok = Send(Flow, State),
     State#state{links = Links#{OutHandle =>
                                Link#link{link_credit = Credit,
-                                         % need to add on the current LCU
-                                         % to ensure we don't overcredit
-                                         link_credit_unsettled = LCU + Credit,
                                          auto_flow = AutoFlow}}}.
 
 build_frames(Channel, Trf, Bin, MaxPayloadSize, Acc)
@@ -631,19 +657,13 @@ make_target(#{role := {sender, #{address := Address} = Target}}) ->
     #'v1_0.target'{address = {utf8, Address},
                    durable = {uint, Durable}}.
 
-make_properties(#{properties := Properties}) ->
-    translate_properties(Properties);
-make_properties(_) ->
+max_message_size(#{max_message_size := Size})
+  when is_integer(Size) andalso
+       Size > 0 ->
+    {ulong, Size};
+max_message_size(_) ->
     undefined.
 
-translate_properties(Properties) when is_map(Properties) andalso map_size(Properties) =< 0 ->
-    undefined;
-translate_properties(Properties) when is_map(Properties) ->
-    {map, maps:fold(fun translate_property/3, [], Properties)}.
-
-translate_property(K, V, Acc) when is_tuple(V) ->
-    [{{symbol, K}, V} | Acc].
-
 translate_terminus_durability(none) -> 0;
 translate_terminus_durability(configuration) -> 1;
 translate_terminus_durability(unsettled_state) -> 2.
@@ -675,7 +695,7 @@ filter_value_type(V)
   when is_integer(V) andalso V >= 0 ->
     {uint, V};
 filter_value_type(VList) when is_list(VList) ->
-    [filter_value_type(V) || V <- VList];
+    {list, [filter_value_type(V) || V <- VList]};
 filter_value_type({T, _} = V) when is_atom(T) ->
     %% looks like an already tagged type, just pass it through
     V.
@@ -719,19 +739,30 @@ detach_with_error_cond(Link = #link{output_handle = OutHandle}, State, Cond) ->
     Link#link{state = detach_sent}.
 
 send_attach(Send, #{name := Name, role := Role} = Args, {FromPid, _},
-      #state{next_link_handle = OutHandle, links = Links,
+      #state{next_link_handle = OutHandle0, links = Links,
              link_index = LinkIndex} = State) ->
 
     Source = make_source(Args),
     Target = make_target(Args),
-    Properties = make_properties(Args),
+    Properties = amqp10_client_types:make_properties(Args),
+
+    {LinkTarget, RoleAsBool, InitialDeliveryCount, MaxMessageSize} =
+    case Role of
+        {receiver, _, Pid} ->
+            {{pid, Pid}, true, undefined, max_message_size(Args)};
+        {sender, #{address := TargetAddr}} ->
+            {TargetAddr, false, uint(?INITIAL_DELIVERY_COUNT), undefined}
+    end,
 
-    {LinkTarget, RoleAsBool} = case Role of
-                                   {receiver, _, Pid} ->
-                                       {{pid, Pid}, true};
-                                   {sender, #{address := TargetAddr}} ->
-                                       {TargetAddr, false}
-                               end,
+    {OutHandle, NextLinkHandle} =
+    case Args of
+        #{handle := Handle} ->
+            %% Client app provided link handle.
+            %% Really only meant for integration tests.
+            {Handle, OutHandle0};
+        _ ->
+            {OutHandle0, OutHandle0 + 1}
+    end,
 
     % create attach performative
     Attach = #'v1_0.attach'{name = {utf8, Name},
@@ -739,11 +770,11 @@ send_attach(Send, #{name := Name, role := Role} = Args, {FromPid, _},
                             handle = {uint, OutHandle},
                             source = Source,
                             properties = Properties,
-                            initial_delivery_count =
-                                {uint, ?INITIAL_DELIVERY_COUNT},
+                            initial_delivery_count = InitialDeliveryCount,
                             snd_settle_mode = snd_settle_mode(Args),
                             rcv_settle_mode = rcv_settle_mode(Args),
-                            target = Target},
+                            target = Target,
+                            max_message_size = MaxMessageSize},
     ok = Send(Attach, State),
 
     Link = #link{name = Name,
@@ -753,10 +784,12 @@ send_attach(Send, #{name := Name, role := Role} = Args, {FromPid, _},
                  role = element(1, Role),
                  notify = FromPid,
                  auto_flow = never,
-                 target = LinkTarget},
+                 target = LinkTarget,
+                 delivery_count = unpack(InitialDeliveryCount),
+                 max_message_size = unpack(MaxMessageSize)},
 
     {State#state{links = Links#{OutHandle => Link},
-                 next_link_handle = OutHandle + 1,
+                 next_link_handle = NextLinkHandle,
                  link_index = LinkIndex#{Name => OutHandle}}, Link#link.ref}.
 
 -spec handle_session_flow(#'v1_0.flow'{}, #state{}) -> #state{}.
@@ -767,41 +800,52 @@ handle_session_flow(#'v1_0.flow'{next_incoming_id = MaybeNII,
        #state{next_outgoing_id = OurNOI} = State) ->
     NII = case MaybeNII of
               {uint, N} -> N;
-              undefined -> ?INITIAL_OUTGOING_ID + 1
+              undefined -> ?INITIAL_OUTGOING_TRANSFER_ID
           end,
+    RemoteIncomingWindow = diff(add(NII, InWin), OurNOI), % see: 2.5.6
     State#state{next_incoming_id = NOI,
-                remote_incoming_window = NII + InWin - OurNOI, % see: 2.5.6
+                remote_incoming_window = RemoteIncomingWindow,
                 remote_outgoing_window = OutWin}.
 
 
 -spec handle_link_flow(#'v1_0.flow'{}, #link{}) -> {ok | send_flow, #link{}}.
-handle_link_flow(#'v1_0.flow'{drain = true, link_credit = {uint, TheirCredit}},
+handle_link_flow(#'v1_0.flow'{drain = true,
+                              link_credit = {uint, TheirCredit}},
                  Link = #link{role = sender,
                               delivery_count = OurDC,
                               available = 0}) ->
     {send_flow, Link#link{link_credit = 0,
-                          delivery_count = OurDC + TheirCredit}};
+                          delivery_count = add(OurDC, TheirCredit)}};
 handle_link_flow(#'v1_0.flow'{delivery_count = MaybeTheirDC,
                               link_credit = {uint, TheirCredit}},
                  Link = #link{role = sender,
                               delivery_count = OurDC}) ->
     TheirDC = case MaybeTheirDC of
-                  undefined -> ?INITIAL_DELIVERY_COUNT;
-                  {uint, DC} -> DC
+                  {uint, DC} -> DC;
+                  undefined -> ?INITIAL_DELIVERY_COUNT
               end,
-    LinkCredit = TheirDC + TheirCredit - OurDC,
-
+    LinkCredit = diff(add(TheirDC, TheirCredit), OurDC),
     {ok, Link#link{link_credit = LinkCredit}};
 handle_link_flow(#'v1_0.flow'{delivery_count = TheirDC,
+                              link_credit = {uint, TheirCredit},
                               available = Available,
                               drain = Drain},
-                 Link = #link{role = receiver}) ->
-
-    {ok, Link#link{delivery_count = unpack(TheirDC),
-                   available = unpack(Available),
-                   drain = Drain}}.
+                 Link0 = #link{role = receiver}) ->
+    Link = case Drain andalso TheirCredit =< 0 of
+               true ->
+                   notify_credit_exhausted(Link0),
+                   Link0#link{delivery_count = unpack(TheirDC),
+                              link_credit = 0,
+                              available = unpack(Available),
+                              drain = Drain};
+               false ->
+                   Link0#link{delivery_count = unpack(TheirDC),
+                              available = unpack(Available),
+                              drain = Drain}
+           end,
+    {ok, Link}.
 
--spec find_link_by_input_handle(link_handle(), #state{}) ->
+-spec find_link_by_input_handle(input_handle(), #state{}) ->
     {ok, #link{}} | not_found.
 find_link_by_input_handle(InHandle, #state{link_handle_index = LHI,
                                            links = Links}) ->
@@ -825,8 +869,11 @@ with_link(InHandle, State, Fun) ->
             {next_state, end_sent, State}
     end.
 
+maybe_uint(undefined) -> undefined;
+maybe_uint(Int) -> uint(Int).
 
 uint(Int) -> {uint, Int}.
+
 unpack(X) -> amqp10_client_types:unpack(X).
 
 snd_settle_mode(#{snd_settle_mode := unsettled}) -> {ubyte, 0};
@@ -864,10 +911,12 @@ translate_delivery_state(received) -> #'v1_0.received'{}.
 translate_role(sender) -> false;
 translate_role(receiver) -> true.
 
-maybe_notify_link_credit(#link{link_credit = 0, role = sender},
-                         #link{link_credit = Credit} = Link)
-  when Credit > 0 ->
-    notify_link(Link, credited);
+maybe_notify_link_credit(#link{role = sender,
+                               link_credit = 0},
+                         #link{role = sender,
+                               link_credit = NewCredit} = NewLink)
+  when NewCredit > 0 ->
+    notify_link(NewLink, credited);
 maybe_notify_link_credit(_Old, _New) ->
     ok.
 
@@ -890,72 +939,70 @@ notify_session_ended(#state{notify = Pid}, Reason) ->
     Pid ! amqp10_session_event({ended, Reason}),
     ok.
 
-notify_disposition({Pid, _}, SessionDeliveryTag) ->
-    Pid ! {amqp10_disposition, SessionDeliveryTag},
+notify_disposition(Pid, DeliveryStateDeliveryTag) ->
+    Pid ! {amqp10_disposition, DeliveryStateDeliveryTag},
     ok.
 
 book_transfer_send(Num, #link{output_handle = Handle} = Link,
-                   #state{next_outgoing_id = NOI,
+                   #state{next_outgoing_id = NextOutgoingId,
+                          outgoing_delivery_id = DeliveryId,
                           remote_incoming_window = RIW,
                           links = Links} = State) ->
-    State#state{next_outgoing_id = NOI+Num,
+    State#state{next_outgoing_id = add(NextOutgoingId, Num),
+                outgoing_delivery_id = add(DeliveryId, 1),
                 remote_incoming_window = RIW-Num,
-                links = Links#{Handle => incr_link_counters(Link)}}.
+                links = Links#{Handle => book_link_transfer_send(Link)}}.
 
 book_partial_transfer_received(#state{next_incoming_id = NID,
                                       remote_outgoing_window = ROW} = State) ->
-    State#state{next_incoming_id = NID+1,
-                remote_outgoing_window = ROW-1}.
+    State#state{next_incoming_id = add(NID, 1),
+                remote_outgoing_window = ROW - 1}.
 
-book_transfer_received(_Settled,
-                       State = #state{connection_config =
+book_transfer_received(State = #state{connection_config =
                                       #{transfer_limit_margin := Margin}},
-                       #link{link_credit = Margin}) ->
-    {transfer_limit_exceeded, State};
-book_transfer_received(Settled,
-                       #state{next_incoming_id = NID,
+                       #link{link_credit = Margin} = Link) ->
+    {transfer_limit_exceeded, Link, State};
+book_transfer_received(#state{next_incoming_id = NID,
                               remote_outgoing_window = ROW,
                               links = Links} = State,
                        #link{output_handle = OutHandle,
                              delivery_count = DC,
                              link_credit = LC,
-                             link_credit_unsettled = LCU0} = Link) ->
-    LCU = case Settled of
-              true -> LCU0-1;
-              _ -> LCU0
-          end,
-
-    Link1 = Link#link{delivery_count = DC+1,
-                      link_credit = LC-1,
-                      link_credit_unsettled = LCU},
+                             available = Avail} = Link) ->
+    Link1 = Link#link{delivery_count = add(DC, 1),
+                      link_credit = LC - 1,
+                      %% "the receiver MUST maintain a floor of zero in its
+                      %% calculation of the value of available" [2.6.7]
+                      available = max(0, Avail - 1)},
     State1 = State#state{links = Links#{OutHandle => Link1},
-                         next_incoming_id = NID+1,
-                         remote_outgoing_window = ROW-1},
+                         next_incoming_id = add(NID, 1),
+                         remote_outgoing_window = ROW - 1},
     case Link1 of
         #link{link_credit = 0,
-              % only notify of credit exhaustion when
-              % not using auto flow.
               auto_flow = never} ->
-            {credit_exhausted, State1};
-        _ -> {ok, State1}
+            {credit_exhausted, Link1, State1};
+        _ ->
+            {ok, Link1, State1}
     end.
 
-auto_flow(#link{link_credit_unsettled = LCU,
-                auto_flow = {auto, Limit, Credit},
+auto_flow(#link{link_credit = LC,
+                auto_flow = {auto, RenewWhenBelow, Credit},
                 output_handle = OutHandle}, State)
-  when LCU =< Limit ->
-    send_flow(fun send/2, OutHandle,
-              #'v1_0.flow'{link_credit = {uint, Credit}},
-              Limit, State);
-auto_flow(_Link, State) ->
+  when LC < RenewWhenBelow ->
+    send_flow_link(fun send/2, OutHandle,
+                   #'v1_0.flow'{link_credit = {uint, Credit}},
+                   RenewWhenBelow, State);
+auto_flow(_, State) ->
     State.
 
 update_link(Link = #link{output_handle = OutHandle},
             State = #state{links = Links}) ->
             State#state{links = Links#{OutHandle => Link}}.
 
-incr_link_counters(#link{link_credit = LC, delivery_count = DC} = Link) ->
-    Link#link{delivery_count = DC+1, link_credit = LC-1}.
+book_link_transfer_send(Link = #link{link_credit = LC,
+                                     delivery_count = DC}) ->
+    Link#link{link_credit = LC - 1,
+              delivery_count = add(DC, 1)}.
 
 append_partial_transfer(Transfer, Payload,
                         #link{partial_transfers = undefined} = Link) ->
@@ -986,6 +1033,12 @@ socket_send(Sock, Data) ->
             throw({stop, normal})
     end.
 
+%% Only notify of credit exhaustion when not using auto flow.
+notify_credit_exhausted(Link = #link{auto_flow = never}) ->
+    ok = notify_link(Link, credit_exhausted);
+notify_credit_exhausted(_Link) ->
+    ok.
+
 -dialyzer({no_fail_call, socket_send0/2}).
 socket_send0({tcp, Socket}, Data) ->
     gen_tcp:send(Socket, Data);
@@ -1026,6 +1079,91 @@ sym(B) when is_binary(B) -> {symbol, B};
 sym(B) when is_list(B) -> {symbol, list_to_binary(B)};
 sym(B) when is_atom(B) -> {symbol, atom_to_binary(B, utf8)}.
 
+reason(undefined) -> normal;
+reason(Other) -> Other.
+
+format_status(Status = #{data := Data0}) ->
+    #state{channel = Channel,
+           remote_channel = RemoteChannel,
+           next_incoming_id = NextIncomingId,
+           incoming_window = IncomingWindow,
+           next_outgoing_id = NextOutgoingId,
+           remote_incoming_window = RemoteIncomingWindow,
+           remote_outgoing_window = RemoteOutgoingWindow,
+           reader = Reader,
+           socket = Socket,
+           links = Links0,
+           link_index = LinkIndex,
+           link_handle_index = LinkHandleIndex,
+           next_link_handle = NextLinkHandle,
+           early_attach_requests = EarlyAttachRequests,
+           connection_config = ConnectionConfig,
+           outgoing_delivery_id = OutgoingDeliveryId,
+           outgoing_unsettled = OutgoingUnsettled,
+           incoming_unsettled = IncomingUnsettled,
+           notify = Notify
+          } = Data0,
+    Links = maps:map(
+              fun(_OutputHandle,
+                  #link{name = Name,
+                        ref = Ref,
+                        state = State,
+                        notify = LinkNotify,
+                        output_handle = OutputHandle,
+                        input_handle = InputHandle,
+                        role = Role,
+                        target = Target,
+                        max_message_size = MaxMessageSize,
+                        delivery_count = DeliveryCount,
+                        link_credit = LinkCredit,
+                        available = Available,
+                        drain = Drain,
+                        partial_transfers = PartialTransfers0,
+                        auto_flow = AutoFlow
+                       }) ->
+                      PartialTransfers = case PartialTransfers0 of
+                                             undefined ->
+                                                 0;
+                                             {#'v1_0.transfer'{}, Binaries} ->
+                                                 length(Binaries)
+                                         end,
+                      #{name => Name,
+                        ref => Ref,
+                        state => State,
+                        notify => LinkNotify,
+                        output_handle => OutputHandle,
+                        input_handle => InputHandle,
+                        role => Role,
+                        target => Target,
+                        max_message_size => MaxMessageSize,
+                        delivery_count => DeliveryCount,
+                        link_credit => LinkCredit,
+                        available => Available,
+                        drain => Drain,
+                        partial_transfers => PartialTransfers,
+                        auto_flow => AutoFlow}
+              end, Links0),
+    Data = #{channel => Channel,
+             remote_channel => RemoteChannel,
+             next_incoming_id => NextIncomingId,
+             incoming_window => IncomingWindow,
+             next_outgoing_id => NextOutgoingId,
+             remote_incoming_window => RemoteIncomingWindow,
+             remote_outgoing_window => RemoteOutgoingWindow,
+             reader => Reader,
+             socket => Socket,
+             links => Links,
+             link_index => LinkIndex,
+             link_handle_index => LinkHandleIndex,
+             next_link_handle => NextLinkHandle,
+             early_attach_requests => length(EarlyAttachRequests),
+             connection_config => maps:remove(sasl, ConnectionConfig),
+             outgoing_delivery_id => OutgoingDeliveryId,
+             outgoing_unsettled => maps:size(OutgoingUnsettled),
+             incoming_unsettled => maps:size(IncomingUnsettled),
+             notify => Notify},
+    Status#{data := Data}.
+
 -ifdef(TEST).
 -include_lib("eunit/include/eunit.hrl").
 
@@ -1051,7 +1189,9 @@ handle_session_flow_pre_begin_test() ->
     State = handle_session_flow(Flow, State0),
     42 = State#state.next_incoming_id,
     2000 = State#state.remote_outgoing_window,
-    ?INITIAL_OUTGOING_ID + 1 + 1000 - 51 = State#state.remote_incoming_window.
+    % using serial number arithmetic:
+    % ?INITIAL_OUTGOING_TRANSFER_ID + 1000 = 998
+    ?assertEqual(998 - 51, State#state.remote_incoming_window).
 
 handle_link_flow_sender_test() ->
     Handle = 45,
@@ -1064,14 +1204,14 @@ handle_link_flow_sender_test() ->
                        },
     {ok, Outcome} = handle_link_flow(Flow, Link),
     % see section 2.6.7
-    Expected = DeliveryCount + 42 - (DeliveryCount + 2),
-    Expected = Outcome#link.link_credit,
+    ?assertEqual(DeliveryCount + 42 - (DeliveryCount + 2), Outcome#link.link_credit),
 
     % receiver does not yet know the delivery_count
     {ok, Outcome2} = handle_link_flow(Flow#'v1_0.flow'{delivery_count = undefined},
-                                Link),
-    Expected2 = ?INITIAL_DELIVERY_COUNT + 42 - (DeliveryCount + 2),
-    Expected2 = Outcome2#link.link_credit.
+                                      Link),
+    % using serial number arithmetic:
+    % ?INITIAL_DELIVERY_COUNT + 42 - (DeliveryCount + 2) = -18
+    ?assertEqual(-18, Outcome2#link.link_credit).
 
 handle_link_flow_sender_drain_test() ->
     Handle = 45,
@@ -1098,7 +1238,8 @@ handle_link_flow_receiver_test() ->
     Flow = #'v1_0.flow'{handle = {uint, Handle},
                         delivery_count = {uint, SenderDC},
                         available = 99,
-                        drain = true % what to do?
+                        drain = true, % what to do?
+                        link_credit = {uint, 0}
                        },
     {ok, Outcome} = handle_link_flow(Flow, Link),
     % see section 2.6.7
@@ -1148,7 +1289,8 @@ translate_filters_legacy_amqp_no_local_filter_test() ->
     {map,
         [{
             {symbol, <<"apache.org:no-local-filter:list">>},
-            {described, {symbol, <<"apache.org:no-local-filter:list">>}, [{utf8, <<"foo">>}, {utf8, <<"bar">>}]}
+            {described, {symbol, <<"apache.org:no-local-filter:list">>},
+             {list, [{utf8, <<"foo">>}, {utf8, <<"bar">>}]}}
         }]
     } = translate_filters(#{<<"apache.org:no-local-filter:list">> => [<<"foo">>, <<"bar">>]}).
 
diff --git a/deps/amqp10_client/src/amqp10_client_sessions_sup.erl b/deps/amqp10_client/src/amqp10_client_sessions_sup.erl
index 6255c53caae1..a3c4af737ba5 100644
--- a/deps/amqp10_client/src/amqp10_client_sessions_sup.erl
+++ b/deps/amqp10_client/src/amqp10_client_sessions_sup.erl
@@ -8,29 +8,20 @@
 
 -behaviour(supervisor).
 
-%% Private API.
+%% API
 -export([start_link/0]).
 
-%% Supervisor callbacks.
+%% Supervisor callbacks
 -export([init/1]).
 
--define(CHILD(Id, Mod, Type, Args), {Id, {Mod, start_link, Args},
-                                     transient, 5000, Type, [Mod]}).
-
-%% -------------------------------------------------------------------
-%% Private API.
-%% -------------------------------------------------------------------
-
--spec start_link() ->
-    {ok, pid()} | ignore | {error, any()}.
-
 start_link() ->
     supervisor:start_link(?MODULE, []).
 
-%% -------------------------------------------------------------------
-%% Supervisor callbacks.
-%% -------------------------------------------------------------------
-
-init(Args) ->
-    Template = ?CHILD(session, amqp10_client_session, worker, Args),
-    {ok, {{simple_one_for_one, 0, 1}, [Template]}}.
+init([]) ->
+    SupFlags = #{strategy => simple_one_for_one,
+                 intensity => 0,
+                 period => 1},
+    ChildSpec = #{id => session,
+                  start => {amqp10_client_session, start_link, []},
+                  restart => transient},
+    {ok, {SupFlags, [ChildSpec]}}.
diff --git a/deps/amqp10_client/src/amqp10_client_sup.erl b/deps/amqp10_client/src/amqp10_client_sup.erl
index 1e7f40c48c87..423aaa150f73 100644
--- a/deps/amqp10_client/src/amqp10_client_sup.erl
+++ b/deps/amqp10_client/src/amqp10_client_sup.erl
@@ -8,27 +8,21 @@
 
 -behaviour(supervisor).
 
-%% Private API.
+%% API
 -export([start_link/0]).
 
-%% Supervisor callbacks.
+%% Supervisor callbacks
 -export([init/1]).
 
--define(CHILD(Id, Mod, Type, Args), {Id, {Mod, start_link, Args},
-                                     temporary, infinity, Type, [Mod]}).
-
-%% -------------------------------------------------------------------
-%% Private API.
-%% -------------------------------------------------------------------
-
 start_link() ->
     supervisor:start_link({local, ?MODULE}, ?MODULE, []).
 
-%% -------------------------------------------------------------------
-%% Supervisor callbacks.
-%% -------------------------------------------------------------------
-
 init([]) ->
-    Template = ?CHILD(connection_sup, amqp10_client_connection_sup,
-                      supervisor, []),
-    {ok, {{simple_one_for_one, 0, 1}, [Template]}}.
+    SupFlags = #{strategy => simple_one_for_one,
+                 intensity => 0,
+                 period => 1},
+    ChildSpec = #{id => connection_sup,
+                  start => {amqp10_client_connection_sup, start_link, []},
+                  restart => temporary,
+                  type => supervisor},
+    {ok, {SupFlags, [ChildSpec]}}.
diff --git a/deps/amqp10_client/src/amqp10_client_types.erl b/deps/amqp10_client/src/amqp10_client_types.erl
index 3f333f2f2747..2a9859bcdf6c 100644
--- a/deps/amqp10_client/src/amqp10_client_types.erl
+++ b/deps/amqp10_client/src/amqp10_client_types.erl
@@ -10,7 +10,8 @@
 
 -export([unpack/1,
          utf8/1,
-         uint/1]).
+         uint/1,
+         make_properties/1]).
 
 -type amqp10_performative() :: #'v1_0.open'{} | #'v1_0.begin'{} | #'v1_0.attach'{} |
                                #'v1_0.flow'{} | #'v1_0.transfer'{} |
@@ -63,10 +64,13 @@
                                 link_event_detail()}.
 -type amqp10_event() :: {amqp10_event, amqp10_event_detail()}.
 
+-type properties() :: #{binary() => tuple()}.
+
 -export_type([amqp10_performative/0, channel/0,
               source/0, target/0, amqp10_msg_record/0,
               delivery_state/0, amqp_error/0, connection_error/0,
-              amqp10_event_detail/0, amqp10_event/0]).
+              amqp10_event_detail/0, amqp10_event/0,
+              properties/0]).
 
 
 unpack(undefined) -> undefined;
@@ -77,3 +81,12 @@ utf8(S) when is_list(S) -> {utf8, list_to_binary(S)};
 utf8(B) when is_binary(B) -> {utf8, B}.
 
 uint(N) -> {uint, N}.
+
+make_properties(#{properties := Props})
+  when is_map(Props) andalso
+       map_size(Props) > 0 ->
+    {map, maps:fold(fun(K, V, L) ->
+                            [{{symbol, K}, V} | L]
+                    end, [], Props)};
+make_properties(_) ->
+    undefined.
diff --git a/deps/amqp10_client/src/amqp10_msg.erl b/deps/amqp10_client/src/amqp10_msg.erl
index f08e747cef04..f356782f8ba7 100644
--- a/deps/amqp10_client/src/amqp10_msg.erl
+++ b/deps/amqp10_client/src/amqp10_msg.erl
@@ -38,7 +38,7 @@
 
 -include_lib("amqp10_common/include/amqp10_framing.hrl").
 
--type maybe(T) :: T | undefined.
+-type opt(T) :: T | undefined.
 
 -type delivery_tag() :: binary().
 -type content_type() :: term(). % TODO: refine
@@ -52,23 +52,23 @@
 
 -type amqp10_header() :: #{durable => boolean(), % false
                            priority => byte(), % 4
-                           ttl => maybe(non_neg_integer()),
+                           ttl => opt(non_neg_integer()),
                            first_acquirer => boolean(), % false
                            delivery_count => non_neg_integer()}. % 0
 
--type amqp10_properties() :: #{message_id => maybe(any()),
-                               user_id => maybe(binary()),
-                               to => maybe(any()),
-                               subject => maybe(binary()),
-                               reply_to => maybe(any()),
-                               correlation_id => maybe(any()),
-                               content_type => maybe(content_type()),
-                               content_encoding => maybe(content_encoding()),
-                               absolute_expiry_time => maybe(non_neg_integer()),
-                               creation_time => maybe(non_neg_integer()),
-                               group_id => maybe(binary()),
-                               group_sequence => maybe(non_neg_integer()),
-                               reply_to_group_id => maybe(binary())}.
+-type amqp10_properties() :: #{message_id => opt(any()),
+                               user_id => opt(binary()),
+                               to => opt(any()),
+                               subject => opt(binary()),
+                               reply_to => opt(any()),
+                               correlation_id => opt(any()),
+                               content_type => opt(content_type()),
+                               content_encoding => opt(content_encoding()),
+                               absolute_expiry_time => opt(non_neg_integer()),
+                               creation_time => opt(non_neg_integer()),
+                               group_id => opt(binary()),
+                               group_sequence => opt(non_neg_integer()),
+                               reply_to_group_id => opt(binary())}.
 
 -type amqp10_body() :: [#'v1_0.data'{}] |
                        [#'v1_0.amqp_sequence'{}] |
@@ -78,13 +78,13 @@
 
 -record(amqp10_msg,
         {transfer :: #'v1_0.transfer'{},
-         header :: maybe(#'v1_0.header'{}),
-         delivery_annotations :: maybe(#'v1_0.delivery_annotations'{}),
-         message_annotations :: maybe(#'v1_0.message_annotations'{}),
-         properties :: maybe(#'v1_0.properties'{}),
-         application_properties :: maybe(#'v1_0.application_properties'{}),
+         header :: opt(#'v1_0.header'{}),
+         delivery_annotations :: opt(#'v1_0.delivery_annotations'{}),
+         message_annotations :: opt(#'v1_0.message_annotations'{}),
+         properties :: opt(#'v1_0.properties'{}),
+         application_properties :: opt(#'v1_0.application_properties'{}),
          body :: amqp10_body() | unset,
-         footer :: maybe(#'v1_0.footer'{})
+         footer :: opt(#'v1_0.footer'{})
          }).
 
 -opaque amqp10_msg() :: #amqp10_msg{}.
@@ -142,7 +142,7 @@ settled(#amqp10_msg{transfer = #'v1_0.transfer'{settled = Settled}}) ->
 % the last 1 octet is the version
 % See 2.8.11 in the spec
 -spec message_format(amqp10_msg()) ->
-    maybe({non_neg_integer(), non_neg_integer()}).
+    opt({non_neg_integer(), non_neg_integer()}).
 message_format(#amqp10_msg{transfer =
                          #'v1_0.transfer'{message_format = undefined}}) ->
     undefined;
@@ -306,7 +306,7 @@ set_headers(Headers, #amqp10_msg{header = Current} = Msg) ->
     H = maps:fold(fun(durable, V, Acc) ->
                           Acc#'v1_0.header'{durable = V};
                      (priority, V, Acc) ->
-                          Acc#'v1_0.header'{priority = {uint, V}};
+                          Acc#'v1_0.header'{priority = {ubyte, V}};
                      (first_acquirer, V, Acc) ->
                           Acc#'v1_0.header'{first_acquirer = V};
                      (ttl, V, Acc) ->
@@ -325,8 +325,8 @@ set_properties(Props, #amqp10_msg{properties = Current} = Msg) ->
     P = maps:fold(fun(message_id, V, Acc) when is_binary(V) ->
                           % message_id can be any type but we restrict it here
                           Acc#'v1_0.properties'{message_id = utf8(V)};
-                     (user_id, V, Acc) ->
-                          Acc#'v1_0.properties'{user_id = utf8(V)};
+                     (user_id, V, Acc) when is_binary(V) ->
+                          Acc#'v1_0.properties'{user_id = {binary, V}};
                      (to, V, Acc) ->
                           Acc#'v1_0.properties'{to = utf8(V)};
                      (subject, V, Acc) ->
diff --git a/deps/amqp10_client/test/system_SUITE.erl b/deps/amqp10_client/test/system_SUITE.erl
index d37026841985..2147b0f156ce 100644
--- a/deps/amqp10_client/test/system_SUITE.erl
+++ b/deps/amqp10_client/test/system_SUITE.erl
@@ -14,21 +14,10 @@
 
 -include("src/amqp10_client.hrl").
 
--compile(export_all).
+-compile([export_all, nowarn_export_all]).
 
--define(UNAUTHORIZED_USER, <<"test_user_no_perm">>).
-
-%% The latch constant defines how many processes are spawned in order
-%% to run certain functionality in parallel. It follows the standard
-%% countdown latch pattern.
--define(LATCH, 100).
-
-%% The wait constant defines how long a consumer waits before it
-%% unsubscribes
--define(WAIT, 200).
-
-%% How to long wait for a process to die after an expected failure
--define(PROCESS_EXIT_TIMEOUT, 5000).
+suite() ->
+    [{timetrap, {seconds, 120}}].
 
 all() ->
     [
@@ -77,7 +66,8 @@ shared() ->
      subscribe,
      subscribe_with_auto_flow,
      outgoing_heartbeat,
-     roundtrip_large_messages
+     roundtrip_large_messages,
+     transfer_id_vs_delivery_id
     ].
 
 %% -------------------------------------------------------------------
@@ -112,17 +102,13 @@ stop_amqp10_client_app(Config) ->
 init_per_group(rabbitmq, Config0) ->
     Config = rabbit_ct_helpers:set_config(Config0,
                                           {sasl, {plain, <<"guest">>, <<"guest">>}}),
-    Config1 = rabbit_ct_helpers:merge_app_env(Config,
-                                              [{rabbitmq_amqp1_0,
-                                                [{protocol_strict_mode, true}]}]),
-    rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps());
+    rabbit_ct_helpers:run_steps(Config, rabbit_ct_broker_helpers:setup_steps());
 init_per_group(rabbitmq_strict, Config0) ->
     Config = rabbit_ct_helpers:set_config(Config0,
                                           {sasl, {plain, <<"guest">>, <<"guest">>}}),
     Config1 = rabbit_ct_helpers:merge_app_env(Config,
-                                              [{rabbitmq_amqp1_0,
-                                                [{default_user, none},
-                                                 {protocol_strict_mode, true}]}]),
+                                              [{rabbit,
+                                                [{amqp1_0_default_user, none}]}]),
     rabbit_ct_helpers:run_steps(Config1, rabbit_ct_broker_helpers:setup_steps());
 init_per_group(activemq, Config0) ->
     Config = rabbit_ct_helpers:set_config(Config0, {sasl, anon}),
@@ -309,9 +295,7 @@ roundtrip_large_messages(Config) ->
     Data1Mb = binary:copy(DataKb, 1024),
     roundtrip(OpenConf, Data1Mb),
     roundtrip(OpenConf, binary:copy(Data1Mb, 8)),
-    roundtrip(OpenConf, binary:copy(Data1Mb, 64)),
-    ok.
-
+    ok = roundtrip(OpenConf, binary:copy(Data1Mb, 64)).
 
 roundtrip(OpenConf) ->
     roundtrip(OpenConf, <<"banana">>).
@@ -319,39 +303,32 @@ roundtrip(OpenConf) ->
 roundtrip(OpenConf, Body) ->
     {ok, Connection} = amqp10_client:open_connection(OpenConf),
     {ok, Session} = amqp10_client:begin_session(Connection),
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    <<"banana-sender">>,
-                                                    <<"test1">>,
-                                                    settled,
-                                                    unsettled_state),
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"banana-sender">>, <<"test1">>, settled, unsettled_state),
     await_link(Sender, credited, link_credit_timeout),
 
     Now = os:system_time(millisecond),
     Props = #{creation_time => Now},
-    Msg0 =  amqp10_msg:set_properties(Props,
-                                      amqp10_msg:new(<<"my-tag">>, Body, true)),
-    Msg1 = amqp10_msg:set_application_properties(#{"a_key" => "a_value"}, Msg0),
-    Msg = amqp10_msg:set_message_annotations(#{<<"x_key">> => "x_value"}, Msg1),
-    % RabbitMQ AMQP 1.0 does not yet support delivery annotations
-    % Msg = amqp10_msg:set_delivery_annotations(#{<<"x_key">> => "x_value"}, Msg2),
+    Msg0 = amqp10_msg:new(<<"my-tag">>, Body, true),
+    Msg1 =  amqp10_msg:set_properties(Props, Msg0),
+    Msg2 = amqp10_msg:set_application_properties(#{"a_key" => "a_value"}, Msg1),
+    Msg3 = amqp10_msg:set_message_annotations(#{<<"x_key">> => "x_value"}, Msg2),
+    Msg = amqp10_msg:set_delivery_annotations(#{<<"y_key">> => "y_value"}, Msg3),
     ok = amqp10_client:send_msg(Sender, Msg),
     ok = amqp10_client:detach_link(Sender),
     await_link(Sender, {detached, normal}, link_detach_timeout),
 
     {error, link_not_found} = amqp10_client:detach_link(Sender),
-    {ok, Receiver} = amqp10_client:attach_receiver_link(Session,
-                                                        <<"banana-receiver">>,
-                                                        <<"test1">>,
-                                                        settled,
-                                                        unsettled_state),
-    {ok, OutMsg} = amqp10_client:get_msg(Receiver, 60000 * 5),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"banana-receiver">>, <<"test1">>, settled, unsettled_state),
+    {ok, OutMsg} = amqp10_client:get_msg(Receiver, 60_000 * 4),
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection),
     % ct:pal(?LOW_IMPORTANCE, "roundtrip message Out: ~tp~nIn: ~tp~n", [OutMsg, Msg]),
     #{creation_time := Now} = amqp10_msg:properties(OutMsg),
     #{<<"a_key">> := <<"a_value">>} = amqp10_msg:application_properties(OutMsg),
     #{<<"x_key">> := <<"x_value">>} = amqp10_msg:message_annotations(OutMsg),
-    % #{<<"x_key">> := <<"x_value">>} = amqp10_msg:delivery_annotations(OutMsg),
+    #{<<"y_key">> := <<"y_value">>} = amqp10_msg:delivery_annotations(OutMsg),
     ?assertEqual([Body], amqp10_msg:body(OutMsg)),
     ok.
 
@@ -379,7 +356,7 @@ filtered_roundtrip(OpenConf, Body) ->
                                                         settled,
                                                         unsettled_state),
     ok = amqp10_client:send_msg(Sender, Msg1),
-    {ok, OutMsg1} = amqp10_client:get_msg(DefaultReceiver, 60000 * 5),
+    {ok, OutMsg1} = amqp10_client:get_msg(DefaultReceiver, 60_000 * 4),
     ?assertEqual(<<"msg-1-tag">>, amqp10_msg:delivery_tag(OutMsg1)),
 
     timer:sleep(5 * 1000),
@@ -398,16 +375,52 @@ filtered_roundtrip(OpenConf, Body) ->
                                                         unsettled_state,
                                                         #{<<"apache.org:selector-filter:string">> => <<"amqp.annotation.x-opt-enqueuedtimeutc > ", Now2Binary/binary>>}),
 
-    {ok, OutMsg2} = amqp10_client:get_msg(DefaultReceiver, 60000 * 5),
+    {ok, OutMsg2} = amqp10_client:get_msg(DefaultReceiver, 60_000 * 4),
     ?assertEqual(<<"msg-2-tag">>, amqp10_msg:delivery_tag(OutMsg2)),
 
-    {ok, OutMsgFiltered} = amqp10_client:get_msg(FilteredReceiver, 60000 * 5),
+    {ok, OutMsgFiltered} = amqp10_client:get_msg(FilteredReceiver, 60_000 * 4),
     ?assertEqual(<<"msg-2-tag">>, amqp10_msg:delivery_tag(OutMsgFiltered)),
 
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection),
     ok.
 
+%% Assert that implementations respect the difference between transfer-id and delivery-id.
+transfer_id_vs_delivery_id(Config) ->
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    OpenConf = #{address => Hostname, port => Port, sasl => anon},
+
+    {ok, Connection} = amqp10_client:open_connection(OpenConf),
+    {ok, Session} = amqp10_client:begin_session(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"banana-sender">>, <<"test1">>, settled, unsettled_state),
+    await_link(Sender, credited, link_credit_timeout),
+
+    P0 = binary:copy(<<0>>, 8_000_000),
+    P1 = <>,
+    P2 = <>,
+    Msg1 = amqp10_msg:new(<<"tag 1">>, P1, true),
+    Msg2 = amqp10_msg:new(<<"tag 2">>, P2, true),
+    ok = amqp10_client:send_msg(Sender, Msg1),
+    ok = amqp10_client:send_msg(Sender, Msg2),
+    ok = amqp10_client:detach_link(Sender),
+    await_link(Sender, {detached, normal}, link_detach_timeout),
+
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"banana-receiver">>, <<"test1">>, settled, unsettled_state),
+    {ok, RcvMsg1} = amqp10_client:get_msg(Receiver, 60_000 * 4),
+    {ok, RcvMsg2} = amqp10_client:get_msg(Receiver, 60_000 * 4),
+    ok = amqp10_client:end_session(Session),
+    ok = amqp10_client:close_connection(Connection),
+
+    ?assertEqual([P1], amqp10_msg:body(RcvMsg1)),
+    ?assertEqual([P2], amqp10_msg:body(RcvMsg2)),
+    %% Despite many transfers, there were only 2 deliveries.
+    %% Therefore, delivery-id should have been increased by just 1.
+    ?assertEqual(serial_number:add(amqp10_msg:delivery_id(RcvMsg1), 1),
+                 amqp10_msg:delivery_id(RcvMsg2)).
+
 % a message is sent before the link attach is guaranteed to
 % have completed and link credit granted
 % also queue a link detached immediately after transfer
@@ -676,11 +689,13 @@ incoming_heartbeat(Config) ->
               idle_time_out => 1000, notify => self()},
     {ok, Connection} = amqp10_client:open_connection(CConf),
     receive
-        {amqp10_event, {connection, Connection,
-         {closed, {resource_limit_exceeded, <<"remote idle-time-out">>}}}} ->
+        {amqp10_event,
+         {connection, Connection0,
+          {closed, {resource_limit_exceeded, <<"remote idle-time-out">>}}}}
+          when Connection0 =:= Connection ->
             ok
     after 5000 ->
-          exit(incoming_heartbeat_assert)
+              exit(incoming_heartbeat_assert)
     end,
     demonitor(MockRef).
 
@@ -704,7 +719,8 @@ publish_messages(Sender, Data, Num) ->
 
 receive_one(Receiver) ->
     receive
-        {amqp10_msg, Receiver, Msg} ->
+        {amqp10_msg, Receiver0, Msg}
+          when Receiver0 =:= Receiver ->
             amqp10_client:accept_msg(Receiver, Msg)
     after 2000 ->
           timeout
@@ -712,7 +728,8 @@ receive_one(Receiver) ->
 
 await_disposition(DeliveryTag) ->
     receive
-        {amqp10_disposition, {accepted, DeliveryTag}} -> ok
+        {amqp10_disposition, {accepted, DeliveryTag0}}
+          when DeliveryTag0 =:= DeliveryTag -> ok
     after 3000 ->
               flush(),
               exit(dispostion_timeout)
@@ -720,9 +737,12 @@ await_disposition(DeliveryTag) ->
 
 await_link(Who, What, Err) ->
     receive
-        {amqp10_event, {link, Who, What}} ->
+        {amqp10_event, {link, Who0, What0}}
+          when Who0 =:= Who andalso
+               What0 =:= What ->
             ok;
-        {amqp10_event, {link, Who, {detached, Why}}} ->
+        {amqp10_event, {link, Who0, {detached, Why}}}
+          when Who0 =:= Who ->
             exit(Why)
     after 5000 ->
               flush(),
diff --git a/deps/amqp10_common/BUILD.bazel b/deps/amqp10_common/BUILD.bazel
index 898c539c16e8..0db9caf0e3c6 100644
--- a/deps/amqp10_common/BUILD.bazel
+++ b/deps/amqp10_common/BUILD.bazel
@@ -116,6 +116,11 @@ rabbitmq_suite(
     name = "binary_parser_SUITE",
 )
 
+rabbitmq_suite(
+    name = "serial_number_SUITE",
+    size = "small",
+)
+
 assert_suites()
 
 alias(
diff --git a/deps/amqp10_common/app.bzl b/deps/amqp10_common/app.bzl
index a83665c5c086..31d849c58528 100644
--- a/deps/amqp10_common/app.bzl
+++ b/deps/amqp10_common/app.bzl
@@ -13,6 +13,7 @@ def all_beam_files(name = "all_beam_files"):
             "src/amqp10_binary_parser.erl",
             "src/amqp10_framing.erl",
             "src/amqp10_framing0.erl",
+            "src/serial_number.erl",
         ],
         hdrs = [":public_and_private_hdrs"],
         app_name = "amqp10_common",
@@ -34,6 +35,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
             "src/amqp10_binary_parser.erl",
             "src/amqp10_framing.erl",
             "src/amqp10_framing0.erl",
+            "src/serial_number.erl",
         ],
         hdrs = [":public_and_private_hdrs"],
         app_name = "amqp10_common",
@@ -62,11 +64,12 @@ def all_srcs(name = "all_srcs"):
             "src/amqp10_binary_parser.erl",
             "src/amqp10_framing.erl",
             "src/amqp10_framing0.erl",
+            "src/serial_number.erl",
         ],
     )
     filegroup(
         name = "public_hdrs",
-        srcs = ["include/amqp10_framing.hrl"],
+        srcs = ["include/amqp10_framing.hrl", "include/amqp10_types.hrl"],
     )
     filegroup(
         name = "private_hdrs",
@@ -96,3 +99,11 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
         app_name = "amqp10_common",
         erlc_opts = "//:test_erlc_opts",
     )
+    erlang_bytecode(
+        name = "serial_number_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/serial_number_SUITE.erl"],
+        outs = ["test/serial_number_SUITE.beam"],
+        app_name = "amqp10_common",
+        erlc_opts = "//:test_erlc_opts",
+    )
diff --git a/deps/amqp10_common/include/amqp10_types.hrl b/deps/amqp10_common/include/amqp10_types.hrl
new file mode 100644
index 000000000000..550c2bc773f3
--- /dev/null
+++ b/deps/amqp10_common/include/amqp10_types.hrl
@@ -0,0 +1,12 @@
+-define(UINT_MAX, 16#ff_ff_ff_ff).
+
+% [1.6.5]
+-type uint() :: 0..?UINT_MAX.
+% [2.8.4]
+-type link_handle() :: uint().
+% [2.8.8]
+-type delivery_number() :: sequence_no().
+% [2.8.9]
+-type transfer_number() :: sequence_no().
+% [2.8.10]
+-type sequence_no() :: uint().
diff --git a/deps/amqp10_common/src/amqp10_binary_parser.erl b/deps/amqp10_common/src/amqp10_binary_parser.erl
index b4c4188b2862..b4259e0571f8 100644
--- a/deps/amqp10_common/src/amqp10_binary_parser.erl
+++ b/deps/amqp10_common/src/amqp10_binary_parser.erl
@@ -117,11 +117,8 @@ parse_compound(UnitSize, Bin) ->
 
 parse_compound1(0, <<>>, List) ->
     lists:reverse(List);
-parse_compound1(_Left, <<>>, List) ->
-    case application:get_env(rabbitmq_amqp1_0, protocol_strict_mode) of
-        {ok, false} -> lists:reverse(List); %% ignore miscount
-        {ok, true}  -> throw(compound_datatype_miscount)
-    end;
+parse_compound1(_Left, <<>>, _List) ->
+    throw(compound_datatype_miscount);
 parse_compound1(Count, Bin, Acc) ->
     {Value, Rest} = parse(Bin),
     parse_compound1(Count - 1, Rest, [Value | Acc]).
diff --git a/deps/amqp10_common/src/serial_number.erl b/deps/amqp10_common/src/serial_number.erl
new file mode 100644
index 000000000000..e3b6e900f875
--- /dev/null
+++ b/deps/amqp10_common/src/serial_number.erl
@@ -0,0 +1,118 @@
+%% 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-2023 VMware, Inc. or its affiliates.  All rights reserved.
+
+%% https://www.ietf.org/rfc/rfc1982.txt
+-module(serial_number).
+-include("amqp10_types.hrl").
+
+-export([add/2,
+         compare/2,
+         ranges/1,
+         diff/2,
+         foldl/4]).
+
+-ifdef(TEST).
+-export([usort/1]).
+-endif.
+
+-type serial_number() :: sequence_no().
+-export_type([serial_number/0]).
+
+%% SERIAL_BITS = 32
+%% 2 ^ SERIAL_BITS
+-define(SERIAL_SPACE, 16#100000000).
+%% 2 ^ (SERIAL_BITS - 1) - 1
+-define(SERIAL_MAX_ADDEND, 16#7fffffff).
+
+-spec add(serial_number(), non_neg_integer()) ->
+    serial_number().
+add(S, N)
+  when N >= 0 andalso
+       N =< ?SERIAL_MAX_ADDEND ->
+    (S + N) rem ?SERIAL_SPACE;
+add(S, N) ->
+    exit({undefined_serial_addition, S, N}).
+
+%% 2 ^ (SERIAL_BITS - 1)
+-define(COMPARE, 2_147_483_648).
+
+-spec compare(serial_number(), serial_number()) ->
+    equal | less | greater.
+compare(A, B) ->
+    if A =:= B ->
+           equal;
+       (A < B andalso B - A < ?COMPARE) orelse
+       (A > B andalso A - B > ?COMPARE) ->
+           less;
+       (A < B andalso B - A > ?COMPARE) orelse
+       (A > B andalso A - B < ?COMPARE) ->
+           greater;
+       true ->
+           exit({undefined_serial_comparison, A, B})
+    end.
+
+-spec usort([serial_number()]) ->
+    [serial_number()].
+usort(L) ->
+    lists:usort(fun(A, B) ->
+                        case compare(A, B) of
+                            greater -> false;
+                            _ -> true
+                        end
+                end, L).
+
+%% Takes a list of serial numbers and returns tuples
+%% {First, Last} representing contiguous serial numbers.
+-spec ranges([serial_number()]) ->
+    [{First :: serial_number(), Last :: serial_number()}].
+ranges([]) ->
+    [];
+ranges(SerialNumbers) ->
+    [First | Rest] = usort(SerialNumbers),
+    ranges0(Rest, [{First, First}]).
+
+ranges0([], Acc) ->
+    lists:reverse(Acc);
+ranges0([H | Rest], [{First, Last} | AccRest] = Acc0) ->
+    case add(Last, 1) of
+        H ->
+            Acc = [{First, H} | AccRest],
+            ranges0(Rest, Acc);
+        _ ->
+            Acc = [{H, H} | Acc0],
+            ranges0(Rest, Acc)
+    end.
+
+-define(SERIAL_DIFF_BOUND, 16#80000000).
+-spec diff(serial_number(), serial_number()) -> integer().
+diff(A, B) ->
+    Diff = A - B,
+    if Diff > (?SERIAL_DIFF_BOUND) ->
+           %% B is actually greater than A
+           - (?SERIAL_SPACE - Diff);
+       Diff < - (?SERIAL_DIFF_BOUND) ->
+           ?SERIAL_SPACE + Diff;
+       Diff < ?SERIAL_DIFF_BOUND andalso Diff > -?SERIAL_DIFF_BOUND ->
+           Diff;
+       true ->
+           exit({undefined_serial_diff, A, B})
+    end.
+
+-spec foldl(Fun, Acc0, First, Last) -> Acc1 when
+      Fun :: fun((serial_number(), AccIn) -> AccOut),
+                 Acc0 :: term(),
+                 Acc1 :: term(),
+                 AccIn :: term(),
+                 AccOut :: term(),
+                 First :: serial_number(),
+                 Last :: serial_number().
+
+foldl(Fun, Acc0, Current, Last) ->
+    Acc = Fun(Current, Acc0),
+    case compare(Current, Last) of
+        less -> foldl(Fun, Acc, add(Current, 1), Last);
+        equal -> Acc
+    end.
diff --git a/deps/amqp10_common/test/serial_number_SUITE.erl b/deps/amqp10_common/test/serial_number_SUITE.erl
new file mode 100644
index 000000000000..637b581b3973
--- /dev/null
+++ b/deps/amqp10_common/test/serial_number_SUITE.erl
@@ -0,0 +1,124 @@
+%% 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-2023 VMware, Inc. or its affiliates.  All rights reserved.
+%%
+
+-module(serial_number_SUITE).
+-include_lib("eunit/include/eunit.hrl").
+
+-compile([export_all,
+          nowarn_export_all]).
+
+-import(serial_number, [add/2,
+                        compare/2,
+                        usort/1,
+                        ranges/1,
+                        diff/2,
+                        foldl/4]).
+
+all() -> [test_add,
+          test_compare,
+          test_usort,
+          test_ranges,
+          test_diff,
+          test_foldl].
+
+test_add(_Config) ->
+    ?assertEqual(1, add(0, 1)),
+    %% "Addition of a value outside the range
+    %% [0 .. (2^(SERIAL_BITS - 1) - 1)] is undefined."
+    MaxAddend = round(math:pow(2, 32 - 1) - 1),
+    MinAddend = 0,
+    ?assertEqual(MaxAddend, add(0, MaxAddend)),
+    ?assertEqual(MinAddend, add(0, MinAddend)),
+    ?assertEqual(0, add(16#ffffffff, 1)),
+    ?assertEqual(1, add(16#ffffffff, 2)),
+    AddendTooLarge = MaxAddend + 1,
+    ?assertExit({undefined_serial_addition, 0, AddendTooLarge},
+                add(0, AddendTooLarge)),
+    AddendTooSmall = MinAddend - 1,
+    ?assertExit({undefined_serial_addition, 0, AddendTooSmall},
+                add(0, AddendTooSmall)).
+
+test_compare(_Config) ->
+    ?assertEqual(equal, compare(0, 0)),
+    ?assertEqual(equal, compare(16#ffffffff, 16#ffffffff)),
+    ?assertEqual(less, compare(0, 1)),
+    ?assertEqual(greater, compare(1, 0)),
+    ?assertEqual(less, compare(0, 2)),
+    ?assertEqual(less, compare(0, round(math:pow(2, 32 - 1)) - 1)),
+    ?assertExit({undefined_serial_comparison, 0, _},
+                compare(0, round(math:pow(2, 32 - 1)))),
+    ?assertEqual(less, compare(16#ffffffff - 5, 30_000)),
+    ?assertEqual(greater, compare(1, 0)),
+    ?assertEqual(greater, compare(2147483647, 0)),
+    ?assertExit({undefined_serial_comparison, 2147483648, 0},
+                compare(2147483648, 0)).
+
+test_usort(_Config) ->
+    ?assertEqual([],
+                 usort([])),
+    ?assertEqual([3],
+                 usort([3])),
+    ?assertEqual([0],
+                 usort([0, 0])),
+    ?assertEqual([4294967000, 4294967293, 4294967294, 4294967295, 0, 3, 4],
+                 usort([3, 4294967295, 4294967295, 4294967293, 4294967000, 4294967294, 0, 4])).
+
+test_ranges(_Config) ->
+    ?assertEqual([],
+                 ranges([])),
+    ?assertEqual([{0, 0}],
+                 ranges([0])),
+    ?assertEqual([{0, 1}],
+                 ranges([0, 1])),
+    ?assertEqual([{0, 1}],
+                 ranges([1, 0])),
+    ?assertEqual([{0, 0}, {2, 2}],
+                 ranges([0, 2])),
+    ?assertEqual([{0, 0}, {2, 2}],
+                 ranges([2, 0])),
+    %% 2 ^ 32 - 1 = 4294967295
+    ?assertEqual([{4294967290, 4294967290}, {4294967295, 4294967295}],
+                 ranges([4294967290, 4294967295])),
+    ?assertEqual([{4294967290, 4294967290}, {4294967295, 4294967295}],
+                 ranges([4294967295, 4294967290])),
+    ?assertEqual([{4294967294, 4294967294}, {0, 0}],
+                 ranges([4294967294, 0])),
+    ?assertEqual([{4294967294, 4294967294}, {0, 0}],
+                 ranges([0, 4294967294])),
+    ?assertEqual([{4294967295, 0}],
+                 ranges([4294967295, 0])),
+    ?assertEqual([{4294967294, 1}, {3, 5}, {10, 10}, {18, 19}],
+                 ranges([4294967294, 4294967295, 0, 1, 3, 4, 5, 10, 18, 19])),
+    ?assertEqual([{4294967294, 1}, {3, 5}, {10, 10}, {18, 19}],
+                 ranges([1, 10, 4294967294, 0, 3, 4, 5, 19, 18, 4294967295])).
+
+test_diff(_Config) ->
+    ?assertEqual(0, diff(0, 0)),
+    ?assertEqual(0, diff(1, 1)),
+    ?assertEqual(0, diff(16#ffffffff, 16#ffffffff)),
+    ?assertEqual(1, diff(1, 0)),
+    ?assertEqual(2, diff(1, 16#ffffffff)),
+    ?assertEqual(6, diff(0, 16#fffffffa)),
+    ?assertEqual(206, diff(200, 16#fffffffa)),
+    ?assertEqual(-2, diff(16#ffffffff, 1)),
+    ?assertExit({undefined_serial_diff, _, _},
+                diff(0, 16#80000000)),
+    ?assertExit({undefined_serial_diff, _, _},
+                diff(16#ffffffff, 16#7fffffff)).
+
+test_foldl(_Config) ->
+    ?assertEqual(
+       [16#ffffffff - 1, 16#ffffffff, 0, 1],
+       foldl(fun(S, Acc) ->
+                     Acc ++ [S]
+             end, [], 16#ffffffff - 1, 1)),
+
+    ?assertEqual(
+       [0],
+       foldl(fun(S, Acc) ->
+                     Acc ++ [S]
+             end, [], 0, 0)).
diff --git a/deps/amqp_client/src/amqp_channel.erl b/deps/amqp_client/src/amqp_channel.erl
index bc1588f06ff4..1f02d88fdb07 100644
--- a/deps/amqp_client/src/amqp_channel.erl
+++ b/deps/amqp_client/src/amqp_channel.erl
@@ -797,11 +797,6 @@ handle_method_from_server1(#'basic.nack'{} = BasicNack, none,
                            #state{confirm_handler = {CH, _Ref}} = State) ->
     CH ! BasicNack,
     {noreply, update_confirm_set(BasicNack, State)};
-
-handle_method_from_server1(#'basic.credit_drained'{} = CreditDrained, none,
-                           #state{consumer = Consumer} = State) ->
-    Consumer ! CreditDrained,
-    {noreply, State};
 handle_method_from_server1(Method, none, State) ->
     {noreply, rpc_bottom_half(Method, State)};
 handle_method_from_server1(Method, Content, State) ->
diff --git a/deps/amqp_client/src/amqp_selective_consumer.erl b/deps/amqp_client/src/amqp_selective_consumer.erl
index 7a99607750b5..205d260cd5d6 100644
--- a/deps/amqp_client/src/amqp_selective_consumer.erl
+++ b/deps/amqp_client/src/amqp_selective_consumer.erl
@@ -176,10 +176,7 @@ handle_info({'DOWN', _MRef, process, Pid, _Info},
                 _         -> {ok, State} %% unnamed consumer went down
                                          %% before receiving consume_ok
             end
-    end;
-handle_info(#'basic.credit_drained'{} = Method, State) ->
-    deliver_to_consumer_or_die(Method, Method, State),
-    {ok, State}.
+    end.
 
 %% @private
 handle_call({register_default_consumer, Pid}, _From,
@@ -246,8 +243,7 @@ tag(#'basic.consume'{consumer_tag = Tag})         -> Tag;
 tag(#'basic.consume_ok'{consumer_tag = Tag})      -> Tag;
 tag(#'basic.cancel'{consumer_tag = Tag})          -> Tag;
 tag(#'basic.cancel_ok'{consumer_tag = Tag})       -> Tag;
-tag(#'basic.deliver'{consumer_tag = Tag})         -> Tag;
-tag(#'basic.credit_drained'{consumer_tag = Tag})  -> Tag.
+tag(#'basic.deliver'{consumer_tag = Tag})         -> Tag.
 
 add_to_monitor_dict(Pid, Monitors) ->
     case maps:find(Pid, Monitors) of
diff --git a/deps/amqp_client/src/rabbit_routing_util.erl b/deps/amqp_client/src/rabbit_routing_util.erl
index c6747d330a09..6e7ddcb0b72b 100644
--- a/deps/amqp_client/src/rabbit_routing_util.erl
+++ b/deps/amqp_client/src/rabbit_routing_util.erl
@@ -9,8 +9,7 @@
 
 -export([init_state/0, dest_prefixes/0, all_dest_prefixes/0]).
 -export([ensure_endpoint/4, ensure_endpoint/5, ensure_binding/3]).
--export([parse_endpoint/1, parse_endpoint/2]).
--export([parse_routing/1, dest_temp_queue/1]).
+-export([dest_temp_queue/1]).
 
 -include("amqp_client.hrl").
 -include("rabbit_routing_prefixes.hrl").
@@ -26,50 +25,6 @@ all_dest_prefixes() -> [?TEMP_QUEUE_PREFIX | dest_prefixes()].
 
 %% --------------------------------------------------------------------------
 
-parse_endpoint(Destination) ->
-    parse_endpoint(Destination, false).
-
-parse_endpoint(undefined, AllowAnonymousQueue) ->
-    parse_endpoint("/queue", AllowAnonymousQueue);
-
-parse_endpoint(Destination, AllowAnonymousQueue) when is_binary(Destination) ->
-    parse_endpoint(unicode:characters_to_list(Destination),
-                                              AllowAnonymousQueue);
-parse_endpoint(Destination, AllowAnonymousQueue) when is_list(Destination) ->
-    case re:split(Destination, "/", [{return, list}]) of
-        [Name] ->
-            {ok, {queue, unescape(Name)}};
-        ["", Type | Rest]
-            when Type =:= "exchange" orelse Type =:= "queue" orelse
-                 Type =:= "topic"    orelse Type =:= "temp-queue" ->
-            parse_endpoint0(atomise(Type), Rest, AllowAnonymousQueue);
-        ["", "amq", "queue" | Rest] ->
-            parse_endpoint0(amqqueue, Rest, AllowAnonymousQueue);
-        ["", "reply-queue" = Prefix | [_|_]] ->
-            parse_endpoint0(reply_queue,
-                            [lists:nthtail(2 + length(Prefix), Destination)],
-                            AllowAnonymousQueue);
-        _ ->
-            {error, {unknown_destination, Destination}}
-    end.
-
-parse_endpoint0(exchange, ["" | _] = Rest,    _) ->
-    {error, {invalid_destination, exchange, to_url(Rest)}};
-parse_endpoint0(exchange, [Name],             _) ->
-    {ok, {exchange, {unescape(Name), undefined}}};
-parse_endpoint0(exchange, [Name, Pattern],    _) ->
-    {ok, {exchange, {unescape(Name), unescape(Pattern)}}};
-parse_endpoint0(queue,    [],                 false) ->
-    {error, {invalid_destination, queue, []}};
-parse_endpoint0(queue,    [],                 true) ->
-    {ok, {queue, undefined}};
-parse_endpoint0(Type,     [[_|_]] = [Name],   _) ->
-    {ok, {Type, unescape(Name)}};
-parse_endpoint0(Type,     Rest,               _) ->
-    {error, {invalid_destination, Type, to_url(Rest)}}.
-
-%% --------------------------------------------------------------------------
-
 ensure_endpoint(Dir, Channel, Endpoint, State) ->
     ensure_endpoint(Dir, Channel, Endpoint, [], State).
 
@@ -140,16 +95,6 @@ ensure_binding(Queue, {Exchange, RoutingKey}, Channel) ->
 
 %% --------------------------------------------------------------------------
 
-parse_routing({exchange, {Name, undefined}}) ->
-    {Name, ""};
-parse_routing({exchange, {Name, Pattern}}) ->
-    {Name, Pattern};
-parse_routing({topic, Name}) ->
-    {"amq.topic", Name};
-parse_routing({Type, Name})
-  when Type =:= queue orelse Type =:= reply_queue orelse Type =:= amqqueue ->
-    {"", Name}.
-
 dest_temp_queue({temp_queue, Name}) -> Name;
 dest_temp_queue(_)                  -> none.
 
@@ -206,17 +151,3 @@ queue_declare_method(#'queue.declare'{} = Method, Type, Params) ->
         _ ->
             Method2
     end.
-
-%% --------------------------------------------------------------------------
-
-to_url([])  -> [];
-to_url(Lol) -> "/" ++ string:join(Lol, "/").
-
-atomise(Name) when is_list(Name) ->
-    list_to_atom(re:replace(Name, "-", "_", [{return,list}, global])).
-
-unescape(Str) -> unescape(Str, []).
-
-unescape("%2F" ++ Str, Acc) -> unescape(Str, [$/ | Acc]);
-unescape([C | Str],    Acc) -> unescape(Str, [C | Acc]);
-unescape([],           Acc) -> lists:reverse(Acc).
diff --git a/deps/amqp_client/test/system_SUITE.erl b/deps/amqp_client/test/system_SUITE.erl
index 870b44036068..6df0d6c2ad8e 100644
--- a/deps/amqp_client/test/system_SUITE.erl
+++ b/deps/amqp_client/test/system_SUITE.erl
@@ -1342,9 +1342,9 @@ channel_writer_death(Config) ->
         Ret = amqp_channel:call(Channel, QoS),
         throw({unexpected_success, Ret})
     catch
-        exit:{{function_clause,
-               [{rabbit_channel, check_user_id_header, _, _} | _]}, _}
-        when ConnType =:= direct -> ok;
+        exit:{{{badrecord, <<>>},
+               [{rabbit_channel, _, _, _} | _]}, _}
+          when ConnType =:= direct -> ok;
 
         exit:{{infrastructure_died, {unknown_properties_record, <<>>}}, _}
         when ConnType =:= network -> ok
diff --git a/deps/amqp_client/test/unit_SUITE.erl b/deps/amqp_client/test/unit_SUITE.erl
index 5504c2abecfc..5aee3bb18664 100644
--- a/deps/amqp_client/test/unit_SUITE.erl
+++ b/deps/amqp_client/test/unit_SUITE.erl
@@ -322,9 +322,9 @@ route_destination_parsing(_Config) ->
     ok.
 
 parse_dest(Destination, Params) ->
-    rabbit_routing_util:parse_endpoint(Destination, Params).
+    rabbit_routing_parser:parse_endpoint(Destination, Params).
 parse_dest(Destination) ->
-    rabbit_routing_util:parse_endpoint(Destination).
+    rabbit_routing_parser:parse_endpoint(Destination).
 
 %% -------------------------------------------------------------------
 %% Topic variable map
diff --git a/deps/rabbit/.gitignore b/deps/rabbit/.gitignore
index dc870136e8ec..6a891df212d9 100644
--- a/deps/rabbit/.gitignore
+++ b/deps/rabbit/.gitignore
@@ -40,3 +40,6 @@ callgraph.dot*
 PACKAGES/*
 
 rabbit-rabbitmq-deps.mk
+
+[Bb]in/
+[Oo]bj/
diff --git a/deps/rabbit/BUILD.bazel b/deps/rabbit/BUILD.bazel
index 3bf4158a6eb3..be46052ea417 100644
--- a/deps/rabbit/BUILD.bazel
+++ b/deps/rabbit/BUILD.bazel
@@ -59,6 +59,8 @@ _APP_ENV = """[
 	    {default_user_tags, [administrator]},
 	    {default_vhost, <<"/">>},
 	    {default_permissions, [<<".*">>, <<".*">>, <<".*">>]},
+	    {amqp1_0_default_user, <<"guest">>},
+	    {amqp1_0_default_vhost, <<"/">>},
 	    {loopback_users, [<<"guest">>]},
 	    {password_hashing_module, rabbit_password_hashing_sha256},
 	    {server_properties, []},
@@ -234,6 +236,9 @@ rabbitmq_app(
 
 xref(
     name = "xref",
+    additional_libs = [
+        "//deps/rabbitmq_cli:erlang_app",  # keep
+    ],
     target = ":erlang_app",
 )
 
@@ -245,8 +250,10 @@ plt(
     ],
     for_target = ":erlang_app",
     ignore_warnings = True,
+    libs = ["//deps/rabbitmq_cli:elixir"],  # keep
     plt = "//:base_plt",
     deps = [
+        "//deps/rabbitmq_cli:erlang_app",  # keep
         "@looking_glass//:erlang_app",  # keep
     ],
 )
@@ -273,6 +280,7 @@ rabbitmq_home(
     plugins = [
         ":test_erlang_app",
         "//deps/rabbitmq_ct_client_helpers:erlang_app",
+        "//deps/rabbitmq_amqp1_0:erlang_app",
         "@inet_tcp_proxy_dist//:erlang_app",
         "@meck//:erlang_app",
     ],
@@ -1236,6 +1244,52 @@ rabbitmq_integration_suite(
     ],
 )
 
+rabbitmq_integration_suite(
+    name = "amqp_client_SUITE",
+    size = "large",
+    additional_beam = [
+        ":test_event_recorder_beam",
+    ],
+    shard_count = 3,
+    runtime_deps = [
+        "//deps/amqp10_client:erlang_app",
+    ],
+)
+
+rabbitmq_integration_suite(
+    name = "amqp_proxy_protocol_SUITE",
+    size = "medium",
+)
+
+rabbitmq_integration_suite(
+    name = "amqp_system_SUITE",
+    flaky = True,
+    shard_count = 2,
+    tags = [
+        "dotnet",
+    ],
+    test_env = {
+        "TMPDIR": "$TEST_TMPDIR",
+    },
+)
+
+rabbitmq_integration_suite(
+    name = "amqp_auth_SUITE",
+    additional_beam = [
+        ":test_event_recorder_beam",
+    ],
+    runtime_deps = [
+        "//deps/amqp10_client:erlang_app",
+    ],
+)
+
+rabbitmq_integration_suite(
+    name = "amqp_credit_api_v2_SUITE",
+    runtime_deps = [
+        "//deps/amqp10_client:erlang_app",
+    ],
+)
+
 assert_suites()
 
 filegroup(
@@ -1332,6 +1386,7 @@ eunit(
         ":test_test_util_beam",
         ":test_test_rabbit_event_handler_beam",
         ":test_clustering_utils_beam",
+        ":test_event_recorder_beam",
     ],
     target = ":test_erlang_app",
     test_env = {
diff --git a/deps/rabbit/Makefile b/deps/rabbit/Makefile
index 6a7e12216fd1..510e938cc9ab 100644
--- a/deps/rabbit/Makefile
+++ b/deps/rabbit/Makefile
@@ -39,6 +39,8 @@ define PROJECT_ENV
 	    {default_user_tags, [administrator]},
 	    {default_vhost, <<"/">>},
 	    {default_permissions, [<<".*">>, <<".*">>, <<".*">>]},
+	    {amqp1_0_default_user, <<"guest">>},
+	    {amqp1_0_default_vhost, <<"/">>},
 	    {loopback_users, [<<"guest">>]},
 	    {password_hashing_module, rabbit_password_hashing_sha256},
 	    {server_properties, []},
@@ -133,8 +135,8 @@ endef
 LOCAL_DEPS = sasl os_mon inets compiler public_key crypto ssl syntax_tools xmerl
 
 BUILD_DEPS = rabbitmq_cli
-DEPS = ranch rabbit_common rabbitmq_prelaunch ra sysmon_handler stdout_formatter recon redbug observer_cli osiris amqp10_common syslog systemd seshat khepri khepri_mnesia_migration
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client meck proper
+DEPS = ranch rabbit_common amqp10_common rabbitmq_prelaunch ra sysmon_handler stdout_formatter recon redbug observer_cli osiris syslog systemd seshat khepri khepri_mnesia_migration
+TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers meck proper amqp_client amqp10_client rabbitmq_amqp1_0
 
 PLT_APPS += mnesia
 
diff --git a/deps/rabbit/app.bzl b/deps/rabbit/app.bzl
index bf09ac25664b..440567fc64f1 100644
--- a/deps/rabbit/app.bzl
+++ b/deps/rabbit/app.bzl
@@ -24,7 +24,7 @@ def all_beam_files(name = "all_beam_files"):
         app_name = "rabbit",
         dest = "ebin",
         erlc_opts = "//:erlc_opts",
-        deps = ["//deps/rabbit_common:erlang_app"],
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app"],
     )
     erlang_bytecode(
         name = "other_beam",
@@ -46,6 +46,12 @@ def all_beam_files(name = "all_beam_files"):
             "src/rabbit.erl",
             "src/rabbit_access_control.erl",
             "src/rabbit_alarm.erl",
+            "src/rabbit_amqp1_0.erl",
+            "src/rabbit_amqp_reader.erl",
+            "src/rabbit_amqp_session.erl",
+            "src/rabbit_amqp_session_sup.erl",
+            "src/rabbit_amqp_util.erl",
+            "src/rabbit_amqp_writer.erl",
             "src/rabbit_amqqueue.erl",
             "src/rabbit_amqqueue_control.erl",
             "src/rabbit_amqqueue_process.erl",
@@ -286,7 +292,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
         app_name = "rabbit",
         dest = "test",
         erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/rabbit_common:erlang_app"],
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/rabbit_common:erlang_app"],
     )
     erlang_bytecode(
         name = "test_other_beam",
@@ -309,6 +315,12 @@ def all_test_beam_files(name = "all_test_beam_files"):
             "src/rabbit.erl",
             "src/rabbit_access_control.erl",
             "src/rabbit_alarm.erl",
+            "src/rabbit_amqp1_0.erl",
+            "src/rabbit_amqp_reader.erl",
+            "src/rabbit_amqp_session.erl",
+            "src/rabbit_amqp_session_sup.erl",
+            "src/rabbit_amqp_util.erl",
+            "src/rabbit_amqp_writer.erl",
             "src/rabbit_amqqueue.erl",
             "src/rabbit_amqqueue_control.erl",
             "src/rabbit_amqqueue_process.erl",
@@ -541,6 +553,7 @@ def all_srcs(name = "all_srcs"):
             "include/gm_specs.hrl",
             "include/internal_user.hrl",
             "include/mc.hrl",
+            "include/rabbit_amqp.hrl",
             "include/rabbit_global_counters.hrl",
             "include/vhost.hrl",
             "include/vhost_v2.hrl",
@@ -586,6 +599,12 @@ def all_srcs(name = "all_srcs"):
             "src/rabbit.erl",
             "src/rabbit_access_control.erl",
             "src/rabbit_alarm.erl",
+            "src/rabbit_amqp1_0.erl",
+            "src/rabbit_amqp_reader.erl",
+            "src/rabbit_amqp_session.erl",
+            "src/rabbit_amqp_session_sup.erl",
+            "src/rabbit_amqp_util.erl",
+            "src/rabbit_amqp_writer.erl",
             "src/rabbit_amqqueue.erl",
             "src/rabbit_amqqueue_control.erl",
             "src/rabbit_amqqueue_process.erl",
@@ -2147,3 +2166,57 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
         erlc_opts = "//:test_erlc_opts",
         deps = ["//deps/amqp_client:erlang_app"],
     )
+
+    erlang_bytecode(
+        name = "test_event_recorder_beam",
+        testonly = True,
+        srcs = ["test/event_recorder.erl"],
+        outs = ["test/event_recorder.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/rabbit_common:erlang_app"],
+    )
+    erlang_bytecode(
+        name = "amqp_auth_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/amqp_auth_SUITE.erl"],
+        outs = ["test/amqp_auth_SUITE.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/amqp_client:erlang_app"],
+    )
+    erlang_bytecode(
+        name = "amqp_client_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/amqp_client_SUITE.erl"],
+        outs = ["test/amqp_client_SUITE.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/amqp_client:erlang_app"],
+    )
+    erlang_bytecode(
+        name = "amqp_credit_api_v2_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/amqp_credit_api_v2_SUITE.erl"],
+        outs = ["test/amqp_credit_api_v2_SUITE.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/amqp_client:erlang_app"],
+    )
+    erlang_bytecode(
+        name = "amqp_proxy_protocol_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/amqp_proxy_protocol_SUITE.erl"],
+        outs = ["test/amqp_proxy_protocol_SUITE.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+    )
+    erlang_bytecode(
+        name = "amqp_system_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/amqp_system_SUITE.erl"],
+        outs = ["test/amqp_system_SUITE.beam"],
+        app_name = "rabbit",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/rabbit_common:erlang_app"],
+    )
diff --git a/deps/rabbit/docs/rabbitmq.conf.example b/deps/rabbit/docs/rabbitmq.conf.example
index 1ff9145f8351..569dd06d2a0b 100644
--- a/deps/rabbit/docs/rabbitmq.conf.example
+++ b/deps/rabbit/docs/rabbitmq.conf.example
@@ -954,10 +954,6 @@
 ##
 # amqp1_0.default_user = guest
 
-## Enable protocol strict mode. See the README for more information.
-##
-# amqp1_0.protocol_strict_mode = false
-
 ## Logging settings.
 ##
 ## See https://rabbitmq.com/logging.html for details.
diff --git a/deps/rabbit/include/rabbit_amqp.hrl b/deps/rabbit/include/rabbit_amqp.hrl
new file mode 100644
index 000000000000..282cad9d4e47
--- /dev/null
+++ b/deps/rabbit/include/rabbit_amqp.hrl
@@ -0,0 +1,74 @@
+%%-define(debug, true).
+
+-ifdef(debug).
+-define(DEBUG0(F), ?SAFE(rabbit_log:debug(F, []))).
+-define(DEBUG(F, A), ?SAFE(rabbit_log:debug(F, A))).
+-else.
+-define(DEBUG0(F), ok).
+-define(DEBUG(F, A), ok).
+-endif.
+
+-define(pprint(F), rabbit_log:debug("~p~n",
+                                    [amqp10_framing:pprint(F)])).
+
+-define(SAFE(F),
+        ((fun() ->
+                  try F
+                  catch __T:__E:__ST ->
+                            rabbit_log:debug("~p:~p thrown debugging~n~p~n",
+                                             [__T, __E, __ST])
+                  end
+          end)())).
+
+%% General consts
+
+%% [2.8.19]
+-define(MIN_MAX_FRAME_1_0_SIZE, 512).
+
+-define(SEND_ROLE, false).
+-define(RECV_ROLE, true).
+
+%% for rabbit_event user_authentication_success and user_authentication_failure
+-define(AUTH_EVENT_KEYS,
+        [name,
+         host,
+         port,
+         peer_host,
+         peer_port,
+         protocol,
+         auth_mechanism,
+         ssl,
+         ssl_protocol,
+         ssl_key_exchange,
+         ssl_cipher,
+         ssl_hash,
+         peer_cert_issuer,
+         peer_cert_subject,
+         peer_cert_validity]).
+
+-define(ITEMS,
+        [pid,
+         frame_max,
+         timeout,
+         vhost,
+         user,
+         node
+        ] ++ ?AUTH_EVENT_KEYS).
+
+-define(INFO_ITEMS,
+        [connection_state,
+         recv_oct,
+         recv_cnt,
+         send_oct,
+         send_cnt
+        ] ++ ?ITEMS).
+
+%% for rabbit_event connection_created
+-define(CONNECTION_EVENT_KEYS,
+        [type,
+         client_properties,
+         connected_at,
+         channel_max
+        ] ++ ?ITEMS).
+
+-include_lib("amqp10_common/include/amqp10_framing.hrl").
diff --git a/deps/rabbit/include/rabbit_global_counters.hrl b/deps/rabbit/include/rabbit_global_counters.hrl
index ae19ce4f0909..023ac44be2c9 100644
--- a/deps/rabbit/include/rabbit_global_counters.hrl
+++ b/deps/rabbit/include/rabbit_global_counters.hrl
@@ -1,5 +1,4 @@
 -define(NUM_PROTOCOL_COUNTERS, 8).
--define(NUM_PROTOCOL_QUEUE_TYPE_COUNTERS, 8).
 
 %% Dead Letter counters:
 %%
diff --git a/deps/rabbit/priv/schema/rabbit.schema b/deps/rabbit/priv/schema/rabbit.schema
index 9e32992a0c48..c271eb751cd6 100644
--- a/deps/rabbit/priv/schema/rabbit.schema
+++ b/deps/rabbit/priv/schema/rabbit.schema
@@ -2586,6 +2586,33 @@ end}.
     end
 }.
 
+% ===============================
+% AMQP 1.0
+% ===============================
+
+%% Connections that skip SASL layer or use SASL mechanism ANONYMOUS will connect as this account.
+%% Setting this to a username will allow clients to connect without authenticating.
+%% For production environments, set this value to 'none'.
+{mapping, "amqp1_0.default_user", "rabbit.amqp1_0_default_user",
+    [{datatype, [{enum, [none]}, string]}]}.
+
+{mapping, "amqp1_0.default_vhost", "rabbit.amqp1_0_default_vhost",
+    [{datatype, string}]}.
+
+{translation, "rabbit.amqp1_0_default_user",
+fun(Conf) ->
+        case cuttlefish:conf_get("amqp1_0.default_user", Conf) of
+            none -> none;
+            User -> list_to_binary(User)
+        end
+end}.
+
+{translation , "rabbit.amqp1_0_default_vhost",
+fun(Conf) ->
+    list_to_binary(cuttlefish:conf_get("amqp1_0.default_vhost", Conf))
+end}.
+
+
 % ===============================
 % Validators
 % ===============================
diff --git a/deps/rabbit/src/mc.erl b/deps/rabbit/src/mc.erl
index 0e76ed4476d9..2420baadc9ee 100644
--- a/deps/rabbit/src/mc.erl
+++ b/deps/rabbit/src/mc.erl
@@ -19,6 +19,7 @@
          is_persistent/1,
          ttl/1,
          correlation_id/1,
+         user_id/1,
          message_id/1,
          timestamp/1,
          priority/1,
@@ -280,6 +281,15 @@ correlation_id(#?MODULE{protocol = Proto,
 correlation_id(BasicMsg) ->
     mc_compat:correlation_id(BasicMsg).
 
+-spec user_id(state()) ->
+    {binary, rabbit_types:username()} |
+    undefined.
+user_id(#?MODULE{protocol = Proto,
+                 data = Data}) ->
+    Proto:property(?FUNCTION_NAME, Data);
+user_id(BasicMsg) ->
+    mc_compat:user_id(BasicMsg).
+
 -spec message_id(state()) ->
     {uuid, binary()} |
     {utf8, binary()} |
diff --git a/deps/rabbit/src/mc_amqp.erl b/deps/rabbit/src/mc_amqp.erl
index 16e25f14d135..24a2d4bd12e2 100644
--- a/deps/rabbit/src/mc_amqp.erl
+++ b/deps/rabbit/src/mc_amqp.erl
@@ -58,14 +58,32 @@
               message_section/0
              ]).
 
-%% mc implementation
+%% TODO
+%% Up to 3.13 the parsed AMQP 1.0 message is never stored on disk.
+%% We want that to hold true for 4.0 as well to save disk space and disk I/O.
+%%
+%% As the essential annotations, durable, priority, ttl and delivery_count
+%% is all we are interested in it isn't necessary to keep hold of the
+%% incoming AMQP header inside the state
+%%
+%% Probably prepare(store, Msg) should serialize the message.
+%% mc:prepare(store, Msg) should also be called from rabbit_stream_queue after converting to mc_amqp.
+%%
+%% When we received the message via AMQP 1.0, our mc_amqp:state() should ideally store a binary of each section.
+%% This way, prepare(store, Msg) wouldn't need to serialize anything because there shouldn't be any changes
+%% in the sections between receiving via AMQP 1.0 and storing the message in queues.
+%%
+%% Also, we don't need to parse each section.
+%% For example, apart from validation we wouldn’t need to parse application properties at all - unless requested by the headers exchange.
+%% Ideally the parser could have a validate mode, that validated the section(s) but didn’t build up an erlang term representation of the data.
+%% Such a validation mode could be used for application properties. Message annotations might not need to be parsed either.
+%% So, message annotations and application properties should be parsed lazily, only if needed.
+%%
+%% Upon sending the message to clients, when converting from AMQP 1.0, the serialized message needs to be parsed into sections.
 init(Sections) when is_list(Sections) ->
     Msg = decode(Sections, #msg{}),
     init(Msg);
 init(#msg{} = Msg) ->
-    %% TODO: as the essential annotations, durable, priority, ttl and delivery_count
-    %% is all we are interested in it isn't necessary to keep hold of the
-    %% incoming AMQP header inside the state
     Anns = essential_properties(Msg),
     {Msg, Anns}.
 
@@ -95,6 +113,8 @@ property(correlation_id, #msg{properties = #'v1_0.properties'{correlation_id = C
     Corr;
 property(message_id, #msg{properties = #'v1_0.properties'{message_id = MsgId}}) ->
     MsgId;
+property(user_id, #msg{properties = #'v1_0.properties'{user_id = UserId}}) ->
+    UserId;
 property(_Prop, #msg{}) ->
     undefined.
 
@@ -134,7 +154,7 @@ get_property(timestamp, Msg) ->
     end;
 get_property(ttl, Msg) ->
     case Msg of
-        #msg{header = #'v1_0.header'{ttl = {_, Ttl}}} ->
+        #msg{header = #'v1_0.header'{ttl = {uint, Ttl}}} ->
             Ttl;
         _ ->
             %% fallback in case the source protocol was AMQP 0.9.1
@@ -158,6 +178,13 @@ get_property(priority, Msg) ->
                 _ ->
                     undefined
             end
+    end;
+get_property(subject, Msg) ->
+    case Msg of
+        #msg{properties = #'v1_0.properties'{subject = {utf8, Subject}}} ->
+            Subject;
+        _ ->
+            undefined
     end.
 
 convert_to(?MODULE, Msg, _Env) ->
@@ -170,10 +197,19 @@ convert_to(TargetProto, Msg, Env) ->
 serialize(Sections) ->
     encode_bin(Sections).
 
-protocol_state(Msg, Anns) ->
+protocol_state(Msg0 = #msg{header = Header0}, Anns) ->
+    Redelivered = maps:get(redelivered, Anns, false),
+    FirstAcquirer = not Redelivered,
+    Header = case Header0 of
+                 undefined ->
+                     #'v1_0.header'{first_acquirer = FirstAcquirer};
+                 #'v1_0.header'{} ->
+                     Header0#'v1_0.header'{first_acquirer = FirstAcquirer}
+             end,
+    Msg = Msg0#msg{header = Header},
+
     #{?ANN_EXCHANGE := Exchange,
       ?ANN_ROUTING_KEYS := [RKey | _]} = Anns,
-
     %% any x-* annotations get added as message annotations
     AnnsToAdd = maps:filter(fun (Key, _) -> mc_util:is_x_header(Key) end, Anns),
 
@@ -394,6 +430,10 @@ essential_properties(#msg{message_annotations = MA} = Msg) ->
     Priority = get_property(priority, Msg),
     Timestamp = get_property(timestamp, Msg),
     Ttl = get_property(ttl, Msg),
+    RoutingKeys = case get_property(subject, Msg) of
+                      undefined -> undefined;
+                      Subject -> [Subject]
+                  end,
 
     Deaths = case message_annotation(<<"x-death">>, Msg, undefined) of
                  {list, DeathMaps}  ->
@@ -418,8 +458,10 @@ essential_properties(#msg{message_annotations = MA} = Msg) ->
                  maps_put_truthy(
                    ttl, Ttl,
                    maps_put_truthy(
-                     deaths, Deaths,
-                     #{}))))),
+                     ?ANN_ROUTING_KEYS, RoutingKeys,
+                     maps_put_truthy(
+                       deaths, Deaths,
+                       #{})))))),
     case MA of
         [] ->
             Anns;
diff --git a/deps/rabbit/src/mc_amqpl.erl b/deps/rabbit/src/mc_amqpl.erl
index 5a6a65a3b5fb..678345998d50 100644
--- a/deps/rabbit/src/mc_amqpl.erl
+++ b/deps/rabbit/src/mc_amqpl.erl
@@ -25,7 +25,8 @@
          message/3,
          message/4,
          message/5,
-         from_basic_message/1
+         from_basic_message/1,
+         to_091/2
         ]).
 
 -import(rabbit_misc,
diff --git a/deps/rabbit/src/mc_compat.erl b/deps/rabbit/src/mc_compat.erl
index 6598ce7812aa..8a2a407ceaae 100644
--- a/deps/rabbit/src/mc_compat.erl
+++ b/deps/rabbit/src/mc_compat.erl
@@ -14,6 +14,7 @@
          is_persistent/1,
          ttl/1,
          correlation_id/1,
+         user_id/1,
          message_id/1,
          timestamp/1,
          priority/1,
@@ -106,6 +107,9 @@ timestamp(#basic_message{content = Content}) ->
 priority(#basic_message{content = Content}) ->
     get_property(?FUNCTION_NAME, Content).
 
+user_id(#basic_message{content = Content}) ->
+    get_property(?FUNCTION_NAME, Content).
+
 correlation_id(#basic_message{content = Content}) ->
     case get_property(?FUNCTION_NAME, Content) of
         undefined ->
@@ -384,6 +388,13 @@ get_property(P, #content{properties = none} = Content) ->
 get_property(durable,
              #content{properties = #'P_basic'{delivery_mode = Mode}}) ->
     Mode == 2;
+get_property(user_id,
+             #content{properties = #'P_basic'{user_id = UserId}}) ->
+    if UserId =:= undefined ->
+           undefined;
+       is_binary(UserId) ->
+           {binary, UserId}
+    end;
 get_property(ttl, #content{properties = Props}) ->
     {ok, MsgTTL} = rabbit_basic:parse_expiration(Props),
     MsgTTL;
diff --git a/deps/rabbit/src/rabbit.erl b/deps/rabbit/src/rabbit.erl
index 6b9e465e87d5..7cbdf687168f 100644
--- a/deps/rabbit/src/rabbit.erl
+++ b/deps/rabbit/src/rabbit.erl
@@ -36,7 +36,7 @@
 
 %%---------------------------------------------------------------------------
 %% Boot steps.
--export([maybe_insert_default_data/0, boot_delegate/0, recover/0]).
+-export([maybe_insert_default_data/0, boot_delegate/0, recover/0, pg_local/0]).
 
 %% for tests
 -export([validate_msg_store_io_batch_size_and_credit_disc_bound/2]).
@@ -267,6 +267,12 @@
                     {mfa,         {logger, debug, ["'networking' boot step skipped and moved to end of startup", [], #{domain => ?RMQLOG_DOMAIN_GLOBAL}]}},
                     {requires,    notify_cluster}]}).
 
+-rabbit_boot_step({pg_local,
+                   [{description, "local-only pg scope"},
+                    {mfa,         {rabbit, pg_local, []}},
+                    {requires,    kernel_ready},
+                    {enables,     core_initialized}]}).
+
 %%---------------------------------------------------------------------------
 
 -include_lib("rabbit_common/include/rabbit_framing.hrl").
@@ -752,7 +758,7 @@ status() ->
                  true ->
                      [{virtual_host_count, rabbit_vhost:count()},
                       {connection_count,
-                       length(rabbit_networking:connections_local()) +
+                       length(rabbit_networking:local_connections()) +
                        length(rabbit_networking:local_non_amqp_connections())},
                       {queue_count, total_queue_count()}];
                  false ->
@@ -1098,6 +1104,9 @@ recover() ->
     ok = rabbit_vhost:recover(),
     ok.
 
+pg_local() ->
+    rabbit_sup:start_child(pg, [node()]).
+
 -spec maybe_insert_default_data() -> 'ok'.
 
 maybe_insert_default_data() ->
@@ -1690,7 +1699,19 @@ persist_static_configuration() ->
        classic_queue_store_v2_max_cache_size,
        classic_queue_store_v2_check_crc32,
        incoming_message_interceptors
-      ]).
+      ]),
+
+    %% Disallow 0 as it means unlimited:
+    %% "If this field is zero or unset, there is no maximum
+    %% size imposed by the link endpoint." [AMQP 1.0 §2.7.3]
+    MaxMsgSize = case application:get_env(?MODULE, max_message_size) of
+                     {ok, Size}
+                       when is_integer(Size) andalso Size > 0 ->
+                         erlang:min(Size, ?MAX_MSG_SIZE);
+                     _ ->
+                         ?MAX_MSG_SIZE
+                 end,
+    ok = persistent_term:put(max_message_size, MaxMsgSize).
 
 persist_static_configuration(Params) ->
     App = ?MODULE,
diff --git a/deps/rabbit/src/rabbit_access_control.erl b/deps/rabbit/src/rabbit_access_control.erl
index c76610d6bda0..6c3da3cd8d9d 100644
--- a/deps/rabbit/src/rabbit_access_control.erl
+++ b/deps/rabbit/src/rabbit_access_control.erl
@@ -10,7 +10,8 @@
 -include_lib("rabbit_common/include/rabbit.hrl").
 
 -export([check_user_pass_login/2, check_user_login/2, check_user_loopback/2,
-         check_vhost_access/4, check_resource_access/4, check_topic_access/4]).
+         check_vhost_access/4, check_resource_access/4, check_topic_access/4,
+         check_user_id/2]).
 
 -export([permission_cache_can_expire/1, update_state/2, expiry_timestamp/1]).
 
@@ -222,6 +223,31 @@ check_access(Fun, Module, ErrStr, ErrArgs, ErrName) ->
             rabbit_misc:protocol_error(ErrName, FullErrStr, FullErrArgs)
     end.
 
+-spec check_user_id(mc:state(), rabbit_types:user()) ->
+    ok | {refused, string(), [term()]}.
+check_user_id(Message, ActualUser) ->
+    case mc:user_id(Message) of
+        undefined ->
+            ok;
+        {binary, ClaimedUserName} ->
+            check_user_id0(ClaimedUserName, ActualUser)
+    end.
+
+check_user_id0(Username, #user{username = Username}) ->
+    ok;
+check_user_id0(_, #user{authz_backends = [{rabbit_auth_backend_dummy, _}]}) ->
+    ok;
+check_user_id0(ClaimedUserName, #user{username = ActualUserName,
+                                      tags = Tags}) ->
+    case lists:member(impersonator, Tags) of
+        true ->
+            ok;
+        false ->
+            {refused,
+             "user_id property set to '~ts' but authenticated user was '~ts'",
+             [ClaimedUserName, ActualUserName]}
+    end.
+
 -spec update_state(User :: rabbit_types:user(), NewState :: term()) ->
     {'ok', rabbit_types:auth_user()} |
     {'refused', string()} |
diff --git a/deps/rabbit/src/rabbit_alarm.erl b/deps/rabbit/src/rabbit_alarm.erl
index b0b02e89afa0..532a72f19b58 100644
--- a/deps/rabbit/src/rabbit_alarm.erl
+++ b/deps/rabbit/src/rabbit_alarm.erl
@@ -50,7 +50,7 @@
 -type resource_alarm() :: {resource_limit, resource_alarm_source(), node()}.
 -type alarm() :: local_alarm() | resource_alarm().
 -type resource_alert() :: {WasAlarmSetForNode :: boolean(),
-                           IsThereAnyAlarmsWithSameSourceInTheCluster :: boolean(),
+                           IsThereAnyAlarmWithSameSourceInTheCluster :: boolean(),
                            NodeForWhichAlarmWasSetOrCleared :: node()}.
 
 %%----------------------------------------------------------------------------
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0.erl b/deps/rabbit/src/rabbit_amqp1_0.erl
similarity index 60%
rename from deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0.erl
rename to deps/rabbit/src/rabbit_amqp1_0.erl
index 34b0278bb38e..cba97ec2a58f 100644
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0.erl
+++ b/deps/rabbit/src/rabbit_amqp1_0.erl
@@ -6,11 +6,16 @@
 %%
 -module(rabbit_amqp1_0).
 
+-define(PROCESS_GROUP_NAME, rabbit_amqp10_connections).
+
+-export([list_local/0,
+         register_connection/1]).
+
+%% Below 2 functions are deprecated.
+%% They could be called in 3.13 / 4.0 mixed version clusters by the old 3.13 CLI command
+%% rabbitmqctl list_amqp10_connections
 -export([emit_connection_info_local/3,
-         emit_connection_info_all/4,
-         list/0,
-         register_connection/1,
-         unregister_connection/1]).
+         emit_connection_info_all/4]).
 
 emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) ->
     Pids = [spawn_link(Node, rabbit_amqp1_0, emit_connection_info_local,
@@ -20,21 +25,19 @@ emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) ->
     ok.
 
 emit_connection_info_local(Items, Ref, AggregatorPid) ->
+    ConnectionPids = list_local(),
     rabbit_control_misc:emitting_map_with_exit_handler(
-      AggregatorPid, Ref,
+      AggregatorPid,
+      Ref,
       fun(Pid) ->
-              rabbit_amqp1_0_reader:info(Pid, Items)
+              rabbit_amqp_reader:info(Pid, Items)
       end,
-      list()).
+      ConnectionPids).
 
--spec list() -> [pid()].
-list() ->
-    pg_local:get_members(rabbit_amqp10_connections).
+-spec list_local() -> [pid()].
+list_local() ->
+    pg:get_local_members(node(), ?PROCESS_GROUP_NAME).
 
 -spec register_connection(pid()) -> ok.
 register_connection(Pid) ->
-    pg_local:join(rabbit_amqp10_connections, Pid).
-
--spec unregister_connection(pid()) -> ok.
-unregister_connection(Pid) ->
-    pg_local:leave(rabbit_amqp10_connections, Pid).
+    ok = pg:join(node(), ?PROCESS_GROUP_NAME, Pid).
diff --git a/deps/rabbit/src/rabbit_amqp_reader.erl b/deps/rabbit/src/rabbit_amqp_reader.erl
new file mode 100644
index 000000000000..3310320722f7
--- /dev/null
+++ b/deps/rabbit/src/rabbit_amqp_reader.erl
@@ -0,0 +1,1042 @@
+%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
+%%
+
+-module(rabbit_amqp_reader).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include("rabbit_amqp.hrl").
+
+-export([init/2,
+         info/2,
+         mainloop/2]).
+
+-export([system_continue/3,
+         system_terminate/4,
+         system_code_change/4]).
+
+-import(rabbit_amqp_util, [protocol_error/3]).
+
+%% same values as in rabbit_reader
+-define(NORMAL_TIMEOUT, 3_000).
+-define(CLOSING_TIMEOUT, 30_000).
+-define(SILENT_CLOSE_DELAY, 3_000).
+
+-type protocol() :: amqp | sasl.
+-type channel_number() :: non_neg_integer().
+
+-record(v1_connection,
+        {name :: binary(),
+         vhost :: none | rabbit_types:vhost(),
+         %% server host
+         host :: inet:ip_address() | inet:hostname(),
+         %% client host
+         peer_host :: inet:ip_address() | inet:hostname(),
+         %% server port
+         port :: inet:port_number(),
+         %% client port
+         peer_port :: inet:port_number(),
+         connected_at :: integer(),
+         user :: rabbit_types:option(rabbit_types:user()),
+         timeout :: non_neg_integer(),
+         incoming_max_frame_size :: pos_integer(),
+         outgoing_max_frame_size :: unlimited | pos_integer(),
+         channel_max :: non_neg_integer(),
+         auth_mechanism :: none | anonymous | {binary(), module()},
+         auth_state :: term(),
+         properties :: undefined | {map, list(tuple())}
+        }).
+
+-record(v1,
+        {
+         parent :: pid(),
+         helper_sup :: pid(),
+         writer :: none | pid(),
+         heartbeater :: none | rabbit_heartbeat:heartbeaters(),
+         session_sup :: rabbit_types:option(pid()),
+         sock :: rabbit_net:socket(),
+         proxy_socket :: undefined | {rabbit_proxy_socket, any(), any()},
+         connection :: #v1_connection{},
+         connection_state :: pre_init | starting | waiting_amqp0100 | securing | running | closing | closed,
+         callback :: handshake |
+                     {frame_header_1_0, protocol()} |
+                     {frame_payload_1_0, protocol(), DataOffset :: pos_integer(), channel_number()},
+         recv_len :: non_neg_integer(),
+         pending_recv :: boolean(),
+         buf :: list(),
+         buf_len :: non_neg_integer(),
+         tracked_channels :: #{channel_number() => Session :: pid()}
+        }).
+
+-type state() :: #v1{}.
+
+-define(IS_RUNNING(State), State#v1.connection_state =:= running).
+
+%%--------------------------------------------------------------------------
+
+unpack_from_0_9_1(
+  {Sock,RecvLen, PendingRecv, Buf, BufLen, ProxySocket,
+   ConnectionName, Host, PeerHost, Port, PeerPort, ConnectedAt},
+  Parent, ConnectionHelperSupPid, HandshakeTimeout) ->
+    #v1{parent              = Parent,
+        sock                = Sock,
+        callback            = handshake,
+        recv_len            = RecvLen,
+        pending_recv        = PendingRecv,
+        connection_state    = pre_init,
+        heartbeater         = none,
+        helper_sup          = ConnectionHelperSupPid,
+        buf                 = Buf,
+        buf_len             = BufLen,
+        proxy_socket        = ProxySocket,
+        tracked_channels    = maps:new(),
+        writer              = none,
+        connection = #v1_connection{
+                        name = ConnectionName,
+                        vhost = none,
+                        host = Host,
+                        peer_host = PeerHost,
+                        port = Port,
+                        peer_port = PeerPort,
+                        connected_at = ConnectedAt,
+                        user = none,
+                        timeout = HandshakeTimeout,
+                        %% "Prior to any explicit negotiation, the maximum
+                        %% frame size is 512 (MIN-MAX-FRAME-SIZE)" [2.4.1]
+                        incoming_max_frame_size = ?MIN_MAX_FRAME_1_0_SIZE,
+                        outgoing_max_frame_size = ?MIN_MAX_FRAME_1_0_SIZE,
+                        channel_max = 0,
+                        auth_mechanism = none,
+                        auth_state = none}}.
+
+-spec system_continue(pid(), [sys:dbg_opt()], state()) -> no_return() | ok.
+system_continue(Parent, Deb, State) ->
+    ?MODULE:mainloop(Deb, State#v1{parent = Parent}).
+
+-spec system_terminate(term(), pid(), [sys:dbg_opt()], term()) -> no_return().
+system_terminate(Reason, _Parent, _Deb, _State) ->
+    exit(Reason).
+
+-spec system_code_change(term(), module(), undefined | term(), term()) -> {ok, term()}.
+system_code_change(Misc, _Module, _OldVsn, _Extra) ->
+    {ok, Misc}.
+
+server_properties() ->
+    %% The atom doesn't match anything, it's just "not 0-9-1".
+    Raw = lists:keydelete(<<"capabilities">>, 1, rabbit_reader:server_properties(amqp_1_0)),
+    {map, [{{symbol, K}, {utf8, V}} || {K, longstr, V}  <- Raw]}.
+
+%%--------------------------------------------------------------------------
+
+inet_op(F) -> rabbit_misc:throw_on_error(inet_error, F).
+
+recvloop(Deb, State = #v1{pending_recv = true}) ->
+    mainloop(Deb, State);
+recvloop(Deb, State = #v1{sock = Sock, recv_len = RecvLen, buf_len = BufLen})
+  when BufLen < RecvLen ->
+    case rabbit_net:setopts(Sock, [{active, once}]) of
+        ok ->
+            mainloop(Deb, State#v1{pending_recv = true});
+        {error, Reason} ->
+            throw({inet_error, Reason})
+    end;
+recvloop(Deb, State = #v1{recv_len = RecvLen, buf = Buf, buf_len = BufLen}) ->
+    {Data, Rest} = split_binary(case Buf of
+                                    [B] -> B;
+                                    _   -> list_to_binary(lists:reverse(Buf))
+                                end, RecvLen),
+    recvloop(Deb, handle_input(State#v1.callback, Data,
+                               State#v1{buf = [Rest],
+                                        buf_len = BufLen - RecvLen})).
+
+-spec mainloop([sys:dbg_opt()], state()) ->
+    no_return() | ok.
+mainloop(Deb, State = #v1{sock = Sock, buf = Buf, buf_len = BufLen}) ->
+    case rabbit_net:recv(Sock) of
+        {data, Data} ->
+            recvloop(Deb, State#v1{buf = [Data | Buf],
+                                   buf_len = BufLen + size(Data),
+                                   pending_recv = false});
+        closed when State#v1.connection_state =:= closed ->
+            ok;
+        closed ->
+            throw(connection_closed_abruptly);
+        {error, Reason} ->
+            throw({inet_error, Reason});
+        {other, {system, From, Request}} ->
+            sys:handle_system_msg(Request, From, State#v1.parent,
+                                  ?MODULE, Deb, State);
+        {other, Other} ->
+            case handle_other(Other, State) of
+                stop     -> ok;
+                NewState -> recvloop(Deb, NewState)
+            end
+    end.
+
+handle_other({'EXIT', Parent, Reason}, State = #v1{parent = Parent}) ->
+    ReasonString = rabbit_misc:format("broker forced connection closure with reason '~w'",
+                                      [Reason]),
+    _ = terminate(ReasonString, State),
+    %% this is what we are expected to do according to
+    %% http://www.erlang.org/doc/man/sys.html
+    %%
+    %% If we wanted to be *really* nice we should wait for a while for
+    %% clients to close the socket at their end, just as we do in the
+    %% ordinary error case. However, since this termination is
+    %% initiated by our parent it is probably more important to exit
+    %% quickly.
+    exit(Reason);
+handle_other({{'DOWN', ChannelNum}, _MRef, process, SessionPid, Reason}, State) ->
+    handle_session_exit(ChannelNum, SessionPid, Reason, State);
+handle_other(handshake_timeout, State)
+  when ?IS_RUNNING(State) orelse
+       State#v1.connection_state =:= closing orelse
+       State#v1.connection_state =:= closed ->
+    State;
+handle_other(handshake_timeout, State) ->
+    throw({handshake_timeout, State#v1.callback});
+handle_other(heartbeat_timeout, State = #v1{connection_state = closed}) ->
+    State;
+handle_other(heartbeat_timeout, State) ->
+    Error = error_frame(?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED,
+                        "no frame received from client within idle timeout threshold", []),
+    handle_exception(State, 0, Error);
+handle_other({'$gen_call', From, {shutdown, Explanation}},
+             State = #v1{connection = #v1_connection{properties = Properties}}) ->
+    Ret = case Explanation =:= "Node was put into maintenance mode" andalso
+               ignore_maintenance(Properties) of
+              true -> State;
+              false -> terminate(Explanation, State)
+          end,
+    gen_server:reply(From, ok),
+    Ret;
+handle_other({'$gen_call', From, {info, Items}}, State) ->
+    Reply = try infos(Items, State) of
+                Infos ->
+                    {ok, Infos}
+            catch Error ->
+                      {error, Error}
+            end,
+    gen_server:reply(From, Reply),
+    State;
+handle_other({'$gen_cast', {force_event_refresh, _Ref}}, State) ->
+    State;
+handle_other(terminate_connection, _State) ->
+    stop;
+handle_other(credential_expired, State) ->
+    Error = error_frame(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, "credential expired", []),
+    handle_exception(State, 0, Error);
+handle_other(Other, _State) ->
+    %% internal error -> something worth dying for
+    exit({unexpected_message, Other}).
+
+switch_callback(State, Callback, Length) ->
+    State#v1{callback = Callback, recv_len = Length}.
+
+terminate(Reason, State)
+  when ?IS_RUNNING(State) ->
+    handle_exception(State, 0,
+                     error_frame(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                 "Connection forced: ~tp", [Reason]));
+terminate(_, _) ->
+    stop.
+
+%%--------------------------------------------------------------------------
+%% error handling / termination
+
+close(Error, State = #v1{sock = Sock,
+                         connection = #v1_connection{timeout = Timeout}}) ->
+    %% Client properties will be emitted in the connection_closed event by rabbit_reader.
+    ClientProperties = i(client_properties, State),
+    put(client_properties, ClientProperties),
+    Time = case Timeout > 0 andalso
+                Timeout < ?CLOSING_TIMEOUT of
+               true -> Timeout;
+               false -> ?CLOSING_TIMEOUT
+           end,
+    _TRef = erlang:send_after(Time, self(), terminate_connection),
+    ok = send_on_channel0(Sock, #'v1_0.close'{error = Error}),
+    State#v1{connection_state = closed}.
+
+handle_session_exit(ChannelNum, SessionPid, Reason, State0) ->
+    State = untrack_channel(ChannelNum, SessionPid, State0),
+    S = case terminated_normally(Reason) of
+            true ->
+                State;
+            false ->
+                R = case Reason of
+                        {RealReason, Trace} ->
+                            error_frame(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                        "Session error: ~tp~n~tp",
+                                        [RealReason, Trace]);
+                        _ ->
+                            error_frame(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                        "Session error: ~tp",
+                                        [Reason])
+                    end,
+                handle_exception(State, SessionPid, R)
+        end,
+    maybe_close(S).
+
+terminated_normally(normal) ->
+    true;
+terminated_normally(shutdown) ->
+    true;
+terminated_normally({shutdown, _Term}) ->
+    true;
+terminated_normally(_Reason) ->
+    false.
+
+maybe_close(State = #v1{connection_state = closing}) ->
+    close(undefined, State);
+maybe_close(State) ->
+    State.
+
+error_frame(Condition, Fmt, Args) ->
+    Description = list_to_binary(rabbit_misc:format(Fmt, Args)),
+    #'v1_0.error'{condition = Condition,
+                  description = {utf8, Description}}.
+
+handle_exception(State = #v1{connection_state = closed}, Channel,
+                 #'v1_0.error'{description = {utf8, Desc}}) ->
+    rabbit_log_connection:error(
+      "Error on AMQP 1.0 connection ~tp (~tp), channel ~tp:~n~tp",
+      [self(), closed, Channel, Desc]),
+    State;
+handle_exception(State = #v1{connection_state = CS}, Channel,
+                 Error = #'v1_0.error'{description = {utf8, Desc}})
+  when ?IS_RUNNING(State) orelse CS =:= closing ->
+    rabbit_log_connection:error(
+      "Error on AMQP 1.0 connection ~tp (~tp), channel ~tp:~n~tp",
+      [self(), CS, Channel, Desc]),
+    close(Error, State);
+handle_exception(State, Channel, Error) ->
+    silent_close_delay(),
+    throw({handshake_error, State#v1.connection_state, Channel, Error}).
+
+is_connection_frame(#'v1_0.open'{})  -> true;
+is_connection_frame(#'v1_0.close'{}) -> true;
+is_connection_frame(_)               -> false.
+
+%% TODO Handle depending on connection state
+%% TODO It'd be nice to only decode up to the descriptor
+
+handle_1_0_frame(Mode, Channel, Payload, State) ->
+    try
+        handle_1_0_frame0(Mode, Channel, Payload, State)
+    catch
+        _:#'v1_0.error'{} = Reason ->
+            handle_exception(State, 0, Reason);
+        _:{error, {not_allowed, Username}} ->
+            %% section 2.8.15 in http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-complete-v1.0-os.pdf
+            handle_exception(State, 0, error_frame(
+                                         ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+                                         "Access for user '~ts' was refused: insufficient permissions",
+                                         [Username]));
+        _:Reason:Trace ->
+            handle_exception(State, 0, error_frame(
+                                         ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                         "Reader error: ~tp~n~tp",
+                                         [Reason, Trace]))
+    end.
+
+%% Nothing specifies that connection methods have to be on a
+%% particular channel.
+handle_1_0_frame0(_Mode, Channel, Payload,
+                  State = #v1{connection_state = CS})
+  when CS =:= closing orelse
+       CS =:= closed ->
+    Sections = parse_1_0_frame(Payload, Channel),
+    case is_connection_frame(Sections) of
+        true  -> handle_1_0_connection_frame(Sections, State);
+        false -> State
+    end;
+handle_1_0_frame0(Mode, Channel, Payload, State) ->
+    Sections = parse_1_0_frame(Payload, Channel),
+    case {Mode, is_connection_frame(Sections)} of
+        {amqp, true}  -> handle_1_0_connection_frame(Sections, State);
+        {amqp, false} -> handle_1_0_session_frame(Channel, Sections, State);
+        {sasl, false} -> handle_1_0_sasl_frame(Sections, State)
+    end.
+
+parse_1_0_frame(Payload, _Channel) ->
+    {PerfDesc, Rest} = amqp10_binary_parser:parse(Payload),
+    Perf = amqp10_framing:decode(PerfDesc),
+    ?DEBUG("~s Channel ~tp ->~n~tp~n~ts~n",
+           [?MODULE, _Channel, amqp10_framing:pprint(Perf),
+            case Rest of
+                <<>> -> <<>>;
+                _    -> rabbit_misc:format(
+                          " followed by ~tp bytes of content", [size(Rest)])
+            end]),
+    case Rest of
+        <<>> -> Perf;
+        _    -> {Perf, Rest}
+    end.
+
+handle_1_0_connection_frame(
+  #'v1_0.open'{max_frame_size = ClientMaxFrame,
+               channel_max = ClientChannelMax,
+               idle_time_out = IdleTimeout,
+               hostname = Hostname,
+               properties = Properties},
+  #v1{connection_state = starting,
+      connection = Connection = #v1_connection{name = ConnectionName,
+                                               user = User = #user{username = Username}},
+      helper_sup = HelperSupPid,
+      sock = Sock} = State0) ->
+
+    Vhost = vhost(Hostname),
+    ok = check_user_loopback(State0),
+    ok = check_vhost_exists(Vhost, State0),
+    ok = check_vhost_alive(Vhost),
+    ok = rabbit_access_control:check_vhost_access(User, Vhost, {socket, Sock}, #{}),
+    ok = check_vhost_connection_limit(Vhost, Username),
+    ok = check_user_connection_limit(Username),
+    ok = ensure_credential_expiry_timer(User),
+    rabbit_core_metrics:auth_attempt_succeeded(<<>>, Username, amqp10),
+    notify_auth(user_authentication_success, Username, State0),
+    rabbit_log_connection:info(
+      "AMQP 1.0 connection: user '~ts' authenticated and granted access to vhost '~ts'",
+      [Username, Vhost]),
+
+    OutgoingMaxFrameSize = case ClientMaxFrame of
+                               undefined ->
+                                   unlimited;
+                               {uint, Bytes}
+                                 when Bytes >= ?MIN_MAX_FRAME_1_0_SIZE ->
+                                   Bytes;
+                               {uint, Bytes} ->
+                                   protocol_error(
+                                     ?V_1_0_AMQP_ERROR_FRAME_SIZE_TOO_SMALL,
+                                     "max_frame_size (~w) < minimum maximum frame size (~w)",
+                                     [Bytes, ?MIN_MAX_FRAME_1_0_SIZE])
+                           end,
+    SendTimeoutSec =
+    case IdleTimeout of
+        undefined ->
+            0;
+        {uint, Interval} ->
+            if Interval =:= 0 ->
+                   0;
+               Interval < 1000 ->
+                   %% "If a peer can not, for any reason support a proposed idle timeout, then it SHOULD
+                   %% close the connection using a close frame with an error explaining why. There is no
+                   %% requirement for peers to support arbitrarily short or long idle timeouts." [2.4.5]
+                   %% rabbit_heartbeat does not want to support sub-second timeouts.
+                   protocol_error(
+                     ?V_1_0_AMQP_ERROR_NOT_ALLOWED,
+                     "idle-time-out (~b ms) < minimum idle-time-out (1000 ms)",
+                     [Interval]);
+               Interval >= 1000 ->
+                   Interval div 1000
+            end
+    end,
+    {ok, ReceiveTimeoutSec} = application:get_env(rabbit, heartbeat),
+    ReceiveTimeoutMillis = ReceiveTimeoutSec * 1000,
+    SendFun = fun() ->
+                      Frame = amqp10_binary_generator:build_heartbeat_frame(),
+                      catch rabbit_net:send(Sock, Frame)
+              end,
+    Parent = self(),
+    ReceiveFun = fun() -> Parent ! heartbeat_timeout end,
+    %% TODO: only start heartbeat receive timer at next next frame
+    Heartbeater = rabbit_heartbeat:start(
+                    HelperSupPid, Sock, ConnectionName,
+                    SendTimeoutSec, SendFun,
+                    ReceiveTimeoutSec, ReceiveFun),
+    {ok, IncomingMaxFrameSize} = application:get_env(rabbit, frame_max),
+    %% TODO enforce channel_max
+    ChannelMax = case ClientChannelMax of
+                     undefined ->
+                         %% default as per 2.7.1
+                         16#ff_ff;
+                     {ushort, N} ->
+                         N
+                 end,
+    State1 = State0#v1{connection_state = running,
+                       connection = Connection#v1_connection{
+                                      vhost = Vhost,
+                                      incoming_max_frame_size = IncomingMaxFrameSize,
+                                      outgoing_max_frame_size = OutgoingMaxFrameSize,
+                                      channel_max = ChannelMax,
+                                      properties = Properties,
+                                      timeout = ReceiveTimeoutMillis},
+                       heartbeater = Heartbeater},
+    State = start_writer(State1),
+    HostnameVal = case Hostname of
+                      undefined -> undefined;
+                      null -> undefined;
+                      {utf8, Val} -> Val
+                  end,
+    rabbit_log:debug(
+      "AMQP 1.0 connection.open frame: hostname = ~ts, extracted vhost = ~ts, idle-time-out = ~p",
+      [HostnameVal, Vhost, IdleTimeout]),
+
+    Infos = infos(?CONNECTION_EVENT_KEYS, State),
+    ok = rabbit_core_metrics:connection_created(
+           proplists:get_value(pid, Infos),
+           Infos),
+    ok = rabbit_event:notify(connection_created, Infos),
+    ok = rabbit_amqp1_0:register_connection(self()),
+    ok = send_on_channel0(
+           Sock,
+           #'v1_0.open'{channel_max    = ClientChannelMax,
+                        max_frame_size = {uint, IncomingMaxFrameSize},
+                        %% "the value in idle-time-out SHOULD be half the peer's actual timeout threshold" [2.4.5]
+                        idle_time_out  = {uint, ReceiveTimeoutMillis div 2},
+                        container_id   = {utf8, rabbit_nodes:cluster_name()},
+                        properties     = server_properties()}),
+    State;
+handle_1_0_connection_frame(#'v1_0.close'{}, State0) ->
+    State = State0#v1{connection_state = closing},
+    close(undefined, State).
+
+start_writer(#v1{helper_sup = SupPid,
+                 sock = Sock,
+                 connection = #v1_connection{outgoing_max_frame_size = MaxFrame}} = State) ->
+    ChildSpec = #{id => writer,
+                  start => {rabbit_amqp_writer, start_link, [Sock, MaxFrame, self()]},
+                  restart => transient,
+                  significant => true,
+                  shutdown => ?WORKER_WAIT,
+                  type => worker
+                 },
+    {ok, Pid} = supervisor:start_child(SupPid, ChildSpec),
+    State#v1{writer = Pid}.
+
+handle_1_0_session_frame(Channel, Frame, #v1{tracked_channels = Channels} = State) ->
+    case Channels of
+        #{Channel := SessionPid} ->
+            rabbit_amqp_session:process_frame(SessionPid, Frame),
+            State;
+        _ ->
+            case ?IS_RUNNING(State) of
+                true ->
+                    case Frame of
+                        #'v1_0.begin'{} ->
+                            send_to_new_1_0_session(Channel, Frame, State);
+                        _ ->
+                            State
+                    end;
+                false ->
+                    throw({channel_frame_while_connection_not_running,
+                           Channel,
+                           State#v1.connection_state,
+                           Frame})
+            end
+    end.
+
+%% TODO: write a proper ANONYMOUS plugin and unify with STOMP
+handle_1_0_sasl_frame(#'v1_0.sasl_init'{mechanism = {symbol, <<"ANONYMOUS">>},
+                                        hostname = _Hostname},
+                      #v1{connection_state = starting,
+                          connection = Connection,
+                          sock = Sock} = State0) ->
+    case default_user() of
+        none ->
+            silent_close_delay(),
+            Outcome = #'v1_0.sasl_outcome'{code = ?V_1_0_SASL_CODE_SYS_PERM},
+            ok = send_on_channel0(Sock, Outcome, rabbit_amqp_sasl),
+            throw(banned_unauthenticated_connection);
+        _ ->
+            %% We only need to send the frame, again start_connection
+            %% will set up the default user.
+            Outcome = #'v1_0.sasl_outcome'{code = ?V_1_0_SASL_CODE_OK},
+            ok = send_on_channel0(Sock, Outcome, rabbit_amqp_sasl),
+            State = State0#v1{connection_state = waiting_amqp0100,
+                              connection = Connection#v1_connection{auth_mechanism = anonymous}},
+            switch_callback(State, handshake, 8)
+    end;
+handle_1_0_sasl_frame(#'v1_0.sasl_init'{mechanism        = {symbol, Mechanism},
+                                        initial_response = {binary, Response},
+                                        hostname         = _Hostname},
+                      State0 = #v1{connection_state = starting,
+                                   connection       = Connection,
+                                   sock             = Sock}) ->
+    AuthMechanism = auth_mechanism_to_module(Mechanism, Sock),
+    State = State0#v1{connection       =
+                          Connection#v1_connection{
+                            auth_mechanism    = {Mechanism, AuthMechanism},
+                            auth_state        = AuthMechanism:init(Sock)},
+                      connection_state = securing},
+    auth_phase_1_0(Response, State);
+handle_1_0_sasl_frame(#'v1_0.sasl_response'{response = {binary, Response}},
+                      State = #v1{connection_state = securing}) ->
+    auth_phase_1_0(Response, State);
+handle_1_0_sasl_frame(Frame, State) ->
+    throw({unexpected_1_0_sasl_frame, Frame, State}).
+
+handle_input(handshake, <<"AMQP", 0, 1, 0, 0>>,
+             #v1{connection_state = waiting_amqp0100} = State) ->
+    start_1_0_connection(amqp, State);
+
+handle_input({frame_header_1_0, Mode},
+             Header = <>,
+             State) when DOff >= 2 ->
+    case {Mode, Type} of
+        {amqp, 0} -> ok;
+        {sasl, 1} -> ok;
+        _         -> throw({bad_1_0_header_type, Header, Mode})
+    end,
+    MaxFrameSize = State#v1.connection#v1_connection.incoming_max_frame_size,
+    if Size =:= 8 ->
+           %% heartbeat
+           State;
+       Size > MaxFrameSize ->
+           handle_exception(
+             State, Channel, error_frame(
+                               ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR,
+                               "frame size (~b bytes) > maximum frame size (~b bytes)",
+                               [Size, MaxFrameSize]));
+       true ->
+           switch_callback(State, {frame_payload_1_0, Mode, DOff, Channel}, Size - 8)
+    end;
+handle_input({frame_header_1_0, _Mode}, Malformed, _State) ->
+    throw({bad_1_0_header, Malformed});
+handle_input({frame_payload_1_0, Mode, DOff, Channel},
+            FrameBin, State) ->
+    SkipBits = (DOff * 32 - 64), % DOff = 4-byte words, we've read 8 already
+    <> = FrameBin,
+    Skip = Skip, %% hide warning when debug is off
+    handle_1_0_frame(Mode, Channel, FramePayload,
+                     switch_callback(State, {frame_header_1_0, Mode}, 8));
+
+handle_input(Callback, Data, _State) ->
+    throw({bad_input, Callback, Data}).
+
+-spec init(protocol(), tuple()) -> no_return().
+init(Mode, PackedState) ->
+    {ok, HandshakeTimeout} = application:get_env(rabbit, handshake_timeout),
+    {parent, Parent} = erlang:process_info(self(), parent),
+    ConnectionHelperSupFlags = #{strategy => one_for_all,
+                                 intensity => 0,
+                                 period => 1,
+                                 auto_shutdown => any_significant},
+    {ok, ConnectionHelperSupPid} = rabbit_connection_sup:start_connection_helper_sup(
+                                     Parent, ConnectionHelperSupFlags),
+    State0 = unpack_from_0_9_1(PackedState, Parent, ConnectionHelperSupPid, HandshakeTimeout),
+    State = start_1_0_connection(Mode, State0),
+    %% By invoking recvloop here we become 1.0.
+    recvloop(sys:debug_options([]), State).
+
+start_1_0_connection(Mode = sasl, State = #v1{sock = Sock}) ->
+    send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
+    %% "The server mechanisms are ordered in decreasing level of preference." [5.3.3.1]
+    Ms0 = [{symbol, atom_to_binary(M)} || M <- auth_mechanisms(Sock)],
+    Ms1 = case default_user() of
+              none -> Ms0;
+              _ -> Ms0 ++ [{symbol, <<"ANONYMOUS">>}]
+          end,
+    Ms2 = {array, symbol, Ms1},
+    Ms = #'v1_0.sasl_mechanisms'{sasl_server_mechanisms = Ms2},
+    ok = send_on_channel0(Sock, Ms, rabbit_amqp_sasl),
+    start_1_0_connection0(Mode, State);
+
+start_1_0_connection(Mode = amqp,
+                     State = #v1{sock = Sock,
+                                 connection = C = #v1_connection{user = User}}) ->
+    case User of
+        none ->
+            %% Client either skipped SASL layer or used SASL mechansim ANONYMOUS.
+            case default_user() of
+                none ->
+                    send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
+                    throw(banned_unauthenticated_connection);
+                NoAuthUsername ->
+                    case rabbit_access_control:check_user_login(NoAuthUsername, []) of
+                        {ok, NoAuthUser} ->
+                            State1 = State#v1{connection = C#v1_connection{user = NoAuthUser}},
+                            send_1_0_handshake(Sock, <<"AMQP",0,1,0,0>>),
+                            start_1_0_connection0(Mode, State1);
+                        {refused, _, _, _} ->
+                            send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
+                            throw(amqp1_0_default_user_missing)
+                    end
+            end;
+        #user{} ->
+            %% Client already got successfully authenticated by SASL.
+            send_1_0_handshake(Sock, <<"AMQP",0,1,0,0>>),
+            start_1_0_connection0(Mode, State)
+    end.
+
+start_1_0_connection0(Mode, State0 = #v1{connection = Connection,
+                                         helper_sup = HelperSup}) ->
+    SessionSup = case Mode of
+                     sasl ->
+                         undefined;
+                     amqp ->
+                         ChildSpec = #{id => session_sup,
+                                       start => {rabbit_amqp_session_sup, start_link, [self()]},
+                                       restart => transient,
+                                       significant => true,
+                                       shutdown => infinity,
+                                       type => supervisor},
+                         {ok, Pid} = supervisor:start_child(HelperSup, ChildSpec),
+                         Pid
+                 end,
+    State = State0#v1{session_sup = SessionSup,
+                      connection_state = starting,
+                      connection = Connection#v1_connection{timeout = ?NORMAL_TIMEOUT}},
+    switch_callback(State, {frame_header_1_0, Mode}, 8).
+
+send_1_0_handshake(Sock, Handshake) ->
+    ok = inet_op(fun () -> rabbit_net:send(Sock, Handshake) end).
+
+send_on_channel0(Sock, Method) ->
+    send_on_channel0(Sock, Method, amqp10_framing).
+
+send_on_channel0(Sock, Method, Framing) ->
+    ok = rabbit_amqp_writer:internal_send_command(Sock, Method, Framing).
+
+%% End 1-0
+
+auth_mechanism_to_module(TypeBin, Sock) ->
+    case rabbit_registry:binary_to_type(TypeBin) of
+        {error, not_found} ->
+            protocol_error(?V_1_0_AMQP_ERROR_NOT_FOUND,
+                           "unknown authentication mechanism '~ts'", [TypeBin]);
+        T ->
+            case {lists:member(T, auth_mechanisms(Sock)),
+                  rabbit_registry:lookup_module(auth_mechanism, T)} of
+                {true, {ok, Module}} ->
+                    Module;
+                _ ->
+                    protocol_error(?V_1_0_AMQP_ERROR_NOT_FOUND,
+                                   "invalid authentication mechanism '~ts'", [T])
+            end
+    end.
+
+auth_mechanisms(Sock) ->
+    {ok, Configured} = application:get_env(rabbit, auth_mechanisms),
+    [Name || {Name, Module} <- rabbit_registry:lookup_all(auth_mechanism),
+             Module:should_offer(Sock), lists:member(Name, Configured)].
+
+%% Begin 1-0
+
+auth_phase_1_0(Response,
+               State = #v1{sock = Sock,
+                           connection = Connection =
+                           #v1_connection{auth_mechanism = {Name, AuthMechanism},
+                                          auth_state     = AuthState}}) ->
+    case AuthMechanism:handle_response(Response, AuthState) of
+        {refused, Username, Msg, Args} ->
+            %% We don't trust the client at this point - force them to wait
+            %% for a bit before sending the sasl outcome frame
+            %% so they can't DOS us with repeated failed logins etc.
+            auth_fail(Username, State),
+            silent_close_delay(),
+            Outcome = #'v1_0.sasl_outcome'{code = ?V_1_0_SASL_CODE_AUTH},
+            ok = send_on_channel0(Sock, Outcome, rabbit_amqp_sasl),
+            protocol_error(
+              ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, "~ts login refused: ~ts",
+              [Name, io_lib:format(Msg, Args)]);
+        {protocol_error, Msg, Args} ->
+            auth_fail(none, State),
+            protocol_error(?V_1_0_AMQP_ERROR_DECODE_ERROR, Msg, Args);
+        {challenge, Challenge, AuthState1} ->
+            Secure = #'v1_0.sasl_challenge'{challenge = {binary, Challenge}},
+            ok = send_on_channel0(Sock, Secure, rabbit_amqp_sasl),
+            State#v1{connection = Connection#v1_connection{auth_state = AuthState1}};
+        {ok, User} ->
+            Outcome = #'v1_0.sasl_outcome'{code = ?V_1_0_SASL_CODE_OK},
+            ok = send_on_channel0(Sock, Outcome, rabbit_amqp_sasl),
+            State1 = State#v1{connection_state = waiting_amqp0100,
+                              connection = Connection#v1_connection{user = User}},
+            switch_callback(State1, handshake, 8)
+    end.
+
+
+auth_fail(Username, State) ->
+    rabbit_core_metrics:auth_attempt_failed(<<>>, Username, amqp10),
+    notify_auth(user_authentication_failure, Username, State).
+
+notify_auth(EventType, Username, State) ->
+    Name = case Username of
+               none -> [];
+               _ -> [{name, Username}]
+           end,
+    AuthEventItems = lists:filtermap(
+                       fun(Item = name) ->
+                               {true, {connection_name, i(Item, State)}};
+                          (Item) ->
+                               case i(Item, State) of
+                                   '' -> false;
+                                   Val -> {true, {Item, Val}}
+                               end
+                       end, ?AUTH_EVENT_KEYS),
+    EventProps = Name ++ AuthEventItems,
+    rabbit_event:notify(EventType, EventProps).
+
+track_channel(ChannelNum, SessionPid, #v1{tracked_channels = Channels} = State) ->
+    rabbit_log:debug("AMQP 1.0 created session process ~p for channel number ~b",
+                     [SessionPid, ChannelNum]),
+    _Ref = erlang:monitor(process, SessionPid, [{tag, {'DOWN', ChannelNum}}]),
+    State#v1{tracked_channels = maps:put(ChannelNum, SessionPid, Channels)}.
+
+untrack_channel(ChannelNum, SessionPid, #v1{tracked_channels = Channels0} = State) ->
+    case maps:take(ChannelNum, Channels0) of
+        {SessionPid, Channels} ->
+            rabbit_log:debug("AMQP 1.0 closed session process ~p with channel number ~b",
+                             [SessionPid, ChannelNum]),
+            State#v1{tracked_channels = Channels};
+        _ ->
+            State
+    end.
+
+send_to_new_1_0_session(
+  ChannelNum, BeginFrame,
+  #v1{session_sup = SessionSup,
+      connection = #v1_connection{outgoing_max_frame_size = MaxFrame,
+                                  vhost = Vhost,
+                                  user = User,
+                                  name = ConnName},
+      writer = WriterPid} = State) ->
+    %% Subtract fixed frame header size.
+    OutgoingMaxFrameSize = case MaxFrame of
+                               unlimited -> unlimited;
+                               _ -> MaxFrame - 8
+                           end,
+    ChildArgs = [WriterPid,
+                 ChannelNum,
+                 OutgoingMaxFrameSize,
+                 User,
+                 Vhost,
+                 ConnName,
+                 BeginFrame],
+    case rabbit_amqp_session_sup:start_session(SessionSup, ChildArgs) of
+        {ok, SessionPid} ->
+            track_channel(ChannelNum, SessionPid, State);
+        {error, _} = E ->
+            throw(E)
+    end.
+
+vhost({utf8, <<"vhost:", VHost/binary>>}) ->
+    VHost;
+vhost(_) ->
+    application:get_env(rabbit, amqp1_0_default_vhost,
+                        application:get_env(rabbit, default_vhost, <<"/">>)).
+
+check_user_loopback(#v1{connection = #v1_connection{user = #user{username = Username}},
+                        sock = Socket} = State) ->
+    case rabbit_access_control:check_user_loopback(Username, Socket) of
+        ok ->
+            ok;
+        not_allowed ->
+            auth_fail(Username, State),
+            protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+                           "user '~ts' can only connect via localhost",
+                           [Username])
+    end.
+
+check_vhost_exists(Vhost, State) ->
+    case rabbit_vhost:exists(Vhost) of
+        true ->
+            ok;
+        false ->
+            auth_fail(State#v1.connection#v1_connection.user#user.username, State),
+            protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+                           "AMQP 1.0 connection failed: virtual host '~s' does not exist",
+                           [Vhost])
+    end.
+
+check_vhost_alive(Vhost) ->
+    case rabbit_vhost_sup_sup:is_vhost_alive(Vhost) of
+        true ->
+            ok;
+        false ->
+            protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                           "AMQP 1.0 connection failed: virtual host '~s' is down",
+                           [Vhost])
+    end.
+
+check_vhost_connection_limit(Vhost, Username) ->
+    case rabbit_vhost_limit:is_over_connection_limit(Vhost) of
+        false ->
+            ok;
+        {true, Limit} ->
+            protocol_error(
+              ?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED,
+              "access to vhost '~ts' refused for user '~ts': vhost connection limit (~p) is reached",
+              [Vhost, Username, Limit])
+    end.
+
+check_user_connection_limit(Username) ->
+    case rabbit_auth_backend_internal:is_over_connection_limit(Username) of
+        false ->
+            ok;
+        {true, Limit} ->
+            protocol_error(
+              ?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED,
+              "connection refused for user '~ts': user connection limit (~p) is reached",
+              [Username, Limit])
+    end.
+
+
+%% TODO Provide a means for the client to refresh the credential.
+%% This could be either via:
+%% 1. SASL (if multiple authentications are allowed on the same AMQP 1.0 connection), see
+%%    https://datatracker.ietf.org/doc/html/rfc4422#section-3.8 , or
+%% 2. Claims Based Security (CBS) extension, see https://docs.oasis-open.org/amqp/amqp-cbs/v1.0/csd01/amqp-cbs-v1.0-csd01.html
+%%    and https://github.com/rabbitmq/rabbitmq-server/issues/9259
+%%
+%% If the user does not refresh their credential on time (the only implementation currently),
+%% close the entire connection as we must assume that vhost access could have been revoked.
+%%
+%% If the user refreshes their credential on time (to be implemented), the AMQP reader should
+%% 1. rabbit_access_control:check_vhost_access/4
+%% 2. send a message to all its sessions which should then erase the topic permission cache and
+%% re-check all link permissions (i.e. whether reading / writing to exchanges / queues is still allowed).
+%% 3. cancel the current timer, and set a new timer
+%% similary as done for Stream connections, see https://github.com/rabbitmq/rabbitmq-server/issues/10292
+ensure_credential_expiry_timer(User) ->
+    case rabbit_access_control:expiry_timestamp(User) of
+        never ->
+            ok;
+        Ts when is_integer(Ts) ->
+            Time = (Ts - os:system_time(second)) * 1000,
+            rabbit_log:debug(
+              "Credential expires in ~b ms frow now (absolute timestamp = ~b seconds since epoch)",
+              [Time, Ts]),
+            case Time > 0 of
+                true ->
+                    _TimerRef = erlang:send_after(Time, self(), credential_expired),
+                    ok;
+                false ->
+                    protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+                                   "Credential expired ~b ms ago", [Time])
+            end
+    end.
+
+-spec default_user() -> none | rabbit_types:username().
+default_user() ->
+    case application:get_env(rabbit, amqp1_0_default_user) of
+        {ok, none} ->
+            none;
+        {ok, Username} when is_binary(Username) ->
+            Username
+    end.
+
+%% We don't trust the client at this point - force them to wait
+%% for a bit so they can't DOS us with repeated failed logins etc.
+silent_close_delay() ->
+    timer:sleep(?SILENT_CLOSE_DELAY).
+
+%% This function is deprecated.
+%% It could be called in 3.13 / 4.0 mixed version clusters by the old 3.13 CLI command
+%% rabbitmqctl list_amqp10_connections
+%%
+%% rabbitmqctl list_connections
+%% listing AMQP 1.0 connections in 4.0 uses rabbit_reader:info/2 instead.
+-spec info(rabbit_types:connection(), rabbit_types:info_keys()) ->
+    rabbit_types:infos().
+info(Pid, InfoItems) ->
+    case InfoItems -- ?INFO_ITEMS of
+        [] ->
+            case gen_server:call(Pid, {info, InfoItems}, infinity) of
+                {ok, InfoList} ->
+                    InfoList;
+                {error, Error} ->
+                    throw(Error)
+            end;
+        UnknownItems ->
+            throw({bad_argument, UnknownItems})
+    end.
+
+infos(Items, State) ->
+    [{Item, i(Item, State)} || Item <- Items].
+
+i(pid, #v1{}) ->
+    self();
+i(type, #v1{}) ->
+    network;
+i(protocol, #v1{}) ->
+    {1, 0};
+i(connection, #v1{connection = Val}) ->
+    Val;
+i(node, #v1{}) ->
+    node();
+i(auth_mechanism, #v1{connection = #v1_connection{auth_mechanism = Val}}) ->
+    case Val of
+        {Name, _Mod} -> Name;
+        _ -> Val
+    end;
+i(frame_max, #v1{connection = #v1_connection{outgoing_max_frame_size = Val}}) ->
+    Val;
+i(timeout, #v1{connection = #v1_connection{timeout = Millis}}) ->
+    Millis div 1000;
+i(user,
+  #v1{connection = #v1_connection{user = #user{username = Val}}}) ->
+    Val;
+i(user,
+  #v1{connection = #v1_connection{user = none}}) ->
+    '';
+i(state, S) ->
+    i(connection_state, S);
+i(connection_state, #v1{connection_state = Val}) ->
+    Val;
+i(connected_at, #v1{connection = #v1_connection{connected_at = Val}}) ->
+    Val;
+i(name, #v1{connection = #v1_connection{name = Val}}) ->
+    Val;
+i(vhost, #v1{connection = #v1_connection{vhost = Val}}) ->
+    Val;
+i(host, #v1{connection = #v1_connection{host = Val}}) ->
+    Val;
+i(port, #v1{connection = #v1_connection{port = Val}}) ->
+    Val;
+i(peer_host, #v1{connection = #v1_connection{peer_host = Val}}) ->
+    Val;
+i(peer_port, #v1{connection = #v1_connection{peer_port = Val}}) ->
+    Val;
+i(SockStat, S) when SockStat =:= recv_oct;
+                    SockStat =:= recv_cnt;
+                    SockStat =:= send_oct;
+                    SockStat =:= send_cnt;
+                    SockStat =:= send_pend ->
+    socket_info(fun (Sock) -> rabbit_net:getstat(Sock, [SockStat]) end,
+                fun ([{_, I}]) -> I end, S);
+i(ssl, #v1{sock = Sock}) -> rabbit_net:is_ssl(Sock);
+i(SSL, #v1{sock = Sock, proxy_socket = ProxySock})
+  when SSL =:= ssl_protocol;
+       SSL =:= ssl_key_exchange;
+       SSL =:= ssl_cipher;
+       SSL =:= ssl_hash ->
+    rabbit_ssl:info(SSL, {Sock, ProxySock});
+i(Cert, #v1{sock = Sock})
+  when Cert =:= peer_cert_issuer;
+       Cert =:= peer_cert_subject;
+       Cert =:= peer_cert_validity ->
+    rabbit_ssl:cert_info(Cert, Sock);
+i(client_properties, #v1{connection = #v1_connection{properties = Props}}) ->
+    %% Connection properties sent by the client.
+    %% Displayed in rabbitmq_management/priv/www/js/tmpl/connection.ejs
+    case Props of
+        undefined ->
+            [];
+        {map, Fields} ->
+            [mc_amqpl:to_091(Key, TypeVal) || {{symbol, Key}, TypeVal} <- Fields]
+    end;
+i(channels, #v1{tracked_channels = Channels}) ->
+    maps:size(Channels);
+i(channel_max, #v1{connection = #v1_connection{channel_max = Max}}) ->
+    Max;
+i(Item, #v1{}) ->
+    throw({bad_argument, Item}).
+
+%% From rabbit_reader
+socket_info(Get, Select, #v1{sock = Sock}) ->
+    case Get(Sock) of
+        {ok,    T} -> Select(T);
+        {error, _} -> ''
+    end.
+
+ignore_maintenance({map, Properties}) ->
+    lists:member(
+      {{symbol, <<"ignore-maintenance">>}, {boolean, true}},
+      Properties);
+ignore_maintenance(_) ->
+    false.
diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
new file mode 100644
index 000000000000..4ffdcb2846ba
--- /dev/null
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -0,0 +1,2325 @@
+%% 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries.  All rights reserved.
+%%
+
+-module(rabbit_amqp_session).
+
+-behaviour(gen_server).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+-include_lib("amqp10_common/include/amqp10_types.hrl").
+-include("rabbit_amqp.hrl").
+-include("mc.hrl").
+
+-define(PROTOCOL, amqp10).
+-define(HIBERNATE_AFTER, 6_000).
+-define(CREDIT_REPLY_TIMEOUT, 30_000).
+-define(UINT_OUTGOING_WINDOW, {uint, ?UINT_MAX}).
+-define(MAX_INCOMING_WINDOW, 400).
+%% "The next-outgoing-id MAY be initialized to an arbitrary value" [2.5.6]
+-define(INITIAL_OUTGOING_TRANSFER_ID, ?UINT_MAX - 3).
+%% "Note that, despite its name, the delivery-count is not a count but a
+%% sequence number initialized at an arbitrary point by the sender." [2.6.7]
+-define(INITIAL_DELIVERY_COUNT, ?UINT_MAX - 4).
+-define(INITIAL_OUTGOING_DELIVERY_ID, 0).
+-define(DEFAULT_MAX_HANDLE, ?UINT_MAX).
+%% [3.4]
+-define(OUTCOMES, [?V_1_0_SYMBOL_ACCEPTED,
+                   ?V_1_0_SYMBOL_REJECTED,
+                   ?V_1_0_SYMBOL_RELEASED,
+                   ?V_1_0_SYMBOL_MODIFIED]).
+-define(MAX_PERMISSION_CACHE_SIZE, 12).
+-define(TOPIC_PERMISSION_CACHE, topic_permission_cache).
+-define(PROCESS_GROUP_NAME, amqp_sessions).
+-define(UINT(N), {uint, N}).
+%% This is the link credit that we grant to sending clients.
+%% We are free to choose whatever we want, sending clients must obey.
+%% Default soft limits / credits in deps/rabbit/Makefile are:
+%% 32 for quorum queues
+%% 256 for streams
+%% 400 for classic queues
+%% If link target is a queue (rather than an exchange), we could use one of these depending
+%% on target queue type. For the time being just use a static value that's something in between.
+%% An even better approach in future would be to dynamically grow (or shrink) the link credit
+%% we grant depending on how fast target queue(s) actually confirm messages.
+-define(LINK_CREDIT_RCV, 128).
+
+-export([start_link/8,
+         process_frame/2,
+         list_local/0,
+         conserve_resources/3]).
+
+-export([init/1,
+         terminate/2,
+         handle_call/3,
+         handle_cast/2, 
+         handle_info/2,
+         format_status/1]).
+
+-import(rabbit_amqp_util,
+        [protocol_error/3]).
+-import(serial_number,
+        [add/2,
+         diff/2,
+         compare/2]).
+
+%% incoming multi transfer delivery [2.6.14]
+-record(multi_transfer_msg, {
+          payload_fragments_rev :: [binary(),...],
+          delivery_id :: delivery_number(),
+          settled :: boolean()
+         }).
+
+-record(incoming_link, {
+          exchange :: rabbit_exchange:name(),
+          routing_key :: undefined | rabbit_types:routing_key(),
+          %% queue_name_bin is only set if the link target address refers to a queue.
+          queue_name_bin :: undefined | rabbit_misc:resource_name(),
+          delivery_count :: sequence_no(),
+          credit = 0 :: non_neg_integer(),
+          %% TRANSFER delivery IDs published to queues but not yet confirmed by queues
+          incoming_unconfirmed_map = #{} :: #{delivery_number() =>
+                                              {#{rabbit_amqqueue:name() := ok},
+                                               IsTransferSettled :: boolean(),
+                                               AtLeastOneQueueConfirmed :: boolean()}},
+          multi_transfer_msg :: undefined | #multi_transfer_msg{}
+         }).
+
+-record(outgoing_link, {
+          %% Although the source address of a link might be an exchange name and binding key
+          %% or a topic filter, an outgoing link will always consume from a queue.
+          queue_name_bin :: rabbit_misc:resource_name(),
+          queue_type :: rabbit_queue_type:queue_type(),
+          send_settled :: boolean(),
+          max_message_size :: unlimited | pos_integer(),
+          %% When credit API v1 is used, our session process holds the delivery-count
+          %% When credit API v2 is used, the queue type implementation holds the delivery-count
+          %% When feature flag credit_api_v2 becomes required, this field should be deleted.
+          delivery_count :: {credit_api_v1, sequence_no()} | credit_api_v2
+         }).
+
+-record(outgoing_unsettled, {
+          %% The queue sent us this consumer scoped sequence number.
+          msg_id :: rabbit_amqqueue:msg_id(),
+          consumer_tag :: rabbit_types:ctag(),
+          queue_name :: rabbit_amqqueue:name(),
+          delivered_at :: integer()
+         }).
+
+-record(pending_transfer, {
+          frames :: iolist(),
+          queue_ack_required :: boolean(),
+          %% queue that sent us this message
+          queue_pid :: pid(),
+          delivery_id :: delivery_number(),
+          outgoing_unsettled :: #outgoing_unsettled{}
+         }).
+
+-record(cfg, {
+          outgoing_max_frame_size :: unlimited | pos_integer(),
+          reader_pid :: rabbit_types:connection(),
+          writer_pid :: pid(),
+          user :: rabbit_types:user(),
+          vhost :: rabbit_types:vhost(),
+          %% We just use the incoming (AMQP 1.0) channel number.
+          channel_num :: non_neg_integer(),
+          %% We tolerate our incoming_window to be violated by up to this number of
+          %% excess TRANSFERs. If the client sends us even more TRANSFERs, we will
+          %% close the session with session error window-violation.
+          %% Unless we decrease our incoming_window dynamically, we are strict by
+          %% default and don't allow for any excess TRANSFERs.
+          incoming_window_margin = 0 :: non_neg_integer(),
+          resource_alarms :: sets:set(rabbit_alarm:resource_alarm_source()),
+          trace_state :: rabbit_trace:state(),
+          conn_name :: binary()
+         }).
+
+-record(state, {
+          cfg :: #cfg{},
+
+          %% The following 5 fields are state for session flow control.
+          %% See section 2.5.6.
+          %%
+          %% We omit outgoing-window. We keep the outgoing-window always large and don't
+          %% restrict ourselves delivering messages fast to AMQP clients because keeping an
+          %% #outgoing_unsettled{} entry in the outgoing_unsettled_map requires far less
+          %% memory than holding the message payload in the outgoing_pending queue.
+          %%
+          %% expected implicit transfer-id of next incoming TRANSFER
+          next_incoming_id :: transfer_number(),
+          %% Defines the maximum number of incoming transfer frames that we can currently receive.
+          %% This value is chosen by us.
+          %% Purpose:
+          %% 1. It protects our session process from being overloaded, and
+          %% 2. Since frames have a maximum size for a given connection, this provides flow control based
+          %% on the number of bytes transmitted, and therefore protects our platform, i.e. RabbitMQ as a
+          %% whole. We will set this window to 0 if a cluster wide memory or disk alarm occurs (see module
+          %% rabbit_alarm) to stop receiving incoming TRANSFERs.
+          %% (It's an optional feature: If we wanted we could always keep that window huge, i.e. not
+          %% shrinking the window when we receive a TRANSFER. However, we do want to use that feature
+          %% due to aforementioned purposes.)
+          %% Can become negative up to -incoming_window_margin when client overshoots our window.
+          incoming_window :: integer(),
+          %% implicit transfer-id of our next outgoing TRANSFER
+          next_outgoing_id :: transfer_number(),
+          %% Defines the maximum number of outgoing transfer frames that we are
+          %% currently allowed to send. This value is chosen by the AMQP client.
+          remote_incoming_window :: non_neg_integer(),
+          %% This field is informational.
+          %% It reflects the maximum number of incoming TRANSFERs that may arrive without exceeding
+          %% the AMQP client's own outgoing-window.
+          %% When this window shrinks, it is an indication of outstanding transfers (from AMQP client
+          %% to us) which we need to settle (after receiving confirmations from target queues) for
+          %% the window to grow again.
+          remote_outgoing_window :: non_neg_integer(),
+
+          %% These messages were received from queues thanks to sufficient link credit.
+          %% However, they are buffered here due to session flow control
+          %% (when remote_incoming_window <= 0) before being sent to the AMQP client.
+          %%
+          %% FLOW frames are stored here as well because for a specific outgoing link the order
+          %% in which we send TRANSFER and FLOW frames is important. An outgoing FLOW frame with link flow
+          %% control information must not overtake a TRANSFER frame for the same link just because
+          %% we are throttled by session flow control. (However, we can still send outgoing FLOW frames
+          %% that contain only session flow control information, i.e. where the FLOW's 'handle' field is not set.)
+          %% Example:
+          %% A receiver grants our queue 2 credits with drain=true and the queue only has 1 message available.
+          %% Even when we are limited by session flow control, we must make sure to first send the TRANSFER to the
+          %% client (once the remote_incoming_window got opened) followed by the FLOW with drain=true and credit=0
+          %% and advanced delivery count. Otherwise, we would violate the AMQP protocol spec.
+          outgoing_pending = queue:new() :: queue:queue(#pending_transfer{} | #'v1_0.flow'{}),
+
+          %% The link or session endpoint assigns each message a unique delivery-id
+          %% from a session scoped sequence number.
+          %%
+          %% Do not confuse this field with next_outgoing_id:
+          %% Both are session scoped sequence numbers, but initialised at different arbitrary values.
+          %%
+          %% next_outgoing_id is an implicit ID, i.e. not sent in the TRANSFER frame.
+          %% outgoing_delivery_id is an explicit ID, i.e. sent in the TRANSFER frame.
+          %%
+          %% next_outgoing_id is incremented per TRANSFER frame.
+          %% outgoing_delivery_id is incremented per message.
+          %% Remember that a large message can be split up into multiple TRANSFER frames.
+          outgoing_delivery_id :: delivery_number(),
+
+          %% Links are unidirectional.
+          %% We receive messages from clients on incoming links.
+          incoming_links = #{} :: #{link_handle() => #incoming_link{}},
+          %% We send messages to clients on outgoing links.
+          outgoing_links = #{} :: #{link_handle() => #outgoing_link{}},
+
+          %% TRANSFER delivery IDs published to consuming clients but not yet acknowledged by clients.
+          outgoing_unsettled_map = #{} :: #{delivery_number() => #outgoing_unsettled{}},
+
+          %% Queue actions that we will process later such that we can confirm and reject
+          %% delivery IDs in ranges to reduce the number of DISPOSITION frames sent to the client.
+          stashed_rejected = [] :: [{rejected, rabbit_amqqueue:name(), [delivery_number(),...]}],
+          stashed_settled = [] :: [{settled, rabbit_amqqueue:name(), [delivery_number(),...]}],
+          %% Classic queues that are down.
+          stashed_down = []:: [rabbit_amqqueue:name()],
+          %% Queues that got deleted.
+          stashed_eol = [] :: [rabbit_amqqueue:name()],
+
+          queue_states = rabbit_queue_type:init() :: rabbit_queue_type:state()
+         }).
+
+start_link(ReaderPid, WriterPid, ChannelNum, FrameMax, User, Vhost, ConnName, BeginFrame) ->
+    Args = {ReaderPid, WriterPid, ChannelNum, FrameMax, User, Vhost, ConnName, BeginFrame},
+    Opts = [{hibernate_after, ?HIBERNATE_AFTER}],
+    gen_server:start_link(?MODULE, Args, Opts).
+
+process_frame(Pid, Frame) ->
+    gen_server:cast(Pid, {frame, Frame}).
+
+init({ReaderPid, WriterPid, ChannelNum, MaxFrameSize, User, Vhost, ConnName,
+      #'v1_0.begin'{next_outgoing_id = ?UINT(RemoteNextOutgoingId),
+                    incoming_window = ?UINT(RemoteIncomingWindow),
+                    outgoing_window = ?UINT(RemoteOutgoingWindow),
+                    handle_max = HandleMax0}}) ->
+    process_flag(trap_exit, true),
+    ok = pg:join(node(), ?PROCESS_GROUP_NAME, self()),
+
+    Alarms0 = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
+    Alarms = sets:from_list(Alarms0, [{version, 2}]),
+
+    NextOutgoingId = ?INITIAL_OUTGOING_TRANSFER_ID,
+    IncomingWindow = case sets:is_empty(Alarms) of
+                         true -> ?MAX_INCOMING_WINDOW;
+                         false -> 0
+                     end,
+
+    HandleMax = case HandleMax0 of
+                    ?UINT(Max) -> Max;
+                    _ -> ?DEFAULT_MAX_HANDLE
+                end,
+    Reply = #'v1_0.begin'{remote_channel = {ushort, ChannelNum},
+                          handle_max = ?UINT(HandleMax),
+                          next_outgoing_id = ?UINT(NextOutgoingId),
+                          incoming_window = ?UINT(IncomingWindow),
+                          outgoing_window = ?UINT_OUTGOING_WINDOW},
+    rabbit_amqp_writer:send_command(WriterPid, ChannelNum, Reply),
+
+    {ok, #state{next_incoming_id = RemoteNextOutgoingId,
+                next_outgoing_id = NextOutgoingId,
+                incoming_window = IncomingWindow,
+                remote_incoming_window = RemoteIncomingWindow,
+                remote_outgoing_window = RemoteOutgoingWindow,
+                outgoing_delivery_id = ?INITIAL_OUTGOING_DELIVERY_ID,
+                cfg = #cfg{reader_pid = ReaderPid,
+                           writer_pid = WriterPid,
+                           outgoing_max_frame_size = MaxFrameSize,
+                           user = User,
+                           vhost = Vhost,
+                           channel_num = ChannelNum,
+                           resource_alarms = Alarms,
+                           trace_state = rabbit_trace:init(Vhost),
+                           conn_name = ConnName
+                          }}}.
+
+terminate(_Reason, #state{incoming_links = IncomingLinks,
+                          outgoing_links = OutgoingLinks,
+                          queue_states = QStates}) ->
+    maps:foreach(
+      fun (_, _) ->
+              rabbit_global_counters:publisher_deleted(?PROTOCOL)
+      end, IncomingLinks),
+    maps:foreach(
+      fun (_, _) ->
+              rabbit_global_counters:consumer_deleted(?PROTOCOL)
+      end, OutgoingLinks),
+    ok = rabbit_queue_type:close(QStates).
+
+-spec list_local() -> [pid()].
+list_local() ->
+    pg:get_local_members(node(), ?PROCESS_GROUP_NAME).
+
+-spec conserve_resources(pid(),
+                         rabbit_alarm:resource_alarm_source(),
+                         rabbit_alarm:resource_alert()) -> ok.
+conserve_resources(Pid, Source, {_, Conserve, _}) ->
+    gen_server:cast(Pid, {conserve_resources, Source, Conserve}).
+
+handle_call(Msg, _From, State) ->
+    Reply = {error, {not_understood, Msg}},
+    reply(Reply, State).
+
+handle_info(timeout, State) ->
+    noreply(State);
+handle_info({bump_credit, _IgnoreMsg}, State) ->
+    noreply(State);
+handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
+            #state{queue_states = QStates0,
+                   stashed_eol = Eol} = State0) ->
+    case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of
+        {ok, QStates, Actions} ->
+            State1 = State0#state{queue_states = QStates},
+            State = handle_queue_actions(Actions, State1),
+            noreply(State);
+        {eol, QStates, QRef} ->
+            State = State0#state{queue_states = QStates,
+                                 stashed_eol = [QRef | Eol]},
+            noreply(State)
+    end.
+
+handle_cast({frame, Frame},
+            #state{cfg = #cfg{writer_pid = WriterPid,
+                              channel_num = Ch}} = State0) ->
+    try handle_control(Frame, State0) of
+        {reply, Replies, State} when is_list(Replies) ->
+            lists:foreach(fun (Reply) ->
+                                  rabbit_amqp_writer:send_command(WriterPid, Ch, Reply)
+                          end, Replies),
+            noreply(State);
+        {reply, Reply, State} ->
+            rabbit_amqp_writer:send_command(WriterPid, Ch, Reply),
+            noreply(State);
+        {noreply, State} ->
+            noreply(State);
+        {stop, _, _} = Stop ->
+            Stop
+    catch exit:#'v1_0.error'{} = Error ->
+              log_error_and_close_session(Error, State0);
+          exit:normal ->
+              {stop, normal, State0};
+          _:Reason:Stacktrace ->
+              {stop, {Reason, Stacktrace}, State0}
+    end;
+handle_cast({queue_event, _, _} = QEvent, State0) ->
+    try handle_queue_event(QEvent, State0) of
+        State ->
+            noreply_coalesce(State)
+    catch exit:#'v1_0.error'{} = Error ->
+              log_error_and_close_session(Error, State0)
+    end;
+handle_cast({conserve_resources, Alarm, Conserve},
+            #state{incoming_window = IncomingWindow0,
+                   cfg = #cfg{resource_alarms = Alarms0,
+                              incoming_window_margin = Margin0,
+                              writer_pid = WriterPid,
+                              channel_num = Ch} = Cfg
+                  } = State0) ->
+    Alarms = case Conserve of
+                 true -> sets:add_element(Alarm, Alarms0);
+                 false -> sets:del_element(Alarm, Alarms0)
+             end,
+    {SendFlow, IncomingWindow, Margin} =
+    case {sets:is_empty(Alarms0), sets:is_empty(Alarms)} of
+        {true, false} ->
+            %% Alarm kicked in.
+            %% Notify the client to not send us any more TRANSFERs. Since we decrase
+            %% our incoming window dynamically, there might be incoming in-flight
+            %% TRANSFERs. So, let's be lax and allow for some excess TRANSFERs.
+            {true, 0, ?MAX_INCOMING_WINDOW};
+        {false, true} ->
+            %% All alarms cleared.
+            %% Notify the client that it can resume sending us TRANSFERs.
+            {true, ?MAX_INCOMING_WINDOW, 0};
+        _ ->
+            {false, IncomingWindow0, Margin0}
+    end,
+    State = State0#state{incoming_window = IncomingWindow,
+                         cfg = Cfg#cfg{resource_alarms = Alarms,
+                                       incoming_window_margin = Margin}},
+    case SendFlow of
+        true ->
+            Flow = session_flow_fields(#'v1_0.flow'{}, State),
+            rabbit_amqp_writer:send_command(WriterPid, Ch, Flow);
+        false ->
+            ok
+    end,
+    noreply(State);
+handle_cast(refresh_config, #state{cfg = #cfg{vhost = Vhost} = Cfg} = State0) ->
+    State = State0#state{cfg = Cfg#cfg{trace_state = rabbit_trace:init(Vhost)}},
+    noreply(State).
+
+log_error_and_close_session(
+  Error, State = #state{cfg = #cfg{reader_pid = ReaderPid,
+                                   writer_pid = WriterPid,
+                                   channel_num = Ch}}) ->
+    End = #'v1_0.end'{error = Error},
+    rabbit_log:warning("Closing session for connection ~p: ~tp",
+                       [ReaderPid, Error]),
+    ok = rabbit_amqp_writer:send_command_sync(WriterPid, Ch, End),
+    {stop, {shutdown, Error}, State}.
+
+%% Batch confirms / rejects to publishers.
+noreply_coalesce(#state{stashed_rejected = [],
+                        stashed_settled = [],
+                        stashed_down = [],
+                        stashed_eol = []} = State) ->
+    noreply(State);
+noreply_coalesce(State) ->
+    Timeout = 0,
+    {noreply, State, Timeout}.
+
+noreply(State0) ->
+    State = send_buffered(State0),
+    {noreply, State}.
+
+reply(Reply, State0) ->
+    State = send_buffered(State0),
+    {reply, Reply, State}.
+
+send_buffered(State0) ->
+    State = send_delivery_state_changes(State0),
+    send_pending(State).
+
+%% Send confirms / rejects to publishers.
+send_delivery_state_changes(#state{stashed_rejected = [],
+                                   stashed_settled = [],
+                                   stashed_down = [],
+                                   stashed_eol = []} = State) ->
+    State;
+send_delivery_state_changes(State0 = #state{cfg = #cfg{writer_pid = Writer,
+                                                       channel_num = ChannelNum}}) ->
+    %% Order is important:
+    %% 1. Process queue rejections.
+    {RejectedIds, GrantCredits0, State1} = handle_stashed_rejected(State0),
+    send_dispositions(RejectedIds, #'v1_0.rejected'{}, Writer, ChannelNum),
+    %% 2. Process queue confirmations.
+    {AcceptedIds0, GrantCredits1, State2} = handle_stashed_settled(GrantCredits0, State1),
+    %% 3. Process unavailable classic queues.
+    {DetachFrames0, State3} = handle_stashed_down(State2),
+    %% 4. Process queue deletions.
+    {ReleasedIds, AcceptedIds1, DetachFrames, GrantCredits, State} = handle_stashed_eol(DetachFrames0, GrantCredits1, State3),
+    send_dispositions(ReleasedIds, #'v1_0.released'{}, Writer, ChannelNum),
+    AcceptedIds = AcceptedIds1 ++ AcceptedIds0,
+    send_dispositions(AcceptedIds, #'v1_0.accepted'{}, Writer, ChannelNum),
+    rabbit_global_counters:messages_confirmed(?PROTOCOL, length(AcceptedIds)),
+    %% Send DETACH frames after DISPOSITION frames such that
+    %% clients can handle DISPOSITIONs before closing their links.
+    lists:foreach(fun(Frame) ->
+                          rabbit_amqp_writer:send_command(Writer, ChannelNum, Frame)
+                  end, DetachFrames),
+    maps:foreach(fun(HandleInt, DeliveryCount) ->
+                         F0 = flow(?UINT(HandleInt), DeliveryCount),
+                         F = session_flow_fields(F0, State),
+                         rabbit_amqp_writer:send_command(Writer, ChannelNum, F)
+                 end, GrantCredits),
+    State.
+
+handle_stashed_rejected(#state{stashed_rejected = []} = State) ->
+    {[], #{}, State};
+handle_stashed_rejected(#state{stashed_rejected = Actions,
+                               incoming_links = Links} = State0) ->
+    {Ids, GrantCredits, Ls} =
+    lists:foldl(
+      fun({rejected, _QName, Correlations}, Accum) ->
+              lists:foldl(
+                fun({HandleInt, DeliveryId}, {Ids0, GrantCreds0, Links0} = Acc) ->
+                        case Links0 of
+                            #{HandleInt := Link0 = #incoming_link{incoming_unconfirmed_map = U0}} ->
+                                case maps:take(DeliveryId, U0) of
+                                    {{_, Settled, _}, U} ->
+                                        Ids1 = case Settled of
+                                                   true -> Ids0;
+                                                   false -> [DeliveryId | Ids0]
+                                               end,
+                                        Link1 = Link0#incoming_link{incoming_unconfirmed_map = U},
+                                        {Link, GrantCreds} = maybe_grant_link_credit(
+                                                               HandleInt, Link1, GrantCreds0),
+                                        {Ids1, GrantCreds, maps:update(HandleInt, Link, Links0)};
+                                    error ->
+                                        Acc
+                                end;
+                            _ ->
+                                Acc
+                        end
+                end, Accum, Correlations)
+      end, {[], #{}, Links}, Actions),
+
+    State = State0#state{stashed_rejected = [],
+                         incoming_links = Ls},
+    {Ids, GrantCredits, State}.
+
+handle_stashed_settled(GrantCredits, #state{stashed_settled = []} = State) ->
+    {[], GrantCredits, State};
+handle_stashed_settled(GrantCredits0, #state{stashed_settled = Actions,
+                                             incoming_links = Links} = State0) ->
+    {Ids, GrantCredits, Ls} =
+    lists:foldl(
+      fun({settled, QName, Correlations}, Accum) ->
+              lists:foldl(
+                fun({HandleInt, DeliveryId}, {Ids0, GrantCreds0, Links0} = Acc) ->
+                        case Links0 of
+                            #{HandleInt := Link0 = #incoming_link{incoming_unconfirmed_map = U0}} ->
+                                case maps:take(DeliveryId, U0) of
+                                    {{#{QName := _} = Qs, Settled, _}, U1} ->
+                                        UnconfirmedQs = map_size(Qs),
+                                        {Ids2, U} =
+                                        if UnconfirmedQs =:= 1 ->
+                                               %% last queue confirmed
+                                               Ids1 = case Settled of
+                                                          true -> Ids0;
+                                                          false -> [DeliveryId | Ids0]
+                                                      end,
+                                               {Ids1, U1};
+                                           UnconfirmedQs > 1 ->
+                                               U2 = maps:update(
+                                                      DeliveryId,
+                                                      {maps:remove(QName, Qs), Settled, true},
+                                                      U0),
+                                               {Ids0, U2}
+                                        end,
+                                        Link1 = Link0#incoming_link{incoming_unconfirmed_map = U},
+                                        {Link, GrantCreds} = maybe_grant_link_credit(
+                                                               HandleInt, Link1, GrantCreds0),
+                                        {Ids2, GrantCreds, maps:update(HandleInt, Link, Links0)};
+                                    _ ->
+                                        Acc
+                                end;
+                            _ ->
+                                Acc
+                        end
+                end, Accum, Correlations)
+      end, {[], GrantCredits0, Links}, Actions),
+
+    State = State0#state{stashed_settled = [],
+                         incoming_links = Ls},
+    {Ids, GrantCredits, State}.
+
+handle_stashed_down(#state{stashed_down = []} = State) ->
+    {[], State};
+handle_stashed_down(#state{stashed_down = QNames,
+                           outgoing_links = OutgoingLinks0} = State0) ->
+    %% We already processed queue actions settled and rejected for classic queues that are down.
+    %% Here, we destroy any outgoing links that consume from unavailable classic queues.
+    %% (This roughly corresponds to consumer_cancel_notify sent from server to client in AMQP 0.9.1.)
+    {DetachFrames, OutgoingLinks} =
+    lists:foldl(fun(#resource{name = QNameBinDown}, Acc = {_, OutgoingLinks1}) ->
+                        maps:fold(fun(Handle, Link = #outgoing_link{queue_name_bin = QNameBin}, {Frames0, Links0})
+                                        when QNameBin =:= QNameBinDown ->
+                                          Detach = detach(Handle, Link, ?V_1_0_AMQP_ERROR_ILLEGAL_STATE),
+                                          Frames = [Detach | Frames0],
+                                          Links = maps:remove(Handle, Links0),
+                                          {Frames, Links};
+                                     (_, _, Accum) ->
+                                          Accum
+                                  end, Acc, OutgoingLinks1)
+                end, {[], OutgoingLinks0}, QNames),
+    State = State0#state{stashed_down = [],
+                         outgoing_links = OutgoingLinks},
+    {DetachFrames, State}.
+
+handle_stashed_eol(DetachFrames, GrantCredits, #state{stashed_eol = []} = State) ->
+    {[], [], DetachFrames, GrantCredits, State};
+handle_stashed_eol(DetachFrames0, GrantCredits0, #state{stashed_eol = Eols} = State0) ->
+    {ReleasedIs, AcceptedIds, DetachFrames, GrantCredits, State1} =
+    lists:foldl(fun(QName, {RIds0, AIds0, DetachFrames1, GrantCreds0, S0 = #state{incoming_links = Links0,
+                                                                                  queue_states = QStates0}}) ->
+                        {RIds, AIds, GrantCreds1, Links} = settle_eol(QName, {RIds0, AIds0, GrantCreds0, Links0}),
+                        QStates = rabbit_queue_type:remove(QName, QStates0),
+                        S1 = S0#state{incoming_links = Links,
+                                      queue_states = QStates},
+                        {DetachFrames2, GrantCreds, S} = destroy_links(QName, DetachFrames1, GrantCreds1, S1),
+                        {RIds, AIds, DetachFrames2, GrantCreds, S}
+                end, {[], [], DetachFrames0, GrantCredits0, State0}, Eols),
+
+    State = State1#state{stashed_eol = []},
+    {ReleasedIs, AcceptedIds, DetachFrames, GrantCredits, State}.
+
+settle_eol(QName, {_ReleasedIds, _AcceptedIds, _GrantCredits, Links} = Acc) ->
+    maps:fold(fun(HandleInt,
+                  #incoming_link{incoming_unconfirmed_map = U0} = Link0,
+                  {RelIds0, AcceptIds0, GrantCreds0, Links0}) ->
+                      {RelIds, AcceptIds, U} = settle_eol0(QName, {RelIds0, AcceptIds0, U0}),
+                      Link1 = Link0#incoming_link{incoming_unconfirmed_map = U},
+                      {Link, GrantCreds} = maybe_grant_link_credit(
+                                             HandleInt, Link1, GrantCreds0),
+                      Links1 = maps:update(HandleInt,
+                                           Link,
+                                           Links0),
+                      {RelIds, AcceptIds, GrantCreds, Links1}
+              end, Acc, Links).
+
+settle_eol0(QName, {_ReleasedIds, _AcceptedIds, UnconfirmedMap} = Acc) ->
+    maps:fold(
+      fun(DeliveryId,
+          {#{QName := _} = Qs, Settled, AtLeastOneQueueConfirmed},
+          {RelIds, AcceptIds, U0}) ->
+              UnconfirmedQs = map_size(Qs),
+              if UnconfirmedQs =:= 1 ->
+                     %% The last queue that this delivery ID was waiting a confirm for got deleted.
+                     U = maps:remove(DeliveryId, U0),
+                     case Settled of
+                         true ->
+                             {RelIds, AcceptIds, U};
+                         false ->
+                             case AtLeastOneQueueConfirmed of
+                                 true ->
+                                     %% Since at least one queue confirmed this message, we reply to
+                                     %% the client with ACCEPTED. This allows e.g. for large fanout
+                                     %% scenarios where temporary target queues are deleted
+                                     %% (think about an MQTT subscriber disconnects).
+                                     {RelIds, [DeliveryId | AcceptIds], U};
+                                 false ->
+                                     %% Since no queue confirmed this message, we reply to the client
+                                     %% with RELEASED. (The client can then re-publish this message.)
+                                     {[DeliveryId | RelIds], AcceptIds, U}
+                             end
+                     end;
+                 UnconfirmedQs > 1 ->
+                     U = maps:update(DeliveryId,
+                                     {maps:remove(QName, Qs), Settled, AtLeastOneQueueConfirmed},
+                                     U0),
+                     {RelIds, AcceptIds, U}
+              end;
+         (_, _, A) ->
+              A
+      end, Acc, UnconfirmedMap).
+
+destroy_links(#resource{kind = queue,
+                        name = QNameBin},
+              Frames0,
+              GrantCredits0,
+              #state{incoming_links = IncomingLinks0,
+                     outgoing_links = OutgoingLinks0,
+                     outgoing_unsettled_map = Unsettled0} = State0) ->
+    {Frames1,
+     GrantCredits,
+     IncomingLinks} = maps:fold(fun(Handle, Link, Acc) ->
+                                        destroy_incoming_link(Handle, Link, QNameBin, Acc)
+                                end, {Frames0, GrantCredits0, IncomingLinks0}, IncomingLinks0),
+    {Frames,
+     Unsettled,
+     OutgoingLinks} = maps:fold(fun(Handle, Link, Acc) ->
+                                        destroy_outgoing_link(Handle, Link, QNameBin, Acc)
+                                end, {Frames1, Unsettled0, OutgoingLinks0}, OutgoingLinks0),
+    State = State0#state{incoming_links = IncomingLinks,
+                         outgoing_links = OutgoingLinks,
+                         outgoing_unsettled_map = Unsettled},
+    {Frames, GrantCredits, State}.
+
+destroy_incoming_link(Handle, Link = #incoming_link{queue_name_bin = QNameBin}, QNameBin, {Frames, GrantCreds, Links}) ->
+    {[detach(Handle, Link, ?V_1_0_AMQP_ERROR_RESOURCE_DELETED) | Frames],
+     %% Don't grant credits for a link that we destroy.
+     maps:remove(Handle, GrantCreds),
+     maps:remove(Handle, Links)};
+destroy_incoming_link(_, _, _, Acc) ->
+    Acc.
+
+destroy_outgoing_link(Handle, Link = #outgoing_link{queue_name_bin = QNameBin}, QNameBin, {Frames, Unsettled0, Links}) ->
+    {Unsettled, _RemovedMsgIds} = remove_link_from_outgoing_unsettled_map(Handle, Unsettled0),
+    {[detach(Handle, Link, ?V_1_0_AMQP_ERROR_RESOURCE_DELETED) | Frames],
+     Unsettled,
+     maps:remove(Handle, Links)};
+destroy_outgoing_link(_, _, _, Acc) ->
+    Acc.
+
+detach(Handle, Link, ErrorCondition) ->
+    rabbit_log:warning("Detaching link handle ~b due to error condition: ~tp",
+                       [Handle, ErrorCondition]),
+    publisher_or_consumer_deleted(Link),
+    #'v1_0.detach'{handle = ?UINT(Handle),
+                   closed = true,
+                   error = #'v1_0.error'{condition = ErrorCondition}}.
+
+send_dispositions(Ids, DeliveryState, Writer, ChannelNum) ->
+    Ranges = serial_number:ranges(Ids),
+    lists:foreach(fun({First, Last}) ->
+                          Disposition = disposition(DeliveryState, First, Last),
+                          rabbit_amqp_writer:send_command(Writer, ChannelNum, Disposition)
+                  end, Ranges).
+
+disposition(DeliveryState, First, Last) ->
+    Last1 = case First of
+                Last ->
+                    %% "If not set, this is taken to be the same as first." [2.7.6]
+                    %% Save a few bytes.
+                    undefined;
+                _ ->
+                    ?UINT(Last)
+            end,
+    #'v1_0.disposition'{
+       role = ?RECV_ROLE,
+       settled = true,
+       state = DeliveryState,
+       first = ?UINT(First),
+       last = Last1}.
+
+handle_control(#'v1_0.attach'{role = ?SEND_ROLE,
+                              name = LinkName,
+                              handle = InputHandle = ?UINT(HandleInt),
+                              source = Source,
+                              snd_settle_mode = SndSettleMode,
+                              target = Target,
+                              initial_delivery_count = ?UINT(DeliveryCount)} = Attach,
+               State0 = #state{incoming_links = IncomingLinks0,
+                               cfg = #cfg{vhost = Vhost,
+                                          user = User}}) ->
+    ok = validate_attach(Attach),
+    case ensure_target(Target, Vhost, User) of
+        {ok, XName, RoutingKey, QNameBin} ->
+            IncomingLink = #incoming_link{
+                              exchange = XName,
+                              routing_key = RoutingKey,
+                              queue_name_bin = QNameBin,
+                              delivery_count = DeliveryCount,
+                              credit = ?LINK_CREDIT_RCV},
+            _Outcomes = outcomes(Source),
+            OutputHandle = output_handle(InputHandle),
+            Reply = #'v1_0.attach'{
+                       name = LinkName,
+                       handle = OutputHandle,
+                       source = Source,
+                       snd_settle_mode = SndSettleMode,
+                       rcv_settle_mode = ?V_1_0_RECEIVER_SETTLE_MODE_FIRST,
+                       target = Target,
+                       %% We are the receiver.
+                       role = ?RECV_ROLE,
+                       %% "ignored if the role is receiver"
+                       initial_delivery_count = undefined,
+                       max_message_size = {ulong, persistent_term:get(max_message_size)}},
+            Flow = #'v1_0.flow'{
+                      handle = OutputHandle,
+                      link_credit = ?UINT(?LINK_CREDIT_RCV),
+                      drain = false,
+                      echo = false},
+            %%TODO check that handle is not present in either incoming_links or outgoing_links:
+            %%"The handle MUST NOT be used for other open links. An attempt to attach
+            %% using a handle which is already associated with a link MUST be responded to
+            %% with an immediate close carrying a handle-in-use session-error."
+            IncomingLinks = IncomingLinks0#{HandleInt => IncomingLink},
+            State = State0#state{incoming_links = IncomingLinks},
+            rabbit_global_counters:publisher_created(?PROTOCOL),
+            reply0([Reply, Flow], State);
+        {error, Reason} ->
+            %% TODO proper link establishment protocol here?
+            protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
+                           "Attach rejected: ~tp",
+                           [Reason])
+    end;
+
+handle_control(#'v1_0.attach'{role = ?RECV_ROLE,
+                              name = LinkName,
+                              handle = InputHandle = ?UINT(HandleInt),
+                              source = Source,
+                              snd_settle_mode = SndSettleMode,
+                              rcv_settle_mode = RcvSettleMode,
+                              max_message_size = MaybeMaxMessageSize} = Attach,
+               State0 = #state{queue_states = QStates0,
+                               outgoing_links = OutgoingLinks0,
+                               cfg = #cfg{vhost = Vhost,
+                                          user = User = #user{username = Username},
+                                          reader_pid = ReaderPid}}) ->
+    ok = validate_attach(Attach),
+    {SndSettled, EffectiveSndSettleMode} = case SndSettleMode of
+                                               ?V_1_0_SENDER_SETTLE_MODE_SETTLED ->
+                                                   {true, SndSettleMode};
+                                               _ ->
+                                                   %% In the future, we might want to support sender settle
+                                                   %% mode mixed where we would expect a settlement from the
+                                                   %% client only for durable messages.
+                                                   {false, ?V_1_0_SENDER_SETTLE_MODE_UNSETTLED}
+                                           end,
+    case ensure_source(Source, Vhost, User) of
+        {error, Reason} ->
+            protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD, "Attach rejected: ~tp", [Reason]);
+        {ok, QNameBin} ->
+            QName = rabbit_misc:r(Vhost, queue, QNameBin),
+            check_read_permitted(QName, User),
+            case rabbit_amqqueue:with(
+                   QName,
+                   fun(Q) ->
+                           try rabbit_amqqueue:check_exclusive_access(Q, ReaderPid)
+                           catch exit:#amqp_error{name = resource_locked} ->
+                                     %% An exclusive queue can only be consumed from by its declaring connection.
+                                     protocol_error(
+                                       ?V_1_0_AMQP_ERROR_RESOURCE_LOCKED,
+                                       "cannot obtain exclusive access to locked ~s",
+                                       [rabbit_misc:rs(QName)])
+                           end,
+                           QType = amqqueue:get_type(Q),
+                           %% Whether credit API v1 or v2 is used is decided only here at link attachment time.
+                           %% This decision applies to the whole life time of the link.
+                           %% This means even when feature flag credit_api_v2 will be enabled later, this consumer will
+                           %% continue to use credit API v1. This is the safest and easiest solution avoiding
+                           %% transferring link flow control state (the delivery-count) at runtime from this session
+                           %% process to the queue process.
+                           %% Eventually, after feature flag credit_api_v2 gets enabled and a subsequent rolling upgrade,
+                           %% all consumers will use credit API v2.
+                           %% Streams always use credit API v2 since the stream client (rabbit_stream_queue) holds the link
+                           %% flow control state. Hence, credit API mixed version isn't an issue for streams.
+                           {Mode,
+                            DeliveryCount} = case rabbit_feature_flags:is_enabled(credit_api_v2) orelse
+                                                  QType =:= rabbit_stream_queue of
+                                                 true ->
+                                                     {{credited, ?INITIAL_DELIVERY_COUNT}, credit_api_v2};
+                                                 false ->
+                                                     {{credited, credit_api_v1}, {credit_api_v1, ?INITIAL_DELIVERY_COUNT}}
+                                             end,
+                           Spec = #{no_ack => SndSettled,
+                                    channel_pid => self(),
+                                    limiter_pid => none,
+                                    limiter_active => false,
+                                    mode => Mode,
+                                    consumer_tag => handle_to_ctag(HandleInt),
+                                    exclusive_consume => false,
+                                    args => source_filters_to_consumer_args(Source),
+                                    ok_msg => undefined,
+                                    acting_user => Username},
+                           case rabbit_queue_type:consume(Q, Spec, QStates0) of
+                               {ok, QStates} ->
+                                   OutputHandle = output_handle(InputHandle),
+                                   A = #'v1_0.attach'{
+                                          name = LinkName,
+                                          handle = OutputHandle,
+                                          initial_delivery_count = ?UINT(?INITIAL_DELIVERY_COUNT),
+                                          snd_settle_mode = EffectiveSndSettleMode,
+                                          rcv_settle_mode = RcvSettleMode,
+                                          %% The queue process monitors our session process. When our session process
+                                          %% terminates (abnormally) any messages checked out to our session process
+                                          %% will be requeued. That's why the we only support RELEASED as the default outcome.
+                                          source = Source#'v1_0.source'{
+                                                            default_outcome = #'v1_0.released'{},
+                                                            outcomes = outcomes(Source)},
+                                          role = ?SEND_ROLE,
+                                          %% Echo back that we will respect the client's requested max-message-size.
+                                          max_message_size = MaybeMaxMessageSize},
+                                   MaxMessageSize = case MaybeMaxMessageSize of
+                                                        {ulong, Size} when Size > 0 ->
+                                                            Size;
+                                                        _ ->
+                                                            %% "If this field is zero or unset, there is no
+                                                            %% maximum size imposed by the link endpoint."
+                                                            unlimited
+                                                    end,
+                                   Link = #outgoing_link{queue_name_bin = QNameBin,
+                                                         queue_type = QType,
+                                                         send_settled = SndSettled,
+                                                         max_message_size = MaxMessageSize,
+                                                         delivery_count = DeliveryCount},
+                                   %%TODO check that handle is not present in either incoming_links or outgoing_links:
+                                   %%"The handle MUST NOT be used for other open links. An attempt to attach
+                                   %% using a handle which is already associated with a link MUST be responded to
+                                   %% with an immediate close carrying a handle-in-use session-error."
+                                   OutgoingLinks = OutgoingLinks0#{HandleInt => Link},
+                                   State1 = State0#state{queue_states = QStates,
+                                                         outgoing_links = OutgoingLinks},
+                                   rabbit_global_counters:consumer_created(?PROTOCOL),
+                                   {ok, [A], State1};
+                               {error, Reason} ->
+                                   protocol_error(
+                                     ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                     "Consuming from ~s failed: ~tp",
+                                     [rabbit_misc:rs(QName), Reason]);
+                               {protocol_error, _Type, Reason, Args} ->
+                                   protocol_error(
+                                     ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                     Reason, Args)
+                           end
+                   end) of
+                {ok, Reply, State} ->
+                    reply0(Reply, State);
+                {error, Reason} ->
+                    protocol_error(
+                      ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                      "Could not operate on ~s: ~tp",
+                      [rabbit_misc:rs(QName), Reason])
+            end
+    end;
+
+handle_control({Txfr = #'v1_0.transfer'{handle = ?UINT(Handle)}, MsgPart},
+               State0 = #state{incoming_links = IncomingLinks}) ->
+    case IncomingLinks of
+        #{Handle := Link0} ->
+            {Flows, State1} = session_flow_control_received_transfer(State0),
+           case incoming_link_transfer(Txfr, MsgPart, Link0, State1) of
+                {ok, Reply0, Link, State2} ->
+                    Reply = Reply0 ++ Flows,
+                    State = State2#state{incoming_links = maps:update(Handle, Link, IncomingLinks)},
+                    reply0(Reply, State);
+                {error, Reply0} ->
+                    %% "When an error occurs at a link endpoint, the endpoint MUST be detached
+                    %% with appropriate error information supplied in the error field of the
+                    %% detach frame. The link endpoint MUST then be destroyed." [2.6.5]
+                    Reply = Reply0 ++ Flows,
+                    State = State1#state{incoming_links = maps:remove(Handle, IncomingLinks)},
+                    reply0(Reply, State)
+            end;
+        _ ->
+            protocol_error(?V_1_0_AMQP_ERROR_ILLEGAL_STATE,
+                           "Unknown link handle: ~p", [Handle])
+    end;
+
+%% Flow control. These frames come with two pieces of information:
+%% the session window, and optionally, credit for a particular link.
+%% We'll deal with each of them separately.
+handle_control(#'v1_0.flow'{handle = Handle} = Flow,
+               #state{incoming_links = IncomingLinks,
+                      outgoing_links = OutgoingLinks} = State0) ->
+    State = session_flow_control_received_flow(Flow, State0),
+    case Handle of
+        undefined ->
+            %% "If not set, the flow frame is carrying only information
+            %% pertaining to the session endpoint." [2.7.4]
+            {noreply, State};
+        ?UINT(HandleInt) ->
+            %% "If set, indicates that the flow frame carries flow state information
+            %% for the local link endpoint associated with the given handle." [2.7.4]
+            case OutgoingLinks of
+                #{HandleInt := OutgoingLink} ->
+                    {noreply, handle_outgoing_link_flow_control(OutgoingLink, Flow, State)};
+                _ ->
+                    case IncomingLinks of
+                        #{HandleInt := _IncomingLink} ->
+                            %% We're being told about available messages at
+                            %% the sender.  Yawn. TODO at least check transfer-count?
+                            {noreply, State};
+                        _ ->
+                            %% "If set to a handle that is not currently associated with
+                            %% an attached link, the recipient MUST respond by ending the
+                            %% session with an unattached-handle session error." [2.7.4]
+                            rabbit_log:warning(
+                              "Received Flow frame for unknown link handle: ~tp", [Flow]),
+                            protocol_error(
+                              ?V_1_0_SESSION_ERROR_UNATTACHED_HANDLE,
+                              "Unattached link handle: ~b", [HandleInt])
+                    end
+            end
+    end;
+
+handle_control(#'v1_0.detach'{handle = Handle = ?UINT(HandleInt),
+                              closed = Closed},
+               State0 = #state{queue_states = QStates0,
+                               incoming_links = IncomingLinks,
+                               outgoing_links = OutgoingLinks0,
+                               outgoing_unsettled_map = Unsettled0,
+                               cfg = #cfg{
+                                        writer_pid = WriterPid,
+                                        vhost = Vhost,
+                                        user = #user{username = Username},
+                                        channel_num = Ch}}) ->
+    Ctag = handle_to_ctag(HandleInt),
+    %% TODO delete queue if closed flag is set to true? see 2.6.6
+    %% TODO keep the state around depending on the lifetime
+    {QStates, Unsettled, OutgoingLinks}
+    = case maps:take(HandleInt, OutgoingLinks0) of
+          {#outgoing_link{queue_name_bin = QNameBin}, OutgoingLinks1} ->
+              QName = rabbit_misc:r(Vhost, queue, QNameBin),
+              case rabbit_amqqueue:lookup(QName) of
+                  {ok, Q} ->
+                      %%TODO Consider adding a new rabbit_queue_type:remove_consumer API that - from the point of view of
+                      %% the queue process - behaves as if our session process terminated: All messages checked out
+                      %% to this consumer should be re-queued automatically instead of us requeueing them here after cancelling
+                      %% consumption.
+                      %% For AMQP legacy (and STOMP / MQTT) consumer cancellation not requeueing messages is a good approach as
+                      %% clients may want to ack any in-flight messages.
+                      %% For AMQP however, the consuming client can stop cancellations via link-credit=0 and drain=true being
+                      %% sure that no messages are in flight before detaching the link. Hence, AMQP doesn't need the
+                      %% rabbit_queue_type:cancel API semantics.
+                      %% A rabbit_queue_type:remove_consumer API has also the advantage to simplify reasoning about clients
+                      %% first detaching and then re-attaching to the same session with the same link handle (the handle
+                      %% becomes available for re-use once a link is closed): This will result in the same consumer tag,
+                      %% and we ideally disallow "updating" an AMQP consumer.
+                      case rabbit_queue_type:cancel(Q, Ctag, undefined, Username, QStates0) of
+                          {ok, QStates1} ->
+                              {Unsettled1, MsgIds} = remove_link_from_outgoing_unsettled_map(Ctag, Unsettled0),
+                              case MsgIds of
+                                  [] ->
+                                      {QStates1, Unsettled0, OutgoingLinks1};
+                                  _ ->
+                                      case rabbit_queue_type_settle(QName, requeue, Ctag, MsgIds, QStates1) of
+                                          {ok, QStates2, _Actions = []} ->
+                                              {QStates2, Unsettled1, OutgoingLinks1};
+                                          {protocol_error, _ErrorType, Reason, ReasonArgs} ->
+                                              protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                                             Reason, ReasonArgs)
+                                      end
+                              end;
+                          {error, Reason} ->
+                              protocol_error(
+                                ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                "Failed to cancel consuming from ~s: ~tp",
+                                [rabbit_misc:rs(amqqueue:get_name(Q)), Reason])
+                      end;
+                  {error, not_found} ->
+                      {Unsettled1, _RemovedMsgIds} = remove_link_from_outgoing_unsettled_map(Ctag, Unsettled0),
+                      {QStates0, Unsettled1, OutgoingLinks1}
+              end;
+          error ->
+              {Unsettled1, _RemovedMsgIds} = remove_link_from_outgoing_unsettled_map(Ctag, Unsettled0),
+              {QStates0, Unsettled1, OutgoingLinks0}
+      end,
+    State = State0#state{queue_states = QStates,
+                         incoming_links = maps:remove(HandleInt, IncomingLinks),
+                         outgoing_links = OutgoingLinks,
+                         outgoing_unsettled_map = Unsettled},
+    ok = rabbit_amqp_writer:send_command(
+           WriterPid, Ch, #'v1_0.detach'{handle = Handle,
+                                         closed = Closed}),
+    publisher_or_consumer_deleted(State, State0),
+    {noreply, State};
+
+handle_control(#'v1_0.end'{},
+               State0 = #state{cfg = #cfg{writer_pid = WriterPid,
+                                          channel_num = Ch}}) ->
+    State = send_delivery_state_changes(State0),
+    ok = try rabbit_amqp_writer:send_command_sync(WriterPid, Ch, #'v1_0.end'{})
+         catch exit:{Reason, {gen_server, call, _ArgList}}
+                 when Reason =:= shutdown orelse
+                      Reason =:= noproc ->
+                   %% AMQP connection and therefore the writer process got already terminated
+                   %% before we had the chance to synchronously end the session.
+                   ok
+         end,
+    {stop, normal, State};
+
+handle_control(#'v1_0.disposition'{role = ?RECV_ROLE,
+                                   first = ?UINT(First),
+                                   last = Last0,
+                                   state = Outcome,
+                                   settled = DispositionSettled} = Disposition,
+               #state{outgoing_unsettled_map = UnsettledMap,
+                      queue_states = QStates0} = State0) ->
+    Last = case Last0 of
+               ?UINT(L) ->
+                   L;
+               undefined ->
+                   %% "If not set, this is taken to be the same as first." [2.7.6]
+                   First
+           end,
+    UnsettledMapSize = map_size(UnsettledMap),
+    case UnsettledMapSize of
+        0 ->
+            {noreply, State0};
+        _ ->
+            DispositionRangeSize = diff(Last, First) + 1,
+            {Settled, UnsettledMap1} =
+            case DispositionRangeSize =< UnsettledMapSize of
+                true ->
+                    %% It is cheaper to iterate over the range of settled delivery IDs.
+                    serial_number:foldl(fun settle_delivery_id/2, {#{}, UnsettledMap}, First, Last);
+                false ->
+                    %% It is cheaper to iterate over the outgoing unsettled map.
+                    maps:fold(
+                      fun (DeliveryId,
+                           #outgoing_unsettled{queue_name = QName,
+                                               consumer_tag = Ctag,
+                                               msg_id = MsgId} = Unsettled,
+                           {SettledAcc, UnsettledAcc}) ->
+                              DeliveryIdComparedToFirst = compare(DeliveryId, First),
+                              DeliveryIdComparedToLast = compare(DeliveryId, Last),
+                              if DeliveryIdComparedToFirst =:= less orelse
+                                 DeliveryIdComparedToLast =:= greater ->
+                                     %% Delivery ID is outside the DISPOSITION range.
+                                     {SettledAcc, UnsettledAcc#{DeliveryId => Unsettled}};
+                                 true ->
+                                     %% Delivery ID is inside the DISPOSITION range.
+                                     SettledAcc1 = maps:update_with(
+                                                     {QName, Ctag},
+                                                     fun(MsgIds) -> [MsgId | MsgIds] end,
+                                                     [MsgId],
+                                                     SettledAcc),
+                                     {SettledAcc1, UnsettledAcc}
+                              end
+                      end,
+                      {#{}, #{}}, UnsettledMap)
+            end,
+
+            SettleOp = settle_op_from_outcome(Outcome),
+            {QStates, Actions} =
+            maps:fold(
+              fun({QName, Ctag}, MsgIds, {QS0, ActionsAcc}) ->
+                      case rabbit_queue_type_settle(QName, SettleOp, Ctag, MsgIds, QS0) of
+                          {ok, QS, Actions0} ->
+                              messages_acknowledged(SettleOp, QName, QS, MsgIds),
+                              {QS, ActionsAcc ++ Actions0};
+                          {protocol_error, _ErrorType, Reason, ReasonArgs} ->
+                              protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                             Reason, ReasonArgs)
+                      end
+              end, {QStates0, []}, Settled),
+
+            State1 = State0#state{outgoing_unsettled_map = UnsettledMap1,
+                                  queue_states = QStates},
+            Reply = case DispositionSettled of
+                        true  -> [];
+                        false -> [Disposition#'v1_0.disposition'{settled = true,
+                                                                 role = ?SEND_ROLE}]
+                    end,
+            State = handle_queue_actions(Actions, State1),
+            reply0(Reply, State)
+    end;
+
+handle_control(Frame, _State) ->
+    protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                   "Unexpected frame ~tp",
+                   [amqp10_framing:pprint(Frame)]).
+
+rabbit_queue_type_settle(QName, SettleOp, Ctag, MsgIds0, QStates) ->
+    %% Classic queues expect message IDs in sorted order.
+    MsgIds = lists:usort(MsgIds0),
+    rabbit_queue_type:settle(QName, SettleOp, Ctag, MsgIds, QStates).
+
+send_pending(#state{remote_incoming_window = Space,
+                    outgoing_pending = Buf0,
+                    cfg = #cfg{writer_pid = WriterPid,
+                               channel_num = Ch}} = State0) ->
+    case queue:out(Buf0) of
+        {empty, _} ->
+            State0;
+        {{value, #'v1_0.flow'{} = Flow0}, Buf} ->
+            Flow = session_flow_fields(Flow0, State0),
+            rabbit_amqp_writer:send_command(WriterPid, Ch, Flow),
+            send_pending(State0#state{outgoing_pending = Buf});
+        {{value, #pending_transfer{
+                    frames = Frames,
+                    queue_pid = QPid,
+                    outgoing_unsettled = #outgoing_unsettled{
+                                            consumer_tag = Ctag,
+                                            queue_name = QName}} = Pending}, Buf1}
+          when Space > 0 ->
+            SendFun = case rabbit_queue_type:module(QName, State0#state.queue_states) of
+                          {ok, rabbit_classic_queue} ->
+                              fun(Transfer, Sections) ->
+                                      rabbit_amqp_writer:send_command_and_notify(
+                                        WriterPid, Ch, QPid, self(), Transfer, Sections)
+                              end;
+                          {ok, _QType} ->
+                              fun(Transfer, Sections) ->
+                                      rabbit_amqp_writer:send_command(
+                                        WriterPid, Ch, Transfer, Sections)
+                              end
+                      end,
+            %% rabbit_basic:maybe_gc_large_msg(Content, GCThreshold)
+            case send_frames(SendFun, Frames, Space) of
+                {all, SpaceLeft} ->
+                    State1 = #state{outgoing_links = OutgoingLinks0} = session_flow_control_sent_transfers(
+                                                                         Space - SpaceLeft, State0),
+                    HandleInt = ctag_to_handle(Ctag),
+                    OutgoingLinks = maps:update_with(
+                                      HandleInt,
+                                      fun(#outgoing_link{delivery_count = {credit_api_v1, C}} = Link) ->
+                                              Link#outgoing_link{delivery_count = {credit_api_v1, add(C, 1)}};
+                                         (#outgoing_link{delivery_count = credit_api_v2} = Link) ->
+                                              Link
+                                      end,
+                                      OutgoingLinks0),
+                    State2 = State1#state{outgoing_links = OutgoingLinks},
+                    State = record_outgoing_unsettled(Pending, State2),
+                    send_pending(State#state{outgoing_pending = Buf1});
+                {some, Rest} ->
+                    State = session_flow_control_sent_transfers(Space, State0),
+                    Buf = queue:in_r(Pending#pending_transfer{frames = Rest}, Buf1),
+                    send_pending(State#state{outgoing_pending = Buf})
+            end;
+        {{value, #pending_transfer{}}, _}
+          when Space =:= 0 ->
+            State0
+    end.
+
+send_frames(_, [], Left) ->
+    {all, Left};
+send_frames(_, Rest, 0) ->
+    {some, Rest};
+send_frames(SendFun, [[Transfer, Sections] | Rest], Left) ->
+    ok = SendFun(Transfer, Sections),
+    send_frames(SendFun, Rest, Left - 1).
+
+record_outgoing_unsettled(#pending_transfer{queue_ack_required = true,
+                                            delivery_id = DeliveryId,
+                                            outgoing_unsettled = Unsettled},
+                          #state{outgoing_unsettled_map = Map0} = State) ->
+    %% Record by DeliveryId such that we will ack this message to the queue
+    %% once we receive the DISPOSITION from the AMQP client.
+    Map = Map0#{DeliveryId => Unsettled},
+    State#state{outgoing_unsettled_map = Map};
+record_outgoing_unsettled(#pending_transfer{queue_ack_required = false}, State) ->
+    %% => 'snd-settle-mode' at attachment must have been 'settled'.
+    %% => 'settled' field in TRANSFER must have been 'true'.
+    %% => AMQP client won't ack this message.
+    %% Also, queue client already acked to queue on behalf of us.
+    State.
+
+reply0([], State) ->
+    {noreply, State};
+reply0(Reply, State) ->
+    {reply, session_flow_fields(Reply, State), State}.
+
+%% Implements section "receiving a transfer" in 2.5.6
+session_flow_control_received_transfer(
+  #state{next_incoming_id = NextIncomingId,
+         incoming_window = InWindow0,
+         remote_outgoing_window = RemoteOutgoingWindow,
+         cfg = #cfg{incoming_window_margin = Margin,
+                    resource_alarms = Alarms}
+        } = State) ->
+    InWindow1 = InWindow0 - 1,
+    case InWindow1 < -Margin of
+        true ->
+            protocol_error(
+              ?V_1_0_SESSION_ERROR_WINDOW_VIOLATION,
+              "incoming window violation (tolerated excess tranfers: ~b)",
+              [Margin]);
+        false ->
+            ok
+    end,
+    {Flows, InWindow} = case InWindow1 =< (?MAX_INCOMING_WINDOW div 2) andalso
+                             sets:is_empty(Alarms) of
+                            true ->
+                                %% We've reached halfway and there are no
+                                %% disk or memory alarm, open the window.
+                                {[#'v1_0.flow'{}], ?MAX_INCOMING_WINDOW};
+                            false ->
+                                {[], InWindow1}
+                        end,
+    {Flows, State#state{incoming_window = InWindow,
+                        next_incoming_id = add(NextIncomingId, 1),
+                        remote_outgoing_window = RemoteOutgoingWindow - 1}}.
+
+%% Implements section "sending a transfer" in 2.5.6
+session_flow_control_sent_transfers(
+  NumTransfers,
+  #state{remote_incoming_window = RemoteIncomingWindow,
+         next_outgoing_id = NextOutgoingId} = State) ->
+    State#state{remote_incoming_window = RemoteIncomingWindow - NumTransfers,
+                next_outgoing_id = add(NextOutgoingId, NumTransfers)}.
+
+settle_delivery_id(Current, {Settled, Unsettled}) ->
+    case maps:take(Current, Unsettled) of
+        {#outgoing_unsettled{queue_name = QName,
+                             consumer_tag = Ctag,
+                             msg_id = MsgId}, Unsettled1} ->
+            Settled1 = maps:update_with(
+                         {QName, Ctag},
+                         fun(MsgIds) -> [MsgId | MsgIds] end,
+                         [MsgId],
+                         Settled),
+            {Settled1, Unsettled1};
+        error ->
+            {Settled, Unsettled}
+    end.
+
+settle_op_from_outcome(#'v1_0.accepted'{}) ->
+    complete;
+settle_op_from_outcome(#'v1_0.rejected'{}) ->
+    discard;
+settle_op_from_outcome(#'v1_0.released'{}) ->
+    requeue;
+%% Keep the same Modified behaviour as in RabbitMQ 3.x
+settle_op_from_outcome(#'v1_0.modified'{delivery_failed = true,
+                                        undeliverable_here = UndelHere})
+  when UndelHere =/= true ->
+    requeue;
+settle_op_from_outcome(#'v1_0.modified'{}) ->
+    %% If delivery_failed is not true, we can't increment its delivery_count.
+    %% So, we will have to reject without requeue.
+    %%
+    %% If undeliverable_here is true, this is not quite correct because
+    %% undeliverable_here refers to the link, and not the message in general.
+    %% However, we cannot filter messages from being assigned to individual consumers.
+    %% That's why we will have to reject it without requeue.
+    discard;
+settle_op_from_outcome(Outcome) ->
+    protocol_error(
+      ?V_1_0_AMQP_ERROR_INVALID_FIELD,
+      "Unrecognised state: ~tp in DISPOSITION",
+      [Outcome]).
+
+flow(Handle, DeliveryCount) ->
+    #'v1_0.flow'{handle = Handle,
+                 delivery_count = ?UINT(DeliveryCount),
+                 link_credit = ?UINT(?LINK_CREDIT_RCV)}.
+
+session_flow_fields(Frames, State)
+  when is_list(Frames) ->
+    [session_flow_fields(F, State) || F <- Frames];
+session_flow_fields(Flow = #'v1_0.flow'{},
+                    #state{next_outgoing_id = NextOutgoingId,
+                           next_incoming_id = NextIncomingId,
+                           incoming_window = IncomingWindow}) ->
+    Flow#'v1_0.flow'{
+           next_outgoing_id = ?UINT(NextOutgoingId),
+           outgoing_window = ?UINT_OUTGOING_WINDOW,
+           next_incoming_id = ?UINT(NextIncomingId),
+           incoming_window = ?UINT(IncomingWindow)};
+session_flow_fields(Frame, _State) ->
+    Frame.
+
+%% Implements section "receiving a flow" in 2.5.6
+session_flow_control_received_flow(
+  #'v1_0.flow'{next_incoming_id = FlowNextIncomingId,
+               incoming_window = ?UINT(FlowIncomingWindow),
+               next_outgoing_id = ?UINT(FlowNextOutgoingId),
+               outgoing_window = ?UINT(FlowOutgoingWindow)},
+  #state{next_outgoing_id = NextOutgoingId} = State) ->
+
+    Seq = case FlowNextIncomingId of
+              ?UINT(Id) ->
+                  case compare(Id, NextOutgoingId) of
+                      greater ->
+                          protocol_error(
+                            ?V_1_0_SESSION_ERROR_WINDOW_VIOLATION,
+                            "next-incoming-id from FLOW (~b) leads next-outgoing-id (~b)",
+                            [Id, NextOutgoingId]);
+                      _ ->
+                          Id
+                  end;
+              undefined ->
+                  %% The AMQP client might not have yet received our #begin.next_outgoing_id
+                  ?INITIAL_OUTGOING_TRANSFER_ID
+          end,
+
+    RemoteIncomingWindow0 = diff(add(Seq, FlowIncomingWindow), NextOutgoingId),
+    %% RemoteIncomingWindow0 can be negative, for example if we sent a TRANSFER to the
+    %% client between the point in time the client sent us a FLOW with updated
+    %% incoming_window=0 and we received that FLOW. Whether 0 or negative doesn't matter:
+    %% In both cases we're blocked sending more TRANSFERs to the client until it sends us
+    %% a new FLOW with a positive incoming_window. For better understandibility
+    %% across the code base, we ensure a floor of 0 here.
+    RemoteIncomingWindow = max(0, RemoteIncomingWindow0),
+
+    State#state{next_incoming_id = FlowNextOutgoingId,
+                remote_outgoing_window = FlowOutgoingWindow,
+                remote_incoming_window = RemoteIncomingWindow}.
+
+% TODO: validate effective settle modes against
+%       those declared during attach
+
+% TODO: handle aborted transfers
+
+handle_queue_event({queue_event, QRef, Evt},
+                   #state{queue_states = QStates0} = S0) ->
+    case rabbit_queue_type:handle_event(QRef, Evt, QStates0) of
+        {ok, QStates1, Actions} ->
+            S = S0#state{queue_states = QStates1},
+            handle_queue_actions(Actions, S);
+        {eol, Actions} ->
+            S = handle_queue_actions(Actions, S0),
+            S#state{stashed_eol = [QRef | S#state.stashed_eol]};
+        {protocol_error, _Type, Reason, ReasonArgs} ->
+            protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR, Reason, ReasonArgs)
+    end.
+
+handle_queue_actions(Actions, State) ->
+    lists:foldl(
+      fun ({settled, _QName, _DelIds} = Action, S = #state{stashed_settled = As}) ->
+              S#state{stashed_settled = [Action | As]};
+          ({rejected, _QName, _DelIds} = Action, S = #state{stashed_rejected = As}) ->
+              S#state{stashed_rejected = [Action | As]};
+          ({deliver, CTag, AckRequired, Msgs}, S0) ->
+              lists:foldl(fun(Msg, S) ->
+                                  handle_deliver(CTag, AckRequired, Msg, S)
+                          end, S0, Msgs);
+          ({credit_reply, Ctag, DeliveryCount, Credit, Available,  Drain},
+           S = #state{outgoing_pending = Pending}) ->
+              %% credit API v2
+              Handle = ctag_to_handle(Ctag),
+              Flow = #'v1_0.flow'{
+                        handle = ?UINT(Handle),
+                        delivery_count = ?UINT(DeliveryCount),
+                        link_credit = ?UINT(Credit),
+                        available = ?UINT(Available),
+                        drain = Drain},
+              S#state{outgoing_pending = queue:in(Flow, Pending)};
+          ({credit_reply_v1, Ctag, Credit0, Available, Drain},
+           S0 = #state{outgoing_links = OutgoingLinks0,
+                       outgoing_pending = Pending}) ->
+              %% credit API v1
+              %% Delete this branch when feature flag credit_api_v2 becomes required.
+              Handle = ctag_to_handle(Ctag),
+              Link = #outgoing_link{delivery_count = {credit_api_v1, Count0}} = maps:get(Handle, OutgoingLinks0),
+              {Count, Credit, S} = case Drain of
+                                       true ->
+                                           Count1 = add(Count0, Credit0),
+                                           OutgoingLinks = maps:update(
+                                                             Handle,
+                                                             Link#outgoing_link{delivery_count = {credit_api_v1,  Count1}},
+                                                             OutgoingLinks0),
+                                           S1 = S0#state{outgoing_links = OutgoingLinks},
+                                           {Count1, 0, S1};
+                                       false ->
+                                           {Count0, Credit0, S0}
+                                   end,
+              Flow = #'v1_0.flow'{
+                        handle = ?UINT(Handle),
+                        delivery_count = ?UINT(Count),
+                        link_credit = ?UINT(Credit),
+                        available = ?UINT(Available),
+                        drain = Drain},
+              S#state{outgoing_pending = queue:in(Flow, Pending)};
+          ({queue_down, QName}, S = #state{stashed_down = L}) ->
+              S#state{stashed_down = [QName | L]};
+          ({Action, _QName}, S)
+            when Action =:= block orelse
+                 Action =:= unblock ->
+              %% Ignore since we rely on our own mechanism to detect if a client sends to fast
+              %% into a link: If the number of outstanding queue confirmations grows,
+              %% we won't grant new credits to publishers.
+              S
+      end, State, Actions).
+
+handle_deliver(ConsumerTag, AckRequired,
+               Msg = {QName, QPid, MsgId, Redelivered, Mc0},
+               State = #state{outgoing_pending = Pending,
+                              outgoing_delivery_id = DeliveryId,
+                              outgoing_links = OutgoingLinks,
+                              cfg = #cfg{outgoing_max_frame_size = MaxFrameSize,
+                                         conn_name = ConnName,
+                                         channel_num = ChannelNum,
+                                         user = #user{username = Username},
+                                         trace_state = Trace}}) ->
+    Handle = ctag_to_handle(ConsumerTag),
+    case OutgoingLinks of
+        #{Handle := #outgoing_link{queue_type = QType,
+                                   send_settled = SendSettled,
+                                   max_message_size = MaxMessageSize}} ->
+            Dtag = delivery_tag(MsgId, SendSettled),
+            Transfer = #'v1_0.transfer'{
+                          handle = ?UINT(Handle),
+                          delivery_id = ?UINT(DeliveryId),
+                          delivery_tag = {binary, Dtag},
+                          message_format = ?UINT(0), % [3.2.16]
+                          settled = SendSettled},
+            Mc1 = mc:convert(mc_amqp, Mc0),
+            Mc = mc:set_annotation(redelivered, Redelivered, Mc1),
+            Sections0 = mc:protocol_state(Mc),
+            Sections = mc_amqp:serialize(Sections0),
+            ?DEBUG("~s Outbound content:~n  ~tp~n",
+                   [?MODULE, [amqp10_framing:pprint(Section) ||
+                              Section <- amqp10_framing:decode_bin(iolist_to_binary(Sections))]]),
+            validate_message_size(Sections, MaxMessageSize),
+            Frames = case MaxFrameSize of
+                         unlimited ->
+                             [[Transfer, Sections]];
+                         _ ->
+                             %% TODO Ugh
+                             TLen = iolist_size(amqp10_framing:encode_bin(Transfer)),
+                             encode_frames(Transfer, Sections, MaxFrameSize - TLen, [])
+                     end,
+            messages_delivered(Redelivered, QType),
+            rabbit_trace:tap_out(Msg, ConnName, ChannelNum, Username, Trace),
+            Del = #outgoing_unsettled{
+                     msg_id = MsgId,
+                     consumer_tag = ConsumerTag,
+                     queue_name = QName,
+                     %% The consumer timeout interval starts already from the point in time the
+                     %% queue sent us the message so that the Ra log can be truncated even if
+                     %% the message is sitting here for a long time.
+                     delivered_at = os:system_time(millisecond)},
+            PendingTransfer = #pending_transfer{
+                                 frames = Frames,
+                                 queue_ack_required = AckRequired,
+                                 queue_pid = QPid,
+                                 delivery_id = DeliveryId,
+                                 outgoing_unsettled = Del},
+            State#state{outgoing_delivery_id = add(DeliveryId, 1),
+                        outgoing_pending = queue:in(PendingTransfer, Pending)};
+        _ ->
+            %% TODO handle missing link -- why does the queue think it's there?
+            rabbit_log:warning(
+              "No link handle ~b exists for delivery with consumer tag ~p from queue ~tp",
+              [Handle, ConsumerTag, QName]),
+            State
+    end.
+
+%% "The delivery-tag MUST be unique amongst all deliveries that could be
+%% considered unsettled by either end of the link." [2.6.12]
+delivery_tag(MsgId, _)
+  when is_integer(MsgId) ->
+    %% We use MsgId (the consumer scoped sequence number from the queue) as
+    %% delivery-tag since delivery-tag must be unique only per link (not per session).
+    %% "A delivery-tag can be up to 32 octets of binary data." [2.8.7]
+    case MsgId =< ?UINT_MAX of
+        true -> <>;
+        false -> <>
+    end;
+delivery_tag(undefined, true) ->
+    %% Both ends of the link will always consider this message settled because
+    %% "the sender will send all deliveries settled to the receiver" [3.8.2].
+    %% Hence, the delivery tag does not have to be unique on this link.
+    %% However, the spec still mandates to send a delivery tag.
+    <<>>;
+%% Message comes from a (classic) priority queue.
+delivery_tag({Priority, undefined}, true)
+  when is_integer(Priority) ->
+    <<>>;
+delivery_tag(MsgId = {Priority, Seq}, _)
+  when is_integer(Priority) andalso
+       is_integer(Seq) ->
+    term_to_binary(MsgId).
+
+%%%%%%%%%%%%%%%%%%%%%
+%%% Incoming Link %%%
+%%%%%%%%%%%%%%%%%%%%%
+
+incoming_link_transfer(
+  #'v1_0.transfer'{more = true,
+                   %% "The delivery-id MUST be supplied on the first transfer of a multi-transfer delivery."
+                   delivery_id = ?UINT(DeliveryId),
+                   settled = Settled},
+  MsgPart,
+  Link0 = #incoming_link{multi_transfer_msg = undefined},
+  State) ->
+    %% This is the first transfer of a multi-transfer message.
+    Link = Link0#incoming_link{
+             multi_transfer_msg = #multi_transfer_msg{
+                                     payload_fragments_rev = [MsgPart],
+                                     delivery_id = DeliveryId,
+                                     %% "If not set on the first (or only) transfer for a (multi-transfer)
+                                     %% delivery, then the settled flag MUST be interpreted as being false."
+                                     settled = default(Settled, false)}},
+    {ok, [], Link, State};
+incoming_link_transfer(
+  #'v1_0.transfer'{more = true,
+                   delivery_id = DeliveryId,
+                   settled = Settled},
+  MsgPart,
+  Link0 = #incoming_link{
+             multi_transfer_msg = Multi = #multi_transfer_msg{
+                                             payload_fragments_rev = PFR0,
+                                             delivery_id = FirstDeliveryId,
+                                             settled = FirstSettled}},
+  State) ->
+    %% This is a continuation transfer with even more transfers to come.
+    validate_multi_transfer_delivery_id(DeliveryId, FirstDeliveryId),
+    validate_multi_transfer_settled(Settled, FirstSettled),
+    PFR = [MsgPart | PFR0],
+    validate_incoming_message_size(PFR),
+    Link = Link0#incoming_link{multi_transfer_msg = Multi#multi_transfer_msg{payload_fragments_rev = PFR}},
+    {ok, [], Link, State};
+incoming_link_transfer(
+  #'v1_0.transfer'{handle = ?UINT(HandleInt)},
+  _,
+  #incoming_link{credit = Credit} = Link,
+  _)
+  when Credit =< 0 ->
+    Detach = detach(HandleInt, Link, ?V_1_0_LINK_ERROR_TRANSFER_LIMIT_EXCEEDED),
+    {error, [Detach]};
+incoming_link_transfer(
+  #'v1_0.transfer'{delivery_id = MaybeDeliveryId,
+                   delivery_tag = DeliveryTag,
+                   settled = MaybeSettled,
+                   rcv_settle_mode = RcvSettleMode,
+                   handle = Handle = ?UINT(HandleInt)},
+  MsgPart,
+  #incoming_link{exchange = XName = #resource{name = XNameBin},
+                 routing_key = LinkRKey,
+                 delivery_count = DeliveryCount0,
+                 incoming_unconfirmed_map = U0,
+                 credit = Credit0,
+                 multi_transfer_msg = MultiTransfer
+                } = Link0,
+  State0 = #state{queue_states = QStates0,
+                  cfg = #cfg{user = User = #user{username = Username},
+                             trace_state = Trace,
+                             conn_name = ConnName,
+                             channel_num = ChannelNum}}) ->
+
+    {MsgBin, DeliveryId, Settled} =
+    case MultiTransfer of
+        undefined ->
+            ?UINT(DeliveryId0) = MaybeDeliveryId,
+            {MsgPart, DeliveryId0, default(MaybeSettled, false)};
+        #multi_transfer_msg{payload_fragments_rev = PFR,
+                            delivery_id = FirstDeliveryId,
+                            settled = FirstSettled} ->
+            MsgBin0 = iolist_to_binary(lists:reverse([MsgPart | PFR])),
+            ok = validate_multi_transfer_delivery_id(MaybeDeliveryId, FirstDeliveryId),
+            ok = validate_multi_transfer_settled(MaybeSettled, FirstSettled),
+            {MsgBin0, FirstDeliveryId, FirstSettled}
+    end,
+    validate_incoming_message_size(MsgBin),
+
+    Sections = amqp10_framing:decode_bin(MsgBin),
+    ?DEBUG("~s Inbound content:~n  ~tp",
+           [?MODULE, [amqp10_framing:pprint(Section) || Section <- Sections]]),
+    Anns0 = #{?ANN_EXCHANGE => XNameBin},
+    Anns = case LinkRKey of
+               undefined -> Anns0;
+               _ -> Anns0#{?ANN_ROUTING_KEYS => [LinkRKey]}
+           end,
+    Mc0 = mc:init(mc_amqp, Sections, Anns),
+    Mc = rabbit_message_interceptor:intercept(Mc0),
+    check_user_id(Mc, User),
+    RoutingKeys = mc:routing_keys(Mc),
+    RoutingKey = routing_key(RoutingKeys, XName),
+    messages_received(Settled),
+    case rabbit_exchange:lookup(XName) of
+        {ok, Exchange} ->
+            check_write_permitted_on_topic(Exchange, User, RoutingKey),
+            QNames = rabbit_exchange:route(Exchange, Mc, #{return_binding_keys => true}),
+            rabbit_trace:tap_in(Mc, QNames, ConnName, ChannelNum, Username, Trace),
+            case not Settled andalso
+                 RcvSettleMode =:= ?V_1_0_RECEIVER_SETTLE_MODE_SECOND of
+                true -> protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+                                       "rcv-settle-mode second not supported", []);
+                false -> ok
+            end,
+            Opts = #{correlation => {HandleInt, DeliveryId}},
+            Qs0 = rabbit_amqqueue:lookup_many(QNames),
+            Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0),
+            case rabbit_queue_type:deliver(Qs, Mc, Opts, QStates0) of
+                {ok, QStates, Actions} ->
+                    State1 = State0#state{queue_states = QStates},
+                    %% Confirms must be registered before processing actions
+                    %% because actions may contain rejections of publishes.
+                    {U, Reply0} = process_routing_confirm(
+                                    Qs, Settled, DeliveryId, U0),
+                    State = handle_queue_actions(Actions, State1),
+                    DeliveryCount = add(DeliveryCount0, 1),
+                    Credit1 = Credit0 - 1,
+                    {Credit, Reply1} = maybe_grant_link_credit(
+                                         Credit1, DeliveryCount, map_size(U), Handle),
+                    Reply = Reply0 ++ Reply1,
+                    Link = Link0#incoming_link{
+                             delivery_count = DeliveryCount,
+                             credit = Credit,
+                             incoming_unconfirmed_map = U,
+                             multi_transfer_msg = undefined},
+                    {ok, Reply, Link, State};
+                {error, Reason} ->
+                    protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
+                                   "Failed to deliver message to queues, "
+                                   "delivery_tag=~p, delivery_id=~p, reason=~p",
+                                   [DeliveryTag, DeliveryId, Reason])
+            end;
+        {error, not_found} ->
+            Disposition = released(DeliveryId),
+            Detach = detach(HandleInt, Link0, ?V_1_0_AMQP_ERROR_RESOURCE_DELETED),
+            {error, [Disposition, Detach]}
+    end.
+
+process_routing_confirm([], _SenderSettles = true, _, U) ->
+    rabbit_global_counters:messages_unroutable_dropped(?PROTOCOL, 1),
+    {U, []};
+process_routing_confirm([], _SenderSettles = false, DeliveryId, U) ->
+    rabbit_global_counters:messages_unroutable_returned(?PROTOCOL, 1),
+    Disposition = released(DeliveryId),
+    {U, [Disposition]};
+process_routing_confirm([_|_] = Qs, SenderSettles, DeliveryId, U0) ->
+    QNames = rabbit_amqqueue:queue_names(Qs),
+    false = maps:is_key(DeliveryId, U0),
+    Map = maps:from_keys(QNames, ok),
+    U = U0#{DeliveryId => {Map, SenderSettles, false}},
+    rabbit_global_counters:messages_routed(?PROTOCOL, map_size(Map)),
+    {U, []}.
+
+released(DeliveryId) ->
+    #'v1_0.disposition'{role = ?RECV_ROLE,
+                        first = ?UINT(DeliveryId),
+                        settled = true,
+                        state = #'v1_0.released'{}}.
+
+maybe_grant_link_credit(Credit, DeliveryCount, NumUnconfirmed, Handle) ->
+    case grant_link_credit(Credit, NumUnconfirmed) of
+        true ->
+            {?LINK_CREDIT_RCV, [flow(Handle, DeliveryCount)]};
+        false ->
+            {Credit, []}
+    end.
+
+maybe_grant_link_credit(
+  HandleInt,
+  Link = #incoming_link{credit = Credit,
+                        incoming_unconfirmed_map = U,
+                        delivery_count = DeliveryCount},
+  AccMap) ->
+    case grant_link_credit(Credit, map_size(U)) of
+        true ->
+            {Link#incoming_link{credit = ?LINK_CREDIT_RCV},
+             AccMap#{HandleInt => DeliveryCount}};
+        false ->
+            {Link, AccMap}
+    end.
+
+grant_link_credit(Credit, NumUnconfirmed) ->
+    Credit =< ?LINK_CREDIT_RCV / 2 andalso
+    NumUnconfirmed < ?LINK_CREDIT_RCV.
+
+%% TODO default-outcome and outcomes, dynamic lifetimes
+ensure_target(#'v1_0.target'{dynamic = true}, _, _) ->
+    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+                   "Dynamic targets not supported", []);
+ensure_target(#'v1_0.target'{address = Address,
+                             durable = Durable}, Vhost, User) ->
+    case Address of
+        {utf8, Destination} ->
+            case rabbit_routing_parser:parse_endpoint(Destination, true) of
+                {ok, Dest} ->
+                    QNameBin = ensure_terminus(target, Dest, Vhost, User, Durable),
+                    {XNameList1, RK} = rabbit_routing_parser:parse_routing(Dest),
+                    XName = rabbit_misc:r(Vhost, exchange, list_to_binary(XNameList1)),
+                    {ok, X} = rabbit_exchange:lookup(XName),
+                    check_internal_exchange(X),
+                    check_write_permitted(XName, User),
+                    RoutingKey = case RK of
+                                     undefined -> undefined;
+                                     []        -> undefined;
+                                     _         -> list_to_binary(RK)
+                                 end,
+                    {ok, XName, RoutingKey, QNameBin};
+                {error, _} = E ->
+                    E
+            end;
+        _Else ->
+            {error, {address_not_utf8_string, Address}}
+    end.
+
+handle_outgoing_link_flow_control(
+  #outgoing_link{queue_name_bin = QNameBin,
+                 delivery_count = MaybeDeliveryCountSnd},
+  #'v1_0.flow'{handle = ?UINT(HandleInt),
+               delivery_count = MaybeDeliveryCountRcv,
+               link_credit = ?UINT(LinkCreditRcv),
+               drain = Drain0,
+               echo = Echo0},
+  State0 = #state{queue_states = QStates0,
+                  cfg = #cfg{vhost = Vhost}}) ->
+    DeliveryCountRcv = case MaybeDeliveryCountRcv of
+                           ?UINT(Count) ->
+                               Count;
+                           undefined ->
+                               %% "In the event that the receiver does not yet know the delivery-count,
+                               %% i.e., delivery-countrcv is unspecified, the sender MUST assume that the
+                               %% delivery-countrcv is the first delivery-countsnd sent from sender to
+                               %% receiver, i.e., the delivery-countsnd specified in the flow state carried
+                               %% by the initial attach frame from the sender to the receiver." [2.6.7]
+                               ?INITIAL_DELIVERY_COUNT
+                       end,
+    Ctag = handle_to_ctag(HandleInt),
+    QName = rabbit_misc:r(Vhost, queue, QNameBin),
+    Drain = default(Drain0, false),
+    Echo = default(Echo0, false),
+    case MaybeDeliveryCountSnd of
+        credit_api_v2 ->
+            {ok, QStates, Actions} = rabbit_queue_type:credit(
+                                       QName, Ctag, DeliveryCountRcv, LinkCreditRcv, Drain, Echo, QStates0),
+            State1 = State0#state{queue_states = QStates},
+            State = handle_queue_actions(Actions, State1),
+            %% We'll handle the credit_reply queue event async later
+            %% thanks to the queue event containing the consumer tag.
+            State;
+        {credit_api_v1, DeliveryCountSnd} ->
+            LinkCreditSnd = diff(add(DeliveryCountRcv, LinkCreditRcv), DeliveryCountSnd),
+            {ok, QStates, Actions} = rabbit_queue_type:credit_v1(QName, Ctag, LinkCreditSnd, Drain, QStates0),
+            State1 = State0#state{queue_states = QStates},
+            State = handle_queue_actions(Actions, State1),
+            process_credit_reply_sync(Ctag, QName, LinkCreditSnd, State)
+    end.
+
+%% The AMQP 0.9.1 credit extension was poorly designed because a consumer granting
+%% credits to a queue has to synchronously wait for a credit reply from the queue:
+%% https://github.com/rabbitmq/rabbitmq-server/blob/b9566f4d02f7ceddd2f267a92d46affd30fb16c8/deps/rabbitmq_codegen/credit_extension.json#L43
+%% This blocks our entire AMQP 1.0 session process. Since the credit reply from the
+%% queue did not contain the consumr tag prior to feature flag credit_api_v2, we
+%% must behave here the same way as non-native AMQP 1.0: We wait until the queue
+%% sends us a credit reply sucht that we can correlate that reply with our consumer tag.
+process_credit_reply_sync(
+  Ctag, QName, Credit, State = #state{queue_states = QStates}) ->
+    case rabbit_queue_type:module(QName, QStates) of
+        {ok, rabbit_classic_queue} ->
+            receive {'$gen_cast',
+                     {queue_event,
+                      QName,
+                      {send_credit_reply, Avail}}} ->
+                        %% Convert to credit_api_v2 action.
+                        Action = {credit_reply_v1, Ctag, Credit, Avail, false},
+                        handle_queue_actions([Action], State)
+            after ?CREDIT_REPLY_TIMEOUT ->
+                      credit_reply_timeout(classic, QName)
+            end;
+        {ok, rabbit_quorum_queue} ->
+            process_credit_reply_sync_quorum_queue(Ctag, QName, Credit, State);
+        {error, not_found} ->
+            State
+    end.
+
+process_credit_reply_sync_quorum_queue(Ctag, QName, Credit, State0) ->
+    receive {'$gen_cast',
+             {queue_event,
+              QName,
+              {QuorumQueue,
+               {applied,
+                Applied0}}}} ->
+
+                {Applied, ReceivedCreditReply}
+                = lists:mapfoldl(
+                    %% Convert v1 send_credit_reply to credit_reply_v1 action.
+                    %% Available refers to *after* and Credit refers to *before*
+                    %% quorum queue sends messages.
+                    %% We therefore keep the same wrong behaviour of RabbitMQ 3.x.
+                    fun({RaIdx, {send_credit_reply, Available}}, _) ->
+                            Action = {credit_reply_v1, Ctag, Credit, Available, false},
+                            {{RaIdx, Action}, true};
+                       ({RaIdx, {multi, [{send_credit_reply, Available},
+                                         {send_drained, _} = SendDrained]}}, _) ->
+                            Action = {credit_reply_v1, Ctag, Credit, Available, false},
+                            {{RaIdx, {multi, [Action, SendDrained]}}, true};
+                       (E, Acc) ->
+                            {E, Acc}
+                    end, false, Applied0),
+
+                Evt = {queue_event, QName, {QuorumQueue, {applied, Applied}}},
+                %% send_drained action must be processed by
+                %% rabbit_fifo_client to advance the delivery count.
+                State = handle_queue_event(Evt, State0),
+                case ReceivedCreditReply of
+                    true ->
+                        State;
+                    false ->
+                        process_credit_reply_sync_quorum_queue(Ctag, QName, Credit, State)
+                end
+    after ?CREDIT_REPLY_TIMEOUT ->
+              credit_reply_timeout(quorum, QName)
+    end.
+
+-spec credit_reply_timeout(atom(), rabbit_types:rabbit_amqqueue_name()) ->
+    no_return().
+credit_reply_timeout(QType, QName) ->
+    Fmt = "Timed out waiting for credit reply from ~s ~s. "
+    "Hint: Enable feature flag credit_api_v2",
+    Args = [QType, rabbit_misc:rs(QName)],
+    rabbit_log:error(Fmt, Args),
+    protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR, Fmt, Args).
+
+default(undefined, Default) -> Default;
+default(Thing,    _Default) -> Thing.
+
+ensure_source(#'v1_0.source'{dynamic = true}, _, _) ->
+    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED, "Dynamic sources not supported", []);
+ensure_source(#'v1_0.source'{address = Address,
+                             durable = Durable},
+              Vhost,
+              User = #user{username = Username}) ->
+    case Address of
+        {utf8, SourceAddr} ->
+            case rabbit_routing_parser:parse_endpoint(SourceAddr, false) of
+                {ok, Src} ->
+                    QNameBin = ensure_terminus(source, Src, Vhost, User, Durable),
+                    case rabbit_routing_parser:parse_routing(Src) of
+                        {"", QNameList} ->
+                            true = string:equal(QNameList, QNameBin),
+                            {ok, QNameBin};
+                        {XNameList, RoutingKeyList} ->
+                            RoutingKey = list_to_binary(RoutingKeyList),
+                            XNameBin = list_to_binary(XNameList),
+                            XName = rabbit_misc:r(Vhost, exchange, XNameBin),
+                            QName = rabbit_misc:r(Vhost, queue, QNameBin),
+                            Binding = #binding{source = XName,
+                                               destination = QName,
+                                               key = RoutingKey},
+                            check_write_permitted(QName, User),
+                            check_read_permitted(XName, User),
+                            {ok, X} = rabbit_exchange:lookup(XName),
+                            check_read_permitted_on_topic(X, User, RoutingKey),
+                            case rabbit_binding:add(Binding, Username) of
+                                ok ->
+                                    {ok, QNameBin};
+                                {error, _} = Err ->
+                                    Err
+                            end
+                    end;
+                {error, _} = Err ->
+                    Err
+            end;
+        _ ->
+            {error, {address_not_utf8_string, Address}}
+    end.
+
+encode_frames(_T, _Msg, MaxContentLen, _Transfers) when MaxContentLen =< 0 ->
+    protocol_error(?V_1_0_AMQP_ERROR_FRAME_SIZE_TOO_SMALL,
+                   "Frame size is too small by ~tp bytes",
+                   [-MaxContentLen]);
+encode_frames(T, Msg, MaxContentLen, Transfers) ->
+    case iolist_size(Msg) > MaxContentLen of
+        true ->
+            <> = iolist_to_binary(Msg),
+            T1 = T#'v1_0.transfer'{more = true},
+            encode_frames(T, Rest, MaxContentLen, [[T1, Chunk] | Transfers]);
+        false ->
+            lists:reverse([[T, Msg] | Transfers])
+    end.
+
+source_filters_to_consumer_args(#'v1_0.source'{filter = {map, KVList}}) ->
+    source_filters_to_consumer_args(
+      [<<"rabbitmq:stream-offset-spec">>,
+       <<"rabbitmq:stream-filter">>,
+       <<"rabbitmq:stream-match-unfiltered">>],
+      KVList,
+      []);
+source_filters_to_consumer_args(_Source) ->
+    [].
+
+source_filters_to_consumer_args([], _KVList, Acc) ->
+    Acc;
+source_filters_to_consumer_args([<<"rabbitmq:stream-offset-spec">> = H | T], KVList, Acc) ->
+    Key = {symbol, H},
+    Arg = case keyfind_unpack_described(Key, KVList) of
+              {_, {timestamp, Ts}} ->
+                  [{<<"x-stream-offset">>, timestamp, Ts div 1000}]; %% 0.9.1 uses second based timestamps
+              {_, {utf8, Spec}} ->
+                  [{<<"x-stream-offset">>, longstr, Spec}]; %% next, last, first and "10m" etc
+              {_, {_, Offset}} when is_integer(Offset) ->
+                  [{<<"x-stream-offset">>, long, Offset}]; %% integer offset
+              _ ->
+                  []
+          end,
+    source_filters_to_consumer_args(T, KVList, Arg ++ Acc);
+source_filters_to_consumer_args([<<"rabbitmq:stream-filter">> = H | T], KVList, Acc) ->
+    Key = {symbol, H},
+    Arg = case keyfind_unpack_described(Key, KVList) of
+              {_, {list, Filters0}} when is_list(Filters0) ->
+                  Filters = lists:foldl(fun({utf8, Filter}, L) ->
+                                                [{longstr, Filter} | L];
+                                           (_, L) ->
+                                                L
+                                        end, [], Filters0),
+                  [{<<"x-stream-filter">>, array, Filters}];
+              {_, {utf8, Filter}} ->
+                  [{<<"x-stream-filter">>, longstr, Filter}];
+              _ ->
+                  []
+          end,
+    source_filters_to_consumer_args(T, KVList, Arg ++ Acc);
+source_filters_to_consumer_args([<<"rabbitmq:stream-match-unfiltered">> = H | T], KVList, Acc) ->
+    Key = {symbol, H},
+    Arg = case keyfind_unpack_described(Key, KVList) of
+              {_, {boolean, MU}} ->
+                  [{<<"x-stream-match-unfiltered">>, bool, MU}];
+              _ ->
+                  []
+          end,
+    source_filters_to_consumer_args(T, KVList, Arg ++ Acc);
+source_filters_to_consumer_args([_ | T], KVList, Acc) ->
+    source_filters_to_consumer_args(T, KVList, Acc).
+
+keyfind_unpack_described(Key, KvList) ->
+    %% filterset values _should_ be described values
+    %% they aren't always however for historical reasons so we need this bit of
+    %% code to return a plain value for the given filter key
+    case lists:keyfind(Key, 1, KvList) of
+        {Key, {described, Key, Value}} ->
+            {Key, Value};
+        {Key, _} = Kv ->
+            Kv;
+        false ->
+            false
+    end.
+
+validate_attach(#'v1_0.attach'{target = #'v1_0.coordinator'{}}) ->
+    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+                   "Transactions not supported", []);
+validate_attach(#'v1_0.attach'{unsettled = Unsettled,
+                               incomplete_unsettled = IncompleteSettled})
+  when Unsettled =/= undefined andalso Unsettled =/= {map, []} orelse
+       IncompleteSettled =:= true ->
+    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+                   "Link recovery not supported", []);
+validate_attach(
+  #'v1_0.attach'{snd_settle_mode = SndSettleMode,
+                 rcv_settle_mode = ?V_1_0_RECEIVER_SETTLE_MODE_SECOND})
+  when SndSettleMode =/= ?V_1_0_SENDER_SETTLE_MODE_SETTLED ->
+    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+                   "rcv-settle-mode second not supported", []);
+validate_attach(#'v1_0.attach'{}) ->
+    ok.
+
+validate_multi_transfer_delivery_id(?UINT(Id), Id) ->
+    ok;
+validate_multi_transfer_delivery_id(undefined, _FirstDeliveryId) ->
+    %% "On continuation transfers the delivery-id MAY be omitted."
+    ok;
+validate_multi_transfer_delivery_id(OtherId, FirstDeliveryId) ->
+    %% "It is an error if the delivery-id on a continuation transfer
+    %% differs from the delivery-id on the first transfer of a delivery."
+    protocol_error(
+      ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR,
+      "delivery-id of continuation transfer (~p) differs from delivery-id on first transfer (~p)",
+      [OtherId, FirstDeliveryId]).
+
+validate_multi_transfer_settled(Settled, Settled)
+  when is_boolean(Settled) ->
+    ok;
+validate_multi_transfer_settled(undefined, Settled)
+  when is_boolean(Settled) ->
+    ok;
+validate_multi_transfer_settled(Other, First)
+  when is_boolean(First) ->
+    protocol_error(
+      ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR,
+      "field 'settled' of continuation transfer (~p) differs from "
+      "(interpreted) field 'settled' on first transfer (~p)",
+      [Other, First]).
+
+validate_incoming_message_size(Message) ->
+    validate_message_size(Message, persistent_term:get(max_message_size)).
+
+validate_message_size(_, unlimited) ->
+    ok;
+validate_message_size(Message, MaxMsgSize)
+  when is_integer(MaxMsgSize) ->
+    MsgSize = iolist_size(Message),
+    case MsgSize =< MaxMsgSize of
+        true ->
+            ok;
+        false ->
+            %% "Any attempt to deliver a message larger than this results in a message-size-exceeded link-error." [2.7.3]
+            %% We apply that sentence to both incoming messages that are too large for us and outgoing messages that are
+            %% too large for the client.
+            %% This is an interesting protocol difference to MQTT where we instead discard outgoing messages that are too
+            %% large to send then behave as if we had completed sending that message [MQTT 5.0, MQTT-3.1.2-25].
+            protocol_error(
+              ?V_1_0_LINK_ERROR_MESSAGE_SIZE_EXCEEDED,
+              "message size (~b bytes) > maximum message size (~b bytes)",
+              [MsgSize, MaxMsgSize])
+    end.
+
+ensure_terminus(Type, {exchange, {XNameList, _RoutingKey}}, Vhost, User, Durability) ->
+    ok = exit_if_absent(exchange, Vhost, XNameList),
+    case Type of
+        target -> undefined;
+        source -> declare_queue(generate_queue_name(), Vhost, User, Durability)
+    end;
+ensure_terminus(target, {topic, _bindingkey}, _, _, _) ->
+    %% exchange amq.topic exists
+    undefined;
+ensure_terminus(source, {topic, _BindingKey}, Vhost, User, Durability) ->
+    %% exchange amq.topic exists
+    declare_queue(generate_queue_name(), Vhost, User, Durability);
+ensure_terminus(_, {queue, QNameList}, Vhost, User, Durability) ->
+    declare_queue(list_to_binary(QNameList), Vhost, User, Durability);
+ensure_terminus(_, {amqqueue, QNameList}, Vhost, _, _) ->
+    %% Target "/amq/queue/" is handled specially due to AMQP legacy:
+    %% "Queue names starting with "amq." are reserved for pre-declared and
+    %% standardised queues. The client MAY declare a queue starting with "amq."
+    %% if the passive option is set, or the queue already exists."
+    QNameBin = list_to_binary(QNameList),
+    ok = exit_if_absent(queue, Vhost, QNameBin),
+    QNameBin.
+
+exit_if_absent(Type, Vhost, Name) ->
+    ResourceName = rabbit_misc:r(Vhost, Type, rabbit_data_coercion:to_binary(Name)),
+    Mod = case Type of
+              exchange -> rabbit_exchange;
+              queue -> rabbit_amqqueue
+          end,
+    case Mod:exists(ResourceName) of
+        true ->
+            ok;
+        false ->
+            protocol_error(?V_1_0_AMQP_ERROR_NOT_FOUND, "no ~ts", [rabbit_misc:rs(ResourceName)])
+    end.
+
+generate_queue_name() ->
+    rabbit_guid:binary(rabbit_guid:gen_secure(), "amq.gen").
+
+declare_queue(QNameBin, Vhost, User = #user{username = Username}, TerminusDurability) ->
+    QName = rabbit_misc:r(Vhost, queue, QNameBin),
+    check_configure_permitted(QName, User),
+    check_vhost_queue_limit(Vhost, QName),
+    rabbit_core_metrics:queue_declared(QName),
+    Q0 = amqqueue:new(QName,
+                      _Pid = none,
+                      queue_is_durable(TerminusDurability),
+                      _AutoDelete = false,
+                      _QOwner = none,
+                      _QArgs = [],
+                      Vhost,
+                      #{user => Username},
+                      rabbit_classic_queue),
+    case rabbit_queue_type:declare(Q0, node()) of
+        {new, _Q}  ->
+            rabbit_core_metrics:queue_created(QName),
+            QNameBin;
+        {existing, _Q} ->
+            QNameBin;
+        Other ->
+            protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR, "Failed to declare ~s: ~p", [rabbit_misc:rs(QName), Other])
+    end.
+
+outcomes(#'v1_0.source'{outcomes = undefined}) ->
+    {array, symbol, ?OUTCOMES};
+outcomes(#'v1_0.source'{outcomes = {array, symbol, Syms} = Outcomes}) ->
+    case lists:filter(fun(O) -> not lists:member(O, ?OUTCOMES) end, Syms) of
+        [] ->
+            Outcomes;
+        Unsupported ->
+            rabbit_amqp_util:protocol_error(
+              ?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+              "Outcomes not supported: ~tp",
+              [Unsupported])
+    end;
+outcomes(#'v1_0.source'{outcomes = Unsupported}) ->
+    rabbit_amqp_util:protocol_error(
+      ?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
+      "Outcomes not supported: ~tp",
+      [Unsupported]);
+outcomes(_) ->
+    {array, symbol, ?OUTCOMES}.
+
+-spec handle_to_ctag(link_handle()) -> rabbit_types:ctag().
+handle_to_ctag(Handle) ->
+    integer_to_binary(Handle).
+
+-spec ctag_to_handle(rabbit_types:ctag()) -> link_handle().
+ctag_to_handle(Ctag) ->
+    binary_to_integer(Ctag).
+
+queue_is_durable(?V_1_0_TERMINUS_DURABILITY_NONE) ->
+    false;
+queue_is_durable(?V_1_0_TERMINUS_DURABILITY_CONFIGURATION) ->
+    true;
+queue_is_durable(?V_1_0_TERMINUS_DURABILITY_UNSETTLED_STATE) ->
+    true;
+queue_is_durable(undefined) ->
+    %% 
+    %% [3.5.3]
+    queue_is_durable(?V_1_0_TERMINUS_DURABILITY_NONE).
+
+%% "The two endpoints are not REQUIRED to use the same handle. This means a peer
+%% is free to independently chose its handle when a link endpoint is associated
+%% with the session. The locally chosen handle is referred to as the output handle.
+%% The remotely chosen handle is referred to as the input handle." [2.6.2]
+%% For simplicity, we choose to use the same handle.
+output_handle(InputHandle) ->
+    _Outputhandle = InputHandle.
+
+-spec remove_link_from_outgoing_unsettled_map(link_handle() | rabbit_types:ctag(), Map) ->
+    {Map, [rabbit_amqqueue:msg_id()]}
+      when Map :: #{delivery_number() => #outgoing_unsettled{}}.
+remove_link_from_outgoing_unsettled_map(Handle, Map)
+  when is_integer(Handle) ->
+    remove_link_from_outgoing_unsettled_map(handle_to_ctag(Handle), Map);
+remove_link_from_outgoing_unsettled_map(Ctag, Map)
+  when is_binary(Ctag) ->
+    maps:fold(fun(DeliveryId,
+                  #outgoing_unsettled{consumer_tag = Tag,
+                                      msg_id = Id},
+                  {M, Ids})
+                    when Tag =:= Ctag ->
+                      {maps:remove(DeliveryId, M), [Id | Ids]};
+                 (_, _, Acc) ->
+                      Acc
+              end, {Map, []}, Map).
+
+routing_key([RoutingKey], _XName) ->
+    RoutingKey;
+routing_key([], XName) ->
+    protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
+                   "Publishing to ~ts failed since no routing key was provided",
+                   [rabbit_misc:rs(XName)]).
+
+messages_received(Settled) ->
+    rabbit_global_counters:messages_received(?PROTOCOL, 1),
+    case Settled of
+        true -> ok;
+        false -> rabbit_global_counters:messages_received_confirm(?PROTOCOL, 1)
+    end.
+
+messages_delivered(Redelivered, QueueType) ->
+    rabbit_global_counters:messages_delivered(?PROTOCOL, QueueType, 1),
+    case Redelivered of
+        true -> rabbit_global_counters:messages_redelivered(?PROTOCOL, QueueType, 1);
+        false -> ok
+    end.
+
+messages_acknowledged(complete, QName, QS, MsgIds) ->
+    case rabbit_queue_type:module(QName, QS) of
+        {ok, QType} ->
+            rabbit_global_counters:messages_acknowledged(?PROTOCOL, QType, length(MsgIds));
+        _ ->
+            ok
+    end;
+messages_acknowledged(_, _, _, _) ->
+    ok.
+
+publisher_or_consumer_deleted(#incoming_link{}) ->
+    rabbit_global_counters:publisher_deleted(?PROTOCOL);
+publisher_or_consumer_deleted(#outgoing_link{}) ->
+    rabbit_global_counters:consumer_deleted(?PROTOCOL).
+
+publisher_or_consumer_deleted(
+  #state{incoming_links = NewIncomingLinks,
+         outgoing_links = NewOutgoingLinks},
+  #state{incoming_links = OldIncomingLinks,
+         outgoing_links = OldOutgoingLinks}) ->
+    if map_size(NewIncomingLinks) < map_size(OldIncomingLinks) ->
+           rabbit_global_counters:publisher_deleted(?PROTOCOL);
+       map_size(NewOutgoingLinks) < map_size(OldOutgoingLinks) ->
+           rabbit_global_counters:consumer_deleted(?PROTOCOL);
+       true ->
+           ok
+    end.
+
+check_internal_exchange(#exchange{internal = true,
+                                  name = XName}) ->
+    protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+                   "attach to internal ~ts is forbidden",
+                   [rabbit_misc:rs(XName)]);
+check_internal_exchange(_) ->
+    ok.
+
+check_write_permitted(Resource, User) ->
+    check_resource_access(Resource, User, write).
+
+check_read_permitted(Resource, User) ->
+    check_resource_access(Resource, User, read).
+
+check_configure_permitted(Resource, User) ->
+    check_resource_access(Resource, User, configure).
+
+check_resource_access(Resource, User, Perm) ->
+    Context = #{},
+    ok = try rabbit_access_control:check_resource_access(User, Resource, Perm, Context)
+         catch exit:#amqp_error{name = access_refused,
+                                explanation = Msg} ->
+                   protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, Msg, [])
+         end.
+
+check_write_permitted_on_topic(Resource, User, RoutingKey) ->
+    check_topic_authorisation(Resource, User, RoutingKey, write).
+
+check_read_permitted_on_topic(Resource, User, RoutingKey) ->
+    check_topic_authorisation(Resource, User, RoutingKey, read).
+
+check_topic_authorisation(#exchange{type = topic,
+                                    name = XName = #resource{virtual_host = VHost}},
+                          User = #user{username = Username},
+                          RoutingKey,
+                          Permission) ->
+    Resource = XName#resource{kind = topic},
+    CacheElem = {Resource, RoutingKey, Permission},
+    Cache = case get(?TOPIC_PERMISSION_CACHE) of
+                undefined -> [];
+                List -> List
+            end,
+    case lists:member(CacheElem, Cache) of
+        true ->
+            ok;
+        false ->
+            VariableMap = #{<<"vhost">> => VHost,
+                            <<"username">> => Username},
+            Context = #{routing_key => RoutingKey,
+                        variable_map => VariableMap},
+            try rabbit_access_control:check_topic_access(User, Resource, Permission, Context) of
+                ok ->
+                    CacheTail = lists:sublist(Cache, ?MAX_PERMISSION_CACHE_SIZE - 1),
+                    put(?TOPIC_PERMISSION_CACHE, [CacheElem | CacheTail])
+            catch
+                exit:#amqp_error{name = access_refused,
+                                 explanation = Msg} ->
+                    protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, Msg, [])
+            end
+    end;
+check_topic_authorisation(_, _, _, _) ->
+    ok.
+
+check_vhost_queue_limit(Vhost, QName) ->
+    case rabbit_vhost_limit:is_over_queue_limit(Vhost) of
+        false ->
+            ok;
+        {true, Limit} ->
+            protocol_error(
+              ?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED,
+              "cannot declare ~ts: vhost queue limit (~p) is reached",
+              [rabbit_misc:rs(QName), Limit])
+    end.
+
+check_user_id(Mc, User) ->
+    case rabbit_access_control:check_user_id(Mc, User) of
+        ok ->
+            ok;
+        {refused, Reason, Args} ->
+            protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, Reason, Args)
+    end.
+
+format_status(
+  #{state := #state{cfg = Cfg,
+                    outgoing_pending = OutgoingPending,
+                    remote_incoming_window = RemoteIncomingWindow,
+                    remote_outgoing_window = RemoteOutgoingWindow,
+                    next_incoming_id = NextIncomingId,
+                    incoming_window = IncomingWindow,
+                    next_outgoing_id = NextOutgoingId,
+                    outgoing_delivery_id = OutgoingDeliveryId,
+                    incoming_links = IncomingLinks,
+                    outgoing_links = OutgoingLinks,
+                    outgoing_unsettled_map = OutgoingUnsettledMap,
+                    stashed_rejected = StashedRejected,
+                    stashed_settled = StashedSettled,
+                    stashed_down = StashedDown,
+                    stashed_eol = StashedEol,
+                    queue_states = QueueStates}} = Status) ->
+    State = #{cfg => Cfg,
+              outgoing_pending => queue:len(OutgoingPending),
+              remote_incoming_window => RemoteIncomingWindow,
+              remote_outgoing_window => RemoteOutgoingWindow,
+              next_incoming_id => NextIncomingId,
+              incoming_window => IncomingWindow,
+              next_outgoing_id => NextOutgoingId,
+              outgoing_delivery_id => OutgoingDeliveryId,
+              incoming_links => IncomingLinks,
+              outgoing_links => OutgoingLinks,
+              outgoing_unsettled_map => OutgoingUnsettledMap,
+              stashed_rejected => StashedRejected,
+              stashed_settled => StashedSettled,
+              stashed_down => StashedDown,
+              stashed_eol => StashedEol,
+              queue_states => rabbit_queue_type:format_status(QueueStates)},
+    maps:update(state, State, Status).
diff --git a/deps/rabbit/src/rabbit_amqp_session_sup.erl b/deps/rabbit/src/rabbit_amqp_session_sup.erl
new file mode 100644
index 000000000000..1c1af2784cfc
--- /dev/null
+++ b/deps/rabbit/src/rabbit_amqp_session_sup.erl
@@ -0,0 +1,39 @@
+%% 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries.  All rights reserved.
+%%
+
+-module(rabbit_amqp_session_sup).
+-behaviour(supervisor).
+
+-include_lib("rabbit_common/include/rabbit.hrl").
+
+%% client API
+-export([start_link/1,
+         start_session/2]).
+
+%% supervisor callback
+-export([init/1]).
+
+-spec start_link(Reader :: pid()) ->
+    supervisor:startlink_ret().
+start_link(ReaderPid) ->
+    supervisor:start_link(?MODULE, ReaderPid).
+
+init(ReaderPid) ->
+    SupFlags = #{strategy => simple_one_for_one,
+                 intensity => 0,
+                 period => 1},
+    ChildSpec = #{id => amqp1_0_session,
+                  start => {rabbit_amqp_session, start_link, [ReaderPid]},
+                  restart => temporary,
+                  shutdown => ?WORKER_WAIT,
+                  type => worker},
+    {ok, {SupFlags, [ChildSpec]}}.
+
+-spec start_session(pid(), list()) ->
+    supervisor:startchild_ret().
+start_session(SessionSupPid, Args) ->
+    supervisor:start_child(SessionSupPid, Args).
diff --git a/deps/rabbit/src/rabbit_amqp_util.erl b/deps/rabbit/src/rabbit_amqp_util.erl
new file mode 100644
index 000000000000..0398c5c38b56
--- /dev/null
+++ b/deps/rabbit/src/rabbit_amqp_util.erl
@@ -0,0 +1,19 @@
+%% 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries.  All rights reserved.
+%%
+
+-module(rabbit_amqp_util).
+-include("rabbit_amqp.hrl").
+
+-export([protocol_error/3]).
+
+-spec protocol_error(term(), io:format(), [term()]) ->
+    no_return().
+protocol_error(Condition, Msg, Args) ->
+    Description = list_to_binary(lists:flatten(io_lib:format(Msg, Args))),
+    Reason = #'v1_0.error'{condition = Condition,
+                           description = {utf8, Description}},
+    exit(Reason).
diff --git a/deps/rabbit/src/rabbit_amqp_writer.erl b/deps/rabbit/src/rabbit_amqp_writer.erl
new file mode 100644
index 000000000000..f5b1b5adc6d3
--- /dev/null
+++ b/deps/rabbit/src/rabbit_amqp_writer.erl
@@ -0,0 +1,218 @@
+%% 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries.  All rights reserved.
+%%
+
+-module(rabbit_amqp_writer).
+-behaviour(gen_server).
+
+-include("rabbit_amqp.hrl").
+
+%% client API
+-export([start_link/3,
+         send_command/3,
+         send_command/4,
+         send_command_sync/3,
+         send_command_and_notify/6,
+         internal_send_command/3]).
+
+%% gen_server callbacks
+-export([init/1,
+         handle_call/3,
+         handle_cast/2,
+         handle_info/2,
+         format_status/1]).
+
+-record(state, {
+          sock :: rabbit_net:socket(),
+          max_frame_size :: unlimited | pos_integer(),
+          reader :: rabbit_types:connection(),
+          pending :: iolist(),
+          %% This field is just an optimisation to minimize the cost of erlang:iolist_size/1
+          pending_size :: non_neg_integer()
+         }).
+
+-define(HIBERNATE_AFTER, 6_000).
+-define(CALL_TIMEOUT, 300_000).
+-define(AMQP_SASL_FRAME_TYPE, 1).
+
+%%%%%%%%%%%%%%%%%%
+%%% client API %%%
+%%%%%%%%%%%%%%%%%%
+
+-spec start_link (rabbit_net:socket(), non_neg_integer(), pid()) ->
+    rabbit_types:ok(pid()).
+start_link(Sock, MaxFrame, ReaderPid) ->
+    Args = {Sock, MaxFrame, ReaderPid},
+    Opts = [{hibernate_after, ?HIBERNATE_AFTER}],
+    gen_server:start_link(?MODULE, Args, Opts).
+
+-spec send_command(pid(),
+                   rabbit_types:channel_number(),
+                   rabbit_framing:amqp_method_record()) -> ok.
+send_command(Writer, ChannelNum, MethodRecord) ->
+    Request = {send_command, ChannelNum, MethodRecord},
+    gen_server:cast(Writer, Request).
+
+-spec send_command(pid(),
+                   rabbit_types:channel_number(),
+                   rabbit_framing:amqp_method_record(),
+                   rabbit_types:content()) -> ok.
+send_command(Writer, ChannelNum, MethodRecord, Content) ->
+    Request = {send_command, ChannelNum, MethodRecord, Content},
+    gen_server:cast(Writer, Request).
+
+-spec send_command_sync(pid(),
+                        rabbit_types:channel_number(),
+                        rabbit_framing:amqp_method_record()) -> ok.
+send_command_sync(Writer, ChannelNum, MethodRecord) ->
+    Request = {send_command, ChannelNum, MethodRecord},
+    gen_server:call(Writer, Request, ?CALL_TIMEOUT).
+
+-spec send_command_and_notify(pid(),
+                              rabbit_types:channel_number(),
+                              pid(),
+                              pid(),
+                              rabbit_framing:amqp_method_record(),
+                              rabbit_types:content()) -> ok.
+send_command_and_notify(Writer, ChannelNum, QueuePid, SessionPid, MethodRecord, Content) ->
+    Request = {send_command_and_notify, ChannelNum, QueuePid, SessionPid, MethodRecord, Content},
+    gen_server:cast(Writer, Request).
+
+-spec internal_send_command(rabbit_net:socket(),
+                            rabbit_framing:amqp_method_record(),
+                            amqp10_framing | rabbit_amqp_sasl) -> ok.
+internal_send_command(Sock, MethodRecord, Protocol) ->
+    Data = assemble_frame(0, MethodRecord, Protocol),
+    ok = tcp_send(Sock, Data).
+
+%%%%%%%%%%%%%%%%%%%%%%%%%%%%
+%%% gen_server callbacks %%%
+%%%%%%%%%%%%%%%%%%%%%%%%%%%%
+
+init({Sock, MaxFrame, ReaderPid}) ->
+    State = #state{sock = Sock,
+                   max_frame_size = MaxFrame,
+                   reader = ReaderPid,
+                   pending = [],
+                   pending_size = 0},
+    {ok, State}.
+
+handle_cast({send_command, ChannelNum, MethodRecord}, State0) ->
+    State = internal_send_command_async(ChannelNum, MethodRecord, State0),
+    no_reply(State);
+handle_cast({send_command, ChannelNum, MethodRecord, Content}, State0) ->
+    State = internal_send_command_async(ChannelNum, MethodRecord, Content, State0),
+    no_reply(State);
+handle_cast({send_command_and_notify, ChannelNum, QueuePid, SessionPid, MethodRecord, Content}, State0) ->
+    State = internal_send_command_async(ChannelNum, MethodRecord, Content, State0),
+    rabbit_amqqueue:notify_sent(QueuePid, SessionPid),
+    no_reply(State).
+
+handle_call({send_command, ChannelNum, MethodRecord}, _From, State0) ->
+    State1 = internal_send_command_async(ChannelNum, MethodRecord, State0),
+    State = flush(State1),
+    {reply, ok, State}.
+
+handle_info(timeout, State0) ->
+    State = flush(State0),
+    {noreply, State};
+handle_info({'DOWN', _MRef, process, QueuePid, _Reason}, State) ->
+    rabbit_amqqueue:notify_sent_queue_down(QueuePid),
+    no_reply(State).
+
+format_status(Status) ->
+    maps:update_with(
+      state,
+      fun(#state{sock = Sock,
+                 max_frame_size = MaxFrame,
+                 reader = Reader,
+                 pending = Pending,
+                 pending_size = PendingSize}) ->
+              #{socket => Sock,
+                max_frame_size => MaxFrame,
+                reader => Reader,
+                %% Below 2 fields should always have the same value.
+                pending => iolist_size(Pending),
+                pending_size => PendingSize}
+      end,
+      Status).
+
+%%%%%%%%%%%%%%%
+%%% Helpers %%%
+%%%%%%%%%%%%%%%
+
+no_reply(State) ->
+    {noreply, State, 0}.
+
+internal_send_command_async(Channel, MethodRecord,
+                            State = #state{pending = Pending,
+                                           pending_size = PendingSize}) ->
+    Frame = assemble_frame(Channel, MethodRecord),
+    maybe_flush(State#state{pending = [Frame | Pending],
+                            pending_size = PendingSize + iolist_size(Frame)}).
+
+internal_send_command_async(Channel, MethodRecord, Content,
+                            State = #state{max_frame_size = MaxFrame,
+                                           pending = Pending,
+                                           pending_size = PendingSize}) ->
+    Frames = assemble_frames(Channel, MethodRecord, Content, MaxFrame),
+    maybe_flush(State#state{pending = [Frames | Pending],
+                            pending_size = PendingSize + iolist_size(Frames)}).
+
+%% Note: a transfer record can be followed by a number of other
+%% records to make a complete frame but unlike 0-9-1 we may have many
+%% content records. However, that's already been handled for us, we're
+%% just sending a chunk, so from this perspective it's just a binary.
+
+%%TODO respect MaxFrame
+assemble_frames(Channel, Performative, Content, _MaxFrame) ->
+    ?DEBUG("~s Channel ~tp <-~n~tp~n followed by ~tp bytes of content~n",
+           [?MODULE, Channel, amqp10_framing:pprint(Performative),
+            iolist_size(Content)]),
+    PerfBin = amqp10_framing:encode_bin(Performative),
+    amqp10_binary_generator:build_frame(Channel, [PerfBin, Content]).
+
+assemble_frame(Channel, Performative) ->
+    assemble_frame(Channel, Performative, amqp10_framing).
+
+assemble_frame(Channel, Performative, amqp10_framing) ->
+    ?DEBUG("~s Channel ~tp <-~n~tp~n",
+           [?MODULE, Channel, amqp10_framing:pprint(Performative)]),
+    PerfBin = amqp10_framing:encode_bin(Performative),
+    amqp10_binary_generator:build_frame(Channel, PerfBin);
+assemble_frame(Channel, Performative, rabbit_amqp_sasl) ->
+    ?DEBUG("~s Channel ~tp <-~n~tp~n",
+           [?MODULE, Channel, amqp10_framing:pprint(Performative)]),
+    PerfBin = amqp10_framing:encode_bin(Performative),
+    amqp10_binary_generator:build_frame(Channel, ?AMQP_SASL_FRAME_TYPE, PerfBin).
+
+tcp_send(Sock, Data) ->
+    rabbit_misc:throw_on_error(
+      inet_error,
+      fun() -> rabbit_net:send(Sock, Data) end).
+
+%% Flush when more than 2.5 * 1460 bytes (TCP over Ethernet MSS) = 3650 bytes of data
+%% has accumulated. The idea is to get the TCP data sections full (i.e. fill 1460 bytes)
+%% as often as possible to reduce the overhead of TCP/IP headers.
+-define(FLUSH_THRESHOLD, 3650).
+
+maybe_flush(State = #state{pending_size = PendingSize}) ->
+    case PendingSize > ?FLUSH_THRESHOLD of
+        true  -> flush(State);
+        false -> State
+    end.
+
+flush(State = #state{pending = []}) ->
+    State;
+flush(State = #state{sock = Sock,
+                     pending = Pending}) ->
+    case rabbit_net:send(Sock, lists:reverse(Pending)) of
+        ok ->
+            State#state{pending = [],
+                        pending_size = 0};
+        {error, Reason} ->
+            exit({writer, send_failed, Reason})
+    end.
diff --git a/deps/rabbit/src/rabbit_amqqueue.erl b/deps/rabbit/src/rabbit_amqqueue.erl
index 73e1ba09b794..ed805154fe0a 100644
--- a/deps/rabbit/src/rabbit_amqqueue.erl
+++ b/deps/rabbit/src/rabbit_amqqueue.erl
@@ -33,7 +33,7 @@
 -export([consumers/1, consumers_all/1,  emit_consumers_all/4, consumer_info_keys/0]).
 -export([basic_get/5, basic_consume/12, basic_cancel/5, notify_decorators/1]).
 -export([notify_sent/2, notify_sent_queue_down/1, resume/2]).
--export([notify_down_all/2, notify_down_all/3, activate_limit_all/2, credit/5]).
+-export([notify_down_all/2, notify_down_all/3, activate_limit_all/2]).
 -export([on_node_up/1, on_node_down/1]).
 -export([update/2, store_queue/1, update_decorators/2, policy_changed/2]).
 -export([update_mirroring/1, sync_mirrors/1, cancel_sync_mirrors/1]).
@@ -92,7 +92,7 @@
 -define(IS_QUORUM(QPid), is_tuple(QPid)).
 %%----------------------------------------------------------------------------
 
--export_type([name/0, qmsg/0, absent_reason/0]).
+-export_type([name/0, qmsg/0, msg_id/0, absent_reason/0]).
 
 -type name() :: rabbit_types:r('queue').
 
@@ -101,7 +101,7 @@
 -type qfun(A) :: fun ((amqqueue:amqqueue()) -> A | no_return()).
 -type qmsg() :: {name(), pid() | {atom(), pid()}, msg_id(),
                  boolean(), mc:state()}.
--type msg_id() :: non_neg_integer().
+-type msg_id() :: undefined | non_neg_integer() | {Priority :: non_neg_integer(), undefined | non_neg_integer()}.
 -type ok_or_errors() ::
         'ok' | {'error', [{'error' | 'exit' | 'throw', any()}]}.
 -type absent_reason() :: 'nodedown' | 'crashed' | stopped | timeout.
@@ -789,11 +789,13 @@ check_exclusive_access(Q, _ReaderPid, _MatchType) ->
       [rabbit_misc:rs(QueueName)]).
 
 -spec with_exclusive_access_or_die(name(), pid(), qfun(A)) ->
-          A | rabbit_types:channel_exit().
-
+    A | rabbit_types:channel_exit().
 with_exclusive_access_or_die(Name, ReaderPid, F) ->
     with_or_die(Name,
-                fun (Q) -> check_exclusive_access(Q, ReaderPid), F(Q) end).
+                fun (Q) ->
+                        check_exclusive_access(Q, ReaderPid),
+                        F(Q)
+                end).
 
 assert_args_equivalence(Q, NewArgs) ->
     ExistingArgs = amqqueue:get_arguments(Q),
@@ -1731,15 +1733,6 @@ deactivate_limit_all(QRefs, ChPid) ->
     delegate:invoke_no_result(QPids, {gen_server2, cast,
                                       [{deactivate_limit, ChPid}]}).
 
--spec credit(amqqueue:amqqueue(),
-             rabbit_types:ctag(),
-             non_neg_integer(),
-             boolean(),
-             rabbit_queue_type:state()) ->
-    {ok, rabbit_queue_type:state(), rabbit_queue_type:actions()}.
-credit(Q, CTag, Credit, Drain, QStates) ->
-    rabbit_queue_type:credit(Q, CTag, Credit, Drain, QStates).
-
 -spec basic_get(amqqueue:amqqueue(), boolean(), pid(), rabbit_types:ctag(),
                 rabbit_queue_type:state()) ->
           {'ok', non_neg_integer(), qmsg(), rabbit_queue_type:state()} |
@@ -1766,7 +1759,7 @@ basic_consume(Q, NoAck, ChPid, LimiterPid,
              channel_pid => ChPid,
              limiter_pid => LimiterPid,
              limiter_active => LimiterActive,
-             prefetch_count => ConsumerPrefetchCount,
+             mode => {simple_prefetch, ConsumerPrefetchCount},
              consumer_tag => ConsumerTag,
              exclusive_consume => ExclusiveConsume,
              args => Args,
diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl
index 0798ea8b739f..3edbce0bd0cf 100644
--- a/deps/rabbit/src/rabbit_amqqueue_process.erl
+++ b/deps/rabbit/src/rabbit_amqqueue_process.erl
@@ -370,6 +370,13 @@ code_change(_OldVsn, State, _Extra) ->
 maybe_notify_decorators(false, State) -> State;
 maybe_notify_decorators(true,  State) -> notify_decorators(State), State.
 
+notify_decorators_if_became_empty(WasEmpty, State) ->
+    case (not WasEmpty) andalso is_empty(State) of
+        true -> notify_decorators(State);
+        false -> ok
+    end,
+    State.
+
 notify_decorators(Event, State) ->
     _ = decorator_callback(qname(State), Event, []),
     ok.
@@ -570,14 +577,6 @@ assert_invariant(State = #q{consumers = Consumers, single_active_consumer_on = f
 
 is_empty(#q{backing_queue = BQ, backing_queue_state = BQS}) -> BQ:is_empty(BQS).
 
-maybe_send_drained(WasEmpty, #q{q = Q} = State) ->
-    case (not WasEmpty) andalso is_empty(State) of
-        true  -> notify_decorators(State),
-                 rabbit_queue_consumers:send_drained(amqqueue:get_name(Q));
-        false -> ok
-    end,
-    State.
-
 confirm_messages([], MTC, _QName) ->
     MTC;
 confirm_messages(MsgIds, MTC, QName) ->
@@ -852,7 +851,7 @@ requeue_and_run(AckTags, State = #q{backing_queue       = BQ,
     WasEmpty = BQ:is_empty(BQS),
     {_MsgIds, BQS1} = BQ:requeue(AckTags, BQS),
     {_Dropped, State1} = maybe_drop_head(State#q{backing_queue_state = BQS1}),
-    run_message_queue(maybe_send_drained(WasEmpty, drop_expired_msgs(State1))).
+    run_message_queue(notify_decorators_if_became_empty(WasEmpty, drop_expired_msgs(State1))).
 
 fetch(AckRequired, State = #q{backing_queue       = BQ,
                               backing_queue_state = BQS}) ->
@@ -861,7 +860,7 @@ fetch(AckRequired, State = #q{backing_queue       = BQ,
     %%       we will send expired messages at times.
     {Result, BQS1} = BQ:fetch(AckRequired, BQS),
     State1 = drop_expired_msgs(State#q{backing_queue_state = BQS1}),
-    {Result, maybe_send_drained(Result =:= empty, State1)}.
+    {Result, notify_decorators_if_became_empty(Result =:= empty, State1)}.
 
 ack(AckTags, ChPid, State) ->
     subtract_acks(ChPid, AckTags, State,
@@ -992,11 +991,6 @@ calculate_msg_expiry(Msg, TTL) ->
             os:system_time(microsecond) + T * 1000
     end.
 
-%% Logically this function should invoke maybe_send_drained/2.
-%% However, that is expensive. Since some frequent callers of
-%% drop_expired_msgs/1, in particular deliver_or_enqueue/3, cannot
-%% possibly cause the queue to become empty, we push the
-%% responsibility to the callers. So be cautious when adding new ones.
 drop_expired_msgs(State) ->
     case is_empty(State) of
         true  -> State;
@@ -1343,9 +1337,8 @@ handle_call({basic_get, ChPid, NoAck, LimiterPid}, _From,
     end;
 
 handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
-             PrefetchCount, ConsumerTag, ExclusiveConsume, Args, OkMsg, ActingUser},
-            _From, State = #q{q = Q,
-                              consumers = Consumers,
+             ModeOrPrefetch, ConsumerTag, ExclusiveConsume, Args, OkMsg, ActingUser},
+            _From, State = #q{consumers = Consumers,
                               active_consumer = Holder,
                               single_active_consumer_on = SingleActiveConsumerOn}) ->
     ConsumerRegistration = case SingleActiveConsumerOn of
@@ -1355,33 +1348,28 @@ handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
                     {error, reply({error, exclusive_consume_unavailable}, State)};
                 false ->
                     Consumers1 = rabbit_queue_consumers:add(
-                                   amqqueue:get_name(Q),
                                    ChPid, ConsumerTag, NoAck,
-                                   LimiterPid, LimiterActive,
-                                   PrefetchCount, Args, is_empty(State),
-                                   ActingUser, Consumers),
-
-                  case Holder of
-                      none ->
-                          NewConsumer = rabbit_queue_consumers:get(ChPid, ConsumerTag, Consumers1),
-                          {state, State#q{consumers          = Consumers1,
-                                          has_had_consumers  = true,
-                                          active_consumer    = NewConsumer}};
-                      _    ->
-                          {state, State#q{consumers          = Consumers1,
-                                          has_had_consumers  = true}}
-                  end
+                                   LimiterPid, LimiterActive, ModeOrPrefetch,
+                                   Args, ActingUser, Consumers),
+                    case Holder of
+                        none ->
+                            NewConsumer = rabbit_queue_consumers:get(ChPid, ConsumerTag, Consumers1),
+                            {state, State#q{consumers          = Consumers1,
+                                            has_had_consumers  = true,
+                                            active_consumer    = NewConsumer}};
+                        _    ->
+                            {state, State#q{consumers          = Consumers1,
+                                            has_had_consumers  = true}}
+                    end
             end;
         false ->
             case check_exclusive_access(Holder, ExclusiveConsume, State) of
               in_use -> {error, reply({error, exclusive_consume_unavailable}, State)};
               ok     ->
                     Consumers1 = rabbit_queue_consumers:add(
-                                   amqqueue:get_name(Q),
                                    ChPid, ConsumerTag, NoAck,
-                                   LimiterPid, LimiterActive,
-                                   PrefetchCount, Args, is_empty(State),
-                                   ActingUser, Consumers),
+                                   LimiterPid, LimiterActive, ModeOrPrefetch,
+                                   Args, ActingUser, Consumers),
                     ExclusiveConsumer =
                         if ExclusiveConsume -> {ChPid, ConsumerTag};
                            true             -> Holder
@@ -1408,7 +1396,8 @@ handle_call({basic_consume, NoAck, ChPid, LimiterPid, LimiterActive,
                     {false, _} ->
                        {true, up}
                 end,
-            rabbit_core_metrics:consumer_created(
+                PrefetchCount = rabbit_queue_consumers:parse_prefetch_count(ModeOrPrefetch),
+                rabbit_core_metrics:consumer_created(
                 ChPid, ConsumerTag, ExclusiveConsume, AckRequired, QName,
                 PrefetchCount, ConsumerIsActive, ActivityStatus, Args),
             emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
@@ -1436,7 +1425,9 @@ handle_call({basic_cancel, ChPid, ConsumerTag, OkMsg, ActingUser}, _From,
             emit_consumer_deleted(ChPid, ConsumerTag, qname(State1), ActingUser),
             notify_decorators(State1),
             case should_auto_delete(State1) of
-                false -> reply(ok, ensure_expiry_timer(State1));
+                false ->
+                    State2 = run_message_queue(Holder =/= Holder1, State1),
+                    reply(ok, ensure_expiry_timer(State2));
                 true  ->
                     log_auto_delete(
                         io_lib:format(
@@ -1467,7 +1458,7 @@ handle_call(purge, _From, State = #q{backing_queue       = BQ,
                                      backing_queue_state = BQS}) ->
     {Count, BQS1} = BQ:purge(BQS),
     State1 = State#q{backing_queue_state = BQS1},
-    reply({ok, Count}, maybe_send_drained(Count =:= 0, State1));
+    reply({ok, Count}, notify_decorators_if_became_empty(Count =:= 0, State1));
 
 handle_call({requeue, AckTags, ChPid}, From, State) ->
     gen_server2:reply(From, ok),
@@ -1638,21 +1629,57 @@ handle_cast(update_mirroring, State = #q{q = Q,
             noreply(update_mirroring(Policy, State1))
     end;
 
-handle_cast({credit, ChPid, CTag, Credit, Drain},
-            State = #q{consumers           = Consumers,
-                       backing_queue       = BQ,
-                       backing_queue_state = BQS,
-                       q = Q}) ->
-    Len = BQ:len(BQS),
-    rabbit_classic_queue:send_credit_reply(ChPid, amqqueue:get_name(Q), Len),
-    noreply(
-      case rabbit_queue_consumers:credit(amqqueue:get_name(Q),
-                                         Len == 0, Credit, Drain, ChPid, CTag,
-                                         Consumers) of
-          unchanged               -> State;
-          {unblocked, Consumers1} -> State1 = State#q{consumers = Consumers1},
-                                     run_message_queue(true, State1)
-      end);
+handle_cast({credit, SessionPid, CTag, Credit, Drain},
+            #q{q = Q,
+               backing_queue = BQ,
+               backing_queue_state = BQS0} = State) ->
+    %% Credit API v1.
+    %% Delete this function clause when feature flag credit_api_v2 becomes required.
+    %% Behave like non-native AMQP 1.0: Send send_credit_reply before deliveries.
+    rabbit_classic_queue:send_credit_reply_credit_api_v1(
+      SessionPid, amqqueue:get_name(Q), BQ:len(BQS0)),
+    handle_cast({credit, SessionPid, CTag, credit_api_v1, Credit, Drain, false}, State);
+handle_cast({credit, SessionPid, CTag, DeliveryCountRcv, Credit, Drain, Echo},
+            #q{consumers = Consumers0,
+               q = Q} = State0) ->
+    QName = amqqueue:get_name(Q),
+    State = #q{backing_queue_state = PostBQS,
+               backing_queue = BQ} = case rabbit_queue_consumers:process_credit(
+                                            DeliveryCountRcv, Credit, SessionPid, CTag, Consumers0) of
+                                         unchanged ->
+                                             State0;
+                                         {unblocked, Consumers1} ->
+                                             State1 = State0#q{consumers = Consumers1},
+                                             run_message_queue(true, State1)
+                                     end,
+    case rabbit_queue_consumers:get_link_state(SessionPid, CTag) of
+        {credit_api_v1, PostCred}
+          when Drain andalso
+               is_integer(PostCred) andalso PostCred > 0 ->
+            %% credit API v1
+            rabbit_queue_consumers:drained(credit_api_v1, SessionPid, CTag),
+            rabbit_classic_queue:send_drained_credit_api_v1(SessionPid, QName, CTag, PostCred);
+        {PostDeliveryCountSnd, PostCred}
+          when is_integer(PostDeliveryCountSnd) andalso
+               Drain andalso
+               is_integer(PostCred) andalso PostCred > 0 ->
+            %% credit API v2
+            AdvancedDeliveryCount = serial_number:add(PostDeliveryCountSnd, PostCred),
+            rabbit_queue_consumers:drained(AdvancedDeliveryCount, SessionPid, CTag),
+            Avail = BQ:len(PostBQS),
+            rabbit_classic_queue:send_credit_reply(
+              SessionPid, QName, CTag, AdvancedDeliveryCount, 0, Avail, Drain);
+        {PostDeliveryCountSnd, PostCred}
+          when is_integer(PostDeliveryCountSnd) andalso
+               Echo ->
+            %% credit API v2
+            Avail = BQ:len(PostBQS),
+            rabbit_classic_queue:send_credit_reply(
+              SessionPid, QName, CTag, PostDeliveryCountSnd, PostCred, Avail, Drain);
+        _ ->
+            ok
+    end,
+    noreply(State);
 
 % Note: https://www.pivotaltracker.com/story/show/166962656
 % This event is necessary for the stats timer to be initialized with
@@ -1731,7 +1758,7 @@ handle_info({maybe_expire, _Vsn}, State) ->
 handle_info({drop_expired, Vsn}, State = #q{args_policy_version = Vsn}) ->
     WasEmpty = is_empty(State),
     State1 = drop_expired_msgs(State#q{ttl_timer_ref = undefined}),
-    noreply(maybe_send_drained(WasEmpty, State1));
+    noreply(notify_decorators_if_became_empty(WasEmpty, State1));
 
 handle_info({drop_expired, _Vsn}, State) ->
     noreply(State);
diff --git a/deps/rabbit/src/rabbit_autoheal.erl b/deps/rabbit/src/rabbit_autoheal.erl
index 9e0a71950301..63f38dc82366 100644
--- a/deps/rabbit/src/rabbit_autoheal.erl
+++ b/deps/rabbit/src/rabbit_autoheal.erl
@@ -411,7 +411,7 @@ make_decision(AllPartitions) ->
 partition_value(Partition) ->
     Connections = [Res || Node <- Partition,
                           Res <- [rpc:call(Node, rabbit_networking,
-                                           connections_local, [])],
+                                           local_connections, [])],
                           is_list(Res)],
     {length(lists:append(Connections)), length(Partition)}.
 
diff --git a/deps/rabbit/src/rabbit_channel.erl b/deps/rabbit/src/rabbit_channel.erl
index 16655b28d5a0..1464794a90c2 100644
--- a/deps/rabbit/src/rabbit_channel.erl
+++ b/deps/rabbit/src/rabbit_channel.erl
@@ -63,7 +63,7 @@
 -export([get_vhost/1, get_user/1]).
 %% For testing
 -export([build_topic_variable_map/3]).
--export([list_queue_states/1, get_max_message_size/0]).
+-export([list_queue_states/1]).
 
 %% Mgmt HTTP API refactor
 -export([handle_method/6]).
@@ -87,13 +87,9 @@
           %% same as reader's name, see #v1.name
           %% in rabbit_reader
           conn_name,
-          %% channel's originating source e.g. rabbit_reader | rabbit_direct | undefined
-          %% or any other channel creating/spawning entity
-          source,
           %% same as #v1.user in the reader, used in
           %% authorisation checks
           user,
-          %% same as #v1.user in the reader
           virtual_host,
           %% when queue.bind's queue field is empty,
           %% this name will be used instead
@@ -107,15 +103,10 @@
           capabilities,
           trace_state :: rabbit_trace:state(),
           consumer_prefetch,
-          %% Message content size limit
-          max_message_size,
           consumer_timeout,
           authz_context,
           %% defines how ofter gc will be executed
-          writer_gc_threshold,
-          %% true with AMQP 1.0 to include the publishing sequence
-          %% in the return callback, false otherwise
-          extended_return_callback
+          writer_gc_threshold
          }).
 
 -record(pending_ack, {
@@ -513,10 +504,8 @@ init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
               end,
     %% Process dictionary is used here because permission cache already uses it. MK.
     put(permission_cache_can_expire, rabbit_access_control:permission_cache_can_expire(User)),
-    MaxMessageSize = get_max_message_size(),
     ConsumerTimeout = get_consumer_timeout(),
     OptionalVariables = extract_variable_map_from_amqp_params(AmqpParams),
-    UseExtendedReturnCallback = use_extended_return_callback(AmqpParams),
     {ok, GCThreshold} = application:get_env(rabbit, writer_gc_threshold),
     State = #ch{cfg = #conf{state = starting,
                             protocol = Protocol,
@@ -532,17 +521,14 @@ init([Channel, ReaderPid, WriterPid, ConnPid, ConnName, Protocol, User, VHost,
                             capabilities = Capabilities,
                             trace_state = rabbit_trace:init(VHost),
                             consumer_prefetch = Prefetch,
-                            max_message_size = MaxMessageSize,
                             consumer_timeout = ConsumerTimeout,
                             authz_context = OptionalVariables,
-                            writer_gc_threshold = GCThreshold,
-                            extended_return_callback = UseExtendedReturnCallback
+                            writer_gc_threshold = GCThreshold
                            },
                 limiter = Limiter,
                 tx                      = none,
                 next_tag                = 1,
                 unacked_message_q       = ?QUEUE:new(),
-                queue_monitors          = pmon:new(),
                 consumer_mapping        = #{},
                 queue_consumers         = #{},
                 confirm_enabled         = false,
@@ -755,8 +741,7 @@ handle_info(emit_stats, State) ->
     {noreply, send_confirms_and_nacks(State1), hibernate};
 
 handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
-            #ch{queue_states = QStates0,
-                queue_monitors = _QMons} = State0) ->
+            #ch{queue_states = QStates0} = State0) ->
     credit_flow:peer_down(QPid),
     case rabbit_queue_type:handle_down(QPid, QName, Reason, QStates0) of
         {ok, QState1, Actions} ->
@@ -812,17 +797,17 @@ terminate(_Reason,
           State = #ch{cfg = #conf{user = #user{username = Username}},
                       consumer_mapping = CM,
                       queue_states = QueueCtxs}) ->
-    _ = rabbit_queue_type:close(QueueCtxs),
+    rabbit_queue_type:close(QueueCtxs),
     {_Res, _State1} = notify_queues(State),
     pg_local:leave(rabbit_channels, self()),
     rabbit_event:if_enabled(State, #ch.stats_timer,
                             fun() -> emit_stats(State) end),
     [delete_stats(Tag) || {Tag, _} <- get()],
     maybe_decrease_global_publishers(State),
-    _ = maps:map(
-          fun (_, _) ->
-                  rabbit_global_counters:consumer_deleted(amqp091)
-          end, CM),
+    maps:foreach(
+      fun (_, _) ->
+              rabbit_global_counters:consumer_deleted(amqp091)
+      end, CM),
     rabbit_core_metrics:channel_closed(self()),
     rabbit_event:notify(channel_closed, [{pid, self()},
                                          {user_who_performed_action, Username},
@@ -839,16 +824,6 @@ code_change(_OldVsn, State, _Extra) ->
 
 format_message_queue(Opt, MQ) -> rabbit_misc:format_message_queue(Opt, MQ).
 
--spec get_max_message_size() -> non_neg_integer().
-
-get_max_message_size() ->
-    case application:get_env(rabbit, max_message_size) of
-        {ok, MS} when is_integer(MS) ->
-            erlang:min(MS, ?MAX_MSG_SIZE);
-        _ ->
-            ?MAX_MSG_SIZE
-    end.
-
 get_consumer_timeout() ->
     case application:get_env(rabbit, consumer_timeout) of
         {ok, MS} when is_integer(MS) ->
@@ -954,30 +929,19 @@ check_write_permitted_on_topic(Resource, User, RoutingKey, AuthzContext) ->
 check_read_permitted_on_topic(Resource, User, RoutingKey, AuthzContext) ->
     check_topic_authorisation(Resource, User, RoutingKey, AuthzContext, read).
 
-check_user_id_header(#'P_basic'{user_id = undefined}, _) ->
-    ok;
-check_user_id_header(#'P_basic'{user_id = Username},
-                     #ch{cfg = #conf{user = #user{username = Username}}}) ->
-    ok;
-check_user_id_header(
-  #'P_basic'{}, #ch{cfg = #conf{user = #user{authz_backends =
-                                 [{rabbit_auth_backend_dummy, _}]}}}) ->
-    ok;
-check_user_id_header(#'P_basic'{user_id = Claimed},
-                     #ch{cfg = #conf{user = #user{username = Actual,
-                                                  tags     = Tags}}}) ->
-    case lists:member(impersonator, Tags) of
-        true  -> ok;
-        false -> rabbit_misc:precondition_failed(
-                   "user_id property set to '~ts' but authenticated user was "
-                   "'~ts'", [Claimed, Actual])
+check_user_id_header(Msg, User) ->
+    case rabbit_access_control:check_user_id(Msg, User) of
+        ok ->
+            ok;
+        {refused, Reason, Args} ->
+            rabbit_misc:precondition_failed(Reason, Args)
     end.
 
 check_expiration_header(Props) ->
     case rabbit_basic:parse_expiration(Props) of
         {ok, _}    -> ok;
         {error, E} -> rabbit_misc:precondition_failed("invalid expiration '~ts': ~tp",
-                                          [Props#'P_basic'.expiration, E])
+                                                      [Props#'P_basic'.expiration, E])
     end.
 
 check_internal_exchange(#exchange{name = Name, internal = true}) ->
@@ -1028,28 +992,21 @@ extract_variable_map_from_amqp_params([Value]) ->
 extract_variable_map_from_amqp_params(_) ->
     #{}.
 
-%% Use tuple representation of amqp_params to avoid a dependency on amqp_client.
-%% Used for AMQP 1.0
-use_extended_return_callback({amqp_params_direct,_,_,_,_,
-                              {amqp_adapter_info,_,_,_,_,_,{'AMQP',"1.0"},_},
-                              _}) ->
-    true;
-use_extended_return_callback(_) ->
-    false.
-
-check_msg_size(Content, MaxMessageSize, GCThreshold) ->
+check_msg_size(Content, GCThreshold) ->
+    MaxMessageSize = persistent_term:get(max_message_size),
     Size = rabbit_basic:maybe_gc_large_msg(Content, GCThreshold),
-    case Size of
-        S when S > MaxMessageSize ->
-            ErrorMessage = case MaxMessageSize of
-                ?MAX_MSG_SIZE ->
-                    "message size ~B is larger than max size ~B";
-                _ ->
-                    "message size ~B is larger than configured max size ~B"
-            end,
-            rabbit_misc:precondition_failed(ErrorMessage,
-                                [Size, MaxMessageSize]);
-        _ -> ok
+    case Size =< MaxMessageSize of
+        true ->
+            ok;
+        false ->
+            Fmt = case MaxMessageSize of
+                      ?MAX_MSG_SIZE ->
+                          "message size ~B is larger than max size ~B";
+                      _ ->
+                          "message size ~B is larger than configured max size ~B"
+                  end,
+            rabbit_misc:precondition_failed(
+              Fmt, [Size, MaxMessageSize])
     end.
 
 check_vhost_queue_limit(#resource{name = QueueName}, VHost) ->
@@ -1226,22 +1183,21 @@ handle_method(#'basic.publish'{immediate = true}, _Content, _State) ->
 handle_method(#'basic.publish'{exchange    = ExchangeNameBin,
                                routing_key = RoutingKey,
                                mandatory   = Mandatory},
-              Content, State = #ch{cfg = #conf{channel = ChannelNum,
-                                               conn_name = ConnName,
-                                               virtual_host = VHostPath,
-                                               user = #user{username = Username} = User,
-                                               trace_state = TraceState,
-                                               max_message_size = MaxMessageSize,
-                                               authz_context = AuthzContext,
-                                               writer_gc_threshold = GCThreshold
-                                              },
+              Content, State0 = #ch{cfg = #conf{channel = ChannelNum,
+                                                conn_name = ConnName,
+                                                virtual_host = VHostPath,
+                                                user = #user{username = Username} = User,
+                                                trace_state = TraceState,
+                                                authz_context = AuthzContext,
+                                                writer_gc_threshold = GCThreshold
+                                               },
                                    tx               = Tx,
                                    confirm_enabled  = ConfirmEnabled,
                                    delivery_flow    = Flow
                                    }) ->
-    State0 = maybe_increase_global_publishers(State),
+    State1 = maybe_increase_global_publishers(State0),
     rabbit_global_counters:messages_received(amqp091, 1),
-    check_msg_size(Content, MaxMessageSize, GCThreshold),
+    check_msg_size(Content, GCThreshold),
     ExchangeName = rabbit_misc:r(VHostPath, exchange, ExchangeNameBin),
     check_write_permitted(ExchangeName, User, AuthzContext),
     Exchange = rabbit_exchange:lookup_or_die(ExchangeName),
@@ -1251,19 +1207,19 @@ handle_method(#'basic.publish'{exchange    = ExchangeNameBin,
     %% certain to want to look at delivery-mode and priority.
     DecodedContent = #content {properties = Props} =
         maybe_set_fast_reply_to(
-          rabbit_binary_parser:ensure_content_decoded(Content), State),
-    check_user_id_header(Props, State),
+          rabbit_binary_parser:ensure_content_decoded(Content), State1),
     check_expiration_header(Props),
     DoConfirm = Tx =/= none orelse ConfirmEnabled,
-    {DeliveryOptions, State1} =
+    {DeliveryOptions, State} =
         case DoConfirm of
             false ->
-                {maps_put_truthy(flow, Flow, #{mandatory => Mandatory}), State0};
+                {maps_put_truthy(flow, Flow, #{mandatory => Mandatory}), State1};
             true  ->
                 rabbit_global_counters:messages_received_confirm(amqp091, 1),
-                SeqNo = State0#ch.publish_seqno,
-                Opts = maps_put_truthy(flow, Flow, #{correlation => SeqNo, mandatory => Mandatory}),
-                {Opts, State0#ch{publish_seqno = SeqNo + 1}}
+                SeqNo = State1#ch.publish_seqno,
+                Opts = maps_put_truthy(flow, Flow, #{correlation => SeqNo,
+                                                     mandatory => Mandatory}),
+                {Opts, State1#ch{publish_seqno = SeqNo + 1}}
         end,
 
     case mc_amqpl:message(ExchangeName,
@@ -1273,6 +1229,7 @@ handle_method(#'basic.publish'{exchange    = ExchangeNameBin,
             rabbit_misc:precondition_failed("invalid message: ~tp", [Reason]);
         {ok, Message0} ->
             Message = rabbit_message_interceptor:intercept(Message0),
+            check_user_id_header(Message, User),
             QNames = rabbit_exchange:route(Exchange, Message, #{return_binding_keys => true}),
             [rabbit_channel:deliver_reply(RK, Message) ||
              {virtual_reply_queue, RK} <- QNames],
@@ -1283,10 +1240,10 @@ handle_method(#'basic.publish'{exchange    = ExchangeNameBin,
             Delivery = {Message, DeliveryOptions, Queues},
             {noreply, case Tx of
                           none ->
-                              deliver_to_queues(ExchangeName, Delivery, State1);
+                              deliver_to_queues(ExchangeName, Delivery, State);
                           {Msgs, Acks} ->
                               Msgs1 = ?QUEUE:in(Delivery, Msgs),
-                              State1#ch{tx = {Msgs1, Acks}}
+                              State#ch{tx = {Msgs1, Acks}}
                       end}
     end;
 
@@ -1729,19 +1686,6 @@ handle_method(#'channel.flow'{active = true}, _, State) ->
 handle_method(#'channel.flow'{active = false}, _, _State) ->
     rabbit_misc:protocol_error(not_implemented, "active=false", []);
 
-handle_method(#'basic.credit'{consumer_tag = CTag,
-                              credit       = Credit,
-                              drain        = Drain},
-              _, State = #ch{consumer_mapping = Consumers,
-                             queue_states = QStates0}) ->
-    case maps:find(CTag, Consumers) of
-        {ok, {Q, _CParams}} ->
-            {ok, QStates, Actions} = rabbit_queue_type:credit(Q, CTag, Credit, Drain, QStates0),
-            {noreply, handle_queue_actions(Actions, State#ch{queue_states = QStates})};
-        error -> rabbit_misc:precondition_failed(
-                   "unknown consumer tag '~ts'", [CTag])
-    end;
-
 handle_method(_MethodRecord, _Content, _State) ->
     rabbit_misc:protocol_error(
       command_invalid, "unimplemented method", []).
@@ -2146,10 +2090,10 @@ deliver_to_queues(XName,
         {ok, QueueStates, Actions} ->
             rabbit_global_counters:messages_routed(amqp091, length(Qs)),
             QueueNames = rabbit_amqqueue:queue_names(Qs),
-            MsgSeqNo = maps:get(correlation, Options, undefined),
             %% NB: the order here is important since basic.returns must be
             %% sent before confirms.
-            ok = process_routing_mandatory(Mandatory, RoutedToQueues, MsgSeqNo, Message, XName, State0),
+            ok = process_routing_mandatory(Mandatory, RoutedToQueues, Message, XName, State0),
+            MsgSeqNo = maps:get(correlation, Options, undefined),
             State1 = process_routing_confirm(MsgSeqNo, QueueNames, XName, State0),
             %% Actions must be processed after registering confirms as actions may
             %% contain rejections of publishes
@@ -2178,32 +2122,23 @@ deliver_to_queues(XName,
 
 process_routing_mandatory(_Mandatory = true,
                           _RoutedToQs = [],
-                          MsgSeqNo,
                           Msg,
                           XName,
-                          State = #ch{cfg = #conf{extended_return_callback = ExtRetCallback}}) ->
+                          State) ->
     rabbit_global_counters:messages_unroutable_returned(amqp091, 1),
     ?INCR_STATS(exchange_stats, XName, 1, return_unroutable, State),
-    Content0 = mc:protocol_state(Msg),
-    Content = case ExtRetCallback of
-                  true ->
-                      %% providing the publishing sequence for AMQP 1.0
-                      {MsgSeqNo, Content0};
-                  false ->
-                      Content0
-              end,
+    Content = mc:protocol_state(Msg),
     [RoutingKey | _] = mc:routing_keys(Msg),
     ok = basic_return(Content, RoutingKey, XName#resource.name, State, no_route);
 process_routing_mandatory(_Mandatory = false,
                           _RoutedToQs = [],
-                          _MsgSeqNo,
                           _Msg,
                           XName,
                           State) ->
     rabbit_global_counters:messages_unroutable_dropped(amqp091, 1),
     ?INCR_STATS(exchange_stats, XName, 1, drop_unroutable, State),
     ok;
-process_routing_mandatory(_, _, _, _, _, _) ->
+process_routing_mandatory(_, _, _, _, _) ->
     ok.
 
 process_routing_confirm(undefined, _, _, State) ->
@@ -2797,12 +2732,11 @@ handle_consumer_timed_out(Timeout,#pending_ack{delivery_tag = DeliveryTag, tag =
 				[Channel, Timeout], none),
     handle_exception(Ex, State).
 
-handle_queue_actions(Actions, #ch{cfg = #conf{writer_pid = WriterPid}} = State0) ->
+handle_queue_actions(Actions, State) ->
     lists:foldl(
-      fun
-          ({settled, QRef, MsgSeqNos}, S0) ->
+      fun({settled, QRef, MsgSeqNos}, S0) ->
               confirm(MsgSeqNos, QRef, S0);
-          ({rejected, _QRef, MsgSeqNos}, S0) ->
+         ({rejected, _QRef, MsgSeqNos}, S0) ->
               {U, Rej} =
               lists:foldr(
                 fun(SeqNo, {U1, Acc}) ->
@@ -2815,26 +2749,17 @@ handle_queue_actions(Actions, #ch{cfg = #conf{writer_pid = WriterPid}} = State0)
                 end, {S0#ch.unconfirmed, []}, MsgSeqNos),
               S = S0#ch{unconfirmed = U},
               record_rejects(Rej, S);
-          ({deliver, CTag, AckRequired, Msgs}, S0) ->
+         ({deliver, CTag, AckRequired, Msgs}, S0) ->
               handle_deliver(CTag, AckRequired, Msgs, S0);
-          ({queue_down, QRef}, S0) ->
+         ({queue_down, QRef}, S0) ->
               handle_consuming_queue_down_or_eol(QRef, S0);
-          ({block, QName}, S0) ->
+         ({block, QName}, S0) ->
               credit_flow:block(QName),
               S0;
-          ({unblock, QName}, S0) ->
+         ({unblock, QName}, S0) ->
               credit_flow:unblock(QName),
-              S0;
-          ({send_credit_reply, Avail}, S0) ->
-              ok = rabbit_writer:send_command(WriterPid,
-                                              #'basic.credit_ok'{available = Avail}),
-              S0;
-          ({send_drained, {CTag, Credit}}, S0) ->
-              ok = rabbit_writer:send_command(WriterPid,
-                                              #'basic.credit_drained'{consumer_tag = CTag,
-                                                                      credit_drained = Credit}),
               S0
-      end, State0, Actions).
+      end, State, Actions).
 
 handle_eol(QName, State0) ->
     State1 = handle_consuming_queue_down_or_eol(QName, State0),
diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl
index 6f8cdfbf57f2..753a7d851c58 100644
--- a/deps/rabbit/src/rabbit_classic_queue.erl
+++ b/deps/rabbit/src/rabbit_classic_queue.erl
@@ -41,7 +41,8 @@
          handle_event/3,
          deliver/3,
          settle/5,
-         credit/5,
+         credit_v1/5,
+         credit/7,
          dequeue/5,
          info/2,
          state_info/1,
@@ -58,8 +59,9 @@
 -export([confirm_to_sender/3,
          send_rejection/3,
          deliver_to_consumer/5,
-         send_drained/3,
-         send_credit_reply/3]).
+         send_credit_reply_credit_api_v1/3,
+         send_drained_credit_api_v1/4,
+         send_credit_reply/7]).
 
 -spec is_enabled() -> boolean().
 is_enabled() -> true.
@@ -237,16 +239,17 @@ consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) ->
       channel_pid := ChPid,
       limiter_pid := LimiterPid,
       limiter_active := LimiterActive,
-      prefetch_count := ConsumerPrefetchCount,
+      mode := Mode,
       consumer_tag := ConsumerTag,
       exclusive_consume := ExclusiveConsume,
-      args := Args,
+      args := Args0,
       ok_msg := OkMsg,
       acting_user :=  ActingUser} = Spec,
+    {ModeOrPrefetch, Args} = consume_backwards_compat(Mode, Args0),
     case delegate:invoke(QPid,
                          {gen_server2, call,
                           [{basic_consume, NoAck, ChPid, LimiterPid,
-                            LimiterActive, ConsumerPrefetchCount, ConsumerTag,
+                            LimiterActive, ModeOrPrefetch, ConsumerTag,
                             ExclusiveConsume, Args, OkMsg, ActingUser},
                            infinity]}) of
         ok ->
@@ -257,6 +260,22 @@ consume(Q, Spec, State0) when ?amqqueue_is_classic(Q) ->
             Err
     end.
 
+%% Delete this function when feature flag credit_api_v2 becomes required.
+consume_backwards_compat({simple_prefetch, PrefetchCount} = Mode, Args) ->
+    case rabbit_feature_flags:is_enabled(credit_api_v2) of
+        true -> {Mode, Args};
+        false -> {PrefetchCount, Args}
+    end;
+consume_backwards_compat({credited, InitialDeliveryCount} = Mode, Args)
+  when is_integer(InitialDeliveryCount) ->
+    %% credit API v2
+    {Mode, Args};
+consume_backwards_compat({credited, credit_api_v1}, Args) ->
+    %% credit API v1
+    {_PrefetchCount = 0,
+     [{<<"x-credit">>, table, [{<<"credit">>, long, 0},
+                               {<<"drain">>,  bool, false}]} | Args]}.
+
 cancel(Q, ConsumerTag, OkMsg, ActingUser, State) ->
     QPid = amqqueue:get_pid(Q),
     case delegate:invoke(QPid, {gen_server2, call,
@@ -282,11 +301,14 @@ settle(_QName, Op, _CTag, MsgIds, State) ->
                                     [{reject, Op == requeue, MsgIds, ChPid}]}),
     {State, []}.
 
-credit(_QName, CTag, Credit, Drain, State) ->
-    ChPid = self(),
-    delegate:invoke_no_result(State#?STATE.pid,
-                              {gen_server2, cast,
-                               [{credit, ChPid, CTag, Credit, Drain}]}),
+credit_v1(_QName, Ctag, LinkCreditSnd, Drain, #?STATE{pid = QPid} = State) ->
+    Request = {credit, self(), Ctag, LinkCreditSnd, Drain},
+    delegate:invoke_no_result(QPid, {gen_server2, cast, [Request]}),
+    {State, []}.
+
+credit(_QName, Ctag, DeliveryCountRcv, LinkCreditRcv, Drain, Echo, #?STATE{pid = QPid} = State) ->
+    Request = {credit, self(), Ctag, DeliveryCountRcv, LinkCreditRcv, Drain, Echo},
+    delegate:invoke_no_result(QPid, {gen_server2, cast, [Request]}),
     {State, []}.
 
 handle_event(QName, {confirm, MsgSeqNos, Pid}, #?STATE{unconfirmed = U0} = State) ->
@@ -352,9 +374,13 @@ handle_event(QName, {down, Pid, Info}, #?STATE{monitored = Monitored,
             {ok, State#?STATE{unconfirmed = U},
              [{rejected, QName, MsgIds} | Actions0]}
     end;
-handle_event(_QName, {send_drained, _} = Action, State) ->
+handle_event(_QName, Action, State)
+  when element(1, Action) =:= credit_reply ->
     {ok, State, [Action]};
-handle_event(_QName, {send_credit_reply, _} = Action, State) ->
+handle_event(_QName, {send_drained, {Ctag, Credit}}, State) ->
+    %% This function clause should be deleted when feature flag
+    %% credit_api_v2 becomes required.
+    Action = {credit_reply_v1, Ctag, Credit, _Available = 0, _Drain = true},
     {ok, State, [Action]}.
 
 settlement_action(_Type, _QRef, [], Acc) ->
@@ -610,26 +636,30 @@ ensure_monitor(Pid, QName, State = #?STATE{monitored = Monitored}) ->
 
 %% part of channel <-> queue api
 confirm_to_sender(Pid, QName, MsgSeqNos) ->
-    Msg = {confirm, MsgSeqNos, self()},
-    gen_server:cast(Pid, {queue_event, QName, Msg}).
+    Evt = {confirm, MsgSeqNos, self()},
+    send_queue_event(Pid, QName, Evt).
 
 send_rejection(Pid, QName, MsgSeqNo) ->
-    Msg = {reject_publish, MsgSeqNo, self()},
-    gen_server:cast(Pid, {queue_event, QName, Msg}).
+    Evt = {reject_publish, MsgSeqNo, self()},
+    send_queue_event(Pid, QName, Evt).
 
 deliver_to_consumer(Pid, QName, CTag, AckRequired, Message) ->
-    Deliver = {deliver, CTag, AckRequired, [Message]},
-    Evt = {queue_event, QName, Deliver},
-    gen_server:cast(Pid, Evt).
-
-send_drained(Pid, QName, CTagCredits) when is_list(CTagCredits) ->
-    lists:foreach(fun(CTagCredit) ->
-                          send_drained(Pid, QName, CTagCredit)
-                  end, CTagCredits);
-send_drained(Pid, QName, CTagCredit) when is_tuple(CTagCredit) ->
-    gen_server:cast(Pid, {queue_event, QName,
-                          {send_drained, CTagCredit}}).
-
-send_credit_reply(Pid, QName, Len) when is_integer(Len) ->
-    gen_server:cast(Pid, {queue_event, QName,
-                          {send_credit_reply, Len}}).
+    Evt = {deliver, CTag, AckRequired, [Message]},
+    send_queue_event(Pid, QName, Evt).
+
+%% Delete this function when feature flag credit_api_v2 becomes required.
+send_credit_reply_credit_api_v1(Pid, QName, Available) ->
+    Evt = {send_credit_reply, Available},
+    send_queue_event(Pid, QName, Evt).
+
+%% Delete this function when feature flag credit_api_v2 becomes required.
+send_drained_credit_api_v1(Pid, QName, Ctag, Credit) ->
+    Evt = {send_drained, {Ctag, Credit}},
+    send_queue_event(Pid, QName, Evt).
+
+send_credit_reply(Pid, QName, Ctag, DeliveryCount, Credit, Available, Drain) ->
+    Evt = {credit_reply, Ctag, DeliveryCount, Credit, Available, Drain},
+    send_queue_event(Pid, QName, Evt).
+
+send_queue_event(Pid, QName, Event) ->
+    gen_server:cast(Pid, {queue_event, QName, Event}).
diff --git a/deps/rabbit/src/rabbit_confirms.erl b/deps/rabbit/src/rabbit_confirms.erl
index b950e6df67eb..2ea00bc9cb39 100644
--- a/deps/rabbit/src/rabbit_confirms.erl
+++ b/deps/rabbit/src/rabbit_confirms.erl
@@ -45,7 +45,7 @@ insert(SeqNo, QNames, #resource{kind = exchange} = XName,
   when is_integer(SeqNo)
        andalso is_list(QNames)
        andalso not is_map_key(SeqNo, U0) ->
-    U = U0#{SeqNo => {XName, maps:from_list([{Q, ok} || Q <- QNames])}},
+    U = U0#{SeqNo => {XName, maps:from_keys(QNames, ok)}},
     S = case S0 of
             undefined -> SeqNo;
             _ -> S0
@@ -58,20 +58,18 @@ insert(SeqNo, QNames, #resource{kind = exchange} = XName,
 confirm(SeqNos, QName, #?MODULE{smallest = Smallest0,
                                 unconfirmed = U0} = State)
   when is_list(SeqNos) ->
-    {Confirmed, U} = lists:foldr(
-                       fun (SeqNo, Acc) ->
-                               confirm_one(SeqNo, QName, Acc)
-                       end, {[], U0}, SeqNos),
-    %% check if smallest is in Confirmed
-    %% TODO: this can be optimised by checking in the preceeding foldr
-    Smallest =
-    case lists:any(fun ({S, _}) -> S == Smallest0 end, Confirmed) of
-        true ->
-            %% work out new smallest
-            next_smallest(Smallest0, U);
-        false ->
-            Smallest0
-    end,
+    {Confirmed, ConfirmedSmallest, U} =
+        lists:foldl(
+          fun (SeqNo, Acc) ->
+                  confirm_one(SeqNo, QName, Smallest0, Acc)
+          end, {[], false, U0}, SeqNos),
+    Smallest = case ConfirmedSmallest of
+                   true ->
+                       %% work out new smallest
+                       next_smallest(Smallest0, U);
+                   false ->
+                       Smallest0
+               end,
     {Confirmed, State#?MODULE{smallest = Smallest,
                               unconfirmed = U}}.
 
@@ -124,17 +122,21 @@ is_empty(State) ->
 
 %% INTERNAL
 
-confirm_one(SeqNo, QName, {Acc, U0}) ->
+confirm_one(SeqNo, QName, Smallest, {Acc, ConfirmedSmallest0, U0}) ->
     case maps:take(SeqNo, U0) of
         {{XName, QS}, U1}
           when is_map_key(QName, QS)
                andalso map_size(QS) == 1 ->
             %% last queue confirm
-            {[{SeqNo, XName} | Acc], U1};
+            ConfirmedSmallest = case SeqNo of
+                                    Smallest -> true;
+                                    _ -> ConfirmedSmallest0
+                                end,
+            {[{SeqNo, XName} | Acc], ConfirmedSmallest, U1};
         {{XName, QS}, U1} ->
-            {Acc, U1#{SeqNo => {XName, maps:remove(QName, QS)}}};
+            {Acc, ConfirmedSmallest0, U1#{SeqNo => {XName, maps:remove(QName, QS)}}};
         error ->
-            {Acc, U0}
+            {Acc, ConfirmedSmallest0, U0}
     end.
 
 next_smallest(_S, U) when map_size(U) == 0 ->
diff --git a/deps/rabbit/src/rabbit_connection_helper_sup.erl b/deps/rabbit/src/rabbit_connection_helper_sup.erl
index 2ac04f73896e..7d665f2ab2be 100644
--- a/deps/rabbit/src/rabbit_connection_helper_sup.erl
+++ b/deps/rabbit/src/rabbit_connection_helper_sup.erl
@@ -18,7 +18,7 @@
 
 -behaviour(supervisor).
 
--export([start_link/0]).
+-export([start_link/1]).
 -export([
     start_channel_sup_sup/1,
     start_queue_collector/2
@@ -30,10 +30,10 @@
 
 %%----------------------------------------------------------------------------
 
--spec start_link() -> rabbit_types:ok_pid_or_error().
-
-start_link() ->
-    supervisor:start_link(?MODULE, []).
+-spec start_link(supervisor:sup_flags()) ->
+    supervisor:startlink_ret().
+start_link(SupFlags) ->
+    supervisor:start_link(?MODULE, SupFlags).
 
 -spec start_channel_sup_sup(pid()) -> rabbit_types:ok_pid_or_error().
 
@@ -62,10 +62,6 @@ start_queue_collector(SupPid, Identity) ->
 
 %%----------------------------------------------------------------------------
 
-init([]) ->
+init(SupFlags) ->
     ?LG_PROCESS_TYPE(connection_helper_sup),
-    SupFlags = #{strategy => one_for_one,
-                 intensity => 10,
-                 period => 10,
-                 auto_shutdown => any_significant},
     {ok, {SupFlags, []}}.
diff --git a/deps/rabbit/src/rabbit_connection_sup.erl b/deps/rabbit/src/rabbit_connection_sup.erl
index 3cdb0d8e0664..92ddaaf7784e 100644
--- a/deps/rabbit/src/rabbit_connection_sup.erl
+++ b/deps/rabbit/src/rabbit_connection_sup.erl
@@ -19,7 +19,10 @@
 -behaviour(supervisor).
 -behaviour(ranch_protocol).
 
--export([start_link/3, reader/1]).
+-export([start_link/3,
+         reader/1,
+         start_connection_helper_sup/2
+        ]).
 
 -export([init/1]).
 
@@ -27,40 +30,17 @@
 
 %%----------------------------------------------------------------------------
 
--spec start_link(any(), module(), any()) ->
+-spec start_link(ranch:ref(), module(), any()) ->
     {'ok', pid(), pid()}.
 
 start_link(Ref, _Transport, _Opts) ->
     {ok, SupPid} = supervisor:start_link(?MODULE, []),
-    %% We need to get channels in the hierarchy here so they get shut
-    %% down after the reader, so the reader gets a chance to terminate
-    %% them cleanly. But for 1.0 readers we can't start the real
-    %% ch_sup_sup (because we don't know if we will be 0-9-1 or 1.0) -
-    %% so we add another supervisor into the hierarchy.
-    %%
-    %% This supervisor also acts as an intermediary for heartbeaters and
-    %% the queue collector process, since these must not be siblings of the
-    %% reader due to the potential for deadlock if they are added/restarted
-    %% whilst the supervision tree is shutting down.
-    {ok, HelperSup} =
-        supervisor:start_child(
-            SupPid,
-            #{
-                id => helper_sup,
-                start => {rabbit_connection_helper_sup, start_link, []},
-                restart => transient,
-                significant => true,
-                shutdown => infinity,
-                type => supervisor,
-                modules => [rabbit_connection_helper_sup]
-            }
-        ),
     {ok, ReaderPid} =
         supervisor:start_child(
             SupPid,
             #{
                 id => reader,
-                start => {rabbit_reader, start_link, [HelperSup, Ref]},
+                start => {rabbit_reader, start_link, [Ref]},
                 restart => transient,
                 significant => true,
                 shutdown => ?WORKER_WAIT,
@@ -75,6 +55,20 @@ start_link(Ref, _Transport, _Opts) ->
 reader(Pid) ->
     hd(rabbit_misc:find_child(Pid, reader)).
 
+-spec start_connection_helper_sup(pid(), supervisor:sup_flags()) ->
+    supervisor:startchild_ret().
+start_connection_helper_sup(ConnectionSupPid, ConnectionHelperSupFlags) ->
+    supervisor:start_child(
+      ConnectionSupPid,
+      #{
+        id => helper_sup,
+        start => {rabbit_connection_helper_sup, start_link, [ConnectionHelperSupFlags]},
+        restart => transient,
+        significant => true,
+        shutdown => infinity,
+        type => supervisor
+       }).
+
 %%--------------------------------------------------------------------------
 
 init([]) ->
diff --git a/deps/rabbit/src/rabbit_core_ff.erl b/deps/rabbit/src/rabbit_core_ff.erl
index e35fdab80f70..d6dd4490056c 100644
--- a/deps/rabbit/src/rabbit_core_ff.erl
+++ b/deps/rabbit/src/rabbit_core_ff.erl
@@ -123,6 +123,8 @@
 -rabbit_feature_flag(
    {message_containers,
     #{desc          => "Message containers.",
+      %%TODO Once lower version node in mixed versions is bumped to 3.13,
+      %% make 'required' for upgrading AMQP 1.0 from 3.13 to 4.0
       stability     => stable,
       depends_on    => [feature_flags_v2]
      }}).
@@ -156,3 +158,9 @@
       stability     => stable,
       depends_on    => [stream_queue]
      }}).
+
+-rabbit_feature_flag(
+   {credit_api_v2,
+    #{desc          => "Credit API v2 between queue clients and queue processes",
+      stability     => stable
+     }}).
diff --git a/deps/rabbit/src/rabbit_fifo.erl b/deps/rabbit/src/rabbit_fifo.erl
index fea9b2a0eaaf..904c4ee39e3d 100644
--- a/deps/rabbit/src/rabbit_fifo.erl
+++ b/deps/rabbit/src/rabbit_fifo.erl
@@ -74,6 +74,8 @@
          chunk_disk_msgs/3]).
 -endif.
 
+-import(serial_number, [add/2, diff/2]).
+
 %% command records representing all the protocol actions that are supported
 -record(enqueue, {pid :: option(pid()),
                   seq :: option(msg_seqno()),
@@ -95,7 +97,7 @@
                   msg_ids :: [msg_id()]}).
 -record(credit, {consumer_id :: consumer_id(),
                  credit :: non_neg_integer(),
-                 delivery_count :: non_neg_integer(),
+                 delivery_count :: rabbit_queue_type:delivery_count(),
                  drain :: boolean()}).
 -record(purge, {}).
 -record(purge_nodes, {nodes :: [node()]}).
@@ -130,7 +132,6 @@
               delivery/0,
               command/0,
               credit_mode/0,
-              consumer_tag/0,
               consumer_meta/0,
               consumer_id/0,
               client_msg/0,
@@ -184,8 +185,8 @@ update_config(Conf, State) ->
 % msg_ids are scoped per consumer
 % ra_indexes holds all raft indexes for enqueues currently on queue
 -spec apply(ra_machine:command_meta_data(), command(), state()) ->
-    {state(), Reply :: term(), ra_machine:effects()} |
-    {state(), Reply :: term()}.
+    {state(), ra_machine:reply(), ra_machine:effects() | ra_machine:effect()} |
+    {state(), ra_machine:reply()}.
 apply(Meta, #enqueue{pid = From, seq = Seq,
                      msg = RawMsg}, State00) ->
     apply_enqueue(Meta, From, Seq, RawMsg, State00);
@@ -276,59 +277,92 @@ apply(#{index := Idx} = Meta,
         _ ->
             {State00, ok, []}
     end;
-apply(Meta, #credit{credit = NewCredit, delivery_count = RemoteDelCnt,
-                    drain = Drain, consumer_id = ConsumerId},
+apply(Meta, #credit{credit = LinkCreditRcv, delivery_count = DeliveryCountRcv,
+                    drain = Drain, consumer_id = ConsumerId = {CTag, CPid}},
       #?MODULE{consumers = Cons0,
                service_queue = ServiceQueue0,
                waiting_consumers = Waiting0} = State0) ->
     case Cons0 of
-        #{ConsumerId := #consumer{delivery_count = DelCnt} = Con0} ->
-            %% this can go below 0 when credit is reduced
-            C = max(0, RemoteDelCnt + NewCredit - DelCnt),
+        #{ConsumerId := #consumer{delivery_count = DeliveryCountSnd,
+                                  cfg = Cfg} = Con0} ->
+            LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, Cfg),
             %% grant the credit
-            Con1 = Con0#consumer{credit = C},
-            ServiceQueue = maybe_queue_consumer(ConsumerId, Con1,
-                                                ServiceQueue0),
-            Cons = maps:put(ConsumerId, Con1, Cons0),
-            {State1, ok, Effects} =
-            checkout(Meta, State0,
-                     State0#?MODULE{service_queue = ServiceQueue,
-                                    consumers = Cons}, []),
-            Response = {send_credit_reply, messages_ready(State1)},
-            %% by this point all checkouts for the updated credit value
-            %% should be processed so we can evaluate the drain
-            case Drain of
-                false ->
-                    %% just return the result of the checkout
-                    {State1, Response, Effects};
+            Con1 = Con0#consumer{credit = LinkCreditSnd},
+            ServiceQueue = maybe_queue_consumer(ConsumerId, Con1, ServiceQueue0),
+            State1 = State0#?MODULE{service_queue = ServiceQueue,
+                                    consumers = maps:update(ConsumerId, Con1, Cons0)},
+            {State2, ok, Effects} = checkout(Meta, State0, State1, []),
+
+            #?MODULE{consumers = Cons1 = #{ConsumerId := Con2}} = State2,
+            #consumer{credit = PostCred,
+                      delivery_count = PostDeliveryCount} = Con2,
+            Available = messages_ready(State2),
+            case credit_api_v2(Cfg) of
                 true ->
-                    Con = #consumer{credit = PostCred} =
-                        maps:get(ConsumerId, State1#?MODULE.consumers),
-                    %% add the outstanding credit to the delivery count
-                    DeliveryCount = Con#consumer.delivery_count + PostCred,
-                    Consumers = maps:put(ConsumerId,
-                                         Con#consumer{delivery_count = DeliveryCount,
-                                                      credit = 0},
-                                         State1#?MODULE.consumers),
-                    Drained = Con#consumer.credit,
-                    {CTag, _} = ConsumerId,
-                    {State1#?MODULE{consumers = Consumers},
-                     %% returning a multi response with two client actions
-                     %% for the channel to execute
-                     {multi, [Response, {send_drained, {CTag, Drained}}]},
-                     Effects}
+                    {Credit, DeliveryCount, State} =
+                    case Drain andalso PostCred > 0 of
+                        true ->
+                            AdvancedDeliveryCount = add(PostDeliveryCount, PostCred),
+                            ZeroCredit = 0,
+                            Con = Con2#consumer{delivery_count = AdvancedDeliveryCount,
+                                                credit = ZeroCredit},
+                            Cons = maps:update(ConsumerId, Con, Cons1),
+                            State3 = State2#?MODULE{consumers = Cons},
+                            {ZeroCredit, AdvancedDeliveryCount, State3};
+                        false ->
+                            {PostCred, PostDeliveryCount, State2}
+                    end,
+                    %% We must send to queue client delivery effects before credit_reply such
+                    %% that session process can send to AMQP 1.0 client TRANSFERs before FLOW.
+                    {State, ok, Effects ++ [{send_msg, CPid,
+                                             {credit_reply, CTag, DeliveryCount, Credit, Available, Drain},
+                                             ?DELIVERY_SEND_MSG_OPTS}]};
+                false ->
+                    %% We must always send a send_credit_reply because basic.credit is synchronous.
+                    %% Additionally, we keep the bug of credit API v1 that we send to queue client the
+                    %% send_drained reply before the delivery effects (resulting in the wrong behaviour
+                    %% that the session process sends to AMQP 1.0 client the FLOW before the TRANSFERs).
+                    %% We have to keep this bug because old rabbit_fifo_client implementations expect
+                    %% a send_drained Ra reply (they can't handle such a Ra effect).
+                    CreditReply = {send_credit_reply, Available},
+                    case Drain of
+                        true ->
+                            AdvancedDeliveryCount = PostDeliveryCount + PostCred,
+                            Con = Con2#consumer{delivery_count = AdvancedDeliveryCount,
+                                                credit = 0},
+                            Cons = maps:update(ConsumerId, Con, Cons1),
+                            State = State2#?MODULE{consumers = Cons},
+                            Reply = {multi, [CreditReply, {send_drained, {CTag, PostCred}}]},
+                            {State, Reply, Effects};
+                        false ->
+                            {State2, CreditReply, Effects}
+                    end
             end;
         _ when Waiting0 /= [] ->
-            %% there are waiting consuemrs
+            %%TODO next time when we bump the machine version:
+            %% 1. Do not put consumer at head of waiting_consumers if NewCredit == 0
+            %%    to reduce likelihood of activating a 0 credit consumer.
+            %% 2. Support Drain == true, i.e. advance delivery-count, consuming all link-credit since there
+            %%    are no messages available for an inactive consumer and send credit_reply with Drain=true.
             case lists:keytake(ConsumerId, 1, Waiting0) of
-                {value, {_, Con0 = #consumer{delivery_count = DelCnt}}, Waiting} ->
-                    %% the consumer is a waiting one
+                {value, {_, Con0 = #consumer{delivery_count = DeliveryCountSnd,
+                                             cfg = Cfg}}, Waiting} ->
+                    LinkCreditSnd = link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, Cfg),
                     %% grant the credit
-                    C = max(0, RemoteDelCnt + NewCredit - DelCnt),
-                    Con = Con0#consumer{credit = C},
+                    Con = Con0#consumer{credit = LinkCreditSnd},
                     State = State0#?MODULE{waiting_consumers =
                                            [{ConsumerId, Con} | Waiting]},
-                    {State, {send_credit_reply, messages_ready(State)}};
+                    %% No messages are available for inactive consumers.
+                    Available = 0,
+                    case credit_api_v2(Cfg) of
+                        true ->
+                            {State, ok,
+                             {send_msg, CPid,
+                              {credit_reply, CTag, DeliveryCountSnd, LinkCreditSnd, Available, false},
+                              ?DELIVERY_SEND_MSG_OPTS}};
+                        false ->
+                            {State, {send_credit_reply, Available}}
+                    end;
                 false ->
                     {State0, ok}
             end;
@@ -1240,12 +1274,12 @@ query_consumers(#?MODULE{consumers = Consumers,
     FromConsumers =
         maps:fold(fun (_, #consumer{status = cancelled}, Acc) ->
                           Acc;
-                      ({Tag, Pid},
+                      (Key = {Tag, Pid},
                        #consumer{cfg = #consumer_cfg{meta = Meta}} = Consumer,
                        Acc) ->
                           {Active, ActivityStatus} =
-                              ActiveActivityStatusFun({Tag, Pid}, Consumer),
-                          maps:put({Tag, Pid},
+                          ActiveActivityStatusFun(Key, Consumer),
+                          maps:put(Key,
                                    {Pid, Tag,
                                     maps:get(ack, Meta, undefined),
                                     maps:get(prefetch, Meta, undefined),
@@ -1258,12 +1292,12 @@ query_consumers(#?MODULE{consumers = Consumers,
     FromWaitingConsumers =
         lists:foldl(fun ({_, #consumer{status = cancelled}}, Acc) ->
                                       Acc;
-                        ({{Tag, Pid},
+                        (Key = {{Tag, Pid},
                           #consumer{cfg = #consumer_cfg{meta = Meta}} = Consumer},
                          Acc) ->
                             {Active, ActivityStatus} =
-                                ActiveActivityStatusFun({Tag, Pid}, Consumer),
-                            maps:put({Tag, Pid},
+                                ActiveActivityStatusFun(Key, Consumer),
+                            maps:put(Key,
                                      {Pid, Tag,
                                       maps:get(ack, Meta, undefined),
                                       maps:get(prefetch, Meta, undefined),
@@ -2032,7 +2066,7 @@ get_next_msg(#?MODULE{returns = Returns0,
 delivery_effect({CTag, CPid}, [{MsgId, ?MSG(Idx,  Header)}],
                 #?MODULE{msg_cache = {Idx, RawMsg}}) ->
     {send_msg, CPid, {delivery, CTag, [{MsgId, {Header, RawMsg}}]},
-     [local, ra_event]};
+     ?DELIVERY_SEND_MSG_OPTS};
 delivery_effect({CTag, CPid}, Msgs, _State) ->
     RaftIdxs = lists:foldr(fun ({_, ?MSG(I, _)}, Acc) ->
                                    [I | Acc]
@@ -2043,7 +2077,7 @@ delivery_effect({CTag, CPid}, Msgs, _State) ->
                          fun (Cmd, {MsgId, ?MSG(_Idx,  Header)}) ->
                                  {MsgId, {Header, get_msg(Cmd)}}
                          end, Log, Msgs),
-             [{send_msg, CPid, {delivery, CTag, DelMsgs}, [local, ra_event]}]
+             [{send_msg, CPid, {delivery, CTag, DelMsgs}, ?DELIVERY_SEND_MSG_OPTS}]
      end,
      {local, node(CPid)}}.
 
@@ -2078,21 +2112,25 @@ checkout_one(#{system_time := Ts} = Meta, ExpiredMsg0, InitState0, Effects0) ->
                             %% recurse without consumer on queue
                             checkout_one(Meta, ExpiredMsg,
                                          InitState#?MODULE{service_queue = SQ1}, Effects1);
-                        #consumer{status = cancelled} ->
-                            checkout_one(Meta, ExpiredMsg,
-                                         InitState#?MODULE{service_queue = SQ1}, Effects1);
-                        #consumer{status = suspected_down} ->
+                        #consumer{status = S}
+                          when S =:= cancelled orelse
+                               S =:= suspected_down ->
                             checkout_one(Meta, ExpiredMsg,
                                          InitState#?MODULE{service_queue = SQ1}, Effects1);
                         #consumer{checked_out = Checked0,
                                   next_msg_id = Next,
                                   credit = Credit,
-                                  delivery_count = DelCnt} = Con0 ->
+                                  delivery_count = DelCnt0,
+                                  cfg = Cfg} = Con0 ->
                             Checked = maps:put(Next, ConsumerMsg, Checked0),
+                            DelCnt = case credit_api_v2(Cfg) of
+                                         true -> add(DelCnt0, 1);
+                                         false -> DelCnt0 + 1
+                                     end,
                             Con = Con0#consumer{checked_out = Checked,
                                                 next_msg_id = Next + 1,
                                                 credit = Credit - 1,
-                                                delivery_count = DelCnt + 1},
+                                                delivery_count = DelCnt},
                             Size = get_header(size, get_msg_header(ConsumerMsg)),
                             State = update_or_remove_sub(
                                        Meta, ConsumerId, Con,
@@ -2186,11 +2224,11 @@ update_or_remove_sub(_Meta, ConsumerId,
                      #?MODULE{consumers = Cons,
                               service_queue = ServiceQueue} = State) ->
     State#?MODULE{consumers = maps:put(ConsumerId, Con, Cons),
-                  service_queue = uniq_queue_in(ConsumerId, Con, ServiceQueue)}.
+                  service_queue = maybe_queue_consumer(ConsumerId, Con, ServiceQueue)}.
 
-uniq_queue_in(Key, #consumer{credit = Credit,
-                             status = up,
-                             cfg = #consumer_cfg{priority = P}}, ServiceQueue)
+maybe_queue_consumer(Key, #consumer{credit = Credit,
+                                    status = up,
+                                    cfg = #consumer_cfg{priority = P}}, ServiceQueue)
   when Credit > 0 ->
     % TODO: queue:member could surely be quite expensive, however the practical
     % number of unique consumers may not be large enough for it to matter
@@ -2200,7 +2238,7 @@ uniq_queue_in(Key, #consumer{credit = Credit,
         false ->
             priority_queue:in(Key, P, ServiceQueue)
     end;
-uniq_queue_in(_Key, _Consumer, ServiceQueue) ->
+maybe_queue_consumer(_Key, _Consumer, ServiceQueue) ->
     ServiceQueue.
 
 update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta,
@@ -2218,7 +2256,8 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta,
                                                      meta = ConsumerMeta,
                                                      priority = Priority,
                                                      credit_mode = Mode},
-                                 credit = Credit}
+                                 credit = Credit,
+                                 delivery_count = initial_delivery_count(ConsumerMeta)}
                end,
     {Consumer, update_or_remove_sub(Meta, ConsumerId, Consumer, State0)};
 update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta,
@@ -2252,8 +2291,8 @@ update_consumer(Meta, {Tag, Pid} = ConsumerId, ConsumerMeta,
                                                      meta = ConsumerMeta,
                                                      priority = Priority,
                                                      credit_mode = Mode},
-                                 credit = Credit},
-
+                                 credit = Credit,
+                                 delivery_count = initial_delivery_count(ConsumerMeta)},
             {Consumer,
              State0#?MODULE{waiting_consumers =
                             Waiting ++ [{ConsumerId, Consumer}]}}
@@ -2277,16 +2316,6 @@ credit_mode(#{machine_version := Vsn}, Credit, simple_prefetch)
 credit_mode(_, _, Mode) ->
     Mode.
 
-maybe_queue_consumer(ConsumerId, #consumer{credit = Credit} = Con,
-                     ServiceQueue0) ->
-    case Credit > 0 of
-        true ->
-            % consumer needs service - check if already on service queue
-            uniq_queue_in(ConsumerId, Con, ServiceQueue0);
-        false ->
-            ServiceQueue0
-    end.
-
 %% creates a dehydrated version of the current state to be cached and
 %% potentially used to for a snaphot at a later point
 dehydrate_state(#?MODULE{cfg = #cfg{},
@@ -2363,8 +2392,8 @@ make_return(ConsumerId, MsgIds) ->
 make_discard(ConsumerId, MsgIds) ->
     #discard{consumer_id = ConsumerId, msg_ids = MsgIds}.
 
--spec make_credit(consumer_id(), non_neg_integer(), non_neg_integer(),
-                  boolean()) -> protocol().
+-spec make_credit(consumer_id(), rabbit_queue_type:credit(),
+                  non_neg_integer(), boolean()) -> protocol().
 make_credit(ConsumerId, Credit, DeliveryCount, Drain) ->
     #credit{consumer_id = ConsumerId,
             credit = Credit,
@@ -2563,3 +2592,26 @@ get_msg(#enqueue{msg = M}) ->
     M;
 get_msg(#requeue{msg = M}) ->
     M.
+
+-spec initial_delivery_count(consumer_meta()) ->
+    rabbit_queue_type:delivery_count().
+initial_delivery_count(#{initial_delivery_count := Count}) ->
+    %% credit API v2
+    Count;
+initial_delivery_count(_) ->
+    %% credit API v1
+    0.
+
+-spec credit_api_v2(#consumer_cfg{}) ->
+    boolean().
+credit_api_v2(#consumer_cfg{meta = ConsumerMeta}) ->
+    maps:is_key(initial_delivery_count, ConsumerMeta).
+
+%% AMQP 1.0 §2.6.7
+link_credit_snd(DeliveryCountRcv, LinkCreditRcv, DeliveryCountSnd, ConsumerCfg) ->
+    C = case credit_api_v2(ConsumerCfg) of
+            true -> diff(add(DeliveryCountRcv, LinkCreditRcv), DeliveryCountSnd);
+            false -> DeliveryCountRcv + LinkCreditRcv - DeliveryCountSnd
+        end,
+    %% C can be negative when receiver decreases credits while messages are in flight.
+    max(0, C).
diff --git a/deps/rabbit/src/rabbit_fifo.hrl b/deps/rabbit/src/rabbit_fifo.hrl
index 9b1078265dc6..65f2db8a601f 100644
--- a/deps/rabbit/src/rabbit_fifo.hrl
+++ b/deps/rabbit/src/rabbit_fifo.hrl
@@ -17,6 +17,8 @@
         is_list(H) orelse
         (is_map(H) andalso is_map_key(size, H))).
 
+-define(DELIVERY_SEND_MSG_OPTS, [local, ra_event]).
+
 -type optimised_tuple(A, B) :: nonempty_improper_list(A, B).
 
 -type option(T) :: undefined | T.
@@ -56,14 +58,10 @@
 -type delivery_msg() :: {msg_id(), {msg_header(), raw_msg()}}.
 %% A tuple consisting of the message id, and the headered message.
 
--type consumer_tag() :: binary().
-%% An arbitrary binary tag used to distinguish between different consumers
-%% set up by the same process. See: {@link rabbit_fifo_client:checkout/3.}
-
--type delivery() :: {delivery, consumer_tag(), [delivery_msg()]}.
+-type delivery() :: {delivery, rabbit_types:ctag(), [delivery_msg()]}.
 %% Represents the delivery of one or more rabbit_fifo messages.
 
--type consumer_id() :: {consumer_tag(), pid()}.
+-type consumer_id() :: {rabbit_types:ctag(), pid()}.
 %% The entity that receives messages. Uniquely identifies a consumer.
 
 -type credit_mode() :: credited |
@@ -81,7 +79,10 @@
 -type consumer_meta() :: #{ack => boolean(),
                            username => binary(),
                            prefetch => non_neg_integer(),
-                           args => list()}.
+                           args => list(),
+                           %% set if and only if credit API v2 is in use
+                           initial_delivery_count => rabbit_queue_type:delivery_count()
+                          }.
 %% static meta data associated with a consumer
 
 -type applied_mfa() :: {module(), atom(), list()}.
@@ -101,7 +102,7 @@
 -record(consumer_cfg,
         {meta = #{} :: consumer_meta(),
          pid :: pid(),
-         tag :: consumer_tag(),
+         tag :: rabbit_types:ctag(),
          %% the mode of how credit is incremented
          %% simple_prefetch: credit is re-filled as deliveries are settled
          %% or returned.
@@ -119,9 +120,8 @@
          %% max number of messages that can be sent
          %% decremented for each delivery
          credit = 0 : non_neg_integer(),
-         %% total number of checked out messages - ever
-         %% incremented for each delivery
-         delivery_count = 0 :: non_neg_integer()
+         %% AMQP 1.0 §2.6.7
+         delivery_count :: rabbit_queue_type:delivery_count()
         }).
 
 -type consumer() :: #consumer{}.
@@ -200,7 +200,7 @@
          dlx = rabbit_fifo_dlx:init() :: rabbit_fifo_dlx:state(),
          msg_bytes_enqueue = 0 :: non_neg_integer(),
          msg_bytes_checkout = 0 :: non_neg_integer(),
-         %% waiting consumers, one is picked active consumer is cancelled or dies
+         %% one is picked if active consumer is cancelled or dies
          %% used only when single active consumer is on
          waiting_consumers = [] :: [{consumer_id(), consumer()}],
          last_active :: option(non_neg_integer()),
diff --git a/deps/rabbit/src/rabbit_fifo_client.erl b/deps/rabbit/src/rabbit_fifo_client.erl
index d6a32aa9769e..f1bea8db3498 100644
--- a/deps/rabbit/src/rabbit_fifo_client.erl
+++ b/deps/rabbit/src/rabbit_fifo_client.erl
@@ -22,7 +22,8 @@
          settle/3,
          return/3,
          discard/3,
-         credit/4,
+         credit_v1/4,
+         credit/6,
          handle_ra_event/4,
          untracked_enqueue/2,
          purge/1,
@@ -39,15 +40,16 @@
 -define(COMMAND_TIMEOUT, 30000).
 
 -type seq() :: non_neg_integer().
--type action() :: {send_credit_reply, Available :: non_neg_integer()} |
-                  {send_drained, CTagCredit ::
-                   {rabbit_fifo:consumer_tag(), non_neg_integer()}} |
-                  rabbit_queue_type:action().
--type actions() :: [action()].
 
 -record(consumer, {last_msg_id :: seq() | -1 | undefined,
                    ack = false :: boolean(),
-                   delivery_count = 0 :: non_neg_integer()}).
+                   %% 'echo' field from latest FLOW, see AMQP 1.0 §2.7.4
+                   %% Quorum queue server will always echo back to us,
+                   %% but we only emit a credit_reply if Echo=true
+                   echo :: boolean(),
+                   %% Remove this field when feature flag credit_api_v2 becomes required.
+                   delivery_count :: {credit_api_v1, rabbit_queue_type:delivery_count()} | credit_api_v2
+                  }).
 
 -record(cfg, {servers = [] :: [ra:server_id()],
               soft_limit = ?SOFT_LIMIT :: non_neg_integer(),
@@ -65,18 +67,14 @@
                                            {[seq()], [seq()], [seq()]}},
                 pending = #{} :: #{seq() =>
                                    {term(), rabbit_fifo:command()}},
-                consumer_deliveries = #{} :: #{rabbit_fifo:consumer_tag() =>
+                consumer_deliveries = #{} :: #{rabbit_types:ctag() =>
                                                #consumer{}},
                 timer_state :: term()
                }).
 
 -opaque state() :: #state{}.
 
--export_type([
-              state/0,
-              actions/0
-             ]).
-
+-export_type([state/0]).
 
 %% @doc Create the initial state for a new rabbit_fifo sessions. A state is needed
 %% to interact with a rabbit_fifo queue using @module.
@@ -111,7 +109,7 @@ init(Servers, SoftLimit) ->
 %% by the {@link handle_ra_event/2. handle_ra_event/2} function.
 -spec enqueue(rabbit_amqqueue:name(), Correlation :: term(),
               Msg :: term(), State :: state()) ->
-    {ok, state(), actions()} | {reject_publish, state()}.
+    {ok, state(), rabbit_queue_type:actions()} | {reject_publish, state()}.
 enqueue(QName, Correlation, Msg,
         #state{queue_status = undefined,
                next_enqueue_seq = 1,
@@ -177,7 +175,7 @@ enqueue(QName, Correlation, Msg,
 %% by the {@link handle_ra_event/2. handle_ra_event/2} function.
 %%
 -spec enqueue(rabbit_amqqueue:name(), Msg :: term(), State :: state()) ->
-    {ok, state(), actions()} | {reject_publish, state()}.
+    {ok, state(), rabbit_queue_type:actions()} | {reject_publish, state()}.
 enqueue(QName, Msg, State) ->
     enqueue(QName, undefined, Msg, State).
 
@@ -193,7 +191,7 @@ enqueue(QName, Msg, State) ->
 %% @param State The {@module} state.
 %%
 %% @returns `{ok, IdMsg, State}' or `{error | timeout, term()}'
--spec dequeue(rabbit_amqqueue:name(), rabbit_fifo:consumer_tag(),
+-spec dequeue(rabbit_amqqueue:name(), rabbit_types:ctag(),
               Settlement :: settled | unsettled, state()) ->
     {ok, non_neg_integer(), term(), non_neg_integer()}
      | {empty, state()} | {error | timeout, term()}.
@@ -239,7 +237,7 @@ add_delivery_count_header(Msg, Count) ->
 %% @param MsgIds the message ids received with the {@link rabbit_fifo:delivery/0.}
 %% @param State the {@module} state
 %%
--spec settle(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
+-spec settle(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) ->
     {state(), list()}.
 settle(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
     ServerId = pick_server(State0),
@@ -267,7 +265,7 @@ settle(ConsumerTag, [_|_] = MsgIds,
 %% @returns
 %% `{State, list()}' if the command was successfully sent.
 %%
--spec return(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
+-spec return(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) ->
     {state(), list()}.
 return(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
     ServerId = pick_server(State0),
@@ -292,7 +290,7 @@ return(ConsumerTag, [_|_] = MsgIds,
 %% @param MsgIds the message ids to discard
 %% from {@link rabbit_fifo:delivery/0.}
 %% @param State the {@module} state
--spec discard(rabbit_fifo:consumer_tag(), [rabbit_fifo:msg_id()], state()) ->
+-spec discard(rabbit_types:ctag(), [rabbit_fifo:msg_id()], state()) ->
     {state(), list()}.
 discard(ConsumerTag, [_|_] = MsgIds, #state{slow = false} = State0) ->
     ServerId = pick_server(State0),
@@ -325,7 +323,7 @@ discard(ConsumerTag, [_|_] = MsgIds,
 %% @param State The {@module} state.
 %%
 %% @returns `{ok, State}' or `{error | timeout, term()}'
--spec checkout(rabbit_fifo:consumer_tag(),
+-spec checkout(rabbit_types:ctag(),
                NumUnsettled :: non_neg_integer(),
                CreditMode :: rabbit_fifo:credit_mode(),
                Meta :: rabbit_fifo:consumer_meta(),
@@ -362,10 +360,18 @@ checkout(ConsumerTag, NumUnsettled, CreditMode, Meta,
                                         NextMsgId - 1
                                 end
                         end,
+            DeliveryCount = case maps:is_key(initial_delivery_count, Meta) of
+                                true -> credit_api_v2;
+                                false -> {credit_api_v1, 0}
+                            end,
             SDels = maps:update_with(
-                      ConsumerTag, fun (C) -> C#consumer{ack = Ack} end,
+                      ConsumerTag,
+                      fun (C) -> C#consumer{ack = Ack} end,
                       #consumer{last_msg_id = LastMsgId,
-                                ack = Ack}, CDels0),
+                                ack = Ack,
+                                echo = false,
+                                delivery_count = DeliveryCount},
+                      CDels0),
             {ok, State0#state{leader = Leader,
                               consumer_deliveries = SDels}};
         Err ->
@@ -385,29 +391,45 @@ query_single_active_consumer(#state{leader = Leader}) ->
             Err
     end.
 
+-spec credit_v1(rabbit_types:ctag(),
+                Credit :: non_neg_integer(),
+                Drain :: boolean(),
+                state()) ->
+    {state(), rabbit_queue_type:actions()}.
+credit_v1(ConsumerTag, Credit, Drain,
+          #state{consumer_deliveries = CDels} = State0) ->
+    ConsumerId = consumer_id(ConsumerTag),
+    #consumer{delivery_count = {credit_api_v1, Count}} = maps:get(ConsumerTag, CDels),
+    ServerId = pick_server(State0),
+    Cmd = rabbit_fifo:make_credit(ConsumerId, Credit, Count, Drain),
+    {send_command(ServerId, undefined, Cmd, normal, State0), []}.
+
 %% @doc Provide credit to the queue
 %%
 %% This only has an effect if the consumer uses credit mode: credited
 %% @param ConsumerTag a unique tag to identify this particular consumer.
-%% @param Credit the amount of credit to provide to theq queue
+%% @param Credit the amount of credit to provide to the queue
 %% @param Drain tells the queue to use up any credit that cannot be immediately
 %% fulfilled. (i.e. there are not enough messages on queue to use up all the
 %% provided credit).
--spec credit(rabbit_fifo:consumer_tag(),
-             Credit :: non_neg_integer(),
+%% @param Reply true if the queue client requests a credit_reply queue action
+-spec credit(rabbit_types:ctag(),
+             rabbit_queue_type:delivery_count(),
+             rabbit_queue_type:credit(),
              Drain :: boolean(),
+             Echo :: boolean(),
              state()) ->
-          {state(), actions()}.
-credit(ConsumerTag, Credit, Drain,
-       #state{consumer_deliveries = CDels} = State0) ->
+    {state(), rabbit_queue_type:actions()}.
+credit(ConsumerTag, DeliveryCount, Credit, Drain, Echo,
+       #state{consumer_deliveries = CDels0} = State0) ->
     ConsumerId = consumer_id(ConsumerTag),
-    %% the last received msgid provides us with the delivery count if we
-    %% add one as it is 0 indexed
-    C = maps:get(ConsumerTag, CDels, #consumer{last_msg_id = -1}),
     ServerId = pick_server(State0),
-    Cmd = rabbit_fifo:make_credit(ConsumerId, Credit,
-                                  C#consumer.last_msg_id + 1, Drain),
-    {send_command(ServerId, undefined, Cmd, normal, State0), []}.
+    Cmd = rabbit_fifo:make_credit(ConsumerId, Credit, DeliveryCount, Drain),
+    CDels = maps:update_with(ConsumerTag,
+                             fun(C) -> C#consumer{echo = Echo} end,
+                             CDels0),
+    State = State0#state{consumer_deliveries = CDels},
+    {send_command(ServerId, undefined, Cmd, normal, State), []}.
 
 %% @doc Cancels a checkout with the rabbit_fifo queue  for the consumer tag
 %%
@@ -418,7 +440,7 @@ credit(ConsumerTag, Credit, Drain,
 %% @param State The {@module} state.
 %%
 %% @returns `{ok, State}' or `{error | timeout, term()}'
--spec cancel_checkout(rabbit_fifo:consumer_tag(), state()) ->
+-spec cancel_checkout(rabbit_types:ctag(), state()) ->
     {ok, state()} | {error | timeout, term()}.
 cancel_checkout(ConsumerTag, #state{consumer_deliveries = CDels} = State0) ->
     Servers = sorted_servers(State0),
@@ -521,25 +543,25 @@ update_machine_state(Server, Conf) ->
 %% with them.
 -spec handle_ra_event(rabbit_amqqueue:name(), ra:server_id(),
                       ra_server_proc:ra_event_body(), state()) ->
-    {internal, Correlators :: [term()], actions(), state()} |
-    {rabbit_fifo:client_msg(), state()} | {eol, actions()}.
+    {internal, Correlators :: [term()], rabbit_queue_type:actions(), state()} |
+    {rabbit_fifo:client_msg(), state()} | {eol, rabbit_queue_type:actions()}.
 handle_ra_event(QName, From, {applied, Seqs},
                 #state{cfg = #cfg{soft_limit = SftLmt}} = State0) ->
 
-    {Corrs, Actions0, State1} = lists:foldl(fun seq_applied/2,
-                                           {[], [], State0#state{leader = From}},
-                                           Seqs),
+    {Corrs, ActionsRev, State1} = lists:foldl(fun seq_applied/2,
+                                              {[], [], State0#state{leader = From}},
+                                              Seqs),
+    Actions0 = lists:reverse(ActionsRev),
     Actions = case Corrs of
                   [] ->
-                      lists:reverse(Actions0);
+                      Actions0;
                   _ ->
                       %%TODO consider using lists:foldr/3 above because
                       %% Corrs is returned in the wrong order here.
                       %% The wrong order does not matter much because the channel sorts the
                       %% sequence numbers before confirming to the client. But rabbit_fifo_client
                       %% is sequence numer agnostic: it handles any correlation terms.
-                      [{settled, QName, Corrs}
-                       | lists:reverse(Actions0)]
+                      [{settled, QName, Corrs} | Actions0]
               end,
     case maps:size(State1#state.pending) < SftLmt of
         true when State1#state.slow == true ->
@@ -572,6 +594,21 @@ handle_ra_event(QName, From, {applied, Seqs},
     end;
 handle_ra_event(QName, From, {machine, {delivery, _ConsumerTag, _} = Del}, State0) ->
     handle_delivery(QName, From, Del, State0);
+handle_ra_event(_QName, _From,
+                {machine, {credit_reply_v1, _CTag, _Credit, _Available, _Drain = false} = Action},
+                State) ->
+    {ok, State, [Action]};
+handle_ra_event(_QName, _From,
+                {machine, {credit_reply, CTag, _DeliveryCount, _Credit, _Available, Drain} = Action},
+                #state{consumer_deliveries = CDels} = State) ->
+    Actions = case CDels of
+                  #{CTag := #consumer{echo = Echo}}
+                    when Echo orelse Drain ->
+                      [Action];
+                  _ ->
+                      []
+              end,
+    {ok, State, Actions};
 handle_ra_event(_QName, _, {machine, {queue_status, Status}},
                 #state{} = State) ->
     %% just set the queue status
@@ -667,14 +704,12 @@ maybe_add_action({multi, Actions}, Acc0, State0) ->
     lists:foldl(fun (Act, {Acc, State}) ->
                         maybe_add_action(Act, Acc, State)
                 end, {Acc0, State0}, Actions);
-maybe_add_action({send_drained, {Tag, Credit}} = Action, Acc,
-                 #state{consumer_deliveries = CDels} = State) ->
-    %% add credit to consumer delivery_count
-    C = maps:get(Tag, CDels),
-    {[Action | Acc],
-     State#state{consumer_deliveries =
-                 update_consumer(Tag, C#consumer.last_msg_id,
-                                 Credit, C, CDels)}};
+maybe_add_action({send_drained, {Tag, Credit}}, Acc, State0) ->
+    %% This function clause should be deleted when
+    %% feature flag credit_api_v2 becomes required.
+    State = add_delivery_count(Credit, Tag, State0),
+    Action = {credit_reply_v1, Tag, Credit, _Avail = 0, _Drain = true},
+    {[Action | Acc], State};
 maybe_add_action(Action, Acc, State) ->
     %% anything else is assumed to be an action
     {[Action | Acc], State}.
@@ -785,13 +820,20 @@ transform_msgs(QName, QRef, Msgs) ->
               {QName, QRef, MsgId, Redelivered, Msg}
       end, Msgs).
 
-update_consumer(Tag, LastId, DelCntIncr,
-                #consumer{delivery_count = D} = C, Consumers) ->
-    maps:put(Tag,
-             C#consumer{last_msg_id = LastId,
-                        delivery_count = D + DelCntIncr},
-             Consumers).
-
+update_consumer(Tag, LastId, DelCntIncr, Consumer, Consumers) ->
+    D = case Consumer#consumer.delivery_count of
+            credit_api_v2 -> credit_api_v2;
+            {credit_api_v1, Count} -> {credit_api_v1, Count + DelCntIncr}
+        end,
+    maps:update(Tag,
+                Consumer#consumer{last_msg_id = LastId,
+                                  delivery_count = D},
+                Consumers).
+
+add_delivery_count(DelCntIncr, Tag, #state{consumer_deliveries = CDels0} = State) ->
+    Con = #consumer{last_msg_id = LastMsgId} = maps:get(Tag, CDels0),
+    CDels = update_consumer(Tag, LastMsgId, DelCntIncr, Con, CDels0),
+    State#state{consumer_deliveries = CDels}.
 
 get_missing_deliveries(State, From, To, ConsumerTag) ->
     %% find local server
diff --git a/deps/rabbit/src/rabbit_global_counters.erl b/deps/rabbit/src/rabbit_global_counters.erl
index 97fedb0d4c0f..b5cdc5b627e1 100644
--- a/deps/rabbit/src/rabbit_global_counters.erl
+++ b/deps/rabbit/src/rabbit_global_counters.erl
@@ -93,7 +93,6 @@
 -define(MESSAGES_GET_EMPTY, 6).
 -define(MESSAGES_REDELIVERED, 7).
 -define(MESSAGES_ACKNOWLEDGED, 8).
-%% Note: ?NUM_PROTOCOL_QUEUE_TYPE_COUNTERS needs to be up-to-date. See include/rabbit_global_counters.hrl
 -define(PROTOCOL_QUEUE_TYPE_COUNTERS,
             [
                 {
@@ -131,13 +130,15 @@
             ]).
 
 boot_step() ->
-    %% Protocol counters
-    init([{protocol, amqp091}]),
-
-    %% Protocol & Queue Type counters
-    init([{protocol, amqp091}, {queue_type, rabbit_classic_queue}]),
-    init([{protocol, amqp091}, {queue_type, rabbit_quorum_queue}]),
-    init([{protocol, amqp091}, {queue_type, rabbit_stream_queue}]),
+    [begin
+         %% Protocol counters
+         init([{protocol, Proto}]),
+
+         %% Protocol & Queue Type counters
+         init([{protocol, Proto}, {queue_type, rabbit_classic_queue}]),
+         init([{protocol, Proto}, {queue_type, rabbit_quorum_queue}]),
+         init([{protocol, Proto}, {queue_type, rabbit_stream_queue}])
+     end || Proto <- [amqp091, amqp10]],
 
     %% Dead Letter counters
     %%
diff --git a/deps/rabbit/src/rabbit_limiter.erl b/deps/rabbit/src/rabbit_limiter.erl
index a28a9caaebe4..f19d20ca4d78 100644
--- a/deps/rabbit/src/rabbit_limiter.erl
+++ b/deps/rabbit/src/rabbit_limiter.erl
@@ -62,12 +62,11 @@
 %%    that's what the limit_prefetch/3, unlimit_prefetch/1,
 %%    get_prefetch_limit/1 API functions are about. They also tell the
 %%    limiter queue state (via the queue) about consumer credit
-%%    changes and message acknowledgement - that's what credit/5 and
+%%    changes and message acknowledgement - that's what credit/4 and
 %%    ack_from_queue/3 are for.
 %%
-%% 2. Queues also tell the limiter queue state about the queue
-%%    becoming empty (via drained/1) and consumers leaving (via
-%%    forget_consumer/2).
+%% 2. Queues also tell the limiter queue state about consumers leaving
+%%    (via forget_consumer/2).
 %%
 %% 3. Queues register with the limiter - this happens as part of
 %%    activate/1.
@@ -120,8 +119,8 @@
          get_prefetch_limit/1, ack/2, pid/1]).
 %% queue API
 -export([client/1, activate/1, can_send/3, resume/1, deactivate/1,
-         is_suspended/1, is_consumer_blocked/2, credit/5, ack_from_queue/3,
-         drained/1, forget_consumer/2]).
+         is_suspended/1, is_consumer_blocked/2, credit/4, ack_from_queue/3,
+         forget_consumer/2]).
 %% callbacks
 -export([init/1, terminate/2, code_change/3, handle_call/3, handle_cast/2,
          handle_info/2, prioritise_call/4]).
@@ -136,7 +135,7 @@
 -type qstate() :: #qstate{pid :: pid() | none,
                           state :: 'dormant' | 'active' | 'suspended'}.
 
--type credit_mode() :: 'manual' | 'drain' | 'auto'.
+-type credit_mode() :: auto | manual.
 
 %%----------------------------------------------------------------------------
 
@@ -259,18 +258,11 @@ is_consumer_blocked(#qstate{credits = Credits}, CTag) ->
         {value, #credit{}}                      -> true
     end.
 
--spec credit
-        (qstate(), rabbit_types:ctag(), non_neg_integer(), credit_mode(),
-         boolean()) ->
-            {boolean(), qstate()}.
-
-credit(Limiter = #qstate{credits = Credits}, CTag, Crd, Mode, IsEmpty) ->
-    {Res, Cr} =
-        case IsEmpty andalso Mode =:= drain of
-            true  -> {true,  #credit{credit = 0,   mode = manual}};
-            false -> {false, #credit{credit = Crd, mode = Mode}}
-        end,
-    {Res, Limiter#qstate{credits = enter_credit(CTag, Cr, Credits)}}.
+-spec credit(qstate(), rabbit_types:ctag(), non_neg_integer(), credit_mode()) ->
+    qstate().
+credit(Limiter = #qstate{credits = Credits}, CTag, Crd, Mode) ->
+    Cr =  #credit{credit = Crd, mode = Mode},
+    Limiter#qstate{credits = enter_credit(CTag, Cr, Credits)}.
 
 -spec ack_from_queue(qstate(), rabbit_types:ctag(), non_neg_integer()) ->
           {boolean(), qstate()}.
@@ -286,20 +278,6 @@ ack_from_queue(Limiter = #qstate{credits = Credits}, CTag, Credit) ->
         end,
     {Unblocked, Limiter#qstate{credits = Credits1}}.
 
--spec drained(qstate()) ->
-          {[{rabbit_types:ctag(), non_neg_integer()}], qstate()}.
-
-drained(Limiter = #qstate{credits = Credits}) ->
-    Drain = fun(C) -> C#credit{credit = 0, mode = manual} end,
-    {CTagCredits, Credits2} =
-        rabbit_misc:gb_trees_fold(
-          fun (CTag, C = #credit{credit = Crd, mode = drain},  {Acc, Creds0}) ->
-                  {[{CTag, Crd} | Acc], update_credit(CTag, Drain(C), Creds0)};
-              (_CTag,   #credit{credit = _Crd, mode = _Mode}, {Acc, Creds0}) ->
-                  {Acc, Creds0}
-          end, {[], Credits}, Credits),
-    {CTagCredits, Limiter#qstate{credits = Credits2}}.
-
 -spec forget_consumer(qstate(), rabbit_types:ctag()) -> qstate().
 
 forget_consumer(Limiter = #qstate{credits = Credits}, CTag) ->
@@ -309,13 +287,6 @@ forget_consumer(Limiter = #qstate{credits = Credits}, CTag) ->
 %% Queue-local code
 %%----------------------------------------------------------------------------
 
-%% We want to do all the AMQP 1.0-ish link level credit calculations
-%% in the queue (to do them elsewhere introduces a ton of
-%% races). However, it's a big chunk of code that is conceptually very
-%% linked to the limiter concept. So we get the queue to hold a bit of
-%% state for us (#qstate.credits), and maintain a fiction that the
-%% limiter is making the decisions...
-
 decrement_credit(CTag, Credits) ->
     case gb_trees:lookup(CTag, Credits) of
         {value, C = #credit{credit = Credit}} ->
@@ -325,16 +296,10 @@ decrement_credit(CTag, Credits) ->
     end.
 
 enter_credit(CTag, C, Credits) ->
-    gb_trees:enter(CTag, ensure_credit_invariant(C), Credits).
+    gb_trees:enter(CTag, C, Credits).
 
 update_credit(CTag, C, Credits) ->
-    gb_trees:update(CTag, ensure_credit_invariant(C), Credits).
-
-ensure_credit_invariant(C = #credit{credit = 0, mode = drain}) ->
-    %% Using up all credit implies no need to send a 'drained' event
-    C#credit{mode = manual};
-ensure_credit_invariant(C) ->
-    C.
+    gb_trees:update(CTag, C, Credits).
 
 %%----------------------------------------------------------------------------
 %% gen_server callbacks
diff --git a/deps/rabbit/src/rabbit_networking.erl b/deps/rabbit/src/rabbit_networking.erl
index 26c111636b04..9ae8561a378c 100644
--- a/deps/rabbit/src/rabbit_networking.erl
+++ b/deps/rabbit/src/rabbit_networking.erl
@@ -49,9 +49,7 @@
 
 -export([
     local_connections/0,
-    local_non_amqp_connections/0,
-    %% prefer local_connections/0
-    connections_local/0
+    local_non_amqp_connections/0
 ]).
 
 -include_lib("rabbit_common/include/rabbit.hrl").
@@ -448,19 +446,15 @@ register_connection(Pid) -> pg_local:join(rabbit_connections, Pid).
 unregister_connection(Pid) -> pg_local:leave(rabbit_connections, Pid).
 
 -spec connections() -> [rabbit_types:connection()].
-
 connections() ->
     Nodes = rabbit_nodes:list_running(),
-    rabbit_misc:append_rpc_all_nodes(Nodes, rabbit_networking, connections_local, [], ?RPC_TIMEOUT).
+    rabbit_misc:append_rpc_all_nodes(Nodes, rabbit_networking, local_connections, [], ?RPC_TIMEOUT).
 
 -spec local_connections() -> [rabbit_types:connection()].
-%% @doc Returns pids of AMQP 0-9-1 and AMQP 1.0 connections local to this node.
 local_connections() ->
-    connections_local().
-
--spec connections_local() -> [rabbit_types:connection()].
-%% @deprecated Prefer {@link local_connections}
-connections_local() -> pg_local:get_members(rabbit_connections).
+    Amqp091Pids = pg_local:get_members(rabbit_connections),
+    Amqp10Pids = rabbit_amqp1_0:list_local(),
+    Amqp10Pids ++ Amqp091Pids.
 
 -spec register_non_amqp_connection(pid()) -> ok.
 
@@ -510,21 +504,16 @@ emit_connection_info_all(Nodes, Items, Ref, AggregatorPid) ->
 emit_connection_info_local(Items, Ref, AggregatorPid) ->
     rabbit_control_misc:emitting_map_with_exit_handler(
       AggregatorPid, Ref, fun(Q) -> connection_info(Q, Items) end,
-      connections_local()).
+      local_connections()).
 
 -spec close_connection(pid(), string()) -> 'ok'.
-
 close_connection(Pid, Explanation) ->
-    case lists:member(Pid, connections()) of
-        true  ->
-            Res = rabbit_reader:shutdown(Pid, Explanation),
-            rabbit_log:info("Closing connection ~tp because ~tp", [Pid, Explanation]),
-            Res;
-        false ->
-            rabbit_log:warning("Asked to close connection ~tp (reason: ~tp) "
-                               "but no running cluster node reported it as an active connection. Was it already closed? ",
-                               [Pid, Explanation]),
-            ok
+    rabbit_log:info("Closing connection ~tp because ~tp",
+                    [Pid, Explanation]),
+    try rabbit_reader:shutdown(Pid, Explanation)
+    catch exit:{Reason, _Location} ->
+              rabbit_log:warning("Could not close connection ~tp (reason: ~tp): ~p",
+                                 [Pid, Explanation, Reason])
     end.
 
 -spec close_connections([pid()], string()) -> 'ok'.
diff --git a/deps/rabbit/src/rabbit_queue_consumers.erl b/deps/rabbit/src/rabbit_queue_consumers.erl
index 77c07a762cab..cb83974fe0a1 100644
--- a/deps/rabbit/src/rabbit_queue_consumers.erl
+++ b/deps/rabbit/src/rabbit_queue_consumers.erl
@@ -8,12 +8,13 @@
 -module(rabbit_queue_consumers).
 
 -export([new/0, max_active_priority/1, inactive/1, all/1, all/3, count/0,
-         unacknowledged_message_count/0, add/11, remove/3, erase_ch/2,
-         send_drained/1, deliver/5, record_ack/3, subtract_acks/3,
+         unacknowledged_message_count/0, add/9, remove/3, erase_ch/2,
+         deliver/5, record_ack/3, subtract_acks/3,
          possibly_unblock/3,
          resume_fun/0, notify_sent_fun/1, activate_limit_fun/0,
-         credit/7, utilisation/1, capacity/1, is_same/3, get_consumer/1, get/3,
-         consumer_tag/1, get_infos/1]).
+         drained/3, process_credit/5, get_link_state/2,
+         utilisation/1, capacity/1, is_same/3, get_consumer/1, get/3,
+         consumer_tag/1, get_infos/1, parse_prefetch_count/1]).
 
 -export([deactivate_limit_fun/0]).
 
@@ -30,7 +31,13 @@
 
 -record(consumer, {tag, ack_required, prefetch, args, user}).
 
+%% AMQP 1.0 link flow control state, see §2.6.7
+%% Delete atom credit_api_v1 when feature flag credit_api_v2 becomes required.
+-record(link_state, {delivery_count :: rabbit_queue_type:delivery_count() | credit_api_v1,
+                     credit :: rabbit_queue_type:credit()}).
+
 %% These are held in our process dictionary
+%% channel record
 -record(cr, {ch_pid,
              monitor_ref,
              acktags,
@@ -41,7 +48,9 @@
              %% The limiter itself
              limiter,
              %% Internal flow control for queue -> writer
-             unsent_message_count}).
+             unsent_message_count,
+             link_states :: #{rabbit_types:ctag() => #link_state{}}
+            }).
 
 %%----------------------------------------------------------------------------
 
@@ -120,33 +129,50 @@ count() -> lists:sum([Count || #cr{consumer_count = Count} <- all_ch_record()]).
 unacknowledged_message_count() ->
     lists:sum([?QUEUE:len(C#cr.acktags) || C <- all_ch_record()]).
 
--spec add(rabbit_amqqueue:name(), ch(), rabbit_types:ctag(), boolean(), pid() | none, boolean(),
-          non_neg_integer(), rabbit_framing:amqp_table(), boolean(),
-          rabbit_types:username(), state())
-         -> state().
-
-add(QName, ChPid, CTag, NoAck, LimiterPid, LimiterActive, Prefetch, Args, IsEmpty,
+-spec add(ch(), rabbit_types:ctag(), boolean(), pid() | none, boolean(),
+          %% credit API v1
+          SimplePrefetch :: non_neg_integer() |
+          %% credit API v2
+          {simple_prefetch, non_neg_integer()} | {credited, rabbit_queue_type:delivery_count()},
+          rabbit_framing:amqp_table(),
+          rabbit_types:username(), state()) ->
+    state().
+
+add(ChPid, CTag, NoAck, LimiterPid, LimiterActive,
+    ModeOrPrefetch, Args,
     Username, State = #state{consumers = Consumers,
                              use       = CUInfo}) ->
-    C = #cr{consumer_count = Count,
-            limiter        = Limiter} = ch_record(ChPid, LimiterPid),
+    C0 = #cr{consumer_count = Count,
+             limiter        = Limiter,
+             link_states = LinkStates} = ch_record(ChPid, LimiterPid),
     Limiter1 = case LimiterActive of
                    true  -> rabbit_limiter:activate(Limiter);
                    false -> Limiter
                end,
-    C1 = C#cr{consumer_count = Count + 1, limiter = Limiter1},
-    update_ch_record(
-      case parse_credit_args(Prefetch, Args) of
-          {0,       auto}            -> C1;
-          {_Credit, auto} when NoAck -> C1;
-          {Credit,  Mode}            -> credit_and_drain(QName,
-                                          C1, CTag, Credit, Mode, IsEmpty)
-      end),
+    C1 = C0#cr{consumer_count = Count + 1,
+               limiter = Limiter1},
+    C = case parse_credit_mode(ModeOrPrefetch, Args) of
+            {0, auto} ->
+                C1;
+            {Credit, auto = Mode} ->
+                case NoAck of
+                    true ->
+                        C1;
+                    false ->
+                        Limiter2 = rabbit_limiter:credit(Limiter1, CTag, Credit, Mode),
+                        C1#cr{limiter = Limiter2}
+                end;
+            {InitialDeliveryCount, manual} ->
+                C1#cr{link_states = LinkStates#{CTag => #link_state{
+                                                           credit = 0,
+                                                           delivery_count = InitialDeliveryCount}}}
+        end,
+    update_ch_record(C),
     Consumer = #consumer{tag          = CTag,
                          ack_required = not NoAck,
-                         prefetch     = Prefetch,
+                         prefetch     = parse_prefetch_count(ModeOrPrefetch),
                          args         = Args,
-                         user          = Username},
+                         user         = Username},
     State#state{consumers = add_consumer({ChPid, Consumer}, Consumers),
                 use       = update_use(CUInfo, active)}.
 
@@ -159,7 +185,8 @@ remove(ChPid, CTag, State = #state{consumers = Consumers}) ->
             not_found;
         C = #cr{consumer_count    = Count,
                 limiter           = Limiter,
-                blocked_consumers = Blocked} ->
+                blocked_consumers = Blocked,
+                link_states = LinkStates} ->
             Blocked1 = remove_consumer(ChPid, CTag, Blocked),
             Limiter1 = case Count of
                            1 -> rabbit_limiter:deactivate(Limiter);
@@ -168,9 +195,10 @@ remove(ChPid, CTag, State = #state{consumers = Consumers}) ->
             Limiter2 = rabbit_limiter:forget_consumer(Limiter1, CTag),
             update_ch_record(C#cr{consumer_count    = Count - 1,
                                   limiter           = Limiter2,
-                                  blocked_consumers = Blocked1}),
+                                  blocked_consumers = Blocked1,
+                                  link_states = maps:remove(CTag, LinkStates)}),
             State#state{consumers =
-                            remove_consumer(ChPid, CTag, Consumers)}
+                        remove_consumer(ChPid, CTag, Consumers)}
     end.
 
 -spec erase_ch(ch(), state()) ->
@@ -192,11 +220,6 @@ erase_ch(ChPid, State = #state{consumers = Consumers}) ->
              State#state{consumers = remove_consumers(ChPid, Consumers)}}
     end.
 
--spec send_drained(rabbit_amqqueue:name()) -> 'ok'.
-send_drained(QName) ->
-    [update_ch_record(send_drained(QName, C)) || C <- all_ch_record()],
-    ok.
-
 -spec deliver(fun ((boolean()) -> {fetch_result(), T}),
               rabbit_amqqueue:name(), state(), boolean(),
               none | {ch(), rabbit_types:ctag()} | {ch(), consumer()}) ->
@@ -252,17 +275,37 @@ deliver_to_consumer(FetchFun, E = {ChPid, Consumer}, QName) ->
         true  ->
             block_consumer(C, E),
             undelivered;
-        false -> case rabbit_limiter:can_send(C#cr.limiter,
-                                              Consumer#consumer.ack_required,
-                                              Consumer#consumer.tag) of
-                     {suspend, Limiter} ->
-                         block_consumer(C#cr{limiter = Limiter}, E),
-                         undelivered;
-                     {continue, Limiter} ->
-                         {delivered, deliver_to_consumer(
-                                       FetchFun, Consumer,
-                                       C#cr{limiter = Limiter}, QName)}
-                 end
+        false ->
+            CTag = Consumer#consumer.tag,
+            LinkStates = C#cr.link_states,
+            case maps:find(CTag, LinkStates) of
+                {ok, #link_state{delivery_count = DeliveryCount0,
+                                 credit = Credit} = LinkState0}
+                  when Credit > 0 ->
+                    DeliveryCount = case DeliveryCount0 of
+                                        credit_api_v1 -> DeliveryCount0;
+                                        _ -> serial_number:add(DeliveryCount0, 1)
+                                    end,
+                    LinkState = LinkState0#link_state{delivery_count = DeliveryCount,
+                                                      credit = Credit - 1},
+                    C1 = C#cr{link_states = maps:update(CTag, LinkState, LinkStates)},
+                    {delivered, deliver_to_consumer(FetchFun, Consumer, C1, QName)};
+                {ok, _Exhausted} ->
+                    block_consumer(C, E),
+                    undelivered;
+                error ->
+                    case rabbit_limiter:can_send(C#cr.limiter,
+                                                 Consumer#consumer.ack_required,
+                                                 CTag) of
+                        {suspend, Limiter} ->
+                            block_consumer(C#cr{limiter = Limiter}, E),
+                            undelivered;
+                        {continue, Limiter} ->
+                            {delivered, deliver_to_consumer(
+                                          FetchFun, Consumer,
+                                          C#cr{limiter = Limiter}, QName)}
+                    end
+            end
     end.
 
 deliver_to_consumer(FetchFun,
@@ -349,11 +392,21 @@ possibly_unblock(Update, ChPid, State) ->
                      end
     end.
 
-unblock(C = #cr{blocked_consumers = BlockedQ, limiter = Limiter},
+unblock(C = #cr{blocked_consumers = BlockedQ,
+                limiter = Limiter,
+                link_states = LinkStates},
         State = #state{consumers = Consumers, use = Use}) ->
     case lists:partition(
            fun({_P, {_ChPid, #consumer{tag = CTag}}}) ->
-                   rabbit_limiter:is_consumer_blocked(Limiter, CTag)
+                   case maps:find(CTag, LinkStates) of
+                       {ok, #link_state{credit = Credits}}
+                         when Credits > 0 ->
+                           false;
+                       {ok, _Exhausted} ->
+                           true;
+                       error ->
+                           rabbit_limiter:is_consumer_blocked(Limiter, CTag)
+                   end
            end, priority_queue:to_list(BlockedQ)) of
         {_, []} ->
             update_ch_record(C),
@@ -395,28 +448,63 @@ deactivate_limit_fun() ->
             C#cr{limiter = rabbit_limiter:deactivate(Limiter)}
     end.
 
--spec credit(rabbit_amqqueue:name(), boolean(), integer(), boolean(), ch(),
-             rabbit_types:ctag(),
-             state()) -> 'unchanged' | {'unblocked', state()}.
+-spec drained(rabbit_queue_type:delivery_count() | credit_api_v1, ch(), rabbit_types:ctag()) ->
+    ok.
+drained(AdvancedDeliveryCount, ChPid, CTag) ->
+    case lookup_ch(ChPid) of
+        C0 = #cr{link_states = LinkStates = #{CTag := LinkState0}} ->
+            LinkState = LinkState0#link_state{delivery_count = AdvancedDeliveryCount,
+                                              credit = 0},
+            C = C0#cr{link_states = maps:update(CTag, LinkState, LinkStates)},
+            update_ch_record(C);
+        _ ->
+            ok
+    end.
 
-credit(QName, IsEmpty, Credit, Drain, ChPid, CTag, State) ->
+-spec process_credit(rabbit_queue_type:delivery_count() | credit_api_v1,
+                     rabbit_queue_type:credit(), ch(), rabbit_types:ctag(), state()) ->
+    'unchanged' | {'unblocked', state()}.
+process_credit(DeliveryCountRcv, LinkCredit, ChPid, CTag, State) ->
     case lookup_ch(ChPid) of
-        not_found ->
-            unchanged;
-        #cr{limiter = Limiter} = C ->
-            C1 = #cr{limiter = Limiter1} =
-                credit_and_drain(QName, C, CTag, Credit, drain_mode(Drain), IsEmpty),
-            case is_ch_blocked(C1) orelse
-                (not rabbit_limiter:is_consumer_blocked(Limiter, CTag)) orelse
-                rabbit_limiter:is_consumer_blocked(Limiter1, CTag) of
-                true  -> update_ch_record(C1),
-                         unchanged;
-                false -> unblock(C1, State)
-            end
+        #cr{link_states = LinkStates = #{CTag := LinkState = #link_state{delivery_count = DeliveryCountSnd,
+                                                                         credit = OldLinkCreditSnd}},
+            unsent_message_count = Count} = C0 ->
+            LinkCreditSnd = case DeliveryCountSnd of
+                                credit_api_v1 ->
+                                    %% LinkCredit refers to LinkCreditSnd
+                                    LinkCredit;
+                                _ ->
+                                    %% credit API v2
+                                    %% LinkCredit refers to LinkCreditRcv
+                                    %% See AMQP §2.6.7
+                                    serial_number:diff(
+                                      serial_number:add(DeliveryCountRcv, LinkCredit),
+                                      DeliveryCountSnd)
+                            end,
+            C = C0#cr{link_states = maps:update(CTag, LinkState#link_state{credit = LinkCreditSnd}, LinkStates)},
+            case Count >= ?UNSENT_MESSAGE_LIMIT orelse
+                 OldLinkCreditSnd > 0 orelse
+                 LinkCreditSnd < 1 of
+                true ->
+                    update_ch_record(C),
+                    unchanged;
+                false ->
+                    unblock(C, State)
+            end;
+        _ ->
+            unchanged
     end.
 
-drain_mode(true)  -> drain;
-drain_mode(false) -> manual.
+-spec get_link_state(pid(), rabbit_types:ctag()) ->
+    {rabbit_queue_type:delivery_count() | credit_api_v1, rabbit_queue_type:credit()} | not_found.
+get_link_state(ChPid, CTag) ->
+    case lookup_ch(ChPid) of
+        #cr{link_states = #{CTag := #link_state{delivery_count = DeliveryCount,
+                                                credit = Credit}}} ->
+            {DeliveryCount, Credit};
+        _ ->
+            not_found
+    end.
 
 -spec utilisation(state()) -> ratio().
 utilisation(State) ->
@@ -465,14 +553,39 @@ consumer_tag(#consumer{tag = CTag}) ->
 
 %%----------------------------------------------------------------------------
 
-parse_credit_args(Default, Args) ->
+%% credit API v2 uses mode
+parse_prefetch_count({simple_prefetch, Prefetch}) ->
+    Prefetch;
+parse_prefetch_count({credited, _InitialDeliveryCount}) ->
+    0;
+%% credit API v1 uses prefetch
+parse_prefetch_count(Prefetch)
+  when is_integer(Prefetch) ->
+    Prefetch.
+
+-spec parse_credit_mode(rabbit_queue_type:consume_mode(), rabbit_framing:amqp_table()) ->
+    {Prefetch :: non_neg_integer(), auto | manual}.
+
+%% credit API v2
+parse_credit_mode({simple_prefetch, Prefetch}, _Args) ->
+    {Prefetch, auto};
+parse_credit_mode({credited, InitialDeliveryCount}, _Args) ->
+    {InitialDeliveryCount, manual};
+%% credit API v1
+%% i.e. below function clause should be deleted when feature flag credit_api_v2 becomes required:
+parse_credit_mode(Prefetch, Args)
+  when is_integer(Prefetch) ->
     case rabbit_misc:table_lookup(Args, <<"x-credit">>) of
-        {table, T} -> case {rabbit_misc:table_lookup(T, <<"credit">>),
-                            rabbit_misc:table_lookup(T, <<"drain">>)} of
-                          {{long, C}, {bool, D}} -> {C, drain_mode(D)};
-                          _                      -> {Default, auto}
-                      end;
-        undefined  -> {Default, auto}
+        {table, T} ->
+            case {rabbit_misc:table_lookup(T, <<"credit">>),
+                  rabbit_misc:table_lookup(T, <<"drain">>)} of
+                {{long, 0}, {bool, false}} ->
+                    {credit_api_v1, manual};
+                _ ->
+                    {Prefetch, auto}
+            end;
+        undefined ->
+            {Prefetch, auto}
     end.
 
 lookup_ch(ChPid) ->
@@ -492,7 +605,8 @@ ch_record(ChPid, LimiterPid) ->
                              consumer_count       = 0,
                              blocked_consumers    = priority_queue:new(),
                              limiter              = Limiter,
-                             unsent_message_count = 0},
+                             unsent_message_count = 0,
+                             link_states = #{}},
                      put(Key, C),
                      C;
         C = #cr{} -> C
@@ -524,31 +638,14 @@ block_consumer(C = #cr{blocked_consumers = Blocked}, QEntry) ->
 is_ch_blocked(#cr{unsent_message_count = Count, limiter = Limiter}) ->
     Count >= ?UNSENT_MESSAGE_LIMIT orelse rabbit_limiter:is_suspended(Limiter).
 
-send_drained(QName, C = #cr{ch_pid = ChPid, limiter = Limiter}) ->
-    case rabbit_limiter:drained(Limiter) of
-        {[],         Limiter}  -> C;
-        {CTagCredits, Limiter2} ->
-            ok = rabbit_classic_queue:send_drained(ChPid, QName, CTagCredits),
-            C#cr{limiter = Limiter2}
-    end.
-
-credit_and_drain(QName, C = #cr{ch_pid = ChPid, limiter = Limiter},
-                 CTag, Credit, Mode, IsEmpty) ->
-    case rabbit_limiter:credit(Limiter, CTag, Credit, Mode, IsEmpty) of
-        {true,  Limiter1} ->
-            ok = rabbit_classic_queue:send_drained(ChPid, QName, [{CTag, Credit}]),
-            C#cr{limiter = Limiter1};
-        {false, Limiter1} -> C#cr{limiter = Limiter1}
-    end.
-
 tags(CList) -> [CTag || {_P, {_ChPid, #consumer{tag = CTag}}} <- CList].
 
-add_consumer({ChPid, Consumer = #consumer{args = Args}}, Queue) ->
+add_consumer(Key = {_ChPid, #consumer{args = Args}}, Queue) ->
     Priority = case rabbit_misc:table_lookup(Args, <<"x-priority">>) of
                    {_, P} -> P;
                    _      -> 0
                end,
-    priority_queue:in({ChPid, Consumer}, Priority, Queue).
+    priority_queue:in(Key, Priority, Queue).
 
 remove_consumer(ChPid, CTag, Queue) ->
     priority_queue:filter(fun ({CP, #consumer{tag = CT}}) ->
diff --git a/deps/rabbit/src/rabbit_queue_type.erl b/deps/rabbit/src/rabbit_queue_type.erl
index 4443372458c1..81f6359727a3 100644
--- a/deps/rabbit/src/rabbit_queue_type.erl
+++ b/deps/rabbit/src/rabbit_queue_type.erl
@@ -11,6 +11,7 @@
 
 -include("amqqueue.hrl").
 -include_lib("rabbit_common/include/rabbit.hrl").
+-include_lib("amqp10_common/include/amqp10_types.hrl").
 
 -export([
          init/0,
@@ -43,7 +44,8 @@
          module/2,
          deliver/4,
          settle/5,
-         credit/5,
+         credit_v1/5,
+         credit/7,
          dequeue/5,
          fold_state/3,
          is_policy_applicable/2,
@@ -63,11 +65,14 @@
 
 -type queue_name() :: rabbit_amqqueue:name().
 -type queue_state() :: term().
--type msg_tag() :: term().
+%% sequence number typically
+-type correlation() :: term().
 -type arguments() :: queue_arguments | consumer_arguments.
 -type queue_type() :: rabbit_classic_queue | rabbit_quorum_queue | rabbit_stream_queue.
-
--export_type([queue_type/0]).
+%% see AMQP 1.0 §2.6.7
+-type delivery_count() :: sequence_no().
+%% Link credit can be negative, see AMQP 1.0 §2.6.7
+-type credit() :: integer().
 
 -define(STATE, ?MODULE).
 
@@ -83,9 +88,15 @@
 -type action() ::
     %% indicate to the queue type module that a message has been delivered
     %% fully to the queue
-    {settled, Success :: boolean(), [msg_tag()]} |
+    {settled, queue_name(), [correlation()]} |
     {deliver, rabbit_types:ctag(), boolean(), [rabbit_amqqueue:qmsg()]} |
-    {block | unblock, QueueName :: term()}.
+    {block | unblock, QueueName :: term()} |
+    %% credit API v2
+    {credit_reply, rabbit_types:ctag(), delivery_count(), credit(),
+     Available :: non_neg_integer(), Drain :: boolean()} |
+    %% credit API v1
+    {credit_reply_v1, rabbit_types:ctag(), credit(),
+     Available :: non_neg_integer(), Drain :: boolean()}.
 
 -type actions() :: [action()].
 
@@ -94,44 +105,42 @@
     term().
 
 -record(ctx, {module :: module(),
-              %% "publisher confirm queue accounting"
-              %% queue type implementation should emit a:
-              %% {settle, Success :: boolean(), msg_tag()}
-              %% to either settle or reject the delivery of a
-              %% message to the queue instance
-              %% The queue type module will then emit a {confirm | reject, [msg_tag()}
-              %% action to the channel or channel like process when a msg_tag
-              %% has reached its conclusion
               state :: queue_state()}).
 
-
 -record(?STATE, {ctxs = #{} :: #{queue_name() => #ctx{}}
                 }).
 
 -opaque state() :: #?STATE{}.
 
+%% Delete atom 'credit_api_v1' when feature flag credit_api_v2 becomes required.
+-type consume_mode() :: {simple_prefetch, non_neg_integer()} | {credited, Initial :: delivery_count() | credit_api_v1}.
 -type consume_spec() :: #{no_ack := boolean(),
                           channel_pid := pid(),
                           limiter_pid => pid() | none,
                           limiter_active => boolean(),
-                          prefetch_count => non_neg_integer(),
+                          mode := consume_mode(),
                           consumer_tag := rabbit_types:ctag(),
                           exclusive_consume => boolean(),
                           args => rabbit_framing:amqp_table(),
                           ok_msg := term(),
                           acting_user :=  rabbit_types:username()}.
 
--type delivery_options() :: #{correlation => term(), %% sequence no typically
+-type delivery_options() :: #{correlation => correlation(),
                               atom() => term()}.
 
 -type settle_op() :: 'complete' | 'requeue' | 'discard'.
 
 -export_type([state/0,
+              consume_mode/0,
               consume_spec/0,
               delivery_options/0,
               action/0,
               actions/0,
-              settle_op/0]).
+              settle_op/0,
+              queue_type/0,
+              credit/0,
+              correlation/0,
+              delivery_count/0]).
 
 -callback is_enabled() -> boolean().
 
@@ -179,7 +188,8 @@
 -callback consume(amqqueue:amqqueue(),
                   consume_spec(),
                   queue_state()) ->
-    {ok, queue_state(), actions()} | {error, term()} |
+    {ok, queue_state(), actions()} |
+    {error, term()} |
     {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
 
 -callback cancel(amqqueue:amqqueue(),
@@ -207,8 +217,13 @@
     {queue_state(), actions()} |
     {'protocol_error', Type :: atom(), Reason :: string(), Args :: term()}.
 
--callback credit(queue_name(), rabbit_types:ctag(),
-                 non_neg_integer(), Drain :: boolean(), queue_state()) ->
+%% Delete this callback when feature flag credit_api_v2 becomes required.
+-callback credit_v1(queue_name(), rabbit_types:ctag(), credit(), Drain :: boolean(), queue_state()) ->
+    {queue_state(), actions()}.
+
+%% credit API v2
+-callback credit(queue_name(), rabbit_types:ctag(), delivery_count(), credit(),
+                 Drain :: boolean(), Echo :: boolean(), queue_state()) ->
     {queue_state(), actions()}.
 
 -callback dequeue(queue_name(), NoAck :: boolean(), LimiterPid :: pid(),
@@ -414,7 +429,9 @@ new(Q, State) when ?is_amqqueue(Q) ->
     set_ctx(Q, Ctx, State).
 
 -spec consume(amqqueue:amqqueue(), consume_spec(), state()) ->
-    {ok, state()} | {error, term()}.
+    {ok, state()} |
+    {error, term()} |
+    {protocol_error, Type :: atom(), Reason :: string(), Args :: term()}.
 consume(Q, Spec, State) ->
     #ctx{state = CtxState0} = Ctx = get_ctx(Q, State),
     Mod = amqqueue:get_type(Q),
@@ -629,15 +646,23 @@ settle(#resource{kind = queue} = QRef, Op, CTag, MsgIds, Ctxs) ->
             end
     end.
 
--spec credit(amqqueue:amqqueue() | queue_name(),
-             rabbit_types:ctag(), non_neg_integer(),
-             boolean(), state()) -> {ok, state(), actions()}.
-credit(Q, CTag, Credit, Drain, Ctxs) ->
+%% Delete this function when feature flag credit_api_v2 becomes required.
+-spec credit_v1(queue_name(), rabbit_types:ctag(), credit(), boolean(), state()) ->
+    {ok, state(), actions()}.
+credit_v1(QName, CTag, LinkCreditSnd, Drain, Ctxs) ->
     #ctx{state = State0,
-         module = Mod} = Ctx = get_ctx(Q, Ctxs),
-    QName = amqqueue:get_name(Q),
-    {State, Actions} = Mod:credit(QName, CTag, Credit, Drain, State0),
-    {ok, set_ctx(Q, Ctx#ctx{state = State}, Ctxs), Actions}.
+         module = Mod} = Ctx = get_ctx(QName, Ctxs),
+    {State, Actions} = Mod:credit_v1(QName, CTag, LinkCreditSnd, Drain, State0),
+    {ok, set_ctx(QName, Ctx#ctx{state = State}, Ctxs), Actions}.
+
+%% credit API v2
+-spec credit(queue_name(), rabbit_types:ctag(), delivery_count(), credit(), boolean(), boolean(), state()) ->
+    {ok, state(), actions()}.
+credit(QName, CTag, DeliveryCount, Credit, Drain, Echo, Ctxs) ->
+    #ctx{state = State0,
+         module = Mod} = Ctx = get_ctx(QName, Ctxs),
+    {State, Actions} = Mod:credit(QName, CTag, DeliveryCount, Credit, Drain, Echo, State0),
+    {ok, set_ctx(QName, Ctx#ctx{state = State}, Ctxs), Actions}.
 
 -spec dequeue(amqqueue:amqqueue(), boolean(),
               pid(), rabbit_types:ctag(), state()) ->
diff --git a/deps/rabbit/src/rabbit_quorum_queue.erl b/deps/rabbit/src/rabbit_quorum_queue.erl
index 5887f141380b..536bcab22afe 100644
--- a/deps/rabbit/src/rabbit_quorum_queue.erl
+++ b/deps/rabbit/src/rabbit_quorum_queue.erl
@@ -25,7 +25,7 @@
          delete_immediately/1]).
 -export([state_info/1, info/2, stat/1, infos/1, infos/2]).
 -export([settle/5, dequeue/5, consume/3, cancel/5]).
--export([credit/5]).
+-export([credit_v1/5, credit/7]).
 -export([purge/1]).
 -export([stateless_deliver/2, deliver/3]).
 -export([dead_letter_publish/5]).
@@ -130,6 +130,7 @@
 -define(DELETE_TIMEOUT, 5000).
 -define(ADD_MEMBER_TIMEOUT, 5000).
 -define(SNAPSHOT_INTERVAL, 8192). %% the ra default is 4096
+-define(UNLIMITED_PREFETCH_COUNT, 2000). %% something large for ra
 
 %%----------- QQ policies ---------------------------------------------------
 
@@ -477,7 +478,7 @@ capabilities() ->
                           <<"x-single-active-consumer">>, <<"x-queue-type">>,
                           <<"x-quorum-initial-group-size">>, <<"x-delivery-limit">>,
                           <<"x-message-ttl">>, <<"x-queue-leader-locator">>],
-      consumer_arguments => [<<"x-priority">>, <<"x-credit">>],
+      consumer_arguments => [<<"x-priority">>],
       server_named => false}.
 
 rpc_delete_metrics(QName) ->
@@ -800,8 +801,11 @@ settle(_QName, requeue, CTag, MsgIds, QState) ->
 settle(_QName, discard, CTag, MsgIds, QState) ->
     rabbit_fifo_client:discard(quorum_ctag(CTag), MsgIds, QState).
 
-credit(_QName, CTag, Credit, Drain, QState) ->
-    rabbit_fifo_client:credit(quorum_ctag(CTag), Credit, Drain, QState).
+credit_v1(_QName, CTag, Credit, Drain, QState) ->
+    rabbit_fifo_client:credit_v1(quorum_ctag(CTag), Credit, Drain, QState).
+
+credit(_QName, CTag, DeliveryCount, Credit, Drain, Echo, QState) ->
+    rabbit_fifo_client:credit(quorum_ctag(CTag), DeliveryCount, Credit, Drain, Echo, QState).
 
 -spec dequeue(rabbit_amqqueue:name(), NoAck :: boolean(), pid(),
               rabbit_types:ctag(), rabbit_fifo_client:state()) ->
@@ -829,7 +833,7 @@ consume(Q, #{limiter_active := true}, _State)
 consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
     #{no_ack := NoAck,
       channel_pid := ChPid,
-      prefetch_count := ConsumerPrefetchCount,
+      mode := Mode,
       consumer_tag := ConsumerTag0,
       exclusive_consume := ExclusiveConsume,
       args := Args,
@@ -840,35 +844,33 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
     QName = amqqueue:get_name(Q),
     maybe_send_reply(ChPid, OkMsg),
     ConsumerTag = quorum_ctag(ConsumerTag0),
-    %% A prefetch count of 0 means no limitation,
-    %% let's make it into something large for ra
-    Prefetch0 = case ConsumerPrefetchCount of
-                    0 -> 2000;
-                    Other -> Other
-                end,
     %% consumer info is used to describe the consumer properties
     AckRequired = not NoAck,
-    ConsumerMeta = #{ack => AckRequired,
-                     prefetch => ConsumerPrefetchCount,
-                     args => Args,
-                     username => ActingUser},
-
-    {CreditMode, Credit, Drain} = parse_credit_args(Prefetch0, Args),
-    %% if the mode is credited we should send a separate credit command
-    %% after checkout and give 0 credits initally
-    Prefetch = case CreditMode of
-                   credited -> 0;
-                   simple_prefetch -> Prefetch0
-               end,
-    {ok, QState1} = rabbit_fifo_client:checkout(ConsumerTag, Prefetch,
-                                                CreditMode, ConsumerMeta,
-                                                QState0),
-    QState = case CreditMode of
-                   credited when Credit > 0 ->
-                     rabbit_fifo_client:credit(ConsumerTag, Credit, Drain,
-                                               QState1);
-                   _ -> QState1
-               end,
+    {CreditMode, EffectivePrefetch, DeclaredPrefetch, ConsumerMeta0} =
+        case Mode of
+            {credited, C} ->
+                Meta = if C =:= credit_api_v1 ->
+                              #{};
+                          is_integer(C) ->
+                              #{initial_delivery_count => C}
+                       end,
+                {credited, 0, 0, Meta};
+            {simple_prefetch = M, Declared} ->
+                Effective = case Declared of
+                                0 -> ?UNLIMITED_PREFETCH_COUNT;
+                                _ -> Declared
+                            end,
+                {M, Effective, Declared, #{}}
+        end,
+    ConsumerMeta = maps:merge(
+                     ConsumerMeta0,
+                     #{ack => AckRequired,
+                       prefetch => DeclaredPrefetch,
+                       args => Args,
+                       username => ActingUser}),
+    {ok, QState} = rabbit_fifo_client:checkout(ConsumerTag, EffectivePrefetch,
+                                               CreditMode, ConsumerMeta,
+                                               QState0),
     case single_active_consumer_on(Q) of
         true ->
             %% get the leader from state
@@ -883,10 +885,10 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
                     rabbit_core_metrics:consumer_created(
                       ChPid, ConsumerTag, ExclusiveConsume,
                       AckRequired, QName,
-                      ConsumerPrefetchCount, ActivityStatus == single_active, %% Active
+                      DeclaredPrefetch, ActivityStatus == single_active, %% Active
                       ActivityStatus, Args),
                     emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
-                                          AckRequired, QName, Prefetch,
+                                          AckRequired, QName, DeclaredPrefetch,
                                           Args, none, ActingUser),
                     {ok, QState};
                 {error, Error} ->
@@ -898,10 +900,10 @@ consume(Q, Spec, QState0) when ?amqqueue_is_quorum(Q) ->
             rabbit_core_metrics:consumer_created(
               ChPid, ConsumerTag, ExclusiveConsume,
               AckRequired, QName,
-              ConsumerPrefetchCount, true, %% Active
+              DeclaredPrefetch, true, %% Active
               up, Args),
             emit_consumer_created(ChPid, ConsumerTag, ExclusiveConsume,
-                                  AckRequired, QName, Prefetch,
+                                  AckRequired, QName, DeclaredPrefetch,
                                   Args, none, ActingUser),
             {ok, QState}
     end.
@@ -1818,20 +1820,6 @@ overflow(<<"reject-publish-dlx">> = V, Def, QName) ->
                        [V, rabbit_misc:rs(QName)]),
     Def.
 
-parse_credit_args(Default, Args) ->
-    case rabbit_misc:table_lookup(Args, <<"x-credit">>) of
-        {table, T} ->
-            case {rabbit_misc:table_lookup(T, <<"credit">>),
-                  rabbit_misc:table_lookup(T, <<"drain">>)} of
-                {{long, C}, {bool, D}} ->
-                    {credited, C, D};
-                _ ->
-                    {simple_prefetch, Default, false}
-            end;
-        undefined ->
-            {simple_prefetch, Default, false}
-    end.
-
 -spec notify_decorators(amqqueue:amqqueue()) -> 'ok'.
 notify_decorators(Q) when ?is_amqqueue(Q) ->
     QName = amqqueue:get_name(Q),
diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl
index fed666c55c02..4f8c9a61b6cd 100644
--- a/deps/rabbit/src/rabbit_reader.erl
+++ b/deps/rabbit/src/rabbit_reader.erl
@@ -43,12 +43,12 @@
 -include_lib("rabbit_common/include/rabbit_framing.hrl").
 -include_lib("rabbit_common/include/rabbit.hrl").
 
--export([start_link/2, info_keys/0, info/1, info/2, force_event_refresh/2,
+-export([start_link/1, info_keys/0, info/1, info/2, force_event_refresh/2,
          shutdown/2]).
 
 -export([system_continue/3, system_terminate/4, system_code_change/4]).
 
--export([init/3, mainloop/4, recvloop/4]).
+-export([init/2, mainloop/4, recvloop/4]).
 
 -export([conserve_resources/3, server_properties/1]).
 
@@ -145,11 +145,10 @@
 
 %%--------------------------------------------------------------------------
 
--spec start_link(pid(), any()) -> rabbit_types:ok(pid()).
-
-start_link(HelperSup, Ref) ->
-    Pid = proc_lib:spawn_link(?MODULE, init, [self(), HelperSup, Ref]),
-
+-spec start_link(ranch:ref()) ->
+    rabbit_types:ok(pid()).
+start_link(Ref) ->
+    Pid = proc_lib:spawn_link(?MODULE, init, [self(), Ref]),
     {ok, Pid}.
 
 -spec shutdown(pid(), string()) -> 'ok'.
@@ -157,14 +156,14 @@ start_link(HelperSup, Ref) ->
 shutdown(Pid, Explanation) ->
     gen_server:call(Pid, {shutdown, Explanation}, infinity).
 
--spec init(pid(), pid(), any()) -> no_return().
-
-init(Parent, HelperSup, Ref) ->
+-spec init(pid(), ranch:ref()) ->
+    no_return().
+init(Parent, Ref) ->
     ?LG_PROCESS_TYPE(reader),
     {ok, Sock} = rabbit_networking:handshake(Ref,
         application:get_env(rabbit, proxy_protocol, false)),
     Deb = sys:debug_options([]),
-    start_connection(Parent, HelperSup, Ref, Deb, Sock).
+    start_connection(Parent, Ref, Deb, Sock).
 
 -spec system_continue(_,_,{[binary()], non_neg_integer(), #v1{}}) -> any().
 
@@ -291,10 +290,10 @@ socket_op(Sock, Fun) ->
                            exit(normal)
     end.
 
--spec start_connection(pid(), pid(), ranch:ref(), any(), rabbit_net:socket()) ->
+-spec start_connection(pid(), ranch:ref(), any(), rabbit_net:socket()) ->
           no_return().
 
-start_connection(Parent, HelperSup, RanchRef, Deb, Sock) ->
+start_connection(Parent, RanchRef, Deb, Sock) ->
     process_flag(trap_exit, true),
     RealSocket = rabbit_net:unwrap_socket(Sock),
     Name = case rabbit_net:connection_string(Sock, inbound) of
@@ -337,7 +336,7 @@ start_connection(Parent, HelperSup, RanchRef, Deb, Sock) ->
                 pending_recv        = false,
                 connection_state    = pre_init,
                 queue_collector     = undefined,  %% started on tune-ok
-                helper_sup          = HelperSup,
+                helper_sup          = none,
                 heartbeater         = none,
                 channel_sup_sup_pid = none,
                 channel_count       = 0,
@@ -356,16 +355,16 @@ start_connection(Parent, HelperSup, RanchRef, Deb, Sock) ->
             %% connection was closed cleanly by the client
             #v1{connection = #connection{user  = #user{username = Username},
                                          vhost = VHost}} ->
-                rabbit_log_connection:info("closing AMQP connection ~tp (~ts, vhost: '~ts', user: '~ts')",
-                    [self(), dynamic_connection_name(Name), VHost, Username]);
+                rabbit_log_connection:info("closing AMQP connection (~ts, vhost: '~ts', user: '~ts')",
+                    [dynamic_connection_name(Name), VHost, Username]);
             %% just to be more defensive
             _ ->
-                rabbit_log_connection:info("closing AMQP connection ~tp (~ts)",
-                    [self(), dynamic_connection_name(Name)])
+                rabbit_log_connection:info("closing AMQP connection (~ts)",
+                    [dynamic_connection_name(Name)])
             end
     catch
         Ex ->
-          log_connection_exception(dynamic_connection_name(Name), Ex)
+            log_connection_exception(dynamic_connection_name(Name), Ex)
     after
         %% We don't call gen_tcp:close/1 here since it waits for
         %% pending output to be sent, which results in unnecessary
@@ -499,8 +498,8 @@ mainloop(Deb, Buf, BufLen, State = #v1{sock = Sock,
             %%
             %% The goal is to not log TCP healthchecks (a connection
             %% with no data received) unless specified otherwise.
-            Fmt = "accepting AMQP connection ~tp (~ts)",
-            Args = [self(), ConnName],
+            Fmt = "accepting AMQP connection ~ts",
+            Args = [ConnName],
             case Recv of
                 closed -> _ = rabbit_log_connection:debug(Fmt, Args);
                 _      -> _ = rabbit_log_connection:info(Fmt, Args)
@@ -1078,75 +1077,64 @@ handle_input({frame_payload, Type, Channel, PayloadSize}, Data, State) ->
                                         Type, Channel, Payload, State)
     end;
 handle_input(handshake, <<"AMQP", A, B, C, D, Rest/binary>>, State) ->
-    {Rest, handshake({A, B, C, D}, State)};
+    {Rest, version_negotiation({A, B, C, D}, State)};
 handle_input(handshake, <>, #v1{sock = Sock}) ->
     refuse_connection(Sock, {bad_header, Other});
 handle_input(Callback, Data, _State) ->
     throw({bad_input, Callback, Data}).
 
-%% The two rules pertaining to version negotiation:
-%%
-%% * If the server cannot support the protocol specified in the
-%% protocol header, it MUST respond with a valid protocol header and
-%% then close the socket connection.
-%%
-%% * The server MUST provide a protocol version that is lower than or
-%% equal to that requested by the client in the protocol header.
-handshake({0, 0, 9, 1}, State) ->
-    start_connection({0, 9, 1}, rabbit_framing_amqp_0_9_1, State);
-
-%% This is the protocol header for 0-9, which we can safely treat as
-%% though it were 0-9-1.
-handshake({1, 1, 0, 9}, State) ->
-    start_connection({0, 9, 0}, rabbit_framing_amqp_0_9_1, State);
-
-%% This is what most clients send for 0-8.  The 0-8 spec, confusingly,
-%% defines the version as 8-0.
-handshake({1, 1, 8, 0}, State) ->
-    start_connection({8, 0, 0}, rabbit_framing_amqp_0_8, State);
-
-%% The 0-8 spec as on the AMQP web site actually has this as the
-%% protocol header; some libraries e.g., py-amqplib, send it when they
-%% want 0-8.
-handshake({1, 1, 9, 1}, State) ->
-    start_connection({8, 0, 0}, rabbit_framing_amqp_0_8, State);
-
-%% ... and finally, the 1.0 spec is crystal clear!
-handshake({Id, 1, 0, 0}, State) ->
-    become_1_0(Id, State);
-
-handshake(Vsn, #v1{sock = Sock}) ->
+%% AMQP 1.0 §2.2
+version_negotiation({Id, 1, 0, 0}, State) ->
+    become_10(Id, State);
+version_negotiation({0, 0, 9, 1}, State) ->
+    start_091_connection({0, 9, 1}, rabbit_framing_amqp_0_9_1, State);
+version_negotiation({1, 1, 0, 9}, State) ->
+    %% This is the protocol header for 0-9, which we can safely treat as though it were 0-9-1.
+    start_091_connection({0, 9, 0}, rabbit_framing_amqp_0_9_1, State);
+version_negotiation(Vsn = {0, 0, Minor, _}, #v1{sock = Sock})
+  when Minor >= 9 ->
+    refuse_connection(Sock, {bad_version, Vsn}, {0, 0, 9, 1});
+version_negotiation(Vsn, #v1{sock = Sock}) ->
     refuse_connection(Sock, {bad_version, Vsn}).
 
 %% Offer a protocol version to the client.  Connection.start only
 %% includes a major and minor version number, Luckily 0-9 and 0-9-1
 %% are similar enough that clients will be happy with either.
-start_connection({ProtocolMajor, ProtocolMinor, _ProtocolRevision},
-                 Protocol,
-                 State = #v1{sock = Sock, connection = Connection}) ->
+start_091_connection({ProtocolMajor, ProtocolMinor, _ProtocolRevision},
+                     Protocol,
+                     #v1{parent = Parent,
+                         sock = Sock,
+                         connection = Connection} = State0) ->
+    ConnectionHelperSupFlags = #{strategy => one_for_one,
+                                 intensity => 10,
+                                 period => 10,
+                                 auto_shutdown => any_significant},
+    {ok, ConnectionHelperSupPid} = rabbit_connection_sup:start_connection_helper_sup(
+                                     Parent, ConnectionHelperSupFlags),
     rabbit_networking:register_connection(self()),
     Start = #'connection.start'{
-      version_major = ProtocolMajor,
-      version_minor = ProtocolMinor,
-      server_properties = server_properties(Protocol),
-      mechanisms = auth_mechanisms_binary(Sock),
-      locales = <<"en_US">> },
+               version_major = ProtocolMajor,
+               version_minor = ProtocolMinor,
+               server_properties = server_properties(Protocol),
+               mechanisms = auth_mechanisms_binary(Sock),
+               locales = <<"en_US">> },
     ok = send_on_channel0(Sock, Start, Protocol),
-    switch_callback(State#v1{connection = Connection#connection{
-                                            timeout_sec = ?NORMAL_TIMEOUT,
-                                            protocol = Protocol},
-                             connection_state = starting},
-                    frame_header, 7).
+    State = State0#v1{connection = Connection#connection{
+                                     timeout_sec = ?NORMAL_TIMEOUT,
+                                     protocol = Protocol},
+                      connection_state = starting,
+                      helper_sup = ConnectionHelperSupPid},
+    switch_callback(State, frame_header, 7).
+
+-spec refuse_connection(rabbit_net:socket(), any()) -> no_return().
+refuse_connection(Sock, Exception) ->
+    refuse_connection(Sock, Exception, {0, 1, 0, 0}).
 
 -spec refuse_connection(_, _, _) -> no_return().
 refuse_connection(Sock, Exception, {A, B, C, D}) ->
     ok = inet_op(fun () -> rabbit_net:send(Sock, <<"AMQP",A,B,C,D>>) end),
     throw(Exception).
 
--spec refuse_connection(rabbit_net:socket(), any()) -> no_return().
-
-refuse_connection(Sock, Exception) ->
-    refuse_connection(Sock, Exception, {0, 0, 9, 1}).
 
 ensure_stats_timer(State = #v1{connection_state = running}) ->
     rabbit_event:ensure_stats_timer(State, #v1.stats_timer, emit_stats);
@@ -1283,9 +1271,8 @@ handle_method0(#'connection.open'{virtual_host = VHost},
     rabbit_event:notify(connection_created, Infos),
     maybe_emit_stats(State1),
     rabbit_log_connection:info(
-        "connection ~tp (~ts): "
-        "user '~ts' authenticated and granted access to vhost '~ts'",
-        [self(), dynamic_connection_name(ConnName), Username, VHost]),
+      "connection ~ts: user '~ts' authenticated and granted access to vhost '~ts'",
+      [dynamic_connection_name(ConnName), Username, VHost]),
     State1;
 handle_method0(#'connection.close'{}, State) when ?IS_RUNNING(State) ->
     lists:foreach(fun rabbit_channel:shutdown/1, all_channels()),
@@ -1309,9 +1296,9 @@ handle_method0(#'connection.update_secret'{new_secret = NewSecret, reason = Reas
                                            log_name   = ConnName} = Conn,
                            sock       = Sock}) when ?IS_RUNNING(State) ->
     rabbit_log_connection:debug(
-        "connection ~tp (~ts) of user '~ts': "
-        "asked to update secret, reason: ~ts",
-        [self(), dynamic_connection_name(ConnName), Username, Reason]),
+      "connection ~ts of user '~ts': "
+      "asked to update secret, reason: ~ts",
+      [dynamic_connection_name(ConnName), Username, Reason]),
     case rabbit_access_control:update_state(User, NewSecret) of
       {ok, User1} ->
         %% User/auth backend state has been updated. Now we can propagate it to channels
@@ -1326,9 +1313,8 @@ handle_method0(#'connection.update_secret'{new_secret = NewSecret, reason = Reas
         end, all_channels()),
         ok = send_on_channel0(Sock, #'connection.update_secret_ok'{}, Protocol),
         rabbit_log_connection:info(
-            "connection ~tp (~ts): "
-            "user '~ts' updated secret, reason: ~ts",
-            [self(), dynamic_connection_name(ConnName), Username, Reason]),
+          "connection ~ts: user '~ts' updated secret, reason: ~ts",
+          [dynamic_connection_name(ConnName), Username, Reason]),
         State#v1{connection = Conn#connection{user = User1}};
       {refused, Message} ->
         rabbit_log_connection:error("Secret update was refused for user '~ts': ~tp",
@@ -1643,32 +1629,34 @@ emit_stats(State) ->
     ensure_stats_timer(State1).
 
 %% 1.0 stub
--spec become_1_0(non_neg_integer(), #v1{}) -> no_return().
-
-become_1_0(Id, State = #v1{sock = Sock}) ->
-    case code:is_loaded(rabbit_amqp1_0_reader) of
-        false -> refuse_connection(Sock, amqp1_0_plugin_not_enabled);
-        _     -> Mode = case Id of
-                            0 -> amqp;
-                            3 -> sasl;
-                            _ -> refuse_connection(
-                                   Sock, {unsupported_amqp1_0_protocol_id, Id},
-                                   {3, 1, 0, 0})
-                        end,
-                 F = fun (_Deb, Buf, BufLen, S) ->
-                             {rabbit_amqp1_0_reader, init,
-                              [Mode, pack_for_1_0(Buf, BufLen, S)]}
-                     end,
-                 State#v1{connection_state = {become, F}}
-    end.
+-spec become_10(non_neg_integer(), #v1{}) -> no_return().
+become_10(Id, State = #v1{sock = Sock}) ->
+    Mode = case Id of
+               0 -> amqp;
+               3 -> sasl;
+               _ -> refuse_connection(
+                      Sock, {unsupported_amqp1_0_protocol_id, Id},
+                      {3, 1, 0, 0})
+           end,
+    F = fun (_Deb, Buf, BufLen, State0) ->
+                {rabbit_amqp_reader, init,
+                 [Mode, pack_for_1_0(Buf, BufLen, State0)]}
+        end,
+    State#v1{connection_state = {become, F}}.
 
-pack_for_1_0(Buf, BufLen, #v1{parent       = Parent,
-                              sock         = Sock,
+pack_for_1_0(Buf, BufLen, #v1{sock         = Sock,
                               recv_len     = RecvLen,
                               pending_recv = PendingRecv,
-                              helper_sup   = SupPid,
-                              proxy_socket = ProxySocket}) ->
-    {Parent, Sock, RecvLen, PendingRecv, SupPid, Buf, BufLen, ProxySocket}.
+                              proxy_socket = ProxySocket,
+                              connection = #connection{
+                                              name = Name,
+                                              host = Host,
+                                              peer_host = PeerHost,
+                                              port = Port,
+                                              peer_port = PeerPort,
+                                              connected_at = ConnectedAt}}) ->
+    {Sock, RecvLen, PendingRecv, Buf, BufLen, ProxySocket,
+     Name, Host, PeerHost, Port, PeerPort, ConnectedAt}.
 
 respond_and_close(State, Channel, Protocol, Reason, LogErr) ->
     log_hard_error(State, Channel, LogErr),
@@ -1802,7 +1790,8 @@ augment_connection_log_name(#connection{name = Name} = Connection) ->
             Connection;
         UserSpecifiedName ->
             LogName = <>,
-            rabbit_log_connection:info("connection ~tp (~ts) has a client-provided name: ~ts", [self(), Name, UserSpecifiedName]),
+            rabbit_log_connection:info("connection ~ts has a client-provided name: ~ts",
+                                       [Name, UserSpecifiedName]),
             ?store_proc_name(LogName),
             Connection#connection{log_name = LogName}
     end.
diff --git a/deps/rabbit/src/rabbit_stream_queue.erl b/deps/rabbit/src/rabbit_stream_queue.erl
index b0f94dd4fd92..df42b94a5361 100644
--- a/deps/rabbit/src/rabbit_stream_queue.erl
+++ b/deps/rabbit/src/rabbit_stream_queue.erl
@@ -23,7 +23,8 @@
          handle_event/3,
          deliver/3,
          settle/5,
-         credit/5,
+         credit_v1/5,
+         credit/7,
          dequeue/5,
          info/2,
          queue_length/1,
@@ -69,25 +70,32 @@
 
 -type appender_seq() :: non_neg_integer().
 
--type msg_id() :: non_neg_integer().
 -type msg() :: term(). %% TODO: refine
 
--record(stream, {credit :: integer(),
-                 max :: non_neg_integer(),
+-record(stream, {mode :: rabbit_queue_type:consume_mode(),
+                 delivery_count :: none | rabbit_queue_type:delivery_count(),
+                 credit :: rabbit_queue_type:credit(),
+                 ack :: boolean(),
                  start_offset = 0 :: non_neg_integer(),
                  listening_offset = 0 :: non_neg_integer(),
+                 last_consumed_offset = 0 :: non_neg_integer(),
                  log :: undefined | osiris_log:state(),
+                 chunk_iterator :: undefined | osiris_log:chunk_iterator(),
+                 %% These messages were already read ahead from the Osiris log,
+                 %% were part of an uncompressed sub batch, and are buffered in
+                 %% reversed order until the consumer has more credits to consume them.
+                 buffer_msgs_rev = [] :: [rabbit_amqqueue:qmsg()],
                  reader_options :: map()}).
 
 -record(stream_client, {stream_id :: string(),
-                        name :: term(),
+                        name :: rabbit_amqqueue:name(),
                         leader :: pid(),
                         local_pid :: undefined | pid(),
                         next_seq = 1 :: non_neg_integer(),
-                        correlation = #{} :: #{appender_seq() => {msg_id(), msg()}},
+                        correlation = #{} :: #{appender_seq() => {rabbit_queue_type:correlation(), msg()}},
                         soft_limit :: non_neg_integer(),
                         slow = false :: boolean(),
-                        readers = #{} :: #{term() => #stream{}},
+                        readers = #{} :: #{rabbit_types:ctag() => #stream{}},
                         writer_id :: binary(),
                         filtering_supported :: boolean()
                        }).
@@ -264,14 +272,15 @@ format(Q, Ctx) ->
              {state, down}]
     end.
 
-consume(Q, #{prefetch_count := 0}, _)
+consume(Q, #{mode := {simple_prefetch, 0}}, _)
   when ?amqqueue_is_stream(Q) ->
-    {protocol_error, precondition_failed, "consumer prefetch count is not set for '~ts'",
+    {protocol_error, precondition_failed, "consumer prefetch count is not set for stream ~ts",
      [rabbit_misc:rs(amqqueue:get_name(Q))]};
-consume(Q, #{no_ack := true}, _)
+consume(Q, #{no_ack := true,
+             mode := {simple_prefetch, _}}, _)
   when ?amqqueue_is_stream(Q) ->
     {protocol_error, not_implemented,
-     "automatic acknowledgement not supported by stream queues ~ts",
+     "automatic acknowledgement not supported by stream ~ts",
      [rabbit_misc:rs(amqqueue:get_name(Q))]};
 consume(Q, #{limiter_active := true}, _State)
   when ?amqqueue_is_stream(Q) ->
@@ -284,7 +293,7 @@ consume(Q, Spec,
         {LocalPid, QState} when is_pid(LocalPid) ->
             #{no_ack := NoAck,
               channel_pid := ChPid,
-              prefetch_count := ConsumerPrefetchCount,
+              mode := Mode,
               consumer_tag := ConsumerTag,
               exclusive_consume := ExclusiveConsume,
               args := Args,
@@ -303,22 +312,24 @@ consume(Q, Spec,
                             {protocol_error, precondition_failed,
                              "Filtering is not supported", []};
                         _ ->
-                            rabbit_core_metrics:consumer_created(ChPid, ConsumerTag,
-                                                                 ExclusiveConsume,
-                                                                 not NoAck, QName,
-                                                                 ConsumerPrefetchCount,
-                                                                 false, up, Args),
+                            ConsumerPrefetchCount = case Mode of
+                                                        {simple_prefetch, C} -> C;
+                                                        _ -> 0
+                                                    end,
+                            AckRequired = not NoAck,
+                            rabbit_core_metrics:consumer_created(
+                              ChPid, ConsumerTag, ExclusiveConsume, AckRequired,
+                              QName, ConsumerPrefetchCount, false, up, Args),
                             %% reply needs to be sent before the stream
                             %% begins sending
                             maybe_send_reply(ChPid, OkMsg),
                             _ = rabbit_stream_coordinator:register_local_member_listener(Q),
-                            begin_stream(QState, ConsumerTag, OffsetSpec,
-                                         ConsumerPrefetchCount, FilterSpec)
+                            begin_stream(QState, ConsumerTag, OffsetSpec, Mode, AckRequired, FilterSpec)
                     end
             end;
         {undefined, _} ->
             {protocol_error, precondition_failed,
-             "queue '~ts' does not have a running replica on the local node",
+             "stream ~ts does not have a running replica on the local node",
              [rabbit_misc:rs(amqqueue:get_name(Q))]}
     end.
 
@@ -405,7 +416,7 @@ query_local_pid(#stream_client{stream_id = StreamId} = State) ->
 begin_stream(#stream_client{name = QName,
                             readers = Readers0,
                             local_pid = LocalPid} = State,
-             Tag, Offset, Max, Options)
+             Tag, Offset, Mode, AckRequired, Options)
   when is_pid(LocalPid) ->
     CounterSpec = {{?MODULE, QName, Tag, self()}, []},
     {ok, Seg0} = osiris:init_reader(LocalPid, Offset, CounterSpec, Options),
@@ -418,14 +429,22 @@ begin_stream(#stream_client{name = QName,
                       {timestamp, _} -> NextOffset;
                       _ -> Offset
                   end,
-    Str0 = #stream{credit = Max,
+    {DeliveryCount, Credit} = case Mode of
+                                  {simple_prefetch, N} ->
+                                      {none, N};
+                                  {credited, InitialDC} ->
+                                      {InitialDC, 0}
+                              end,
+    Str0 = #stream{mode = Mode,
+                   delivery_count = DeliveryCount,
+                   credit = Credit,
+                   ack = AckRequired,
                    start_offset = StartOffset,
                    listening_offset = NextOffset,
+                   last_consumed_offset = StartOffset,
                    log = Seg0,
-                   max = Max,
                    reader_options = Options},
-    {ok, State#stream_client{local_pid = LocalPid,
-                             readers = Readers0#{Tag => Str0}}}.
+    {ok, State#stream_client{readers = Readers0#{Tag => Str0}}}.
 
 cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0,
                                                           name = QName} = State) ->
@@ -444,34 +463,54 @@ cancel(_Q, ConsumerTag, OkMsg, ActingUser, #stream_client{readers = Readers0,
             {ok, State}
     end.
 
-credit(QName, CTag, Credit, Drain, #stream_client{readers = Readers0,
-                                                  name = Name,
-                                                  local_pid = LocalPid} = State) ->
-    case Readers0 of
-        #{CTag := #stream{credit = Credit0} = Str0} ->
-            Str1 = Str0#stream{credit = Credit0 + Credit},
-            {Str, Msgs} = stream_entries(QName, Name, LocalPid, Str1),
-            Actions = case Msgs of
-                          [] ->
-                              [{send_credit_reply, 0}];
-                          _ ->
-                              [{send_credit_reply, length(Msgs)},
-                               {deliver, CTag, true, Msgs}]
+-dialyzer({nowarn_function, credit_v1/5}).
+credit_v1(_, _, _, _, _) ->
+    erlang:error(credit_v1_unsupported).
+
+credit(QName, CTag, DeliveryCountRcv, LinkCreditRcv, Drain, Echo,
+       #stream_client{readers = Readers,
+                      name = Name,
+                      local_pid = LocalPid} = State0) ->
+    case Readers of
+        #{CTag := Str0 = #stream{delivery_count = DeliveryCountSnd}} ->
+            LinkCreditSnd = serial_number:diff(
+                              serial_number:add(DeliveryCountRcv, LinkCreditRcv),
+                              DeliveryCountSnd),
+            Str1 = Str0#stream{credit = LinkCreditSnd},
+            {Str2 = #stream{delivery_count = DeliveryCount,
+                            credit = Credit,
+                            ack = Ack}, Msgs} = stream_entries(QName, Name, LocalPid, Str1),
+            DrainedInsufficientMsgs = Drain andalso Credit > 0,
+            Str = case DrainedInsufficientMsgs of
+                      true ->
+                          Str2#stream{delivery_count = serial_number:add(DeliveryCount, Credit),
+                                      credit = 0};
+                      false ->
+                          Str2
+                  end,
+            DeliverActions = deliver_actions(CTag, Ack, Msgs),
+            State = State0#stream_client{readers = maps:update(CTag, Str, Readers)},
+            Actions = case Echo orelse DrainedInsufficientMsgs of
+                          true ->
+                              DeliverActions ++ [{credit_reply,
+                                                  CTag,
+                                                  Str#stream.delivery_count,
+                                                  Str#stream.credit,
+                                                  available_messages(Str),
+                                                  Drain}];
+                          false ->
+                              DeliverActions
                       end,
-            case Drain of
-                true ->
-                    Readers = Readers0#{CTag => Str#stream{credit = 0}},
-                    {State#stream_client{readers = Readers},
-                     %% send_drained needs to come after deliver
-                     Actions ++ [{send_drained, {CTag, Str#stream.credit}}]};
-                false ->
-                    Readers = Readers0#{CTag => Str},
-                    {State#stream_client{readers = Readers}, Actions}
-            end;
+            {State, Actions};
         _ ->
-            {State, []}
+            {State0, []}
     end.
 
+%% Returns only an approximation.
+available_messages(#stream{log = Log,
+                           last_consumed_offset = LastConsumedOffset}) ->
+    max(0, osiris_log:committed_offset(Log) - LastConsumedOffset).
+
 deliver(QSs, Msg, Options) ->
     lists:foldl(
       fun({Q, stateless}, {Qs, Actions}) ->
@@ -500,7 +539,7 @@ deliver0(MsgId, Msg,
     Correlation = case MsgId of
                       undefined ->
                           Correlation0;
-                      _ when is_number(MsgId) ->
+                      _ ->
                           Correlation0#{Seq => {MsgId, Msg}}
                   end,
     {Slow, Actions} = case maps:size(Correlation) >= SftLmt of
@@ -513,16 +552,21 @@ deliver0(MsgId, Msg,
                          correlation = Correlation,
                          slow = Slow}, Actions}.
 
-stream_message(Msg, _FilteringSupported = true) ->
-    MsgData = msg_to_iodata(Msg),
-    case mc:x_header(<<"x-stream-filter-value">>, Msg) of
-        undefined ->
-            MsgData;
-        {utf8, Value} ->
-            {Value, MsgData}
-    end;
-stream_message(Msg, _FilteringSupported = false) ->
-    msg_to_iodata(Msg).
+stream_message(Msg, FilteringSupported) ->
+    McAmqp = mc:convert(mc_amqp, Msg),
+    Sections = mc:protocol_state(McAmqp),
+    MsgData = mc_amqp:serialize(Sections),
+    case FilteringSupported of
+        true ->
+            case mc:x_header(<<"x-stream-filter-value">>, McAmqp) of
+                undefined ->
+                    MsgData;
+                {utf8, Value} ->
+                    {Value, MsgData}
+            end;
+        false ->
+            MsgData
+    end.
 
 -spec dequeue(_, _, _, _, client()) -> no_return().
 dequeue(_, _, _, _, #stream_client{name = Name}) ->
@@ -530,43 +574,41 @@ dequeue(_, _, _, _, #stream_client{name = Name}) ->
      [rabbit_misc:rs(Name)]}.
 
 handle_event(_QName, {osiris_written, From, _WriterId, Corrs},
-             State = #stream_client{correlation = Correlation0,
-                                    soft_limit = SftLmt,
-                                    slow = Slow0,
-                                    name = Name}) ->
+             State0 = #stream_client{correlation = Correlation0,
+                                     soft_limit = SftLmt,
+                                     slow = Slow0,
+                                     name = Name}) ->
     MsgIds = lists:sort(maps:fold(
                           fun (_Seq, {I, _M}, Acc) ->
                                   [I | Acc]
                           end, [], maps:with(Corrs, Correlation0))),
 
     Correlation = maps:without(Corrs, Correlation0),
-    {Slow, Actions} = case maps:size(Correlation) < SftLmt of
-                          true when Slow0 ->
-                              {false, [{unblock, Name}]};
-                          _ ->
-                              {Slow0, []}
-                      end,
-    {ok, State#stream_client{correlation = Correlation,
-                             slow = Slow}, [{settled, From, MsgIds} | Actions]};
+    {Slow, Actions0} = case maps:size(Correlation) < SftLmt of
+                           true when Slow0 ->
+                               {false, [{unblock, Name}]};
+                           _ ->
+                               {Slow0, []}
+                       end,
+    Actions = case MsgIds of
+                  [] -> Actions0;
+                  [_|_] -> [{settled, From, MsgIds} | Actions0]
+              end,
+    State = State0#stream_client{correlation = Correlation,
+                                 slow = Slow},
+    {ok, State, Actions};
 handle_event(QName, {osiris_offset, _From, _Offs},
              State = #stream_client{local_pid = LocalPid,
                                     readers = Readers0,
                                     name = Name}) ->
-    Ack = true,
     %% offset isn't actually needed as we use the atomic to read the
     %% current committed
-    {Readers, Deliveries} =
-        maps:fold(
-          fun (Tag, Str0, {Acc, TM}) ->
-                  case stream_entries(QName, Name, LocalPid, Str0) of
-                      {Str, []} ->
-                          {Acc#{Tag => Str}, TM};
-                      {Str, Msgs} ->
-                          {Acc#{Tag => Str},
-                           [{deliver, Tag, Ack, Msgs} | TM]}
-                  end
-          end, {#{}, []}, Readers0),
-    {ok, State#stream_client{readers = Readers}, Deliveries};
+    {Readers, Actions} = maps:fold(
+                           fun (Tag, Str0, {Rds, As}) ->
+                                   {Str, Msgs} = stream_entries(QName, Name, LocalPid, Str0),
+                                   {Rds#{Tag => Str}, deliver_actions(Tag, Str#stream.ack, Msgs) ++ As}
+                           end, {#{}, []}, Readers0),
+    {ok, State#stream_client{readers = Readers}, Actions};
 handle_event(_QName, {stream_leader_change, Pid}, State) ->
     {ok, update_leader_pid(Pid, State), []};
 handle_event(_QName, {stream_local_member_change, Pid},
@@ -611,19 +653,22 @@ recover(_VHost, Queues) ->
       end, {[], []}, Queues).
 
 settle(QName, _, CTag, MsgIds, #stream_client{readers = Readers0,
-                                                     local_pid = LocalPid,
-                                                     name = Name} = State) ->
-    %% all settle reasons will "give credit" to the stream queue
-    Credit = length(MsgIds),
-    {Readers, Msgs} = case Readers0 of
-                          #{CTag := #stream{credit = Credit0} = Str0} ->
-                              Str1 = Str0#stream{credit = Credit0 + Credit},
-                              {Str, Msgs0} = stream_entries(QName, Name, LocalPid, Str1),
-                              {Readers0#{CTag => Str}, Msgs0};
-                          _ ->
-                              {Readers0, []}
-                      end,
-    {State#stream_client{readers = Readers}, [{deliver, CTag, true, Msgs}]}.
+                                              local_pid = LocalPid,
+                                              name = Name} = State) ->
+    case Readers0 of
+        #{CTag := #stream{mode = {simple_prefetch, _MaxCredit},
+                          ack = Ack,
+                          credit = Credit0} = Str0} ->
+            %% all settle reasons will "give credit" to the stream queue
+            Credit = length(MsgIds),
+            Str1 = Str0#stream{credit = Credit0 + Credit},
+            {Str, Msgs} = stream_entries(QName, Name, LocalPid, Str1),
+            Readers = maps:update(CTag, Str, Readers0),
+            {State#stream_client{readers = Readers},
+             deliver_actions(CTag, Ack, Msgs)};
+        _ ->
+            {State, []}
+    end.
 
 info(Q, all_keys) ->
     info(Q, ?INFO_KEYS);
@@ -1064,72 +1109,164 @@ recover(Q) ->
 maybe_send_reply(_ChPid, undefined) -> ok;
 maybe_send_reply(ChPid, Msg) -> ok = rabbit_channel:send_command(ChPid, Msg).
 
+stream_entries(QName, Name, LocalPid,
+               #stream{chunk_iterator = undefined,
+                       credit = Credit} = Str0) ->
+    case Credit > 0 of
+        true ->
+            case chunk_iterator(Str0, LocalPid) of
+                {ok, Str} ->
+                    stream_entries(QName, Name, LocalPid, Str);
+                {end_of_stream, Str} ->
+                    {Str, []}
+            end;
+        false ->
+            {Str0, []}
+    end;
+stream_entries(QName, Name, LocalPid,
+               #stream{delivery_count = DC,
+                       credit = Credit,
+                       buffer_msgs_rev = Buf0,
+                       last_consumed_offset = LastOff} = Str0)
+  when Credit > 0 andalso Buf0 =/= [] ->
+    BufLen = length(Buf0),
+    case Credit =< BufLen of
+        true ->
+            %% Entire credit worth of messages can be served from the buffer.
+            {Buf, BufMsgsRev} = lists:split(BufLen - Credit, Buf0),
+            {Str0#stream{delivery_count = delivery_count_add(DC, Credit),
+                         credit = 0,
+                         buffer_msgs_rev = Buf,
+                         last_consumed_offset = LastOff + Credit},
+             lists:reverse(BufMsgsRev)};
+        false ->
+            Str = Str0#stream{delivery_count = delivery_count_add(DC, BufLen),
+                              credit = Credit - BufLen,
+                              buffer_msgs_rev = [],
+                              last_consumed_offset = LastOff + BufLen},
+            stream_entries(QName, Name, LocalPid, Str, Buf0)
+    end;
 stream_entries(QName, Name, LocalPid, Str) ->
     stream_entries(QName, Name, LocalPid, Str, []).
 
+stream_entries(_, _, _, #stream{credit = Credit} = Str, Acc)
+  when Credit < 1 ->
+    {Str, lists:reverse(Acc)};
 stream_entries(QName, Name, LocalPid,
-               #stream{credit = Credit,
-                       start_offset = StartOffs,
-                       listening_offset = LOffs,
-                       log = Seg0} = Str0, MsgIn)
-  when Credit > 0 ->
-    case osiris_log:read_chunk_parsed(Seg0) of
-        {end_of_stream, Seg} ->
-            NextOffset = osiris_log:next_offset(Seg),
-            case NextOffset > LOffs of
-                true ->
-                    osiris:register_offset_listener(LocalPid, NextOffset),
-                    {Str0#stream{log = Seg,
-                                 listening_offset = NextOffset}, MsgIn};
-                false ->
-                    {Str0#stream{log = Seg}, MsgIn}
+               #stream{chunk_iterator = Iter0,
+                       delivery_count = DC,
+                       credit = Credit,
+                       start_offset = StartOffset} = Str0, Acc0) ->
+    case osiris_log:iterator_next(Iter0) of
+        end_of_chunk ->
+            case chunk_iterator(Str0, LocalPid) of
+                {ok, Str} ->
+                    stream_entries(QName, Name, LocalPid, Str, Acc0);
+                {end_of_stream, Str} ->
+                    {Str, lists:reverse(Acc0)}
             end;
+        {{Offset, Entry}, Iter} ->
+            {Str, Acc} = case Entry of
+                             {batch, _NumRecords, 0, _Len, BatchedEntries} ->
+                                 {MsgsRev, NumMsgs} = parse_uncompressed_subbatch(
+                                                        BatchedEntries, Offset, StartOffset,
+                                                        QName, Name, LocalPid, {[], 0}),
+                                 case Credit >= NumMsgs of
+                                     true ->
+                                         {Str0#stream{chunk_iterator = Iter,
+                                                      delivery_count = delivery_count_add(DC, NumMsgs),
+                                                      credit = Credit - NumMsgs,
+                                                      last_consumed_offset = Offset + NumMsgs - 1},
+                                          MsgsRev ++ Acc0};
+                                     false ->
+                                         %% Consumer doesn't have sufficient credit.
+                                         %% Buffer the remaining messages.
+                                         [] = Str0#stream.buffer_msgs_rev, % assertion
+                                         {Buf, MsgsRev1} = lists:split(NumMsgs - Credit, MsgsRev),
+                                         {Str0#stream{chunk_iterator = Iter,
+                                                      delivery_count = delivery_count_add(DC, Credit),
+                                                      credit = 0,
+                                                      buffer_msgs_rev = Buf,
+                                                      last_consumed_offset = Offset + Credit - 1},
+                                          MsgsRev1 ++ Acc0}
+                                 end;
+                             {batch, _, _CompressionType, _, _} ->
+                                 %% Skip compressed sub batch.
+                                 %% It can only be consumed by Stream protocol clients.
+                                 {Str0#stream{chunk_iterator = Iter}, Acc0};
+                             _SimpleEntry ->
+                                 case Offset >= StartOffset of
+                                     true ->
+                                         Msg = entry_to_msg(Entry, Offset, QName, Name, LocalPid),
+                                         {Str0#stream{chunk_iterator = Iter,
+                                                      delivery_count = delivery_count_add(DC, 1),
+                                                      credit = Credit - 1,
+                                                      last_consumed_offset = Offset},
+                                          [Msg | Acc0]};
+                                     false ->
+                                         {Str0#stream{chunk_iterator = Iter}, Acc0}
+                                 end
+                         end,
+            stream_entries(QName, Name, LocalPid, Str, Acc)
+    end.
+
+chunk_iterator(#stream{credit = Credit,
+                       listening_offset = LOffs,
+                       log = Log0} = Str0, LocalPid) ->
+    case osiris_log:chunk_iterator(Log0, Credit) of
+        {ok, _ChunkHeader, Iter, Log} ->
+            {ok, Str0#stream{chunk_iterator = Iter,
+                             log = Log}};
+        {end_of_stream, Log} ->
+            NextOffset = osiris_log:next_offset(Log),
+            Str = case NextOffset > LOffs of
+                      true ->
+                          osiris:register_offset_listener(LocalPid, NextOffset),
+                          Str0#stream{log = Log,
+                                      listening_offset = NextOffset};
+                      false ->
+                          Str0#stream{log = Log}
+                  end,
+            {end_of_stream, Str};
         {error, Err} ->
-            rabbit_log:debug("stream client: error reading chunk ~w", [Err]),
-            exit(Err);
-        {Records, Seg} ->
-            Msgs = [begin
-                        Msg0 = binary_to_msg(QName, B),
-                        Msg = mc:set_annotation(<<"x-stream-offset">>, O, Msg0),
-                        {Name, LocalPid, O, false, Msg}
-                    end || {O, B} <- Records,
-                           O >= StartOffs],
-
-            NumMsgs = length(Msgs),
-
-            Str = Str0#stream{credit = Credit - NumMsgs,
-                              log = Seg},
-            case Str#stream.credit < 1 of
-                true ->
-                    %% we are done here
-                    {Str, MsgIn ++ Msgs};
-                false ->
-                    %% if there are fewer Msgs than Entries0 it means there were non-events
-                    %% in the log and we should recurse and try again
-                    stream_entries(QName, Name, LocalPid, Str, MsgIn ++ Msgs)
-            end
-    end;
-stream_entries(_QName, _Name, _LocalPid, Str, Msgs) ->
-    {Str, Msgs}.
+            rabbit_log:info("stream client: failed to create chunk iterator ~p", [Err]),
+            exit(Err)
+    end.
+
+%% Deliver each record of an uncompressed sub batch individually.
+parse_uncompressed_subbatch(<<>>, _Offset, _StartOffset, _QName, _Name, _LocalPid, Acc) ->
+    Acc;
+parse_uncompressed_subbatch(
+  <<0:1, %% simple entry
+    Len:31/unsigned,
+    Entry:Len/binary,
+    Rem/binary>>,
+  Offset, StartOffset, QName, Name, LocalPid, Acc0 = {AccList, AccCount}) ->
+    Acc = case Offset >= StartOffset of
+              true ->
+                  Msg = entry_to_msg(Entry, Offset, QName, Name, LocalPid),
+                  {[Msg | AccList], AccCount + 1};
+              false ->
+                  Acc0
+          end,
+    parse_uncompressed_subbatch(Rem, Offset + 1, StartOffset, QName, Name, LocalPid, Acc).
 
-binary_to_msg(#resource{kind = queue,
-                        name = QName}, Data) ->
-    Mc0 = mc:init(mc_amqp, amqp10_framing:decode_bin(Data), #{}),
-    %% If exchange or routing_keys annotation isn't present the data most likely came
+entry_to_msg(Entry, Offset, #resource{kind = queue,
+                                      name = QName}, Name, LocalPid) ->
+    Mc0 = mc:init(mc_amqp, amqp10_framing:decode_bin(Entry), #{}),
+    %% If exchange or routing_keys annotation isn't present the entry most likely came
     %% from the rabbitmq-stream plugin so we'll choose defaults that simulate use
     %% of the direct exchange.
-    Mc = case mc:exchange(Mc0) of
-             undefined -> mc:set_annotation(?ANN_EXCHANGE, <<>>, Mc0);
-             _ -> Mc0
-         end,
-    case mc:routing_keys(Mc) of
-        [] -> mc:set_annotation(?ANN_ROUTING_KEYS, [QName], Mc);
-        _ -> Mc
-    end.
-
-msg_to_iodata(Msg0) ->
-    Sections = mc:protocol_state(mc:convert(mc_amqp, Msg0)),
-    mc_amqp:serialize(Sections).
+    Mc1 = case mc:exchange(Mc0) of
+              undefined -> mc:set_annotation(?ANN_EXCHANGE, <<>>, Mc0);
+              _ -> Mc0
+          end,
+    Mc2 = case mc:routing_keys(Mc1) of
+              [] -> mc:set_annotation(?ANN_ROUTING_KEYS, [QName], Mc1);
+              _ -> Mc1
+          end,
+    Mc = mc:set_annotation(<<"x-stream-offset">>, Offset, Mc2),
+    {Name, LocalPid, Offset, false, Mc}.
 
 capabilities() ->
     #{unsupported_policies => [%% Classic policies
@@ -1146,7 +1283,7 @@ capabilities() ->
       queue_arguments => [<<"x-max-length-bytes">>, <<"x-queue-type">>,
                           <<"x-max-age">>, <<"x-stream-max-segment-size-bytes">>,
                           <<"x-initial-cluster-size">>, <<"x-queue-leader-locator">>],
-      consumer_arguments => [<<"x-stream-offset">>, <<"x-credit">>],
+      consumer_arguments => [<<"x-stream-offset">>],
       server_named => false}.
 
 notify_decorators(Q) when ?is_amqqueue(Q) ->
@@ -1211,3 +1348,13 @@ get_nodes(Q) when ?is_amqqueue(Q) ->
 is_minority(All, Up) ->
     MinQuorum = length(All) div 2 + 1,
     length(Up) < MinQuorum.
+
+deliver_actions(_, _, []) ->
+    [];
+deliver_actions(CTag, Ack, Msgs) ->
+    [{deliver, CTag, Ack, Msgs}].
+
+delivery_count_add(none, _) ->
+    none;
+delivery_count_add(Count, N) ->
+    serial_number:add(Count, N).
diff --git a/deps/rabbit/src/rabbit_trace.erl b/deps/rabbit/src/rabbit_trace.erl
index 36a7c572b955..8a447bd064e1 100644
--- a/deps/rabbit/src/rabbit_trace.erl
+++ b/deps/rabbit/src/rabbit_trace.erl
@@ -26,26 +26,24 @@
 %%----------------------------------------------------------------------------
 
 -spec init(rabbit_types:vhost()) -> state().
-
 init(VHost)
   when is_binary(VHost) ->
     case enabled(VHost) of
-        false -> none;
-        true  -> {ok, X} = rabbit_exchange:lookup(
-                             rabbit_misc:r(VHost, exchange, ?XNAME)),
-                 X
+        false ->
+            none;
+        true ->
+            {ok, X} = rabbit_exchange:lookup(rabbit_misc:r(VHost, exchange, ?XNAME)),
+            X
     end.
 
 -spec enabled(rabbit_types:vhost() | state()) -> boolean().
-
-enabled(VHost)
-  when is_binary(VHost) ->
-    {ok, VHosts} = application:get_env(rabbit, ?TRACE_VHOSTS),
-    lists:member(VHost, VHosts);
 enabled(none) ->
     false;
 enabled(#exchange{}) ->
-    true.
+    true;
+enabled(VHost)
+  when is_binary(VHost) ->
+    lists:member(VHost, vhosts_with_tracing_enabled()).
 
 -spec tap_in(mc:state(), rabbit_exchange:route_return(),
              binary(), rabbit_types:username(), state()) -> 'ok'.
@@ -55,7 +53,8 @@ tap_in(Msg, QNames, ConnName, Username, State) ->
 -spec tap_in(mc:state(), rabbit_exchange:route_return(),
              binary(), rabbit_channel:channel_number(),
              rabbit_types:username(), state()) -> 'ok'.
-tap_in(_Msg, _QNames, _ConnName, _ChannelNum, _Username, none) -> ok;
+tap_in(_Msg, _QNames, _ConnName, _ChannelNum, _Username, none) ->
+    ok;
 tap_in(Msg, QNames, ConnName, ChannelNum, Username, TraceX) ->
     XName = mc:exchange(Msg),
     #exchange{name = #resource{virtual_host = VHost}} = TraceX,
@@ -79,11 +78,15 @@ tap_out(Msg, ConnName, Username, State) ->
 -spec tap_out(rabbit_amqqueue:qmsg(), binary(),
               rabbit_channel:channel_number(),
               rabbit_types:username(), state()) -> 'ok'.
-tap_out(_Msg, _ConnName, _ChannelNum, _Username, none) -> ok;
+tap_out(_Msg, _ConnName, _ChannelNum, _Username, none) ->
+    ok;
 tap_out({#resource{name = QName, virtual_host = VHost},
          _QPid, _QMsgId, Redelivered, Msg},
         ConnName, ChannelNum, Username, TraceX) ->
-    RedeliveredNum = case Redelivered of true -> 1; false -> 0 end,
+    RedeliveredNum = case Redelivered of
+                         true -> 1;
+                         false -> 0
+                     end,
     trace(TraceX, Msg, <<"deliver">>, QName,
           [{<<"redelivered">>, signedint, RedeliveredNum},
            {<<"vhost">>,       longstr,   VHost},
@@ -94,28 +97,24 @@ tap_out({#resource{name = QName, virtual_host = VHost},
 %%----------------------------------------------------------------------------
 
 -spec start(rabbit_types:vhost()) -> 'ok'.
-
 start(VHost)
   when is_binary(VHost) ->
-    case lists:member(VHost, vhosts_with_tracing_enabled()) of
+    case enabled(VHost) of
         true  ->
             rabbit_log:info("Tracing is already enabled for vhost '~ts'", [VHost]),
             ok;
         false ->
             rabbit_log:info("Enabling tracing for vhost '~ts'", [VHost]),
-            update_config(fun (VHosts) ->
-                            lists:usort([VHost | VHosts])
-                          end)
+            update_config(fun(VHosts) -> lists:usort([VHost | VHosts]) end)
     end.
 
 -spec stop(rabbit_types:vhost()) -> 'ok'.
-
 stop(VHost)
   when is_binary(VHost) ->
-    case lists:member(VHost, vhosts_with_tracing_enabled()) of
+    case enabled(VHost) of
         true  ->
             rabbit_log:info("Disabling tracing for vhost '~ts'", [VHost]),
-            update_config(fun (VHosts) -> VHosts -- [VHost] end);
+            update_config(fun(VHosts) -> VHosts -- [VHost] end);
         false ->
             rabbit_log:info("Tracing is already disabled for vhost '~ts'", [VHost]),
             ok
@@ -125,17 +124,20 @@ update_config(Fun) ->
     VHosts0 = vhosts_with_tracing_enabled(),
     VHosts = Fun(VHosts0),
     application:set_env(rabbit, ?TRACE_VHOSTS, VHosts),
+    Sessions = rabbit_amqp_session:list_local(),
     NonAmqpPids = rabbit_networking:local_non_amqp_connections(),
-    rabbit_log:debug("Will now refresh state of channels and of ~b non AMQP 0.9.1 "
-                     "connections after virtual host tracing changes",
-                     [length(NonAmqpPids)]),
-    lists:foreach(fun(Pid) -> gen_server:cast(Pid, refresh_config) end, NonAmqpPids),
-    {Time, _} = timer:tc(fun rabbit_channel:refresh_config_local/0),
-    rabbit_log:debug("Refreshed channel state in ~fs", [Time/1_000_000]),
+    rabbit_log:debug("Refreshing state of channels, ~b sessions and ~b non "
+                     "AMQP 0.9.1 connections after virtual host tracing changes...",
+                     [length(Sessions), length(NonAmqpPids)]),
+    Pids = Sessions ++ NonAmqpPids,
+    lists:foreach(fun(Pid) -> gen_server:cast(Pid, refresh_config) end, Pids),
+    {Time, ok} = timer:tc(fun rabbit_channel:refresh_config_local/0),
+    rabbit_log:debug("Refreshed channel states in ~fs", [Time / 1_000_000]),
     ok.
 
 vhosts_with_tracing_enabled() ->
-    application:get_env(rabbit, ?TRACE_VHOSTS, []).
+    {ok, Vhosts} = application:get_env(rabbit, ?TRACE_VHOSTS),
+    Vhosts.
 
 %%----------------------------------------------------------------------------
 
@@ -148,9 +150,7 @@ trace(X, Msg0, RKPrefix, RKSuffix, Extra) ->
             RoutingKeys = mc:routing_keys(Msg0),
             %% for now convert into amqp legacy
             Msg = mc:prepare(read, mc:convert(mc_amqpl, Msg0)),
-            %% check exchange name in case it is same as target
-            #content{properties = Props} = Content0 =
-                mc:protocol_state(Msg),
+            #content{properties = Props} = Content0 = mc:protocol_state(Msg),
 
             Key = <>,
             Content = Content0#content{properties =
@@ -159,26 +159,23 @@ trace(X, Msg0, RKPrefix, RKSuffix, Extra) ->
                                        properties_bin = none},
             TargetXName = SourceXName#resource{name = ?XNAME},
             {ok, TraceMsg} = mc_amqpl:message(TargetXName, Key, Content),
-            ok = rabbit_queue_type:publish_at_most_once(X, TraceMsg),
-            ok
+            ok = rabbit_queue_type:publish_at_most_once(X, TraceMsg)
     end.
 
 msg_to_table(XName, RoutingKeys, Props) ->
     {PropsTable, _Ix} =
-        lists:foldl(fun (K, {L, Ix}) ->
+        lists:foldl(fun(K, {L, Ix}) ->
                             V = element(Ix, Props),
                             NewL = case V of
                                        undefined -> L;
-                                       _         -> [{a2b(K), type(V), V} | L]
+                                       _ -> [{atom_to_binary(K), type(V), V} | L]
                                    end,
                             {NewL, Ix + 1}
                     end, {[], 2}, record_info(fields, 'P_basic')),
     [{<<"exchange_name">>, longstr, XName},
      {<<"routing_keys">>,  array,   [{longstr, K} || K <- RoutingKeys]},
      {<<"properties">>,    table,   PropsTable},
-     {<<"node">>,          longstr, a2b(node())}].
-
-a2b(A) -> list_to_binary(atom_to_list(A)).
+     {<<"node">>,          longstr, atom_to_binary(node())}].
 
 type(V) when is_list(V)    -> table;
 type(V) when is_integer(V) -> signedint;
diff --git a/deps/rabbit/test/amqp_auth_SUITE.erl b/deps/rabbit/test/amqp_auth_SUITE.erl
new file mode 100644
index 000000000000..eafa0f1cabd4
--- /dev/null
+++ b/deps/rabbit/test/amqp_auth_SUITE.erl
@@ -0,0 +1,621 @@
+%% 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-2023 VMware, Inc. or its affiliates.  All rights reserved.
+
+-module(amqp_auth_SUITE).
+
+-compile([export_all,
+          nowarn_export_all]).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("amqp10_common/include/amqp10_framing.hrl").
+
+-import(rabbit_ct_broker_helpers,
+        [rpc/4]).
+-import(rabbit_ct_helpers,
+        [eventually/1]).
+-import(event_recorder,
+        [assert_event_type/2,
+         assert_event_prop/2]).
+
+all() ->
+    [
+     {group, tests}
+    ].
+
+groups() ->
+    [
+     {tests, [shuffle],
+      [
+       attach_target_queue,
+       attach_source_exchange,
+       send_to_topic,
+       send_to_topic_using_subject,
+       attach_source_topic,
+       attach_target_internal_exchange,
+       authn_failure_event,
+       sasl_anonymous_success,
+       sasl_none_success,
+       sasl_plain_success,
+       sasl_anonymous_failure,
+       sasl_none_failure,
+       sasl_plain_failure,
+       vhost_absent,
+       vhost_connection_limit,
+       user_connection_limit,
+       vhost_queue_limit
+      ]
+     }
+    ].
+
+init_per_suite(Config) ->
+    application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    Config.
+
+end_per_suite(Config) ->
+    Config.
+
+init_per_group(_Group, Config0) ->
+    Config = rabbit_ct_helpers:run_setup_steps(
+               Config0,
+               rabbit_ct_broker_helpers:setup_steps() ++
+               rabbit_ct_client_helpers:setup_steps()),
+    Vhost = <<"test vhost">>,
+    User = <<"test user">>,
+    ok = rabbit_ct_broker_helpers:add_vhost(Config, Vhost),
+    ok = rabbit_ct_broker_helpers:add_user(Config, User),
+    [{test_vhost, Vhost},
+     {test_user, User}] ++ Config.
+
+end_per_group(_Group, Config) ->
+    ok = rabbit_ct_broker_helpers:delete_user(Config, ?config(test_user, Config)),
+    ok = rabbit_ct_broker_helpers:delete_vhost(Config, ?config(test_vhost, Config)),
+    rabbit_ct_helpers:run_teardown_steps(
+      Config,
+      rabbit_ct_client_helpers:teardown_steps() ++
+      rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(Testcase, Config) ->
+    ok = set_permissions(Config, <<>>, <<>>, <<"^some vhost permission">>),
+    rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config) ->
+    delete_all_queues(Config),
+    ok = clear_permissions(Config),
+    rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+attach_target_queue(Config) ->
+    QName = <<"test queue">>,
+    %% This target address means RabbitMQ will create a queue
+    %% requiring configure access on the queue.
+    %% We will also need write access to the default exchange to send to this queue.
+    TargetAddress = <<"/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Sender1} = amqp10_client:attach_sender_link(
+                       Session1, <<"test-sender-1">>, TargetAddress),
+    ExpectedErr1 = error_unauthorized(
+                     <<"configure access to queue 'test queue' in vhost "
+                       "'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session1, {ended, ExpectedErr1}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure permissions on the queue.
+    ok = set_permissions(Config, QName, <<>>, <<>>),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Sender2} = amqp10_client:attach_sender_link(
+                       Session2, <<"test-sender-2">>, TargetAddress),
+    ExpectedErr2 = error_unauthorized(
+                     <<"write access to exchange 'amq.default' in vhost "
+                       "'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session2, {ended, ExpectedErr2}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure permissions on the queue and
+    %% write access to the default exchange.
+    ok = set_permissions(Config, QName, <<"amq\.default">>, <<>>),
+    {ok, Session3} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender3} = amqp10_client:attach_sender_link(
+                      Session3, <<"test-sender-3">>, TargetAddress),
+    receive {amqp10_event, {link, Sender3, attached}} -> ok
+    after 5000 -> flush(missing_attached),
+                  ct:fail("missing ATTACH from server")
+    end,
+
+    ok = close_connection_sync(Connection).
+
+attach_source_exchange(Config) ->
+    %% This source address means RabbitMQ will create a queue with a generated name
+    %% prefixed with amq.gen requiring configure access on the queue.
+    %% The queue is bound to the fanout exchange requiring write access on the queue
+    %% and read access on the fanout exchange.
+    %% To consume from the queue, we will also need read access on the queue.
+    SourceAddress = <<"/exchange/amq.fanout/ignored">>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Recv1} = amqp10_client:attach_receiver_link(
+                     Session1, <<"receiver-1">>, SourceAddress),
+    receive
+        {amqp10_event,
+         {session, Session1,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+              description = {utf8, <<"configure access to queue 'amq.gen", _/binary>>}}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure permissions on the queue.
+    ok = set_permissions(Config, <<"^amq\.gen">>, <<>>, <<>>),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Recv2} = amqp10_client:attach_receiver_link(
+                     Session2, <<"receiver-2">>, SourceAddress),
+    receive
+        {amqp10_event,
+         {session, Session2,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+              description = {utf8, <<"write access to queue 'amq.gen", _/binary>>}}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure and write permissions on the queue.
+    ok = set_permissions(Config, <<"^amq\.gen">>, <<"^amq\.gen">>, <<>>),
+    {ok, Session3} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Recv3} = amqp10_client:attach_receiver_link(
+                     Session3, <<"receiver-3">>, SourceAddress),
+    ExpectedErr1 = error_unauthorized(
+                     <<"read access to exchange 'amq.fanout' in vhost "
+                       "'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session3, {ended, ExpectedErr1}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure and write permissions on the queue, and read access on the exchange.
+    ok = set_permissions(Config, <<"^amq\.gen">>, <<"^amq\.gen">>, <<"amq\.fanout">>),
+    {ok, Session4} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Recv4} = amqp10_client:attach_receiver_link(
+                     Session4, <<"receiver-4">>, SourceAddress),
+    receive
+        {amqp10_event,
+         {session, Session4,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+              description = {utf8, <<"read access to queue 'amq.gen", _/binary>>}}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    %% Give the user configure, write, and read permissions on the queue,
+    %% and read access on the exchange.
+    ok = set_permissions(Config, <<"^amq\.gen">>, <<"^amq\.gen">>, <<"^(amq\.gen|amq\.fanout)">>),
+    {ok, Session5} = amqp10_client:begin_session_sync(Connection),
+    {ok, Recv5} = amqp10_client:attach_receiver_link(
+                    Session5, <<"receiver-5">>, SourceAddress),
+    receive {amqp10_event, {link, Recv5, attached}} -> ok
+    after 5000 -> flush(missing_attached),
+                  ct:fail("missing ATTACH from server")
+    end,
+
+    ok = close_connection_sync(Connection).
+
+send_to_topic(Config) ->
+    TargetAddresses = [<<"/topic/test vhost.test user.a.b">>,
+                       <<"/exchange/amq.topic/test vhost.test user.a.b">>],
+    lists:foreach(fun(Address) ->
+                          ok = send_to_topic0(Address, Config)
+                  end, TargetAddresses).
+
+send_to_topic0(TargetAddress, Config) ->
+    User = ?config(test_user, Config),
+    Vhost = ?config(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, Vhost),
+    ok = set_topic_permissions(Config, <<"amq.topic">>, <<"^$">>, <<"^$">>),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender1} = amqp10_client:attach_sender_link_sync(
+                      Session1, <<"sender-1">>, TargetAddress),
+    ok = wait_for_credit(Sender1),
+    Msg1 = amqp10_msg:new(<<255>>, <<1>>, true),
+    ok = amqp10_client:send_msg(Sender1, Msg1),
+
+    ExpectedErr = error_unauthorized(
+                    <<"write access to topic 'test vhost.test user.a.b' in exchange "
+                      "'amq.topic' in vhost 'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session1, {ended, ExpectedErr}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    ok = set_topic_permissions(Config, <<"amq.topic">>, <<"^{vhost}\.{username}\.a\.b$">>, <<"^$">>),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender2} = amqp10_client:attach_sender_link_sync(
+                      Session2, <<"sender-2">>, TargetAddress),
+    ok = wait_for_credit(Sender2),
+    Dtag = <<0, 0>>,
+    Msg2 = amqp10_msg:new(Dtag, <<2>>, false),
+    ok = amqp10_client:send_msg(Sender2, Msg2),
+    %% We expect RELEASED since no queue is bound.
+    receive {amqp10_disposition, {released, Dtag}} -> ok
+    after 5000 -> ct:fail(released_timeout)
+    end,
+
+    ok = amqp10_client:detach_link(Sender2),
+    ok = close_connection_sync(Connection).
+
+send_to_topic_using_subject(Config) ->
+    TargetAddress = <<"/exchange/amq.topic">>,
+    User = ?config(test_user, Config),
+    Vhost = ?config(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, Vhost),
+    ok = set_topic_permissions(Config, <<"amq.topic">>, <<"^\.a$">>, <<"^$">>),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link_sync(
+                     Session, <<"sender">>, TargetAddress),
+    ok = wait_for_credit(Sender),
+
+    Dtag1 = <<"dtag 1">>,
+    Msg1a = amqp10_msg:new(Dtag1, <<"m1">>, false),
+    Msg1b = amqp10_msg:set_properties(#{subject => <<".a">>}, Msg1a),
+    ok = amqp10_client:send_msg(Sender, Msg1b),
+    %% We have sufficient authorization, but expect RELEASED since no queue is bound.
+    receive {amqp10_disposition, {released, Dtag1}} -> ok
+    after 5000 -> ct:fail(released_timeout)
+    end,
+
+    Dtag2 = <<"dtag 2">>,
+    Msg2a = amqp10_msg:new(Dtag2, <<"m2">>, false),
+    %% We don't have sufficient authorization.
+    Msg2b = amqp10_msg:set_properties(#{subject => <<".a.b">>}, Msg2a),
+    ok = amqp10_client:send_msg(Sender, Msg2b),
+    ExpectedErr = error_unauthorized(
+                    <<"write access to topic '.a.b' in exchange 'amq.topic' in "
+                      "vhost 'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session, {ended, ExpectedErr}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    ok = close_connection_sync(Connection).
+
+attach_source_topic(Config) ->
+    %% These source addresses mean RabbitMQ will bind a queue to the default topic
+    %% exchange with binding key 'test vhost.test user.a.b'.
+    %% Therefore, we need read access to that topic.
+    %% We also test variable expansion in topic permission patterns.
+    SourceAddresses = [<<"/topic/test vhost.test user.a.b">>,
+                       <<"/exchange/amq.topic/test vhost.test user.a.b">>],
+    lists:foreach(fun(Address) ->
+                          ok = attach_source_topic0(Address, Config)
+                  end, SourceAddresses).
+
+attach_source_topic0(SourceAddress, Config) ->
+    User = ?config(test_user, Config),
+    Vhost = ?config(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:set_full_permissions(Config, User, Vhost),
+    ok = set_topic_permissions(Config, <<"amq.topic">>, <<"^$">>, <<"^$">>),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection),
+    {ok, _Recv1} = amqp10_client:attach_receiver_link(
+                     Session1, <<"receiver-1">>, SourceAddress),
+    ExpectedErr = error_unauthorized(
+                    <<"read access to topic 'test vhost.test user.a.b' in exchange "
+                      "'amq.topic' in vhost 'test vhost' refused for user 'test user'">>),
+    receive {amqp10_event, {session, Session1, {ended, ExpectedErr}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    ok = set_topic_permissions(Config, <<"amq.topic">>, <<"^$">>, <<"^{vhost}\.{username}\.a\.b$">>),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection),
+    {ok, Recv2} = amqp10_client:attach_receiver_link(
+                    Session2, <<"receiver-2">>, SourceAddress),
+    receive {amqp10_event, {link, Recv2, attached}} -> ok
+    after 5000 -> flush(missing_attached),
+                  ct:fail("missing ATTACH from server")
+    end,
+
+    ok = close_connection_sync(Connection).
+
+attach_target_internal_exchange(Config) ->
+    XName = <<"test exchange">>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'exchange.declare_ok'{} =  amqp_channel:call(Ch, #'exchange.declare'{internal = true,
+                                                                          exchange = XName}),
+
+    OpnConf0 = connection_config(Config, <<"/">>),
+    OpnConf = OpnConf0#{sasl := anon},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/", XName/binary, "/some-routing-key">>,
+    {ok, _} = amqp10_client:attach_sender_link(
+                Session, <<"test-sender">>, Address),
+    ExpectedErr = error_unauthorized(
+                    <<"attach to internal exchange 'test exchange' in vhost '/' is forbidden">>),
+    receive {amqp10_event, {session, Session, {ended, ExpectedErr}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive AMQP_ERROR_UNAUTHORIZED_ACCESS")
+    end,
+
+    ok = amqp10_client:close_connection(Connection),
+    #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = XName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+authn_failure_event(Config) ->
+    ok = event_recorder:start(Config),
+
+    Host = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    Vhost = ?config(test_vhost, Config),
+    User = ?config(test_user, Config),
+    OpnConf = #{address => Host,
+                port => Port,
+                container_id => <<"my container">>,
+                sasl => {plain, User, <<"wrong password">>},
+                hostname => <<"vhost:", Vhost/binary>>},
+
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, {closed, sasl_auth_failure}}} -> ok
+    after 5000 -> flush(missing_closed),
+                  ct:fail("did not receive sasl_auth_failure")
+    end,
+
+    [E | _] = event_recorder:get_events(Config),
+    ok = event_recorder:stop(Config),
+
+    assert_event_type(user_authentication_failure, E),
+    assert_event_prop([{name, <<"test user">>},
+                       {auth_mechanism, <<"PLAIN">>},
+                       {ssl, false},
+                       {protocol, {1, 0}}],
+                      E).
+
+sasl_anonymous_success(Config) ->
+    Mechanism = anon,
+    ok = sasl_success(Mechanism, Config).
+
+sasl_none_success(Config) ->
+    Mechanism = none,
+    ok = sasl_success(Mechanism, Config).
+
+sasl_plain_success(Config) ->
+    Mechanism = {plain, <<"guest">>, <<"guest">>},
+    ok = sasl_success(Mechanism, Config).
+
+sasl_success(Mechanism, Config) ->
+    OpnConf0 = connection_config(Config, <<"/">>),
+    OpnConf = OpnConf0#{sasl := Mechanism},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail(missing_opened)
+    end,
+    ok = amqp10_client:close_connection(Connection).
+
+sasl_anonymous_failure(Config) ->
+    Mechanism = anon,
+    ?assertEqual(
+       {sasl_not_supported, Mechanism},
+       sasl_failure(Mechanism, Config)
+      ).
+
+sasl_none_failure(Config) ->
+    Mechanism = none,
+    sasl_failure(Mechanism, Config).
+
+sasl_plain_failure(Config) ->
+    Mechanism = {plain, <<"guest">>, <<"wrong password">>},
+    ?assertEqual(
+       sasl_auth_failure,
+       sasl_failure(Mechanism, Config)
+      ).
+
+sasl_failure(Mechanism, Config) ->
+    App = rabbit,
+    Par = amqp1_0_default_user,
+    {ok, Default} = rpc(Config, application, get_env, [App, Par]),
+    ok = rpc(Config, application, set_env, [App, Par, none]),
+
+    OpnConf0 = connection_config(Config, <<"/">>),
+    OpnConf = OpnConf0#{sasl := Mechanism},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    Reason = receive {amqp10_event, {connection, Connection, {closed, Reason0}}} -> Reason0
+             after 5000 -> ct:fail(missing_closed)
+             end,
+
+    ok = rpc(Config, application, set_env, [App, Par, Default]),
+    Reason.
+
+vhost_absent(Config) ->
+    OpnConf = connection_config(Config, <<"vhost does not exist">>),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, {closed, _}}} -> ok
+    after 5000 -> ct:fail(missing_closed)
+    end.
+
+vhost_connection_limit(Config) ->
+    Vhost = proplists:get_value(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, Vhost, max_connections, 1),
+
+    OpnConf = connection_config(Config),
+    {ok, C1} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, C1, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    {ok, C2} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, C2, {closed, _}}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    OpnConf0 = connection_config(Config, <<"/">>),
+    OpnConf1 = OpnConf0#{sasl := anon},
+    {ok, C3} = amqp10_client:open_connection(OpnConf1),
+    receive {amqp10_event, {connection, C3, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    {ok, C4} = amqp10_client:open_connection(OpnConf1),
+    receive {amqp10_event, {connection, C4, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    [ok = close_connection_sync(C) || C <- [C1, C3, C4]],
+    ok = rabbit_ct_broker_helpers:clear_vhost_limit(Config, 0, Vhost).
+
+user_connection_limit(Config) ->
+    DefaultUser = <<"guest">>,
+    Limit = max_connections,
+    ok = rabbit_ct_broker_helpers:set_user_limits(Config, DefaultUser, #{Limit => 0}),
+    OpnConf0 = connection_config(Config, <<"/">>),
+    OpnConf = OpnConf0#{sasl := anon},
+    {ok, C1} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, C1, {closed, _}}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    {ok, C2} = amqp10_client:open_connection(connection_config(Config)),
+    receive {amqp10_event, {connection, C2, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    ok = close_connection_sync(C2),
+    ok = rabbit_ct_broker_helpers:clear_user_limits(Config, DefaultUser, Limit).
+
+vhost_queue_limit(Config) ->
+    Vhost = proplists:get_value(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, Vhost, max_queues, 0),
+    QName = <<"q1">>,
+    ok = set_permissions(Config, QName, <<>>, <<>>),
+
+    OpnConf1 = connection_config(Config),
+    {ok, C1} = amqp10_client:open_connection(OpnConf1),
+    {ok, Session1} = amqp10_client:begin_session_sync(C1),
+    TargetAddress = <<"/queue/", QName/binary>>,
+    {ok, _Sender1} = amqp10_client:attach_sender_link(
+                       Session1, <<"test-sender-1">>, TargetAddress),
+    ExpectedErr = amqp_error(
+                    ?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED,
+                    <<"cannot declare queue 'q1' in vhost 'test vhost': vhost queue limit (0) is reached">>),
+    receive {amqp10_event, {session, Session1, {ended, ExpectedErr}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive expected error")
+    end,
+
+    OpnConf2 = connection_config(Config, <<"/">>),
+    OpnConf3 = OpnConf2#{sasl := anon},
+    {ok, C2} = amqp10_client:open_connection(OpnConf3),
+    {ok, Session2} = amqp10_client:begin_session_sync(C2),
+    {ok, Sender2} = amqp10_client:attach_sender_link(
+                      Session2, <<"test-sender-2">>, TargetAddress),
+    receive {amqp10_event, {link, Sender2, attached}} -> ok
+    after 5000 -> flush(missing_attached),
+                  ct:fail("missing ATTACH from server")
+    end,
+
+    ok = close_connection_sync(C1),
+    ok = close_connection_sync(C2),
+    ok = rabbit_ct_broker_helpers:clear_vhost_limit(Config, 0, Vhost).
+
+connection_config(Config) ->
+    Vhost = ?config(test_vhost, Config),
+    connection_config(Config, Vhost).
+
+connection_config(Config, Vhost) ->
+    Host = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    User = Password = ?config(test_user, Config),
+    #{address => Host,
+      port => Port,
+      container_id => <<"my container">>,
+      sasl => {plain, User, Password},
+      hostname => <<"vhost:", Vhost/binary>>}.
+
+set_permissions(Config, ConfigurePerm, WritePerm, ReadPerm) ->
+    ok = rabbit_ct_broker_helpers:set_permissions(Config,
+                                                  ?config(test_user, Config),
+                                                  ?config(test_vhost, Config),
+                                                  ConfigurePerm,
+                                                  WritePerm,
+                                                  ReadPerm).
+
+set_topic_permissions(Config, Exchange, WritePat, ReadPat) ->
+    ok = rpc(Config,
+             rabbit_auth_backend_internal,
+             set_topic_permissions,
+             [?config(test_user, Config),
+              ?config(test_vhost, Config),
+              Exchange,
+              WritePat,
+              ReadPat,
+              <<"acting-user">>]).
+
+clear_permissions(Config) ->
+    User = ?config(test_user, Config),
+    Vhost = ?config(test_vhost, Config),
+    ok = rabbit_ct_broker_helpers:clear_permissions(Config, User, Vhost),
+    ok = rpc(Config,
+             rabbit_auth_backend_internal,
+             clear_topic_permissions,
+             [User, Vhost, <<"acting-user">>]).
+
+error_unauthorized(Description) ->
+    amqp_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, Description).
+
+amqp_error(Condition, Description)
+  when is_binary(Description) ->
+    #'v1_0.error'{
+       condition = Condition,
+       description = {utf8, Description}}.
+
+% before we can send messages we have to wait for credit from the server
+wait_for_credit(Sender) ->
+    receive
+        {amqp10_event, {link, Sender, credited}} ->
+            flush(?FUNCTION_NAME),
+            ok
+    after 5000 ->
+              flush("wait_for_credit timed out"),
+              ct:fail(credited_timeout)
+    end.
+
+flush(Prefix) ->
+    receive Msg ->
+                ct:pal("~ts flushed: ~p~n", [Prefix, Msg]),
+                flush(Prefix)
+    after 1 ->
+              ok
+    end.
+
+delete_all_queues(Config) ->
+    Qs = rpc(Config, rabbit_amqqueue, list, []),
+    [{ok, _QLen} = rpc(Config, rabbit_amqqueue, delete, [Q, false, false, <<"fake-user">>])
+     || Q <- Qs].
+
+close_connection_sync(Connection)
+  when is_pid(Connection) ->
+    ok = amqp10_client:close_connection(Connection),
+    receive {amqp10_event, {connection, Connection, {closed, normal}}} -> ok
+    after 5000 -> flush(missing_closed),
+                  ct:fail("missing CLOSE from server")
+    end.
diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl
new file mode 100644
index 000000000000..98c67da26336
--- /dev/null
+++ b/deps/rabbit/test/amqp_client_SUITE.erl
@@ -0,0 +1,3654 @@
+%% 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-2023 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries.  All rights reserved.
+%%
+
+-module(amqp_client_SUITE).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("amqp10_common/include/amqp10_framing.hrl").
+
+-compile([nowarn_export_all,
+          export_all]).
+
+-import(rabbit_ct_broker_helpers,
+        [get_node_config/3,
+         rpc/4,
+         rpc/5,
+         drain_node/2,
+         revive_node/2
+        ]).
+-import(rabbit_ct_helpers,
+        [eventually/1, eventually/3]).
+-import(event_recorder,
+        [assert_event_type/2,
+         assert_event_prop/2]).
+
+all() ->
+    [
+      {group, cluster_size_1},
+      {group, cluster_size_3},
+      {group, metrics}
+    ].
+
+groups() ->
+    [
+     {cluster_size_1, [shuffle],
+      [
+       reliable_send_receive_with_outcomes_classic_queue,
+       reliable_send_receive_with_outcomes_quorum_queue,
+       sender_settle_mode_unsettled,
+       sender_settle_mode_unsettled_fanout,
+       sender_settle_mode_mixed,
+       quorum_queue_rejects,
+       receiver_settle_mode_first,
+       publishing_to_non_existing_queue_should_settle_with_released,
+       open_link_to_non_existing_destination_should_end_session,
+       roundtrip_with_drain_classic_queue,
+       roundtrip_with_drain_quorum_queue,
+       roundtrip_with_drain_stream,
+       amqp_stream_amqpl,
+       message_headers_conversion,
+       multiple_sessions,
+       server_closes_link_classic_queue,
+       server_closes_link_quorum_queue,
+       server_closes_link_stream,
+       server_closes_link_exchange,
+       link_target_classic_queue_deleted,
+       link_target_quorum_queue_deleted,
+       target_queues_deleted_accepted,
+       no_routing_key,
+       events,
+       sync_get_unsettled_classic_queue,
+       sync_get_unsettled_quorum_queue,
+       sync_get_unsettled_stream,
+       sync_get_unsettled_2_classic_queue,
+       sync_get_unsettled_2_quorum_queue,
+       sync_get_unsettled_2_stream,
+       sync_get_settled_classic_queue,
+       sync_get_settled_quorum_queue,
+       sync_get_settled_stream,
+       timed_get_classic_queue,
+       timed_get_quorum_queue,
+       timed_get_stream,
+       stop_classic_queue,
+       stop_quorum_queue,
+       stop_stream,
+       single_active_consumer_classic_queue,
+       single_active_consumer_quorum_queue,
+       detach_requeues,
+       resource_alarm_before_session_begin,
+       resource_alarm_after_session_begin,
+       max_message_size_client_to_server,
+       max_message_size_server_to_client,
+       receive_transfer_flow_order,
+       global_counters,
+       stream_filtering,
+       available_messages_classic_queue,
+       available_messages_quorum_queue,
+       available_messages_stream,
+       incoming_message_interceptors,
+       trace,
+       user_id,
+       message_ttl,
+       plugin,
+       idle_time_out_on_server,
+       idle_time_out_on_client,
+       idle_time_out_too_short,
+       rabbit_status_connection_count,
+       handshake_timeout,
+       credential_expires,
+       attach_to_exclusive_queue,
+       classic_priority_queue
+      ]},
+
+     {cluster_size_3, [shuffle],
+      [
+       last_queue_confirms,
+       target_queue_deleted,
+       target_classic_queue_down,
+       async_notify_settled_classic_queue,
+       async_notify_settled_quorum_queue,
+       async_notify_settled_stream,
+       async_notify_unsettled_classic_queue,
+       async_notify_unsettled_quorum_queue,
+       async_notify_unsettled_stream,
+       link_flow_control,
+       classic_queue_on_old_node,
+       classic_queue_on_new_node,
+       quorum_queue_on_old_node,
+       quorum_queue_on_new_node,
+       maintenance,
+       list_connections
+      ]},
+
+     {metrics, [shuffle],
+      [
+       auth_attempt_metrics
+      ]}
+    ].
+
+%% -------------------------------------------------------------------
+%% Testsuite setup/teardown.
+%% -------------------------------------------------------------------
+
+init_per_suite(Config) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    Config.
+
+end_per_suite(Config) ->
+    Config.
+
+init_per_group(Group, Config) ->
+    Nodes = case Group of
+                cluster_size_3 -> 3;
+                _ -> 1
+            end,
+    Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
+    Config1 = rabbit_ct_helpers:set_config(
+                Config, [{rmq_nodes_count, Nodes},
+                         {rmq_nodename_suffix, Suffix}]),
+    rabbit_ct_helpers:run_setup_steps(
+      Config1,
+      rabbit_ct_broker_helpers:setup_steps() ++
+      rabbit_ct_client_helpers:setup_steps()).
+
+end_per_group(_, Config) ->
+    rabbit_ct_helpers:run_teardown_steps(Config,
+      rabbit_ct_client_helpers:teardown_steps() ++
+      rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(T, Config)
+  when T =:= message_headers_conversion orelse
+       T =:= roundtrip_with_drain_quorum_queue orelse
+       T =:= timed_get_quorum_queue orelse
+       T =:= available_messages_quorum_queue ->
+    case rabbit_ct_broker_helpers:rpc(
+           Config, rabbit_feature_flags, is_enabled, [credit_api_v2]) of
+        true ->
+            rabbit_ct_helpers:testcase_started(Config, T);
+        false ->
+            {skip, "Receiving with drain from quorum queues in credit API v1 have a known "
+             "bug that they reply with send_drained before delivering the message."}
+    end;
+init_per_testcase(T, Config)
+  when T =:= classic_queue_on_new_node orelse
+       T =:= quorum_queue_on_new_node ->
+    %% If node 1 runs 4.x, this is the new no-op plugin.
+    %% If node 1 runs 3.x, this is the old real plugin.
+    ok = rabbit_ct_broker_helpers:enable_plugin(Config, 1, rabbitmq_amqp1_0),
+    rabbit_ct_helpers:testcase_started(Config, T);
+init_per_testcase(Testcase, Config) ->
+    rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config) ->
+    %% Assert that every testcase cleaned up.
+    eventually(?_assertEqual([], rpc(Config, rabbit_amqqueue, list, []))),
+    %% Assert that global counters count correctly.
+    eventually(?_assertMatch(#{publishers := 0,
+                               consumers := 0},
+                             get_global_counters(Config))),
+    rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+reliable_send_receive_with_outcomes_classic_queue(Config) ->
+    reliable_send_receive_with_outcomes(<<"classic">>, Config).
+
+reliable_send_receive_with_outcomes_quorum_queue(Config) ->
+    reliable_send_receive_with_outcomes(<<"quorum">>, Config).
+
+reliable_send_receive_with_outcomes(QType, Config) ->
+    Outcomes = [
+                accepted,
+                modified,
+                {modified, true, false, #{<<"fruit">> => <<"banana">>}},
+                {modified, false, true, #{}},
+                rejected,
+                released
+               ],
+    [ok = reliable_send_receive(QType, Outcome, Config) || Outcome <- Outcomes].
+
+reliable_send_receive(QType, Outcome, Config) ->
+    OutcomeBin = if is_atom(Outcome) ->
+                        atom_to_binary(Outcome);
+                    is_tuple(Outcome) ->
+                        O1 = atom_to_binary(element(1, Outcome)),
+                        O2 = atom_to_binary(element(2, Outcome)),
+                        <>
+                 end,
+    ct:pal("~s testing ~s", [?FUNCTION_NAME, OutcomeBin]),
+
+    QName = <>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    %% reliable send and consume
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    flush(credited),
+    DTag1 = <<"dtag-1">>,
+    %% create an unsettled message,
+    %% link will be in "mixed" mode by default
+    Body = <<"body-1">>,
+    Msg1 = amqp10_msg:new(DTag1, Body, false),
+    ok = amqp10_client:send_msg(Sender, Msg1),
+    ok = wait_for_settlement(DTag1),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    flush("post sender close"),
+
+    {ok, Connection2} = amqp10_client:open_connection(OpnConf),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection2),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session2, <<"test-receiver">>, Address, unsettled),
+    {ok, Msg} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([Body], amqp10_msg:body(Msg)),
+
+    ok = amqp10_client:settle_msg(Receiver, Msg, Outcome),
+    flush("post accept"),
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session2),
+    ok = amqp10_client:close_connection(Connection2),
+    ok = delete_queue(Config, QName).
+
+%% Tests that confirmations are returned correctly
+%% when sending many messages async to a quorum queue.
+sender_settle_mode_unsettled(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+
+    %% Send many messages aync.
+    NumMsgs = 30,
+    DTags = [begin
+                 DTag = integer_to_binary(N),
+                 Msg = amqp10_msg:new(DTag, <<"body">>, false),
+                 ok = amqp10_client:send_msg(Sender, Msg),
+                 DTag
+             end  || N <- lists:seq(1, NumMsgs)],
+
+    %% Wait for confirms.
+    [receive {amqp10_disposition, {accepted, DTag}} -> ok
+     after 5000 -> ct:fail({missing_accepted, DTag})
+     end || DTag <- DTags],
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ?assertEqual(#'queue.delete_ok'{message_count = NumMsgs},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QName})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+sender_settle_mode_unsettled_fanout(Config) ->
+    QNames = [<<"q1">>, <<"q2">>, <<"q3">>],
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    [begin
+         #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+         #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = QName,
+                                                                  exchange = <<"amq.fanout">>})
+     end || QName <- QNames],
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/amq.fanout/ignored">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+
+    %% Send many messages aync.
+    NumMsgs = 20,
+    DTags = [begin
+                 DTag = integer_to_binary(N),
+                 Msg = amqp10_msg:new(DTag, <<"body">>, false),
+                 ok = amqp10_client:send_msg(Sender, Msg),
+                 DTag
+             end  || N <- lists:seq(1, NumMsgs)],
+
+    %% Wait for confirms.
+    [receive {amqp10_disposition, {accepted, DTag}} -> ok
+     after 5000 -> ct:fail({missing_accepted, DTag})
+     end || DTag <- DTags],
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    [?assertEqual(#'queue.delete_ok'{message_count = NumMsgs},
+                  amqp_channel:call(Ch, #'queue.delete'{queue = QName})) ||
+     QName <- QNames],
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+%% Tests that confirmations are returned correctly
+%% when sending many messages async to a quorum queue where
+%% every 3rd message is settled by the sender.
+sender_settle_mode_mixed(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, mixed),
+    ok = wait_for_credit(Sender),
+
+    %% Send many messages aync.
+    NumMsgs = 30,
+    DTags = lists:filtermap(
+              fun(N) ->
+                      DTag = integer_to_binary(N),
+                      {Settled, Ret} = case N rem 3 of
+                                           0 -> {true, false};
+                                           _ -> {false, {true, DTag}}
+                                       end,
+                      Msg = amqp10_msg:new(DTag, <<"body">>, Settled),
+                      ok = amqp10_client:send_msg(Sender, Msg),
+                      Ret
+              end,  lists:seq(1, NumMsgs)),
+    20 = length(DTags),
+
+    %% Wait for confirms.
+    [receive {amqp10_disposition, {accepted, DTag}} -> ok
+     after 5000 -> ct:fail({missing_accepted, DTag})
+     end || DTag <- DTags],
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ?assertEqual(#'queue.delete_ok'{message_count = NumMsgs},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QName})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+quorum_queue_rejects(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>},
+                                                  {<<"x-max-length">>, long, 1},
+                                                  {<<"x-overflow">>, longstr, <<"reject-publish">>}
+                                                 ]}),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, mixed),
+    ok = wait_for_credit(Sender),
+
+    %% Quorum queue's x-max-length limit is known to be off by 1.
+    %% Therefore, we expect the first 2 messages to be accepted.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag a">>, <<>>, false)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag b">>, <<>>, false)),
+    [receive {amqp10_disposition, {accepted, DTag}} -> ok
+     after 5000 -> ct:fail({missing_accepted, DTag})
+     end || DTag <- [<<"tag a">>, <<"tag b">>]],
+
+    %% From now on the quorum queue should reject our publishes.
+    %% Send many messages aync.
+    NumMsgs = 20,
+    DTags = [begin
+                 DTag = integer_to_binary(N),
+                 Msg = amqp10_msg:new(DTag, <<"body">>, false),
+                 ok = amqp10_client:send_msg(Sender, Msg),
+                 DTag
+             end  || N <- lists:seq(1, NumMsgs)],
+    %% Since our sender settle mode is mixed, let's also test sending one as settled.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag c">>, <<>>, true)),
+    %% and the final one as unsettled again
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag d">>, <<>>, false)),
+
+    [receive {amqp10_disposition, {rejected, DTag}} -> ok
+     after 5000 -> ct:fail({missing_rejected, DTag})
+     end || DTag <- DTags ++ [<<"tag d">>]],
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:end_session(Session),
+    ok = amqp10_client:close_connection(Connection),
+
+    ?assertEqual(#'queue.delete_ok'{message_count = 2},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QName})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+receiver_settle_mode_first(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, settled),
+    ok = wait_for_credit(Sender),
+
+    %% Send 10 messages.
+    [begin
+         DTag = Body = integer_to_binary(N),
+         Msg = amqp10_msg:new(DTag, Body, true),
+         ok = amqp10_client:send_msg(Sender, Msg)
+     end  || N <- lists:seq(1, 10)],
+    ok = amqp10_client:detach_link(Sender),
+    flush("post sender close"),
+
+    %% Receive the first 9 messages.
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, unsettled),
+    ok = amqp10_client:flow_link_credit(Receiver, 9, never),
+    Msgs_1_to_9 = receive_messages(Receiver, 9),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    assert_messages(QName, 10, 9, Config),
+
+    %% What follows is white box testing: We want to hit a few different branches in the
+    %% server code. Although this test is a bit artificial, the AMQP spec does not prohibit
+    %% clients to ack in such ranges.
+
+    %% 1. Ack a range smaller than the number of unacked messages where some delivery IDs
+    %% are outside the [min, max] range of unacked messages.
+    {Msgs_1_to_7, [Msg8, Msg9]} = lists:split(7, Msgs_1_to_9),
+    DeliveryIdMsg8 = amqp10_msg:delivery_id(Msg8),
+    DeliveryIdMsg9 = amqp10_msg:delivery_id(Msg9),
+    ?assertEqual(DeliveryIdMsg9, serial_number_increment(DeliveryIdMsg8)),
+    Last1 = serial_number_increment(serial_number_increment(DeliveryIdMsg9)),
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg8, Last1, true, accepted),
+    assert_messages(QName, 8, 7, Config),
+
+    %% 2. Ack a range smaller than the number of unacked messages where all delivery IDs
+    %% are inside the [min, max] range of unacked messages.
+    [Msg1, Msg2, _Msg3, Msg4, _Msg5, Msg6, Msg7] = Msgs_1_to_7,
+    DeliveryIdMsg4 = amqp10_msg:delivery_id(Msg4),
+    DeliveryIdMsg6 = amqp10_msg:delivery_id(Msg6),
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg4, DeliveryIdMsg6, true, accepted),
+    assert_messages(QName, 5, 4, Config),
+
+    %% 3. Ack a range larger than the number of unacked messages where all delivery IDs
+    %% are inside the [min, max] range of unacked messages.
+    DeliveryIdMsg2 = amqp10_msg:delivery_id(Msg2),
+    DeliveryIdMsg7 = amqp10_msg:delivery_id(Msg7),
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg2, DeliveryIdMsg7, true, accepted),
+    assert_messages(QName, 2, 1, Config),
+
+    %% Consume the last message.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    [Msg10] = receive_messages(Receiver, 1),
+    ?assertEqual([<<"10">>], amqp10_msg:body(Msg10)),
+
+    %% 4. Ack a range larger than the number of unacked messages where some delivery IDs
+    %% are outside the [min, max] range of unacked messages.
+    DeliveryIdMsg1 = amqp10_msg:delivery_id(Msg1),
+    DeliveryIdMsg10 = amqp10_msg:delivery_id(Msg10),
+    Last2 = serial_number_increment(DeliveryIdMsg10),
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg1, Last2, true, accepted),
+    assert_messages(QName, 0, 0, Config),
+
+    %% 5. Ack single delivery ID when there are no unacked messages.
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg1, DeliveryIdMsg1, true, accepted),
+
+    %% 6. Ack multiple delivery IDs when there are no unacked messages.
+    ok = amqp10_client_session:disposition(
+           Session, receiver, DeliveryIdMsg1, DeliveryIdMsg6, true, accepted),
+    assert_messages(QName, 0, 0, Config),
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:end_session(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ok = delete_queue(Config, QName).
+
+publishing_to_non_existing_queue_should_settle_with_released(Config) ->
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    QName = <<"queue does not exist">>,
+    Address = <<"/exchange/amq.direct/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    DTag1 = <<"dtag-1">>,
+    %% create an unsettled message,
+    %% link will be in "mixed" mode by default
+    Msg1 = amqp10_msg:new(DTag1, <<"body-1">>, false),
+    ok = amqp10_client:send_msg(Sender, Msg1),
+    ok = wait_for_settlement(DTag1, released),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:close_connection(Connection),
+    ok = flush("post sender close").
+
+open_link_to_non_existing_destination_should_end_session(Config) ->
+    OpnConf = connection_config(Config),
+    Name = "non-existing-destination",
+    Addresses = ["/exchange/" ++ Name ++ "/bar",
+                 "/amq/queue/" ++ Name],
+    SenderLinkName = <<"test-sender">>,
+    [begin
+         {ok, Connection} = amqp10_client:open_connection(OpnConf),
+         {ok, Session} = amqp10_client:begin_session_sync(Connection),
+         ct:pal("Address ~p", [Address]),
+         {ok, _} = amqp10_client:attach_sender_link(
+                     Session, SenderLinkName, list_to_binary(Address)),
+         wait_for_session_end(Session),
+         ok = amqp10_client:close_connection(Connection),
+         flush("post sender close")
+     end || Address <- Addresses],
+    ok.
+
+roundtrip_with_drain_classic_queue(Config) ->
+    QName  = atom_to_binary(?FUNCTION_NAME),
+    roundtrip_with_drain(Config, <<"classic">>, QName).
+
+roundtrip_with_drain_quorum_queue(Config) ->
+    QName  = atom_to_binary(?FUNCTION_NAME),
+    roundtrip_with_drain(Config, <<"quorum">>, QName).
+
+roundtrip_with_drain_stream(Config) ->
+    QName  = atom_to_binary(?FUNCTION_NAME),
+    roundtrip_with_drain(Config, <<"stream">>, QName).
+
+roundtrip_with_drain(Config, QueueType, QName)
+  when is_binary(QueueType) ->
+    Address = <<"/amq/queue/", QName/binary>>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    Args = [{<<"x-queue-type">>, longstr, QueueType}],
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = Args}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    wait_for_credit(Sender),
+
+    OutMsg = amqp10_msg:new(<<"tag-1">>, <<"my-body">>, false),
+    ok = amqp10_client:send_msg(Sender, OutMsg),
+    ok = wait_for_accepts(1),
+
+    flush("pre-receive"),
+    % create a receiver link
+    TerminusDurability = none,
+    Filter = consume_from_first(QueueType),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, unsettled,
+                       TerminusDurability, Filter),
+
+    % grant credit and drain
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never, true),
+
+    % wait for a delivery
+    receive {amqp10_msg, Receiver, InMsg} ->
+                ok = amqp10_client:accept_msg(Receiver, InMsg)
+    after 2000 ->
+              Reason = delivery_timeout,
+              flush(Reason),
+              ct:fail(Reason)
+    end,
+    OutMsg2 = amqp10_msg:new(<<"tag-2">>, <<"my-body2">>, false),
+    ok = amqp10_client:send_msg(Sender, OutMsg2),
+    ok = wait_for_settlement(<<"tag-2">>),
+
+    %% no delivery should be made at this point
+    receive {amqp10_msg, _, _} -> ct:fail(unexpected_delivery)
+    after 500 -> ok
+    end,
+
+    flush("final"),
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:close_connection(Connection),
+    ok = delete_queue(Config, QName).
+
+%% Send a message with a body containing a single AMQP 1.0 value section
+%% to a stream and consume via AMQP 0.9.1.
+amqp_stream_amqpl(Config) ->
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    QName = atom_to_binary(?FUNCTION_NAME),
+
+    amqp_channel:call(Ch, #'queue.declare'{
+                             queue = QName,
+                             durable = true,
+                             arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}),
+
+    Address = <<"/amq/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    wait_for_credit(Sender),
+    OutMsg = amqp10_msg:new(<<"my-tag">>, {'v1_0.amqp_value', {binary, <<0, 255>>}}, true),
+    ok = amqp10_client:send_msg(Sender, OutMsg),
+    flush("final"),
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:close_connection(Connection),
+
+    #'basic.qos_ok'{} =  amqp_channel:call(Ch, #'basic.qos'{global = false,
+                                                            prefetch_count = 1}),
+    CTag = <<"my-tag">>,
+    #'basic.consume_ok'{} = amqp_channel:subscribe(
+                              Ch,
+                              #'basic.consume'{
+                                 queue = QName,
+                                 consumer_tag = CTag,
+                                 arguments = [{<<"x-stream-offset">>, longstr, <<"first">>}]},
+                              self()),
+    receive
+        {#'basic.deliver'{consumer_tag = CTag,
+                          redelivered  = false},
+         #amqp_msg{props = #'P_basic'{type = <<"amqp-1.0">>}}} ->
+            ok
+    after 5000 ->
+              exit(basic_deliver_timeout)
+    end,
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+message_headers_conversion(Config) ->
+    QName  = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/amq/queue/", QName/binary>>,
+    %% declare a quorum queue
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    amqp_channel:call(Ch, #'queue.declare'{
+                             queue = QName,
+                             durable = true,
+                             arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session(Connection),
+
+    amqp10_to_amqp091_header_conversion(Session, Ch, QName, Address),
+    amqp091_to_amqp10_header_conversion(Session, Ch, QName, Address),
+
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = delete_queue(Config, QName),
+    ok = amqp10_client:close_connection(Connection).
+
+amqp10_to_amqp091_header_conversion(Session,Ch, QName, Address) -> 
+    {ok, Sender} = create_amqp10_sender(Session, Address),
+
+    OutMsg1 = amqp10_msg:new(<<"my-tag">>, <<"my-body">>, false),
+    OutMsg2 = amqp10_msg:set_application_properties(
+                #{"string" => "string-val",
+                  "int" => 2,
+                  "bool" => false},
+                OutMsg1),
+    OutMsg3 = amqp10_msg:set_message_annotations(
+                #{"x-string" => "string-value",
+                  "x-int" => 3,
+                  "x-bool" => true},
+                OutMsg2),
+    ok = amqp10_client:send_msg(Sender, OutMsg3),
+    ok = wait_for_accepts(1),
+
+    {ok, Headers} = amqp091_get_msg_headers(Ch, QName),
+
+    ?assertEqual({longstr, <<"string-val">>}, rabbit_misc:table_lookup(Headers, <<"string">>)),
+    ?assertEqual({unsignedint, 2}, rabbit_misc:table_lookup(Headers, <<"int">>)),
+    ?assertEqual({bool, false}, rabbit_misc:table_lookup(Headers, <<"bool">>)),
+
+    ?assertEqual({longstr, <<"string-value">>}, rabbit_misc:table_lookup(Headers, <<"x-string">>)),
+    ?assertEqual({unsignedint, 3}, rabbit_misc:table_lookup(Headers, <<"x-int">>)),
+    ?assertEqual({bool, true}, rabbit_misc:table_lookup(Headers, <<"x-bool">>)).
+
+amqp091_to_amqp10_header_conversion(Session, Ch, QName, Address) -> 
+    Amqp091Headers = [{<<"x-forwarding">>, array, 
+                       [{table, [{<<"uri">>, longstr,
+                                  <<"amqp://localhost/%2F/upstream">>}]}]},
+                      {<<"x-string">>, longstr, "my-string"},
+                      {<<"x-int">>, long, 92},
+                      {<<"x-bool">>, bool, true},
+                      {<<"string">>, longstr, "my-str"},
+                      {<<"int">>, long, 101},
+                      {<<"bool">>, bool, false}],
+
+    amqp_channel:cast(
+      Ch,
+      #'basic.publish'{routing_key = QName},
+      #amqp_msg{props = #'P_basic'{headers = Amqp091Headers},
+                payload = <<"foobar">>}),
+
+    {ok, [Msg]} = drain_queue(Session, Address, 1),
+    Amqp10MA = amqp10_msg:message_annotations(Msg),
+    ?assertEqual(<<"my-string">>, maps:get(<<"x-string">>, Amqp10MA, undefined)),
+    ?assertEqual(92, maps:get(<<"x-int">>, Amqp10MA, undefined)),
+    ?assertEqual(true, maps:get(<<"x-bool">>, Amqp10MA, undefined)),
+
+    Amqp10Props = amqp10_msg:application_properties(Msg),
+    ?assertEqual(<<"my-str">>, maps:get(<<"string">>, Amqp10Props, undefined)),
+    ?assertEqual(101, maps:get(<<"int">>, Amqp10Props, undefined)),
+    ?assertEqual(false, maps:get(<<"bool">>, Amqp10Props, undefined)).
+
+%% Test sending and receiving concurrently on multiple sessions of the same connection.
+multiple_sessions(Config) ->
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    %% Create 2 sessions on the same connection.
+    {ok, Session1} = amqp10_client:begin_session(Connection),
+    {ok, Session2} = amqp10_client:begin_session(Connection),
+
+    %% Receive on each session.
+    Q1 = <<"q1">>,
+    Q2 = <<"q2">>,
+    Qs = [Q1, Q2],
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(
+                        Session1, <<"receiver link 1">>, Q1, settled, configuration),
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(
+                        Session2, <<"receiver link 2">>, Q2, settled, configuration),
+    receive {amqp10_event, {link, Receiver1, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    receive {amqp10_event, {link, Receiver2, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    NMsgsPerSender = 20,
+    NMsgsPerReceiver = NMsgsPerSender * 2, % due to fanout
+    ok = amqp10_client:flow_link_credit(Receiver1, NMsgsPerReceiver, never),
+    ok = amqp10_client:flow_link_credit(Receiver2, NMsgsPerReceiver, never),
+    flush("receiver attached"),
+
+    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
+    [#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = QName,
+                                                              exchange = <<"amq.fanout">>})
+     || QName <- Qs],
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    %% Send on each session.
+    TargetAddr = <<"/exchange/amq.fanout/ignored">>,
+    {ok, Sender1} = amqp10_client:attach_sender_link_sync(
+                      Session1, <<"sender link 1">>, TargetAddr, settled, configuration),
+    ok = wait_for_credit(Sender1),
+    {ok, Sender2} = amqp10_client:attach_sender_link_sync(
+                      Session2, <<"sender link 2">>, TargetAddr, settled, configuration),
+    ok = wait_for_credit(Sender2),
+
+    %% Send concurrently.
+    Group1 = <<"group 1">>,
+    Group2 = <<"group 2">>,
+    spawn_link(?MODULE, send_messages_with_group_id, [Sender1, NMsgsPerSender, Group1]),
+    spawn_link(?MODULE, send_messages_with_group_id, [Sender2, NMsgsPerSender, Group2]),
+
+    Q1Msgs = receive_messages(Receiver1, NMsgsPerReceiver),
+    Q2Msgs = receive_messages(Receiver2, NMsgsPerReceiver),
+    ExpectedBodies = [integer_to_binary(I) || I <- lists:seq(1, NMsgsPerSender)],
+    [begin
+         {G1Msgs, G2Msgs} = lists:partition(
+                              fun(Msg) ->
+                                      #{group_id := GroupId} = amqp10_msg:properties(Msg),
+                                      case GroupId of
+                                          Group1 -> true;
+                                          Group2 -> false
+                                      end
+                              end, Msgs),
+         [begin
+              Bodies = [begin
+                            [Bin] = amqp10_msg:body(M),
+                            Bin
+                        end || M <- GMsgs],
+              ?assertEqual(ExpectedBodies, Bodies)
+          end || GMsgs <- [G1Msgs, G2Msgs]]
+     end || Msgs <- [Q1Msgs, Q2Msgs]],
+
+    %% Clean up.
+    [ok = amqp10_client:detach_link(Link) || Link <- [Receiver1, Receiver2, Sender1, Sender2]],
+    ok = end_session_sync(Session1),
+    ok = end_session_sync(Session2),
+    ok = amqp10_client:close_connection(Connection),
+    [ok = delete_queue(Config, Q) || Q <- Qs].
+
+server_closes_link_classic_queue(Config) ->
+    server_closes_link(<<"classic">>, Config).
+
+server_closes_link_quorum_queue(Config) ->
+    server_closes_link(<<"quorum">>, Config).
+
+server_closes_link_stream(Config) ->
+    server_closes_link(<<"stream">>, Config).
+
+server_closes_link(QType, Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, unsettled),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing ATTACH frame from server")
+    end,
+    ok = amqp10_client:flow_link_credit(Receiver, 5, never),
+
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    flush(credited),
+    DTag = <<0>>,
+    Body = <<"body">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag, Body, false)),
+    ok = wait_for_settlement(DTag),
+
+    receive {amqp10_msg, Receiver, Msg} ->
+                ?assertEqual([Body], amqp10_msg:body(Msg))
+    after 5000 -> ct:fail("missing msg")
+    end,
+
+    [SessionPid] = rpc(Config, rabbit_amqp_session, list_local, []),
+    %% Received delivery is unsettled.
+    eventually(?_assertEqual(
+                  1,
+                  begin
+                      #{outgoing_unsettled_map := UnsettledMap} = gen_server_state(SessionPid),
+                      maps:size(UnsettledMap)
+                  end)),
+
+    %% Server closes the link endpoint due to some AMQP 1.0 external condition:
+    %% In this test, the external condition is that an AMQP 0.9.1 client deletes the queue.
+    delete_queue(Config, QName),
+
+    %% We expect that the server closes the link endpoints,
+    %% i.e. the server sends us DETACH frames.
+    ExpectedError = #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_RESOURCE_DELETED},
+    receive {amqp10_event, {link, Sender, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our outgoing link")
+    end,
+
+    receive {amqp10_event, {link, Receiver, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our incoming link")
+    end,
+
+    %% Our client has not and will not settle the delivery since the source queue got deleted and
+    %% the link detached with an error condition. Nevertheless the server session should clean up its
+    %% session state by removing the unsettled delivery from its session state.
+    eventually(?_assertEqual(
+                  0,
+                  begin
+                      #{outgoing_unsettled_map := UnsettledMap} = gen_server_state(SessionPid),
+                      maps:size(UnsettledMap)
+                  end)),
+
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+server_closes_link_exchange(Config) ->
+    XName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'exchange.declare_ok'{} = amqp_channel:call(Ch, #'exchange.declare'{exchange = XName}),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/", XName/binary, "/some-routing-key">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    ?assertMatch(#{publishers := 1}, get_global_counters(Config)),
+
+    %% Server closes the link endpoint due to some AMQP 1.0 external condition:
+    %% In this test, the external condition is that an AMQP 0.9.1 client deletes the exchange.
+    #'exchange.delete_ok'{} = amqp_channel:call(Ch, #'exchange.delete'{exchange = XName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    %% When we publish the next message, we expect:
+    %% 1. that the message is released because the exchange doesn't exist anymore, and
+    DTag = <<255>>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag, <<"body">>, false)),
+    ok = wait_for_settlement(DTag, released),
+    %% 2. that the server closes the link, i.e. sends us a DETACH frame.
+    ExpectedError = #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_RESOURCE_DELETED},
+    receive {amqp10_event, {link, Sender, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our outgoing link")
+    end,
+    ?assertMatch(#{publishers := 0}, get_global_counters(Config)),
+
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+link_target_classic_queue_deleted(Config) ->
+    link_target_queue_deleted(<<"classic">>, Config).
+
+link_target_quorum_queue_deleted(Config) ->
+    link_target_queue_deleted(<<"quorum">>, Config).
+
+link_target_queue_deleted(QType, Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    flush(credited),
+    DTag1 = <<1>>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag1, <<"m1">>, false)),
+    ok = wait_for_settlement(DTag1),
+
+    %% Mock delivery to the target queue to do nothing.
+    rabbit_ct_broker_helpers:setup_meck(Config, [?MODULE]),
+    Mod = rabbit_queue_type,
+    ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+    ok = rpc(Config, meck, expect, [Mod, deliver, fun ?MODULE:rabbit_queue_type_deliver_noop/4]),
+
+    %% Send 2nd message.
+    DTag2 = <<2>>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag2, <<"m2">>, false)),
+    receive {amqp10_disposition, Unexpected} -> ct:fail({unexpected_disposition, Unexpected})
+    after 200 -> ok
+    end,
+
+    %% Now, the server AMQP session contains a delivery that did not get confirmed by the target queue.
+    %% If we now delete that target queue, RabbitMQ must not reply to us with ACCEPTED.
+    %% Instead, we expect RabbitMQ to reply with RELEASED since no queue ever received our 2nd message.
+    delete_queue(Config, QName),
+    ok = wait_for_settlement(DTag2, released),
+
+    %% After the 2nd message got released, we additionally expect RabbitMQ to close the link given
+    %% that the target link endpoint - the queue - got deleted.
+    ExpectedError = #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_RESOURCE_DELETED},
+    receive {amqp10_event, {link, Sender, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our outgoing link")
+    end,
+
+    ?assert(rpc(Config, meck, validate, [Mod])),
+    ok = rpc(Config, meck, unload, [Mod]),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+rabbit_queue_type_deliver_noop(_TargetQs, _Msg, _Opts, QTypeState) ->
+    Actions = [],
+    {ok, QTypeState, Actions}.
+
+target_queues_deleted_accepted(Config) ->
+    Q1 = <<"q1">>,
+    Q2 = <<"q2">>,
+    Q3 = <<"q3">>,
+    QNames = [Q1, Q2, Q3],
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    [begin
+         #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+         #'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = QName,
+                                                                  exchange = <<"amq.fanout">>})
+     end || QName <- QNames],
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/amq.fanout/ignored">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+    flush(credited),
+
+    DTag1 = <<1>>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag1, <<"m1">>, false)),
+    ok = wait_for_settlement(DTag1),
+
+    %% Mock to deliver only to q1.
+    rabbit_ct_broker_helpers:setup_meck(Config, [?MODULE]),
+    Mod = rabbit_queue_type,
+    ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+    ok = rpc(Config, meck, expect, [Mod, deliver, fun ?MODULE:rabbit_queue_type_deliver_to_q1/4]),
+
+    %% Send 2nd message.
+    DTag2 = <<2>>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag2, <<"m2">>, false)),
+    receive {amqp10_disposition, Disp1} -> ct:fail({unexpected_disposition, Disp1})
+    after 200 -> ok
+    end,
+
+    %% Now, the server AMQP session contains a delivery that got confirmed by only q1.
+    %% If we delete q2, we should still receive no DISPOSITION since q3 hasn't confirmed.
+    ?assertEqual(#'queue.delete_ok'{message_count = 1},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = Q2})),
+    receive {amqp10_disposition, Disp2} -> ct:fail({unexpected_disposition, Disp2})
+    after 100 -> ok
+    end,
+    %% If we delete q3, RabbitMQ should reply with ACCEPTED since at least one target queue (q1) confirmed.
+    ?assertEqual(#'queue.delete_ok'{message_count = 1},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = Q3})),
+    receive {amqp10_disposition, {accepted, DTag2}} -> ok
+    after 5000 -> ct:fail(accepted_timeout)
+    end,
+
+    ?assertEqual(#'queue.delete_ok'{message_count = 2},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = Q1})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ?assert(rpc(Config, meck, validate, [Mod])),
+    ok = rpc(Config, meck, unload, [Mod]),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+rabbit_queue_type_deliver_to_q1(Qs, Msg, Opts, QTypeState) ->
+    %% Drop q2 and q3.
+    3 = length(Qs),
+    Q1 = lists:filter(fun({Q, _RouteInos}) ->
+                              amqqueue:get_name(Q) =:= rabbit_misc:r(<<"/">>, queue, <<"q1">>)
+                      end, Qs),
+    1 = length(Q1),
+    meck:passthrough([Q1, Msg, Opts, QTypeState]).
+
+%% Set routing key neither in target address nor in message subject.
+no_routing_key(Config) ->
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/amq.direct">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    Msg = amqp10_msg:new(<<0>>, <<1>>, true),
+    ok = amqp10_client:send_msg(Sender, Msg),
+    receive
+        {amqp10_event,
+         {session, Session,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_INVALID_FIELD,
+              description = {utf8, <<"Publishing to exchange 'amq.direct' in vhost '/' "
+                                     "failed since no routing key was provided">>}}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive expected error")
+    end,
+    ok = amqp10_client:close_connection(Connection).
+
+events(Config) ->
+    ok = event_recorder:start(Config),
+
+    OpnConf0 = connection_config(Config),
+    OpnConf = OpnConf0#{properties => #{<<"ignore-maintenance">> => {boolean, true}}},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail(opened_timeout)
+    end,
+    ok = close_connection_sync(Connection),
+
+    [E0, E1, E2] = event_recorder:get_events(Config),
+    ok = event_recorder:stop(Config),
+
+    assert_event_type(user_authentication_success, E0),
+    Protocol = {protocol, {1, 0}},
+    assert_event_prop([{name, <<"guest">>},
+                       {auth_mechanism, <<"PLAIN">>},
+                       {ssl, false},
+                       Protocol],
+                      E0),
+
+    assert_event_type(connection_created, E1),
+    Node = get_node_config(Config, 0, nodename),
+    assert_event_prop(
+      [Protocol,
+       {node, Node},
+       {vhost, <<"/">>},
+       {user, <<"guest">>},
+       {type, network}],
+      E1),
+    Props = E1#event.props,
+    Name = proplists:lookup(name, Props),
+    Pid = proplists:lookup(pid, Props),
+    ClientProperties = {client_properties, List} = proplists:lookup(client_properties, Props),
+    ?assert(lists:member(
+              {<<"product">>, longstr, <<"AMQP 1.0 client from the RabbitMQ Project">>},
+              List)),
+    ?assert(lists:member(
+              {<<"ignore-maintenance">>, bool, true},
+              List)),
+
+    assert_event_type(connection_closed, E2),
+    assert_event_prop(
+      [{node, Node},
+       Name,
+       Pid,
+       ClientProperties],
+      E2).
+
+sync_get_unsettled_classic_queue(Config) ->
+    sync_get_unsettled(<<"classic">>, Config).
+
+sync_get_unsettled_quorum_queue(Config) ->
+    sync_get_unsettled(<<"quorum">>, Config).
+
+sync_get_unsettled_stream(Config) ->
+    sync_get_unsettled(<<"stream">>, Config).
+
+%% Test synchronous get, figure 2.43 with sender settle mode unsettled.
+sync_get_unsettled(QType, Config) ->
+    SenderSettleMode = unsettled,
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+
+    %% Attach 1 sender and 1 receiver to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, SenderSettleMode),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    %% Grant 1 credit to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+
+    %% Since the queue has no messages yet, we shouldn't receive any message.
+    receive {amqp10_msg, _, _} = Unexp1 -> ct:fail("received unexpected message ~p", [Unexp1])
+    after 10 -> ok
+    end,
+
+    %% Let's send 4 messages to the queue.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag1">>, <<"m1">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag2">>, <<"m2">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag3">>, <<"m3">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag4">>, <<"m4">>, true)),
+
+    %% Since we previously granted only 1 credit, we should get only the 1st message.
+    M1 = receive {amqp10_msg, Receiver, Msg1} ->
+                     ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)),
+                     Msg1
+         after 5000 -> ct:fail("missing m1")
+         end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp2 -> ct:fail("received unexpected message ~p", [Unexp2])
+    after 10 -> ok
+    end,
+
+    %% Synchronously get the 2nd message.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    M2 = receive {amqp10_msg, Receiver, Msg2} ->
+                     ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2)),
+                     Msg2
+         after 5000 -> ct:fail("missing m2")
+         end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp3 -> ct:fail("received unexpected message ~p", [Unexp3])
+    after 10 -> ok
+    end,
+
+    %% Accept the first 2 messages.
+    ok = amqp10_client:accept_msg(Receiver, M1),
+    ok = amqp10_client:accept_msg(Receiver, M2),
+    %% Settlements should not top up credit. We are still out of credits.
+    receive {amqp10_msg, _, _} = Unexp4 -> ct:fail("received unexpected message ~p", [Unexp4])
+    after 10 -> ok
+    end,
+
+    %% Synchronously get the 3rd message.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    receive {amqp10_msg, Receiver, Msg3} ->
+                ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3))
+    after 5000 -> ct:fail("missing m3")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp5 -> ct:fail("received unexpected message ~p", [Unexp5])
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+sync_get_unsettled_2_classic_queue(Config) ->
+    sync_get_unsettled_2(<<"classic">>, Config).
+
+sync_get_unsettled_2_quorum_queue(Config) ->
+    sync_get_unsettled_2(<<"quorum">>, Config).
+
+sync_get_unsettled_2_stream(Config) ->
+    sync_get_unsettled_2(<<"stream">>, Config).
+
+%% Synchronously get 2 messages from queue.
+sync_get_unsettled_2(QType, Config) ->
+    SenderSettleMode = unsettled,
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+
+    %% Attach a sender and a receiver to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session,
+                       <<"test-receiver">>,
+                       Address,
+                       SenderSettleMode),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    %% Grant 2 credits to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never),
+
+    %% Let's send 5 messages to the queue.
+    [ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, true)) ||
+     Bin <- [<<"m1">>, <<"m2">>, <<"m3">>, <<"m4">>, <<"m5">>]],
+
+    %% We should receive exactly 2 messages.
+    receive {amqp10_msg, Receiver, Msg1} -> ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1))
+    after 5000 -> ct:fail("missing m1")
+    end,
+    receive {amqp10_msg, Receiver, Msg2} -> ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2))
+    after 5000 -> ct:fail("missing m2")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp1 -> ct:fail("received unexpected message ~p", [Unexp1])
+    after 50 -> ok
+    end,
+
+    %% Grant 2 more credits to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never),
+    %% Again, we should receive exactly 2 messages.
+    receive {amqp10_msg, Receiver, Msg3} -> ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3))
+    after 5000 -> ct:fail("missing m3")
+    end,
+    receive {amqp10_msg, Receiver, Msg4} -> ?assertEqual([<<"m4">>], amqp10_msg:body(Msg4))
+    after 5000 -> ct:fail("missing m4")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp2 -> ct:fail("received unexpected message ~p", [Unexp2])
+    after 50 -> ok
+    end,
+
+    %% Grant 2 more credits to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never),
+
+    %% We should receive the last (5th) message.
+    receive {amqp10_msg, Receiver, Msg5} -> ?assertEqual([<<"m5">>], amqp10_msg:body(Msg5))
+    after 5000 -> ct:fail("missing m5")
+    end,
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+sync_get_settled_classic_queue(Config) ->
+    sync_get_settled(<<"classic">>, Config).
+
+sync_get_settled_quorum_queue(Config) ->
+    sync_get_settled(<<"quorum">>, Config).
+
+sync_get_settled_stream(Config) ->
+    sync_get_settled(<<"stream">>, Config).
+
+%% Test synchronous get, figure 2.43 with sender settle mode settled.
+sync_get_settled(QType, Config) ->
+    SenderSettleMode = settled,
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+
+    %% Attach 1 sender and 1 receivers to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"my receiver">>, Address, SenderSettleMode),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    %% Grant 1 credit to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+
+    %% Since the queue has no messages yet, we shouldn't receive any message.
+    receive {amqp10_msg, _, _} = Unexp1 -> ct:fail("received unexpected message ~p", [Unexp1])
+    after 10 -> ok
+    end,
+
+    %% Let's send 3 messages to the queue.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag1">>, <<"m1">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag2">>, <<"m2">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag3">>, <<"m3">>, true)),
+
+    %% Since we previously granted only 1 credit, we should get only the 1st message.
+    receive {amqp10_msg, Receiver, Msg1} ->
+                ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1))
+    after 5000 -> ct:fail("missing m1")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp2 -> ct:fail("received unexpected message ~p", [Unexp2])
+    after 10 -> ok
+    end,
+
+    %% Synchronously get the 2nd message.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    receive {amqp10_msg, Receiver, Msg2} ->
+                ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2))
+    after 5000 -> ct:fail("missing m2")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp3 -> ct:fail("received unexpected message ~p", [Unexp3])
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+timed_get_classic_queue(Config) ->
+    timed_get(<<"classic">>, Config).
+
+timed_get_quorum_queue(Config) ->
+    timed_get(<<"quorum">>, Config).
+
+timed_get_stream(Config) ->
+    timed_get(<<"stream">>, Config).
+
+%% Synchronous get with a timeout, figure 2.44.
+timed_get(QType, Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+
+    %% Attach a sender and a receiver to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session,
+                       <<"test-receiver">>,
+                       Address,
+                       unsettled),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never, false),
+
+    Timeout = 10,
+    receive Unexpected0 -> ct:fail("received unexpected ~p", [Unexpected0])
+    after Timeout -> ok
+    end,
+
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never, true),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"my tag">>, <<"my msg">>, true)),
+
+    %% Since our consumer didn't grant any new credit, we shouldn't receive the message we
+    %% just sent.
+    receive Unexpected1 -> ct:fail("received unexpected ~p", [Unexpected1])
+    after 50 -> ok
+    end,
+
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never, true),
+    receive {amqp10_msg, Receiver, Msg1} -> ?assertEqual([<<"my msg">>], amqp10_msg:body(Msg1))
+    after 5000 -> ct:fail("missing 'my msg'")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+stop_classic_queue(Config) ->
+    stop(<<"classic">>, Config).
+
+stop_quorum_queue(Config) ->
+    stop(<<"quorum">>, Config).
+
+stop_stream(Config) ->
+    stop(<<"stream">>, Config).
+
+%% Test stopping a link, figure 2.46.
+stop(QType, Config) ->
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    QName = atom_to_binary(?FUNCTION_NAME),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+    %% Attach 1 sender and 1 receiver to the queue.
+    OpnConf0 = connection_config(Config),
+    NumSent = 300,
+    %% Allow in flight messages to be received after stopping the link.
+    OpnConf = OpnConf0#{transfer_limit_margin => -NumSent},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, settled),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    ok = amqp10_client:flow_link_credit(Receiver, 10, 5),
+    ok = send_messages(Sender, NumSent, true),
+
+    %% Let's await the first 20 messages.
+    NumReceived = 20,
+    Msgs = receive_messages(Receiver, NumReceived),
+
+    %% Stop the link.
+    %% "Stopping the transfers on a given link is accomplished by updating
+    %% the link-credit to be zero and sending the updated flow state." [2.6.10]
+    ok = amqp10_client:stop_receiver_link(Receiver),
+    %% "It is possible that some transfers could be in flight at the time the flow
+    %% state is sent, so incoming transfers could still arrive on the link." [2.6.10]
+    NumInFlight = count_received_messages(Receiver),
+
+    ct:pal("After receiving the first ~b messages and stopping the link, "
+           "we received ~b more in flight messages", [NumReceived, NumInFlight]),
+    ?assert(NumInFlight > 0,
+            "expected some in flight messages, but there were actually none"),
+    ?assert(NumInFlight < NumSent - NumReceived,
+            "expected the link to stop, but actually received all messages"),
+
+    %% Check that contents of the first 20 messages are correct.
+    FirstMsg = hd(Msgs),
+    LastMsg = lists:last(Msgs),
+    ?assertEqual([integer_to_binary(NumSent)], amqp10_msg:body(FirstMsg)),
+    ?assertEqual([integer_to_binary(NumSent - NumReceived + 1)], amqp10_msg:body(LastMsg)),
+
+    %% Let's resume the link.
+    ok = amqp10_client:flow_link_credit(Receiver, 50, 40),
+
+    %% We expect to receive all remaining messages.
+    NumRemaining = NumSent - NumReceived - NumInFlight,
+    ct:pal("Waiting for the remaining ~b messages", [NumRemaining]),
+    Msgs1 = receive_messages(Receiver, NumRemaining),
+    ?assertEqual([<<"1">>], amqp10_msg:body(lists:last(Msgs1))),
+
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = amqp10_client:close_connection(Connection).
+
+single_active_consumer_classic_queue(Config) ->
+    single_active_consumer(<<"classic">>, Config).
+
+single_active_consumer_quorum_queue(Config) ->
+    single_active_consumer(<<"quorum">>, Config).
+
+single_active_consumer(QType, Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-single-active-consumer">>, bool, true},
+                                                  {<<"x-queue-type">>, longstr, QType}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    %% Attach 1 sender and 2 receivers to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    flush(sender_attached),
+
+    %% The 1st consumer will become active.
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(
+                        Session,
+                        <<"test-receiver-1">>,
+                        Address,
+                        unsettled),
+    receive {amqp10_event, {link, Receiver1, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    ok = amqp10_client:flow_link_credit(Receiver1, 2, never),
+
+    %% The 2nd consumer will become inactive.
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(
+                        Session,
+                        <<"test-receiver-2">>,
+                        Address,
+                        unsettled),
+    receive {amqp10_event, {link, Receiver2, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    ok = amqp10_client:flow_link_credit(Receiver2, 2, never),
+
+    NumMsgs = 5,
+    [begin
+         Bin = integer_to_binary(N),
+         ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, true))
+     end || N <- lists:seq(1, NumMsgs)],
+
+    %% Only the active consumer should receive messages.
+    receive {amqp10_msg, Receiver1, Msg1} -> ?assertEqual([<<"1">>], amqp10_msg:body(Msg1))
+    after 5000 -> ct:fail("missing 1st msg")
+    end,
+    receive {amqp10_msg, Receiver1, Msg2} -> ?assertEqual([<<"2">>], amqp10_msg:body(Msg2))
+    after 5000 -> ct:fail("missing 2nd msg")
+    end,
+    receive {amqp10_event, {link, Receiver1, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive Unexpected0 -> ct:fail("received unexpected ~p", [Unexpected0])
+    after 10 -> ok
+    end,
+
+    %% Cancelling the active consumer should cause the inactive to become active.
+    ok = amqp10_client:detach_link(Receiver1),
+    receive {amqp10_event, {link, Receiver1, {detached, normal}}} -> ok
+    after 5000 -> ct:fail("missing detached")
+    end,
+    receive {amqp10_msg, Receiver2, Msg3} -> ?assertEqual([<<"3">>], amqp10_msg:body(Msg3))
+    after 5000 -> ct:fail("missing 3rd msg")
+    end,
+    receive {amqp10_msg, Receiver2, Msg4} -> ?assertEqual([<<"4">>], amqp10_msg:body(Msg4))
+    after 5000 -> ct:fail("missing 4th msg")
+    end,
+    receive {amqp10_event, {link, Receiver2, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive Unexpected1 -> ct:fail("received unexpected ~p", [Unexpected1])
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:detach_link(Receiver2),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    delete_queue(Config, QName).
+
+%% "A session endpoint can choose to unmap its output handle for a link. In this case, the endpoint MUST
+%% send a detach frame to inform the remote peer that the handle is no longer attached to the link endpoint.
+%% If both endpoints do this, the link MAY return to a fully detached state. Note that in this case the
+%% link endpoints MAY still indirectly communicate via the session, as there could still be active deliveries
+%% on the link referenced via delivery-id." [2.6.4]
+%%
+%% "The disposition performative MAY refer to deliveries on links that are no longer attached. As long as
+%% the links have not been closed or detached with an error then the deliveries are still "live" and the
+%% updated state MUST be applied." [2.7.6]
+%%
+%% Although the spec allows to settle delivery IDs on detached links, RabbitMQ does not respect the 'closed'
+%% field of the DETACH frame and therefore handles every DETACH frame as closed. Since the link is closed,
+%% we expect every outstanding delivery to be requeued.
+%%
+%% In addition to consumer cancellation, detaching a link therefore causes in flight deliveries to be requeued.
+%%
+%% Note that this behaviour is different from merely consumer cancellation in AMQP legacy:
+%% "After a consumer is cancelled there will be no future deliveries dispatched to it. Note that there can
+%% still be "in flight" deliveries dispatched previously. Cancelling a consumer will neither discard nor requeue them."
+%% [https://www.rabbitmq.com/consumers.html#unsubscribing]
+detach_requeues(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+
+    %% Attach 1 sender and 2 receivers to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address, settled),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(
+                        Session, <<"recv 1">>, Address, unsettled),
+    receive {amqp10_event, {link, Receiver1, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(
+                        Session, <<"recv 2">>, Address, unsettled),
+    receive {amqp10_event, {link, Receiver2, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(attached),
+
+    ok = amqp10_client:flow_link_credit(Receiver1, 50, never),
+    ok = amqp10_client:flow_link_credit(Receiver2, 50, never),
+
+    %% Let's send 4 messages to the queue.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag1">>, <<"m1">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag2">>, <<"m2">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag3">>, <<"m3">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag4">>, <<"m4">>, true)),
+    ok = amqp10_client:detach_link(Sender),
+
+    %% The queue should serve round robin.
+    [Msg1, Msg3] = receive_messages(Receiver1, 2),
+    [Msg2, Msg4] = receive_messages(Receiver2, 2),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)),
+    ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2)),
+    ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3)),
+    ?assertEqual([<<"m4">>], amqp10_msg:body(Msg4)),
+
+    %% Let's detach the 1st receiver.
+    ok = amqp10_client:detach_link(Receiver1),
+    receive {amqp10_event, {link, Receiver1, {detached, normal}}} -> ok
+    after 5000 -> ct:fail("missing detached")
+    end,
+
+    %% Since Receiver1 hasn't settled its 2 deliveries,
+    %% we expect them to be re-queued and re-delivered to Receiver2.
+    [Msg1b, Msg3b] = receive_messages(Receiver2, 2),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1b)),
+    ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3b)),
+
+    %% Receiver2 accepts all 4 messages.
+    ok = amqp10_client_session:disposition(
+           Session, receiver,
+           amqp10_msg:delivery_id(Msg2),
+           amqp10_msg:delivery_id(Msg3b),
+           true, accepted),
+
+    %% Double check that there are no in flight deliveries in the server session.
+    [SessionPid] = rpc(Config, rabbit_amqp_session, list_local, []),
+    eventually(?_assertEqual(
+                  0,
+                  begin
+                      #{outgoing_unsettled_map := UnsettledMap} = gen_server_state(SessionPid),
+                      maps:size(UnsettledMap)
+                  end)),
+
+    ok = amqp10_client:detach_link(Receiver2),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+resource_alarm_before_session_begin(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+
+    %% Set memory alarm before beginning the session.
+    DefaultWatermark = rpc(Config, vm_memory_monitor, get_vm_memory_high_watermark, []),
+    ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
+    timer:sleep(100),
+
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session1, <<"test-sender">>, Address, unsettled),
+    %% We should still receive link credit since the target queue is fine.
+    ok = wait_for_credit(Sender),
+    %% However, RabbitMQ's incoming window shouldn't allow our client to send any TRANSFER.
+    %% In other words, the client is limited by session flow control, but not by link flow control.
+    Tag1 = <<"tag1">>,
+    Msg1 = amqp10_msg:new(Tag1, <<"m1">>, false),
+    ?assertEqual({error, remote_incoming_window_exceeded},
+                 amqp10_client:send_msg(Sender, Msg1)),
+
+    %% Set additionally disk alarm.
+    DefaultDiskFreeLimit = rpc(Config, rabbit_disk_monitor, get_disk_free_limit, []),
+    ok = rpc(Config, rabbit_disk_monitor, set_disk_free_limit, [999_000_000_000_000]), % 999 TB
+    timer:sleep(100),
+
+    ?assertEqual({error, remote_incoming_window_exceeded},
+                 amqp10_client:send_msg(Sender, Msg1)),
+
+    %% Clear memory alarm.
+    ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [DefaultWatermark]),
+    timer:sleep(100),
+
+    ?assertEqual({error, remote_incoming_window_exceeded},
+                 amqp10_client:send_msg(Sender, Msg1)),
+
+    %% Clear disk alarm.
+    ok = rpc(Config, rabbit_disk_monitor, set_disk_free_limit, [DefaultDiskFreeLimit]),
+    timer:sleep(100),
+
+    %% All alarms are cleared now.
+    %% Hence, RabbitMQ should open its incoming window allowing our client to send TRANSFERs.
+    ?assertEqual(ok,
+                 amqp10_client:send_msg(Sender, Msg1)),
+    ok = wait_for_settlement(Tag1),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session1),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+resource_alarm_after_session_begin(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    Address = <<"/amq/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+
+    {ok, Connection1} = amqp10_client:open_connection(OpnConf),
+    {ok, Session1} = amqp10_client:begin_session_sync(Connection1),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session1, <<"sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(Session1, <<"receiver 1">>, Address, unsettled),
+
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t1">>, <<"m1">>, false)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t2">>, <<"m2">>, false)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t3">>, <<"m3">>, false)),
+    ok = wait_for_accepts(3),
+
+    %% Set memory alarm.
+    DefaultWatermark = rpc(Config, vm_memory_monitor, get_vm_memory_high_watermark, []),
+    ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
+    timer:sleep(100),
+
+    %% Our existing receiver should still be able to receive.
+    {ok, Msg1} = amqp10_client:get_msg(Receiver1),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)),
+    ok = amqp10_client:accept_msg(Receiver1, Msg1),
+
+    %% Attaching a new receiver to the same session and receiving should also work.
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(Session1, <<"receiver 2">>, Address, unsettled),
+    {ok, Msg2} = amqp10_client:get_msg(Receiver2),
+    ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2)),
+    ok = amqp10_client:accept_msg(Receiver2, Msg2),
+
+    %% Even creating a new connection and receiving should work.
+    {ok, Connection2} = amqp10_client:open_connection(OpnConf#{container_id => <<"my container 2">>}),
+    {ok, Session2} = amqp10_client:begin_session_sync(Connection2),
+    {ok, Receiver3} = amqp10_client:attach_receiver_link(Session2, <<"receiver 3">>, Address, unsettled),
+    {ok, Msg3} = amqp10_client:get_msg(Receiver3),
+    ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3)),
+    ok = amqp10_client:accept_msg(Receiver3, Msg3),
+
+    %% However, we shouldn't be able to send any TRANSFER.
+    Msg4 = amqp10_msg:new(<<"t4">>, <<"m4">>, false),
+    ?assertEqual({error, remote_incoming_window_exceeded},
+                 amqp10_client:send_msg(Sender, Msg4)),
+
+    %% Clear memory alarm.
+    ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [DefaultWatermark]),
+    timer:sleep(100),
+
+    %% Now, we should be able to send again.
+    ?assertEqual(ok,
+                 amqp10_client:send_msg(Sender, Msg4)),
+    ok = wait_for_settlement(<<"t4">>),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver1),
+    ok = amqp10_client:detach_link(Receiver2),
+    ok = amqp10_client:detach_link(Receiver3),
+    ok = end_session_sync(Session1),
+    ok = end_session_sync(Session2),
+    ok = amqp10_client:close_connection(Connection1),
+    ok = amqp10_client:close_connection(Connection2),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+auth_attempt_metrics(Config) ->
+    open_and_close_connection(Config),
+    [Attempt1] = rpc(Config, rabbit_core_metrics, get_auth_attempts, []),
+    ?assertEqual(false, proplists:is_defined(remote_address, Attempt1)),
+    ?assertEqual(false, proplists:is_defined(username, Attempt1)),
+    ?assertEqual(<<"amqp10">>, proplists:get_value(protocol, Attempt1)),
+    ?assertEqual(1, proplists:get_value(auth_attempts, Attempt1)),
+    ?assertEqual(0, proplists:get_value(auth_attempts_failed, Attempt1)),
+    ?assertEqual(1, proplists:get_value(auth_attempts_succeeded, Attempt1)),
+
+    rpc(Config, rabbit_core_metrics, reset_auth_attempt_metrics, []),
+    ok = rpc(Config, application, set_env, [rabbit, track_auth_attempt_source, true]),
+    open_and_close_connection(Config),
+    Attempts = rpc(Config, rabbit_core_metrics, get_auth_attempts_by_source, []),
+    [Attempt2] = lists:filter(fun(Props) ->
+                                      proplists:is_defined(remote_address, Props)
+                              end, Attempts),
+    ?assertEqual(<<>>, proplists:get_value(remote_address, Attempt2)),
+    ?assertEqual(<<"guest">>, proplists:get_value(username, Attempt2)),
+    ?assertEqual(<<"amqp10">>, proplists:get_value(protocol, Attempt2)),
+    ?assertEqual(1, proplists:get_value(auth_attempts, Attempt2)),
+    ?assertEqual(0, proplists:get_value(auth_attempts_failed, Attempt2)),
+    ?assertEqual(1, proplists:get_value(auth_attempts_succeeded, Attempt2)).
+
+max_message_size_client_to_server(Config) ->
+    DefaultMaxMessageSize = rpc(Config, persistent_term, get, [max_message_size]),
+    %% Limit the server to only accept messages up to 2KB.
+    MaxMessageSize = 2_000,
+    ok = rpc(Config, persistent_term, put, [max_message_size, MaxMessageSize]),
+
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    Address = <<"/amq/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address, mixed),
+    ok = wait_for_credit(Sender),
+
+    PayloadSmallEnough = binary:copy(<<0>>, MaxMessageSize - 10),
+    ?assertEqual(ok,
+                 amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t1">>, PayloadSmallEnough, false))),
+    ok = wait_for_settlement(<<"t1">>),
+
+    PayloadTooLarge = binary:copy(<<0>>, MaxMessageSize + 1),
+    ?assertEqual({error, message_size_exceeded},
+                 amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t2">>, PayloadTooLarge, false))),
+    ?assertEqual({error, message_size_exceeded},
+                 amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t3">>, PayloadTooLarge, true))),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = rpc(Config, persistent_term, put, [max_message_size, DefaultMaxMessageSize]).
+
+max_message_size_server_to_client(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    Address = <<"/amq/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+
+    MaxMessageSize = 2000,
+    %% Leave a bit of headroom for additional sections sent from RabbitMQ to us,
+    %% e.g. message annotations with routing key and exchange name.
+    PayloadSmallEnough = binary:copy(<<0>>, MaxMessageSize - 200),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t1">>, PayloadSmallEnough, false)),
+    ok = wait_for_settlement(<<"t1">>),
+
+    AttachArgs = #{max_message_size => MaxMessageSize,
+                   name => <<"test-receiver">>,
+                   role => {receiver, #{address => Address,
+                                        durable => configuration}, self()},
+                   snd_settle_mode => unsettled,
+                   rcv_settle_mode => first,
+                   filter => #{}},
+    {ok, Receiver} = amqp10_client:attach_link(Session, AttachArgs),
+    {ok, Msg} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([PayloadSmallEnough], amqp10_msg:body(Msg)),
+
+    PayloadTooLarge = binary:copy(<<0>>, MaxMessageSize + 1),
+    %% The sending link has no maximum message size set.
+    %% Hence, sending this large message from client to server should work.
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"t2">>, PayloadTooLarge, false)),
+    ok = wait_for_settlement(<<"t2">>),
+
+    %% However, the receiving link has a maximum message size set.
+    %% Hence, when the server attempts to deliver this large message,
+    %% it should throw link error message-size-exceeded.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never),
+    receive
+        {amqp10_event,
+         {session, Session,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_LINK_ERROR_MESSAGE_SIZE_EXCEEDED}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive expected error")
+    end,
+
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+%% This test ensures that the server sends us TRANSFER and FLOW frames in the correct order
+%% even if the server is temporarily not allowed to send us any TRANSFERs due to our incoming
+%% window being closed.
+receive_transfer_flow_order(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    Address = <<"/amq/queue/", QName/binary>>,
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address),
+    ok = wait_for_credit(Sender),
+    DTag = <<"my tag">>,
+    Body = <<"my body">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag, Body, false)),
+    ok = wait_for_settlement(DTag),
+    ok = amqp10_client:detach_link(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"receiver">>, Address, unsettled),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail(missing_attached)
+    end,
+    flush(receiver_attached),
+
+    %% Close our incoming window.
+    gen_statem:cast(Session, {flow_session, #'v1_0.flow'{incoming_window = {uint, 0}}}),
+
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never, true),
+    %% Given our incoming window is closed, we shouldn't receive the TRANSFER yet, and threfore
+    %% must not yet receive the FLOW that comes thereafter with drain=true, credit=0, and advanced delivery-count.
+    receive Unexpected -> ct:fail({unexpected, Unexpected})
+    after 300 -> ok
+    end,
+
+    %% Open our incoming window
+    gen_statem:cast(Session, {flow_session, #'v1_0.flow'{incoming_window = {uint, 5}}}),
+    %% Important: We should first receive the TRANSFER,
+    %% and only thereafter the FLOW (and hence the credit_exhausted notification).
+    receive First ->
+                {amqp10_msg, Receiver, Msg} = First,
+                ?assertEqual([Body], amqp10_msg:body(Msg))
+    after 5000 -> ct:fail("timeout receiving message")
+    end,
+    receive Second ->
+                ?assertEqual({amqp10_event, {link, Receiver, credit_exhausted}}, Second)
+    after 5000 -> ct:fail("timeout receiving credit_exhausted")
+    end,
+
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ok = delete_queue(Config, QName).
+
+last_queue_confirms(Config) ->
+    ClassicQ = <<"my classic queue">>,
+    QuorumQ = <<"my quorum queue">>,
+    Qs = [ClassicQ, QuorumQ],
+    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{queue = ClassicQ}),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QuorumQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>},
+                                                  {<<"x-quorum-initial-group-size">>, long, 3}
+                                                 ]}),
+    [#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = QName,
+                                                              exchange = <<"amq.fanout">>})
+     || QName <- Qs],
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+
+    AddressFanout = <<"/exchange/amq.fanout/ignored">>,
+    {ok, SenderFanout} = amqp10_client:attach_sender_link(
+                           Session, <<"sender-1">>, AddressFanout, unsettled),
+    ok = wait_for_credit(SenderFanout),
+
+    AddressClassicQ = <<"/amq/queue/", ClassicQ/binary>>,
+    {ok, SenderClassicQ} = amqp10_client:attach_sender_link(
+                             Session, <<"sender-2">>, AddressClassicQ, unsettled),
+    ok = wait_for_credit(SenderClassicQ),
+
+    DTag1 = <<"t1">>,
+    ok = amqp10_client:send_msg(SenderFanout, amqp10_msg:new(DTag1, <<"m1">>, false)),
+    receive {amqp10_disposition, {accepted, DTag1}} -> ok
+    after 5000 -> ct:fail({missing_accepted, DTag1})
+    end,
+
+    %% Make quorum queue unavailable.
+    ok = rabbit_ct_broker_helpers:stop_node(Config, 2),
+    ok = rabbit_ct_broker_helpers:stop_node(Config, 1),
+
+    DTag2 = <<"t2">>,
+    DTag3 = <<"t3">>,
+    ok = amqp10_client:send_msg(SenderFanout, amqp10_msg:new(DTag2, <<"m2">>, false)),
+    ok = amqp10_client:send_msg(SenderClassicQ, amqp10_msg:new(DTag3, <<"m3">>, false)),
+
+    %% Since quorum queue is down, we should only get a confirmation for m3.
+    receive {amqp10_disposition, {accepted, DTag3}} -> ok
+    after 5000 -> ct:fail({missing_accepted, DTag3})
+    end,
+    receive {amqp10_disposition, Unexpected} -> ct:fail({unexpected_disposition, Unexpected})
+    after 200 -> ok
+    end,
+
+    ok = rabbit_ct_broker_helpers:start_node(Config, 1),
+    ok = rabbit_ct_broker_helpers:start_node(Config, 2),
+    %% Since the quorum queue has become available, we should now get a confirmation for m2.
+    receive {amqp10_disposition, {accepted, DTag2}} -> ok
+    after 10_000 -> ct:fail({missing_accepted, DTag2})
+    end,
+
+    ok = amqp10_client:detach_link(SenderClassicQ),
+    ok = amqp10_client:detach_link(SenderFanout),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ?assertEqual(#'queue.delete_ok'{message_count = 3},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = ClassicQ})),
+    ?assertEqual(#'queue.delete_ok'{message_count = 2},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QuorumQ})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+target_queue_deleted(Config) ->
+    ClassicQ = <<"my classic queue">>,
+    QuorumQ = <<"my quorum queue">>,
+    Qs = [ClassicQ, QuorumQ],
+    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{queue = ClassicQ}),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QuorumQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>},
+                                                  %% Use 2 replica quorum queue, such that we can stop 1 node
+                                                  %% later to make quorum queue unavailable, but still have
+                                                  %% 2 out of 3 nodes running for Khepri being available.
+                                                  {<<"x-quorum-initial-group-size">>, long, 2}
+                                                 ]}),
+    [#'queue.bind_ok'{} = amqp_channel:call(Ch, #'queue.bind'{queue = QName,
+                                                              exchange = <<"amq.fanout">>})
+     || QName <- Qs],
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+
+    Address = <<"/exchange/amq.fanout/ignored">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+
+    DTag1 = <<"t1">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag1, <<"m1">>, false)),
+    receive {amqp10_disposition, {accepted, DTag1}} -> ok
+    after 5000 -> ct:fail({missing_accepted, DTag1})
+    end,
+
+    N0 = rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename),
+    RaName = ra_name(QuorumQ),
+    ServerId0 = {RaName, N0},
+    {ok, Members, _Leader} = ra:members(ServerId0),
+    ?assertEqual(2, length(Members)),
+    [{RaName, ReplicaNode}] = Members -- [ServerId0],
+    ct:pal("Stopping node ~s to make quorum queue unavailable...", [ReplicaNode]),
+    ok = rabbit_ct_broker_helpers:stop_node(Config, ReplicaNode),
+    flush("quorum queue is down"),
+
+    DTag2 = <<"t2">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag2, <<"m2">>, false)),
+    %% Target classic queue should receive m2.
+    assert_messages(ClassicQ, 2, 0, Config),
+    %% Delete target classic queue. (Khepri is still available.)
+    ?assertEqual(#'queue.delete_ok'{message_count = 2},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = ClassicQ})),
+
+    %% Since quorum queue is down, we should still receive no DISPOSITION.
+    receive {amqp10_disposition, Unexpected} -> ct:fail({unexpected_disposition, Unexpected})
+    after 100 -> ok
+    end,
+
+    ok = rabbit_ct_broker_helpers:start_node(Config, ReplicaNode),
+    %% Since the quorum queue has become available, we should now get a confirmation for m2.
+    receive {amqp10_disposition, {accepted, DTag2}} -> ok
+    after 10_000 -> ct:fail({missing_accepted, DTag2})
+    end,
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ?assertEqual(#'queue.delete_ok'{message_count = 2},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QuorumQ})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+target_classic_queue_down(Config) ->
+    ClassicQueueNode = 2,
+    Ch = rabbit_ct_client_helpers:open_channel(Config, ClassicQueueNode),
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/amq/queue/", QName/binary>>,
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"classic">>}]}),
+    ok = rabbit_ct_client_helpers:close_channels_and_connection(Config, ClassicQueueNode),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(Session, <<"receiver 1">>, Address),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"sender">>, Address, unsettled),
+    ok = wait_for_credit(Sender),
+
+    DTag1 = <<"t1">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag1, <<"m1">>, false)),
+    ok = wait_for_settlement(DTag1),
+
+    {ok, Msg1} = amqp10_client:get_msg(Receiver1),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)),
+
+    %% Make classic queue down.
+    flush("stopping node"),
+    ok = rabbit_ct_broker_helpers:stop_node(Config, ClassicQueueNode),
+
+    %% We expect that the server closes links that receive from classic queues that are down.
+    ExpectedError = #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_ILLEGAL_STATE},
+    receive {amqp10_event, {link, Receiver1, {detached, ExpectedError}}} -> ok
+    after 10_000 -> ct:fail({missing_event, ?LINE})
+    end,
+    %% However the server should not close links that send to classic queues that are down.
+    receive Unexpected -> ct:fail({unexpected, Unexpected})
+    after 20 -> ok
+    end,
+    %% Instead, the server should reject messages that are sent to classic queues that are down.
+    DTag2 = <<"t2">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag2, <<"m2">>, false)),
+    ok = wait_for_settlement(DTag2, rejected),
+
+    ok = rabbit_ct_broker_helpers:start_node(Config, ClassicQueueNode),
+    %% Now that the classic queue is up again, we should be able to attach a new receiver
+    %% and be able to send to and receive from the classic queue.
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(Session, <<"receiver 2">>, Address),
+    receive {amqp10_event, {link, Receiver2, attached}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    DTag3 = <<"t3">>,
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(DTag3, <<"m3">>, false)),
+    ok = wait_for_settlement(DTag3),
+    {ok, Msg3} = amqp10_client:get_msg(Receiver2),
+    ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3)),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver2),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    delete_queue(Config, QName).
+
+async_notify_settled_classic_queue(Config) ->
+    %% TODO Bump old version in mixed version tests to 3.13.x,
+    %% require ff message_containers and always run this test case.
+    case rabbit_ct_broker_helpers:enable_feature_flag(Config, message_containers) of
+        ok -> async_notify(settled, <<"classic">>, Config);
+        {skip, _} = Skip -> Skip
+    end.
+
+async_notify_settled_quorum_queue(Config) ->
+    async_notify(settled, <<"quorum">>, Config).
+
+async_notify_settled_stream(Config) ->
+    async_notify(settled, <<"stream">>, Config).
+
+async_notify_unsettled_classic_queue(Config) ->
+    %% TODO Bump old version in mixed version tests to 3.13.x,
+    %% require ff message_containers and always run this test case.
+    case rabbit_ct_broker_helpers:enable_feature_flag(Config, message_containers) of
+        ok -> async_notify(unsettled, <<"classic">>, Config);
+        {skip, _} = Skip -> Skip
+    end.
+
+async_notify_unsettled_quorum_queue(Config) ->
+    async_notify(unsettled, <<"quorum">>, Config).
+
+async_notify_unsettled_stream(Config) ->
+    async_notify(unsettled, <<"stream">>, Config).
+
+%% Test asynchronous notification, figure 2.45.
+async_notify(SenderSettleMode, QType, Config) ->
+    %% Place queue leader on the old node.
+    Ch = rabbit_ct_client_helpers:open_channel(Config, 1),
+    QName = atom_to_binary(?FUNCTION_NAME),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+    %% Connect AMQP client to the new node causing queue client to run the new code.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+
+    %% Send 30 messages to the queue.
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    NumMsgs = 30,
+    [begin
+         Bin = integer_to_binary(N),
+         ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, false))
+     end || N <- lists:seq(1, NumMsgs)],
+    %% Wait for last message to be confirmed.
+    ok = wait_for_settlement(integer_to_binary(NumMsgs)),
+    flush(settled),
+    ok = detach_link_sync(Sender),
+
+    case QType of
+        <<"stream">> ->
+            %% If it is a stream we need to wait until there is a local member
+            %% on the node we want to subscibe from before proceeding.
+            rabbit_ct_helpers:await_condition(
+              fun() -> rabbit_ct_broker_helpers:rpc(
+                         Config, 0, ?MODULE, has_local_member,
+                         [rabbit_misc:r(<<"/">>, queue, QName)])
+              end, 30_000);
+        _ ->
+            ok
+    end,
+    Filter = consume_from_first(QType),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address,
+                       SenderSettleMode, configuration, Filter),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+
+    %% Initially, grant 10 credits to the sending queue.
+    %% Whenever credits drops below 5, renew back to 10.
+    ok = amqp10_client:flow_link_credit(Receiver, 10, 5),
+
+    %% We should receive all messages.
+    Msgs = receive_messages(Receiver, NumMsgs),
+    FirstMsg = hd(Msgs),
+    LastMsg = lists:last(Msgs),
+    ?assertEqual([<<"1">>], amqp10_msg:body(FirstMsg)),
+    ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(LastMsg)),
+
+    case SenderSettleMode of
+        settled ->
+            ok;
+        unsettled ->
+            ok = amqp10_client_session:disposition(
+                   Session,
+                   receiver,
+                   amqp10_msg:delivery_id(FirstMsg),
+                   amqp10_msg:delivery_id(LastMsg),
+                   true,
+                   accepted)
+    end,
+
+    %% No further messages should be delivered.
+    receive Unexpected -> ct:fail({received_unexpected_message, Unexpected})
+    after 50 -> ok
+    end,
+
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = end_session_sync(Session),
+    ok = close_connection_sync(Connection).
+
+%% For TRANSFERS from AMQP client to RabbitMQ, this test asserts that a single slow link receiver
+%% (slow queue) does not impact other link receivers (fast queues) on the **same** session.
+%% (This is unlike AMQP legacy where a single slow queue will block the entire connection.)
+link_flow_control(Config) ->
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    CQ = <<"cq">>,
+    QQ = <<"qq">>,
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = CQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"classic">>}]}),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+
+    AddressCQ = <<"/amq/queue/", CQ/binary>>,
+    AddressQQ = <<"/amq/queue/", QQ/binary>>,
+    {ok, ReceiverCQ} = amqp10_client:attach_receiver_link(Session, <<"cq-receiver">>, AddressCQ, settled),
+    {ok, ReceiverQQ} = amqp10_client:attach_receiver_link(Session, <<"qq-receiver">>, AddressQQ, settled),
+    {ok, SenderCQ} = amqp10_client:attach_sender_link(Session, <<"cq-sender">>, AddressCQ, settled),
+    {ok, SenderQQ} = amqp10_client:attach_sender_link(Session, <<"qq-sender">>, AddressQQ, settled),
+    ok = wait_for_credit(SenderCQ),
+    ok = wait_for_credit(SenderQQ),
+    flush(attached),
+
+    %% Send and receive a single message on both queues.
+    ok = amqp10_client:send_msg(SenderCQ, amqp10_msg:new(<<0>>, <<0>>, true)),
+    ok = amqp10_client:send_msg(SenderQQ, amqp10_msg:new(<<1>>, <<1>>, true)),
+    {ok, Msg0} = amqp10_client:get_msg(ReceiverCQ),
+    ?assertEqual([<<0>>], amqp10_msg:body(Msg0)),
+    {ok, Msg1} = amqp10_client:get_msg(ReceiverQQ),
+    ?assertEqual([<<1>>], amqp10_msg:body(Msg1)),
+
+    %% Make quorum queue unavailable.
+    ok = rabbit_ct_broker_helpers:stop_node(Config, 2),
+    ok = rabbit_ct_broker_helpers:stop_node(Config, 1),
+
+    NumMsgs = 1000,
+    %% Since the quorum queue is unavailable, we expect our quorum queue sender to run
+    %% out of credits and RabbitMQ should not grant our quorum queue sender any new credits.
+    ok = assert_link_credit_runs_out(SenderQQ, NumMsgs),
+    %% Despite the quorum queue being unavailable, the classic queue can perfectly receive messages.
+    %% So, we expect that on the same AMQP session, link credit will be renewed for our classic queue sender.
+    ok = send_messages(SenderCQ, NumMsgs, true),
+
+    %% Check that all 1k messages can be received from the classic queue.
+    ok = amqp10_client:flow_link_credit(ReceiverCQ, NumMsgs, never),
+    ReceivedCQ = receive_messages(ReceiverCQ, NumMsgs),
+    FirstMsg = hd(ReceivedCQ),
+    LastMsg = lists:last(ReceivedCQ),
+    ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(FirstMsg)),
+    ?assertEqual([<<"1">>], amqp10_msg:body(LastMsg)),
+
+    %% We expect still that RabbitMQ won't grant our quorum queue sender any new credits.
+    receive {amqp10_event, {link, SenderQQ, credited}} ->
+                ct:fail({unexpected_credited, ?LINE})
+    after 5 -> ok
+    end,
+
+    %% Make quorum queue available again.
+    ok = rabbit_ct_broker_helpers:start_node(Config, 1),
+    ok = rabbit_ct_broker_helpers:start_node(Config, 2),
+
+    %% Now, we exepct that the messages sent earlier make it actually into the quorum queue.
+    %% Therefore, RabbitMQ should grant our quorum queue sender more credits.
+    receive {amqp10_event, {link, SenderQQ, credited}} ->
+                ct:pal("quorum queue sender got credited")
+    after 30_000 -> ct:fail({credited_timeout, ?LINE})
+    end,
+
+    [ok = amqp10_client:detach_link(Link) || Link <- [ReceiverCQ, ReceiverQQ, SenderCQ, SenderQQ]],
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    delete_queue(Config, QQ),
+    delete_queue(Config, CQ).
+
+classic_queue_on_old_node(Config) ->
+    %% TODO Bump old version in mixed version tests to 3.13.x,
+    %% require ff message_containers and always run this test case.
+    case rabbit_ct_broker_helpers:enable_feature_flag(Config, message_containers) of
+        ok -> queue_and_client_different_nodes(1, 0, <<"classic">>, Config);
+        {skip, _} = Skip -> Skip
+    end.
+
+classic_queue_on_new_node(Config) ->
+    queue_and_client_different_nodes(0, 1, <<"classic">>, Config).
+
+quorum_queue_on_old_node(Config) ->
+    queue_and_client_different_nodes(1, 0, <<"quorum">>, Config).
+
+quorum_queue_on_new_node(Config) ->
+    queue_and_client_different_nodes(0, 1, <<"quorum">>, Config).
+
+%% In mixed version tests, run the queue leader with old code
+%% and queue client with new code, or vice versa.
+queue_and_client_different_nodes(QueueLeaderNode, ClientNode, QueueType, Config) ->
+    Ch = rabbit_ct_client_helpers:open_channel(Config, QueueLeaderNode),
+    QName = atom_to_binary(?FUNCTION_NAME),
+    #'queue.declare_ok'{} =  amqp_channel:call(
+                               Ch, #'queue.declare'{queue = QName,
+                                                    durable = true,
+                                                    arguments = [{<<"x-queue-type">>, longstr, QueueType}]}),
+    %% Connect AMQP client to the new (or old) node causing queue client to run the new (or old) code.
+    OpnConf = connection_config(ClientNode, Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session,
+                       <<"test-receiver">>,
+                       Address,
+                       unsettled),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    NumMsgs = 10,
+    [begin
+         Bin = integer_to_binary(N),
+         ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, true))
+     end || N <- lists:seq(1, NumMsgs)],
+
+    %% Grant credits to the sending queue.
+    ok = amqp10_client:flow_link_credit(Receiver, NumMsgs, never),
+
+    %% We should receive all messages.
+    Msgs = receive_messages(Receiver, NumMsgs),
+    FirstMsg = hd(Msgs),
+    LastMsg = lists:last(Msgs),
+    ?assertEqual([<<"1">>], amqp10_msg:body(FirstMsg)),
+    ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(LastMsg)),
+    ok = amqp10_client_session:disposition(
+           Session,
+           receiver,
+           amqp10_msg:delivery_id(FirstMsg),
+           amqp10_msg:delivery_id(LastMsg),
+           true,
+           accepted),
+
+    CreditApiV2 = rpc(Config, rabbit_feature_flags, is_enabled, [credit_api_v2]),
+    case QueueType =:= <<"quorum">> andalso not CreditApiV2 of
+        true ->
+            ct:pal("Quorum queues in credit API v1 have a known bug that they "
+                   "reply with send_drained before delivering the message.");
+        false ->
+            %% Send another message and drain.
+            Tag = <<"tag">>,
+            Body = <<"body">>,
+            ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Tag, Body, false)),
+            ok = wait_for_settlement(Tag),
+            ok = amqp10_client:flow_link_credit(Receiver, 999, never, true),
+            [Msg] = receive_messages(Receiver, 1),
+            ?assertEqual([Body], amqp10_msg:body(Msg)),
+            receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+            after 5000 -> ct:fail("expected credit_exhausted")
+            end,
+            ok = amqp10_client:accept_msg(Receiver, Msg)
+    end,
+
+    ExpectedReadyMsgs = 0,
+    ?assertEqual(#'queue.delete_ok'{message_count = ExpectedReadyMsgs},
+                 amqp_channel:call(Ch, #'queue.delete'{queue = QName})),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = amqp10_client:close_connection(Connection).
+
+maintenance(Config) ->
+    {ok, C0} = amqp10_client:open_connection(connection_config(0, Config)),
+    {ok, C2} = amqp10_client:open_connection(connection_config(2, Config)),
+    receive {amqp10_event, {connection, C0, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    receive {amqp10_event, {connection, C2, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    ok = drain_node(Config, 2),
+    receive
+        {amqp10_event,
+         {connection, C2,
+          {closed,
+           {internal_error, <<"Connection forced: \"Node was put into maintenance mode\"">>}}}} ->
+            ok
+    after 5000 ->
+              flush(?LINE),
+              ct:fail({missing_event, ?LINE})
+    end,
+    ok = revive_node(Config, 2),
+
+    ok = close_connection_sync(C0).
+
+%% rabbitmqctl list_connections
+%% should list both AMQP 1.0 and AMQP 0.9.1 connections.
+list_connections(Config) ->
+    %% Close any open AMQP 0.9.1 connections from previous test cases.
+    [ok = rabbit_ct_client_helpers:close_channels_and_connection(Config, Node) || Node <- [0, 1, 2]],
+
+    Connection091 = rabbit_ct_client_helpers:open_unmanaged_connection(Config, 0),
+    {ok, C0} = amqp10_client:open_connection(connection_config(0, Config)),
+    {ok, C2} = amqp10_client:open_connection(connection_config(2, Config)),
+    receive {amqp10_event, {connection, C0, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    receive {amqp10_event, {connection, C2, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    {ok, StdOut} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["list_connections", "--silent", "protocol"]),
+    Protocols0 = re:split(StdOut, <<"\n">>, [trim]),
+    %% Remove any whitespaces.
+    Protocols1 = [binary:replace(Subject, <<" ">>, <<>>, [global]) || Subject <- Protocols0],
+    Protocols = lists:sort(Protocols1),
+    ?assertEqual([<<"{0,9,1}">>,
+                  <<"{1,0}">>,
+                  <<"{1,0}">>],
+                 Protocols),
+
+    ok = rabbit_ct_client_helpers:close_connection(Connection091),
+    ok = close_connection_sync(C0),
+    ok = close_connection_sync(C2).
+
+global_counters(Config) ->
+    #{publishers := 0,
+      consumers := 0,
+      messages_received_total := Received0,
+      messages_received_confirm_total := ReceivedConfirm0,
+      messages_confirmed_total := Confirmed0,
+      messages_routed_total := Routed0,
+      messages_unroutable_dropped_total := UnroutableDropped0,
+      messages_unroutable_returned_total := UnroutableReturned0} = get_global_counters(Config),
+
+    #{messages_delivered_total := CQDelivered0,
+      messages_redelivered_total := CQRedelivered0,
+      messages_acknowledged_total := CQAcknowledged0} = get_global_counters(Config, rabbit_classic_queue),
+
+    #{messages_delivered_total := QQDelivered0,
+      messages_redelivered_total := QQRedelivered0,
+      messages_acknowledged_total := QQAcknowledged0} = get_global_counters(Config, rabbit_quorum_queue),
+
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    CQ = <<"my classic queue">>,
+    QQ = <<"my quorum queue">>,
+    CQAddress = <<"/amq/queue/", CQ/binary>>,
+    QQAddress = <<"/amq/queue/", QQ/binary>>,
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = CQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"classic">>}]}),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, CQSender} = amqp10_client:attach_sender_link(Session, <<"test-sender-cq">>, CQAddress),
+    {ok, QQSender} = amqp10_client:attach_sender_link(Session, <<"test-sender-qq">>, QQAddress),
+    ok = wait_for_credit(CQSender),
+    ok = wait_for_credit(QQSender),
+    {ok, CQReceiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver-cq">>, CQAddress, settled),
+    {ok, QQReceiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver-qq">>, QQAddress, unsettled),
+    ok = amqp10_client:send_msg(CQSender, amqp10_msg:new(<<0>>, <<"m0">>, true)),
+    ok = amqp10_client:send_msg(QQSender, amqp10_msg:new(<<1>>, <<"m1">>, false)),
+    ok = wait_for_settlement(<<1>>),
+
+    {ok, Msg0} = amqp10_client:get_msg(CQReceiver),
+    ?assertEqual([<<"m0">>], amqp10_msg:body(Msg0)),
+
+    {ok, Msg1} = amqp10_client:get_msg(QQReceiver),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(Msg1)),
+    ok = amqp10_client:accept_msg(QQReceiver, Msg1),
+
+    #{publishers := 2,
+      consumers := 2,
+      messages_received_total := Received1,
+      messages_received_confirm_total := ReceivedConfirm1,
+      messages_confirmed_total := Confirmed1,
+      messages_routed_total := Routed1,
+      messages_unroutable_dropped_total := UnroutableDropped1,
+      messages_unroutable_returned_total := UnroutableReturned1} = get_global_counters(Config),
+    ?assertEqual(Received0 + 2, Received1),
+    ?assertEqual(ReceivedConfirm0 + 1, ReceivedConfirm1),
+    ?assertEqual(Confirmed0 + 1, Confirmed1),
+    ?assertEqual(Routed0 + 2, Routed1),
+    ?assertEqual(UnroutableDropped0, UnroutableDropped1),
+    ?assertEqual(UnroutableReturned0, UnroutableReturned1),
+
+    #{messages_delivered_total := CQDelivered1,
+      messages_redelivered_total := CQRedelivered1,
+      messages_acknowledged_total := CQAcknowledged1} = get_global_counters(Config, rabbit_classic_queue),
+    ?assertEqual(CQDelivered0 + 1, CQDelivered1),
+    ?assertEqual(CQRedelivered0, CQRedelivered1),
+    ?assertEqual(CQAcknowledged0, CQAcknowledged1),
+
+    #{messages_delivered_total := QQDelivered1,
+      messages_redelivered_total := QQRedelivered1,
+      messages_acknowledged_total := QQAcknowledged1} = get_global_counters(Config, rabbit_quorum_queue),
+    ?assertEqual(QQDelivered0 + 1, QQDelivered1),
+    ?assertEqual(QQRedelivered0, QQRedelivered1),
+    ?assertEqual(QQAcknowledged0 + 1, QQAcknowledged1),
+
+    %% Test re-delivery.
+    ok = amqp10_client:send_msg(QQSender, amqp10_msg:new(<<2>>, <<"m2">>, false)),
+    ok = wait_for_settlement(<<2>>),
+    {ok, Msg2a} = amqp10_client:get_msg(QQReceiver),
+    ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2a)),
+    %% Releasing causes the message to be requeued.
+    ok = amqp10_client:settle_msg(QQReceiver, Msg2a, released),
+    %% The message should be re-delivered.
+    {ok, Msg2b} = amqp10_client:get_msg(QQReceiver),
+    ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2b)),
+    #{messages_delivered_total := QQDelivered2,
+      messages_redelivered_total := QQRedelivered2,
+      messages_acknowledged_total := QQAcknowledged2} = get_global_counters(Config, rabbit_quorum_queue),
+    %% m2 was delivered 2 times
+    ?assertEqual(QQDelivered1 + 2, QQDelivered2),
+    %% m2 was re-delivered 1 time
+    ?assertEqual(QQRedelivered1 + 1, QQRedelivered2),
+    %% Releasing a message shouldn't count as acknowledged.
+    ?assertEqual(QQAcknowledged1, QQAcknowledged2),
+    ok = amqp10_client:accept_msg(QQReceiver, Msg2b),
+
+    %% Server closes the link endpoint due to some AMQP 1.0 external condition:
+    %% In this test, the external condition is that an AMQP 0.9.1 client deletes the queue.
+    %% Gauges for publishers and consumers should be decremented.
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QQ}),
+    ExpectedError = #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_RESOURCE_DELETED},
+    receive {amqp10_event, {link, QQSender, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our sending link")
+    end,
+    receive {amqp10_event, {link, QQReceiver, {detached, ExpectedError}}} -> ok
+    after 5000 -> ct:fail("server did not close our receiving link")
+    end,
+    ?assertMatch(#{publishers := 1,
+                   consumers := 1},
+                 get_global_counters(Config)),
+
+    %% Gauges for publishers and consumers should also be decremented for normal link detachments.
+    ok = detach_link_sync(CQSender),
+    ok = detach_link_sync(CQReceiver),
+    ?assertMatch(#{publishers := 0,
+                   consumers := 0},
+                 get_global_counters(Config)),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = CQ}),
+
+    flush("testing unroutable..."),
+    %% Send 2 messages to the fanout exchange that has no bound queues.
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender-fanout">>, <<"/exchange/amq.fanout/ignored">>),
+    ok = wait_for_credit(Sender),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<3>>, <<"m3">>, true)),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<4>>, <<"m4">>, false)),
+    ok = wait_for_settlement(<<4>>, released),
+    #{messages_unroutable_dropped_total := UnroutableDropped2,
+      messages_unroutable_returned_total := UnroutableReturned2} = get_global_counters(Config),
+    %% m3 was dropped
+    ?assertEqual(UnroutableDropped1 + 1, UnroutableDropped2),
+    %% m4 was returned
+    ?assertEqual(UnroutableReturned1 + 1, UnroutableReturned2),
+
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+stream_filtering(Config) ->
+    ok = rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FUNCTION_NAME),
+    Stream = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/amq/queue/", Stream/binary>>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    amqp_channel:call(Ch, #'queue.declare'{
+                             queue = Stream,
+                             durable = true,
+                             arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session(Connection),
+    SenderLinkName = <<"test-sender">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session,
+                                                    SenderLinkName,
+                                                    Address),
+    wait_for_credit(Sender),
+
+    %% We are going to publish several waves of messages with and without filter values.
+    %% We will then create subscriptions with various filter options
+    %% and make sure we receive only what we asked for and not all the messages.
+    WaveCount = 1000,
+    %% logic to publish a wave of messages with or without a filter value
+    Publish = fun(FilterValue) ->
+                      lists:foreach(fun(Seq) ->
+                                            {AppProps, Anns} =
+                                            case FilterValue of
+                                                undefined ->
+                                                    {#{}, #{}};
+                                                _ ->
+                                                    {#{<<"filter">> => FilterValue},
+                                                     #{<<"x-stream-filter-value">> => FilterValue}}
+                                            end,
+                                            FilterBin = rabbit_data_coercion:to_binary(FilterValue),
+                                            SeqBin = integer_to_binary(Seq),
+                                            DTag = <>,
+                                            Msg0 = amqp10_msg:new(DTag, <<"my-body">>, false),
+                                            Msg1 = amqp10_msg:set_application_properties(AppProps, Msg0),
+                                            Msg2 = amqp10_msg:set_message_annotations(Anns, Msg1),
+                                            ok = amqp10_client:send_msg(Sender, Msg2),
+                                            ok = wait_for_settlement(DTag)
+                                    end, lists:seq(1, WaveCount))
+              end,
+
+    %% Publish messages with the "apple" filter value.
+    Publish(<<"apple">>),
+    %% Publish messages with no filter value.
+    Publish(undefined),
+    %% Publish messages with the "orange" filter value.
+    Publish(<<"orange">>),
+    ok = amqp10_client:detach_link(Sender),
+
+    % filtering on "apple"
+    TerminusDurability = none,
+    {ok, AppleReceiver} = amqp10_client:attach_receiver_link(
+                            Session,
+                            <<"test-receiver-1">>,
+                            Address,
+                            unsettled,
+                            TerminusDurability,
+                            #{<<"rabbitmq:stream-offset-spec">> => <<"first">>,
+                              <<"rabbitmq:stream-filter">> => <<"apple">>}),
+    ok = amqp10_client:flow_link_credit(AppleReceiver, 100, 10),
+    AppleMessages = receive_all_messages(AppleReceiver, []),
+    %% we should get less than all the waves combined
+    ?assert(length(AppleMessages) < WaveCount * 3),
+    %% client-side filtering
+    AppleFilteredMessages = lists:filter(fun(Msg) ->
+                                                 AP = amqp10_msg:application_properties(Msg),
+                                                 maps:get(<<"filter">>, AP) =:= <<"apple">>
+                                         end, AppleMessages),
+    ?assertEqual(WaveCount, length(AppleFilteredMessages)),
+    ok = amqp10_client:detach_link(AppleReceiver),
+
+    %% filtering on "apple" and "orange"
+    {ok, AppleOrangeReceiver} = amqp10_client:attach_receiver_link(
+                                  Session,
+                                  <<"test-receiver-2">>,
+                                  Address,
+                                  unsettled,
+                                  TerminusDurability,
+                                  #{<<"rabbitmq:stream-offset-spec">> => <<"first">>,
+                                    <<"rabbitmq:stream-filter">> => [<<"apple">>, <<"orange">>]}),
+    ok = amqp10_client:flow_link_credit(AppleOrangeReceiver, 100, 10),
+    AppleOrangeMessages = receive_all_messages(AppleOrangeReceiver, []),
+    %% we should get less than all the waves combined
+    ?assert(length(AppleOrangeMessages) < WaveCount * 3),
+    %% client-side filtering
+    AppleOrangeFilteredMessages = lists:filter(fun(Msg) ->
+                                                       AP = amqp10_msg:application_properties(Msg),
+                                                       Filter = maps:get(<<"filter">>, AP),
+                                                       Filter =:= <<"apple">> orelse Filter =:= <<"orange">>
+                                               end, AppleOrangeMessages),
+    ?assertEqual(WaveCount * 2, length(AppleOrangeFilteredMessages)),
+    ok = amqp10_client:detach_link(AppleOrangeReceiver),
+
+    %% filtering on "apple" and messages without a filter value
+    {ok, AppleUnfilteredReceiver} = amqp10_client:attach_receiver_link(
+                                      Session,
+                                      <<"test-receiver-3">>,
+                                      Address,
+                                      unsettled,
+                                      TerminusDurability,
+                                      #{<<"rabbitmq:stream-offset-spec">> => <<"first">>,
+                                        <<"rabbitmq:stream-filter">> => <<"apple">>,
+                                        <<"rabbitmq:stream-match-unfiltered">> => {boolean, true}}),
+    ok = amqp10_client:flow_link_credit(AppleUnfilteredReceiver, 100, 10),
+
+    AppleUnfilteredMessages = receive_all_messages(AppleUnfilteredReceiver, []),
+    %% we should get less than all the waves combined
+    ?assert(length(AppleUnfilteredMessages) < WaveCount * 3),
+    %% client-side filtering
+    AppleUnfilteredFilteredMessages = lists:filter(fun(Msg) ->
+                                                           AP = amqp10_msg:application_properties(Msg),
+                                                           not maps:is_key(<<"filter">>, AP) orelse
+                                                           maps:get(<<"filter">>, AP) =:= <<"apple">>
+                                                   end, AppleUnfilteredMessages),
+    ?assertEqual(WaveCount * 2, length(AppleUnfilteredFilteredMessages)),
+    ok = amqp10_client:detach_link(AppleUnfilteredReceiver),
+
+    delete_queue(Config, Stream),
+    ok = amqp10_client:close_connection(Connection).
+
+available_messages_classic_queue(Config) ->
+    available_messages(<<"classic">>, Config).
+
+available_messages_quorum_queue(Config) ->
+    available_messages(<<"quorum">>, Config).
+
+available_messages_stream(Config) ->
+    available_messages(<<"stream">>, Config).
+
+available_messages(QType, Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, QType}]}),
+
+    %% Attach 1 sender and 1 receiver to the queue.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address),
+    receive {amqp10_event, {link, Receiver, attached}} -> ok
+    after 5000 -> ct:fail("missing attached")
+    end,
+    flush(receiver_attached),
+
+    ?assertEqual(0, get_available_messages(Receiver)),
+
+    ok = send_messages(Sender, 3, false),
+    %% We know that Streams only return an approximation for available messages.
+    %% The committed Stream offset is queried by chunk ID.
+    %% So, we wait here a bit such that the 4th message goes into its own new chunk.
+    timer:sleep(50),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"4">>, <<"4">>, false)),
+    ok = wait_for_accepts(4),
+
+    OutputHandle = element(4, Receiver),
+    Flow = #'v1_0.flow'{
+              %% Grant 1 credit to the sending queue.
+              link_credit = {uint, 1},
+              %% Request sending queue to send us a FLOW including available messages.
+              echo = true},
+    ok = amqp10_client_session:flow(Session, OutputHandle, Flow, never),
+    receive_messages(Receiver, 1),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    eventually(?_assertEqual(3, get_available_messages(Receiver))),
+
+    %% Send a FLOW with echo=false and drain=false from client to server.
+    %% Even if the server doesn't reply with a FLOW, our client lib should
+    %% maintain the 'available' variable correctly.
+    ok = amqp10_client:flow_link_credit(Receiver, 1, never, false),
+    receive_messages(Receiver, 1),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    ?assertEqual(2, get_available_messages(Receiver)),
+
+    %% We force the queue to send us a FLOW including available messages
+    %% by granting more credits than messages being available and drain=true.
+    ok = amqp10_client:flow_link_credit(Receiver, 99, never, true),
+    receive_messages(Receiver, 2),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    ?assertEqual(0, get_available_messages(Receiver)),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+incoming_message_interceptors(Config) ->
+    Key = {rabbit, ?FUNCTION_NAME},
+    ok = rpc(Config, persistent_term, put, [Key, [{set_header_routing_node, false}]]),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/queue/", QName/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver">>, Address),
+    Address = <<"/queue/", QName/binary>>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address, settled),
+    ok = wait_for_credit(Sender),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag">>, <<"body">>, true)),
+
+    {ok, Msg} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"body">>], amqp10_msg:body(Msg)),
+    Node = atom_to_binary(get_node_config(Config, 0, nodename)),
+    ?assertMatch(#{<<"x-routed-by">> := Node},
+                 amqp10_msg:message_annotations(Msg)),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    delete_queue(Config, QName),
+    true = rpc(Config, persistent_term, erase, [Key]).
+
+trace(Config) ->
+    Node = atom_to_binary(get_node_config(Config, 0, nodename)),
+    TraceQ = <<"my trace queue">>,
+    Q = <<"my queue">>,
+    Qs = [Q, TraceQ],
+    RoutingKey = <<"my routing key">>,
+    Payload = <<"my payload">>,
+    CorrelationId = <<"my correlation 👀"/utf8>>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    [#'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = Q0}) || Q0 <- Qs],
+    #'queue.bind_ok'{} = amqp_channel:call(
+                           Ch, #'queue.bind'{queue       = TraceQ,
+                                             exchange    = <<"amq.rabbitmq.trace">>,
+                                             routing_key = <<"#">>}),
+    #'queue.bind_ok'{} = amqp_channel:call(
+                           Ch, #'queue.bind'{queue       = Q,
+                                             exchange    = <<"amq.direct">>,
+                                             routing_key = RoutingKey}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+
+    %% We expect traced messages for sessions created before and
+    %% sessions created after tracing is enabled.
+    {ok, SessionSender} = amqp10_client:begin_session_sync(Connection),
+    {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_on"]),
+    {ok, SessionReceiver} = amqp10_client:begin_session_sync(Connection),
+
+    {ok, Sender} = amqp10_client:attach_sender_link(SessionSender,
+                                                    <<"test-sender">>,
+                                                    <<"/exchange/amq.direct/", RoutingKey/binary>>),
+    ok = wait_for_credit(Sender),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(SessionReceiver,
+                                                        <<"test-receiver">>,
+                                                        <<"/amq/queue/", Q/binary>>),
+    Msg0 = amqp10_msg:new(<<"tag 1">>, Payload, true),
+    Msg = amqp10_msg:set_properties(#{correlation_id => CorrelationId}, Msg0),
+    ok = amqp10_client:send_msg(Sender, Msg),
+    {ok, _} = amqp10_client:get_msg(Receiver),
+
+    timer:sleep(20),
+    {#'basic.get_ok'{routing_key = <<"publish.amq.direct">>},
+     #amqp_msg{props = #'P_basic'{headers = PublishHeaders},
+               payload = Payload}} =
+    amqp_channel:call(Ch, #'basic.get'{queue = TraceQ}),
+    ?assertMatch(#{<<"exchange_name">> := <<"amq.direct">>,
+                   <<"routing_keys">> := [RoutingKey],
+                   <<"connection">> := <<"127.0.0.1:", _/binary>>,
+                   <<"node">> := Node,
+                   <<"vhost">> := <<"/">>,
+                   <<"channel">> := 1,
+                   <<"user">> := <<"guest">>,
+                   <<"properties">> := #{<<"correlation_id">> := CorrelationId},
+                   <<"routed_queues">> := [Q]},
+                 rabbit_misc:amqp_table(PublishHeaders)),
+
+    {#'basic.get_ok'{routing_key = <<"deliver.", Q/binary>>},
+     #amqp_msg{props = #'P_basic'{headers = DeliverHeaders},
+               payload = Payload}} =
+    amqp_channel:call(Ch, #'basic.get'{queue = TraceQ}),
+    ?assertMatch(#{<<"exchange_name">> := <<"amq.direct">>,
+                   <<"routing_keys">> := [RoutingKey],
+                   <<"connection">> := <<"127.0.0.1:", _/binary>>,
+                   <<"node">> := Node,
+                   <<"vhost">> := <<"/">>,
+                   <<"channel">> := 2,
+                   <<"user">> := <<"guest">>,
+                   <<"properties">> := #{<<"correlation_id">> := CorrelationId},
+                   <<"redelivered">> := 0},
+                 rabbit_misc:amqp_table(DeliverHeaders)),
+
+    {ok, _} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["trace_off"]),
+    ok = amqp10_client:send_msg(Sender, amqp10_msg:new(<<"tag 2">>, Payload, true)),
+    {ok, _} = amqp10_client:get_msg(Receiver),
+    timer:sleep(20),
+    ?assertMatch(#'basic.get_empty'{},
+                 amqp_channel:call(Ch, #'basic.get'{queue = TraceQ})),
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = end_session_sync(SessionSender),
+    ok = end_session_sync(SessionReceiver),
+    ok = amqp10_client:close_connection(Connection),
+    [delete_queue(Config, Q0) || Q0 <- Qs].
+
+%% https://www.rabbitmq.com/validated-user-id.html
+user_id(Config) ->
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/exchange/amq.direct/some-routing-key">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address),
+    ok = wait_for_credit(Sender),
+    flush(attached),
+
+    Msg1 = amqp10_msg:set_properties(#{user_id => <<"guest">>},
+                                     amqp10_msg:new(<<"t1">>, <<"m1">>, true)),
+    ok = amqp10_client:send_msg(Sender, Msg1),
+    receive Unexpected -> ct:fail({received_unexpected_message, Unexpected})
+    after 10 -> ok
+    end,
+
+    Msg2 = amqp10_msg:set_properties(#{user_id => <<"fake user">>},
+                                     amqp10_msg:new(<<"t2">>, <<"m2">>, true)),
+    ok = amqp10_client:send_msg(Sender, Msg2),
+    receive
+        {amqp10_event,
+         {session, Session,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
+              description = {utf8, <<"user_id property set to 'fake user' but authenticated user was 'guest'">>}}}}} -> ok
+    after 5000 -> flush(missing_ended),
+                  ct:fail("did not receive expected error")
+    end,
+
+    ok = amqp10_client:close_connection(Connection).
+
+message_ttl(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/amq/queue/", QName/binary>>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver">>, Address),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address),
+    wait_for_credit(Sender),
+
+    M1 = amqp10_msg:set_headers(#{ttl => 1}, amqp10_msg:new(<<"t1">>, <<"m1">>, false)),
+    M2 = amqp10_msg:set_headers(#{ttl => 60 * 1000}, amqp10_msg:new(<<"t2">>, <<"m2">>, false)),
+    ok = amqp10_client:send_msg(Sender, M1),
+    ok = amqp10_client:send_msg(Sender, M2),
+    ok = wait_for_accepts(2),
+    %% Wait for the first message to expire.
+    timer:sleep(50),
+    flush(pre_receive),
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never, true),
+    receive {amqp10_msg, Receiver, Msg} ->
+                ?assertEqual([<<"m2">>], amqp10_msg:body(Msg))
+    after 5000 -> ct:fail(delivery_timeout)
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+    receive Unexpected -> ct:fail({received_unexpected_message, Unexpected})
+    after 5 -> ok
+    end,
+
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:close_connection(Connection),
+    ok = delete_queue(Config, QName).
+
+%% For backward compatibility, deployment tools should be able to
+%% enable and disable the deprecated no-op AMQP 1.0 plugin.
+plugin(Config) ->
+    Node = 0,
+    Plugin = rabbitmq_amqp1_0,
+    %% rabbit/Makefile and rabbit/BUILD.bazel declare a test dependency on the rabbitmq_amqp1_0 plugin.
+    %% Therefore, we first disable, and then enable.
+    ?assertEqual(ok, rabbit_ct_broker_helpers:disable_plugin(Config, Node, Plugin)),
+    ?assertEqual(ok, rabbit_ct_broker_helpers:enable_plugin(Config, Node, Plugin)).
+
+%% Test that the idle timeout threshold is exceeded on the server
+%% when no frames are sent from client to server.
+idle_time_out_on_server(Config) ->
+    App = rabbit,
+    Par = heartbeat,
+    {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]),
+    %% Configure RabbitMQ to use an idle-time-out of 1 second.
+    ok = rpc(Config, application, set_env, [App, Par, 1]),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    %% Mock the server socket to not have received any bytes.
+    rabbit_ct_broker_helpers:setup_meck(Config),
+    Mod = rabbit_net,
+    ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+    ok = rpc(Config, meck, expect, [Mod, getstat, 2, {ok, [{recv_oct, 999}]}]),
+    %% The server "SHOULD try to gracefully close the connection using a close
+    %% frame with an error explaining why" [2.4.5].
+    %% Since we chose a heartbeat value of 1 second, the server should easily
+    %% close the connection within 5 seconds.
+    receive
+        {amqp10_event,
+         {connection, Connection,
+          {closed,
+           {resource_limit_exceeded,
+            <<"no frame received from client within idle timeout threshold">>}}}} -> ok
+    after 5000 ->
+              ct:fail({missing_event, ?LINE})
+    end,
+
+    ?assert(rpc(Config, meck, validate, [Mod])),
+    ok = rpc(Config, meck, unload, [Mod]),
+    ok = rpc(Config, application, set_env, [App, Par, DefaultVal]).
+
+%% Test that the idle timeout threshold is exceeded on the client
+%% when no frames are sent from server to client.
+idle_time_out_on_client(Config) ->
+    OpnConf0 = connection_config(Config),
+    %% Request the server to send us frames every second.
+    OpnConf = OpnConf0#{idle_time_out => 1000},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    receive Unexpected -> ct:fail({unexpected, Unexpected})
+    after 3100 -> ok
+    end,
+    ?assert(is_process_alive(Connection)),
+    %% All good, the server sent us frames every second.
+
+    %% Mock the server to not send anything.
+    rabbit_ct_broker_helpers:setup_meck(Config),
+    Mod = rabbit_net,
+    ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+    ok = rpc(Config, meck, expect, [Mod, send, 2, ok]),
+
+    %% Our client should time out within less than 5 seconds given that the
+    %% idle-time-out is 1 second.
+    receive
+        {amqp10_event,
+         {connection, Connection,
+          {closed,
+           {resource_limit_exceeded,
+            <<"remote idle-time-out">>}}}} -> ok
+    after 5000 ->
+              ct:fail({missing_event, ?LINE})
+    end,
+
+    ?assert(rpc(Config, meck, validate, [Mod])),
+    ok = rpc(Config, meck, unload, [Mod]).
+
+%% Test that RabbitMQ does not support idle timeouts smaller than 1 second.
+idle_time_out_too_short(Config) ->
+    OpnConf0 = connection_config(Config),
+    OpnConf = OpnConf0#{idle_time_out => 900},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, {closed, _}}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end.
+
+rabbit_status_connection_count(Config) ->
+    %% Close any open AMQP 0.9.1 connections from previous test cases.
+    ok = rabbit_ct_client_helpers:close_channels_and_connection(Config, 0),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    {ok, String} = rabbit_ct_broker_helpers:rabbitmqctl(Config, 0, ["status"]),
+    ?assertNotEqual(nomatch, string:find(String, "Connection count: 1")),
+
+    ok = amqp10_client:close_connection(Connection).
+
+handshake_timeout(Config) ->
+    App = rabbit,
+    Par = ?FUNCTION_NAME,
+    {ok, DefaultVal} = rpc(Config, application, get_env, [App, Par]),
+    ok = rpc(Config, application, set_env, [App, Par, 200]),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    {ok, Socket} = gen_tcp:connect("localhost", Port, [{active, false}]),
+    ?assertEqual({error, closed}, gen_tcp:recv(Socket, 0, 400)),
+    ok = rpc(Config, application, set_env, [App, Par, DefaultVal]).
+
+credential_expires(Config) ->
+    rabbit_ct_broker_helpers:setup_meck(Config),
+    Mod = rabbit_auth_backend_internal,
+    ok = rpc(Config, meck, new, [Mod, [no_link, passthrough]]),
+    ExpiryTimestamp = os:system_time(second) + 3,
+    ok = rpc(Config, meck, expect, [Mod, expiry_timestamp, 1, ExpiryTimestamp]),
+
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 2000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    %% Since we don't renew our credential, we expect the server to close our connection.
+    receive
+        {amqp10_event,
+         {connection, Connection,
+          {closed,
+           {unauthorized_access, <<"credential expired">>}}}} -> ok
+    after 10_000 ->
+              flush(?LINE),
+              ct:fail({missing_event, ?LINE})
+    end,
+
+    ?assert(rpc(Config, meck, validate, [Mod])),
+    ok = rpc(Config, meck, unload, [Mod]).
+
+%% Attaching to an exclusive source queue should fail.
+attach_to_exclusive_queue(Config) ->
+    QName = <<"my queue">>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{queue = QName,
+                                                   durable = true,
+                                                   exclusive = true}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", QName/binary>>,
+    {ok, _Receiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver">>, Address),
+    receive
+        {amqp10_event,
+         {session, Session,
+          {ended,
+           #'v1_0.error'{
+              condition = ?V_1_0_AMQP_ERROR_RESOURCE_LOCKED,
+              description = {utf8, <<"cannot obtain exclusive access to locked "
+                                     "queue 'my queue' in vhost '/'">>}}}}} -> ok
+    after 5000 -> ct:fail({missing_event, ?LINE})
+    end,
+
+    ok = amqp10_client:close_connection(Connection),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+classic_priority_queue(Config) ->
+    QName = atom_to_binary(?FUNCTION_NAME),
+    Address = <<"/amq/queue/", QName/binary>>,
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QName,
+                                     durable = true,
+                                     arguments = [{<<"x-max-priority">>, long, 10}]}),
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address),
+    wait_for_credit(Sender),
+
+    Out1 = amqp10_msg:set_headers(#{priority => 3,
+                                    durable => true}, amqp10_msg:new(<<"t1">>, <<"low prio">>, false)),
+    Out2 = amqp10_msg:set_headers(#{priority => 5,
+                                    durable => true}, amqp10_msg:new(<<"t2">>, <<"high prio">>, false)),
+    ok = amqp10_client:send_msg(Sender, Out1),
+    ok = amqp10_client:send_msg(Sender, Out2),
+    ok = wait_for_accepts(2),
+    flush(accepted),
+
+    %% The high prio message should be delivered first.
+    {ok, Receiver1} = amqp10_client:attach_receiver_link(Session, <<"receiver 1">>, Address, unsettled),
+    {ok, In1} = amqp10_client:get_msg(Receiver1),
+    ?assertEqual([<<"high prio">>], amqp10_msg:body(In1)),
+    ?assertEqual(5, amqp10_msg:header(priority, In1)),
+    ?assert(amqp10_msg:header(durable, In1)),
+    ok = amqp10_client:accept_msg(Receiver1, In1),
+
+    {ok, Receiver2} = amqp10_client:attach_receiver_link(Session, <<"receiver 2">>, Address, settled),
+    {ok, In2} = amqp10_client:get_msg(Receiver2),
+    ?assertEqual([<<"low prio">>], amqp10_msg:body(In2)),
+    ?assertEqual(3, amqp10_msg:header(priority, In2)),
+    ?assert(amqp10_msg:header(durable, In2)),
+
+    ok = amqp10_client:detach_link(Receiver1),
+    ok = amqp10_client:detach_link(Receiver2),
+    ok = amqp10_client:detach_link(Sender),
+    ok = end_session_sync(Session),
+    ok = amqp10_client:close_connection(Connection),
+    ok = delete_queue(Config, QName).
+
+%% internal
+%%
+
+receive_all_messages(Receiver, Acc) ->
+    receive {amqp10_msg, Receiver, Msg} ->
+                ok = amqp10_client:accept_msg(Receiver, Msg),
+                receive_all_messages(Receiver, [Msg | Acc])
+    after 500 ->
+              lists:reverse(Acc)
+    end.
+
+connection_config(Config) ->
+    connection_config(0, Config).
+
+connection_config(Node, Config) ->
+    Host = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, Node, tcp_port_amqp),
+    #{address => Host,
+      port => Port,
+      container_id => <<"my container">>,
+      sasl => {plain, <<"guest">>, <<"guest">>}}.
+
+flush(Prefix) ->
+    receive
+        Msg ->
+            ct:pal("~p flushed: ~p~n", [Prefix, Msg]),
+            flush(Prefix)
+    after 1 ->
+              ok
+    end.
+
+open_and_close_connection(Config) ->
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, Connection, opened}} -> ok
+    after 5000 -> ct:fail(opened_timeout)
+    end,
+    ok = close_connection_sync(Connection).
+
+% before we can send messages we have to wait for credit from the server
+wait_for_credit(Sender) ->
+    receive
+        {amqp10_event, {link, Sender, credited}} ->
+            ok
+    after 5000 ->
+              flush("wait_for_credit timed out"),
+              ct:fail(credited_timeout)
+    end.
+
+detach_link_sync(Link) ->
+    ok = amqp10_client:detach_link(Link),
+    ok = wait_for_link_detach(Link).
+
+wait_for_link_detach(Link) ->
+    receive
+        {amqp10_event, {link, Link, {detached, normal}}} ->
+            flush(?FUNCTION_NAME),
+            ok
+    after 5000 ->
+              flush("wait_for_link_detach timed out"),
+              ct:fail({link_detach_timeout, Link})
+    end.
+
+end_session_sync(Session) ->
+    ok = amqp10_client:end_session(Session),
+    ok = wait_for_session_end(Session).
+
+wait_for_session_end(Session) ->
+    receive
+        {amqp10_event, {session, Session, {ended, _}}} ->
+            flush(?FUNCTION_NAME),
+            ok
+    after 5000 ->
+              flush("wait_for_session_end timed out"),
+              ct:fail({session_end_timeout, Session})
+    end.
+
+close_connection_sync(Connection) ->
+    ok = amqp10_client:close_connection(Connection),
+    ok = wait_for_connection_close(Connection).
+
+wait_for_connection_close(Connection) ->
+    receive
+        {amqp10_event, {connection, Connection, {closed, normal}}} ->
+            flush(?FUNCTION_NAME),
+            ok
+    after 5000 ->
+              flush("wait_for_connection_close timed out"),
+              ct:fail({connection_close_timeout, Connection})
+    end.
+
+wait_for_settlement(Tag) ->
+    wait_for_settlement(Tag, accepted).
+
+wait_for_settlement(Tag, State) ->
+    receive
+        {amqp10_disposition, {State, Tag}} ->
+            ok
+    after 5000 ->
+              flush("wait_for_settlement timed out"),
+              ct:fail({settled_timeout, Tag})
+    end.
+
+wait_for_accepts(0) ->
+    ok;
+wait_for_accepts(N) ->
+    receive
+        {amqp10_disposition,{accepted,_}} ->
+            wait_for_accepts(N - 1)
+    after 5000 ->
+              ct:fail({missing_accepted, N})
+    end.
+
+delete_queue(Config, QName) -> 
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch).
+
+amqp091_get_msg_headers(Channel, QName) -> 
+    {#'basic.get_ok'{}, #amqp_msg{props = #'P_basic'{ headers= Headers}}}
+        = amqp_channel:call(Channel, #'basic.get'{queue = QName, no_ack = true}),
+    {ok, Headers}.
+
+create_amqp10_sender(Session, Address) -> 
+    {ok, Sender} = amqp10_client:attach_sender_link(
+                     Session, <<"test-sender">>, Address),
+    wait_for_credit(Sender),
+    {ok, Sender}.
+
+drain_queue(Session, Address, N) ->
+    flush("Before drain_queue"),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session,
+                       <<"test-receiver">>,
+                       Address,
+                       settled,
+                       configuration),
+    ok = amqp10_client:flow_link_credit(Receiver, 1000, never, true),
+    Msgs = receive_messages(Receiver, N),
+    flush("after drain"),
+    ok = amqp10_client:detach_link(Receiver),
+    {ok, Msgs}.
+
+receive_messages(Receiver, N) ->
+    receive_messages0(Receiver, N, []).
+
+receive_messages0(_Receiver, 0, Acc) ->
+    lists:reverse(Acc);
+receive_messages0(Receiver, N, Acc) ->
+    receive
+        {amqp10_msg, Receiver, Msg} -> 
+            receive_messages0(Receiver, N - 1, [Msg | Acc])
+    after 5000  ->
+              exit({timeout, {num_received, length(Acc)}, {num_missing, N}})
+    end.
+
+count_received_messages(Receiver) ->
+    count_received_messages0(Receiver, 0).
+
+count_received_messages0(Receiver, Count) ->
+    receive
+        {amqp10_msg, Receiver, _Msg} ->
+            count_received_messages0(Receiver, Count + 1)
+    after 200 ->
+              Count
+    end.
+
+send_messages(_Sender, 0, _Settled) ->
+    ok;
+send_messages(Sender, Left, Settled) ->
+    Bin = integer_to_binary(Left),
+    Msg = amqp10_msg:new(Bin, Bin, Settled),
+    case amqp10_client:send_msg(Sender, Msg) of
+        ok ->
+            send_messages(Sender, Left - 1, Settled);
+        {error, insufficient_credit} ->
+            ok = wait_for_credit(Sender),
+            %% The credited event we just processed could have been received some time ago,
+            %% i.e. we might have 0 credits right now. This happens in the following scenario:
+            %% 1. We (test case proc) send a message successfully, the client session proc decrements remaining link credit from 1 to 0.
+            %% 2. The server grants our client session proc new credits.
+            %% 3. The client session proc sends us (test case proc) a credited event.
+            %% 4. We didn't even notice that we ran out of credits temporarily. We send the next message, it succeeds,
+            %%    but do not process the credited event in our mailbox.
+            %% So, we must be defensive here and assume that the next amqp10_client:send/2 call might return {error, insufficient_credit}
+            %% again causing us then to really wait to receive a credited event (instead of just processing an old credited event).
+            send_messages(Sender, Left, Settled)
+    end.
+
+assert_link_credit_runs_out(_Sender, 0) ->
+    ct:fail(sufficient_link_credit);
+assert_link_credit_runs_out(Sender, Left) ->
+    Bin = integer_to_binary(Left),
+    Msg = amqp10_msg:new(Bin, Bin, true),
+    case amqp10_client:send_msg(Sender, Msg) of
+        ok ->
+            assert_link_credit_runs_out(Sender, Left - 1);
+        {error, insufficient_credit} ->
+            receive {amqp10_event, {link, Sender, credited}} ->
+                        ct:pal("credited with ~b messages left", [Left]),
+                        assert_link_credit_runs_out(Sender, Left - 1)
+            after 500 ->
+                      ct:pal("insufficient link credit with ~b messages left", [Left]),
+                      ok
+            end
+    end.
+
+send_messages_with_group_id(Sender, N, GroupId) ->
+    [begin
+         Bin = integer_to_binary(I),
+         Msg0 = amqp10_msg:new(Bin, Bin, true),
+         Props = #{group_id => GroupId},
+         Msg = amqp10_msg:set_properties(Props, Msg0),
+         ok = amqp10_client:send_msg(Sender, Msg)
+     end || I <- lists:seq(1, N)].
+
+assert_messages(QNameBin, NumTotalMsgs, NumUnackedMsgs, Config) ->
+    Vhost = ?config(rmq_vhost, Config),
+    eventually(
+      ?_assertEqual(
+         lists:sort([{messages, NumTotalMsgs}, {messages_unacknowledged, NumUnackedMsgs}]),
+         begin
+             {ok, Q} = rpc(Config, rabbit_amqqueue, lookup, [QNameBin, Vhost]),
+             Infos = rpc(Config, rabbit_amqqueue, info, [Q, [messages, messages_unacknowledged]]),
+             lists:sort(Infos)
+         end
+        ), 500, 5).
+
+serial_number_increment(S) ->
+    case S + 1 of
+        16#ffffffff + 1 -> 0;
+        S1 -> S1
+    end.
+
+consume_from_first(<<"stream">>) ->
+    #{<<"rabbitmq:stream-offset-spec">> => <<"first">>};
+consume_from_first(_) ->
+    #{}.
+
+%% Return the formatted state of a gen_server via sys:get_status/1.
+%% (sys:get_state/1 is unformatted)
+gen_server_state(Pid) ->
+    {status, _, _, L0} = sys:get_status(Pid, 20_000),
+    L1 = lists:last(L0),
+    {data, L2} = lists:last(L1),
+    proplists:get_value("State", L2).
+
+get_global_counters(Config) ->
+    get_global_counters0(Config, [{protocol, amqp10}]).
+
+get_global_counters(Config, QType) ->
+    get_global_counters0(Config, [{protocol, amqp10},
+                                  {queue_type, QType}]).
+
+get_global_counters0(Config, Key) ->
+    Overview = rpc(Config, rabbit_global_counters, overview, []),
+    maps:get(Key, Overview).
+
+get_available_messages({link_ref, receiver, Session, OutputHandle}) ->
+    {status, _Pid, _Mod, [_, _, _, _, Misc]} = sys:get_status(Session),
+    [State] = [S || {data, [{"State", S}]} <- Misc],
+    {_StateName, StateData} = State,
+    {ok, Links} = maps:find(links, StateData),
+    {ok, Link} = maps:find(OutputHandle, Links),
+    {ok, Available} = maps:find(available, Link),
+    Available.
+
+ra_name(Q) ->
+    binary_to_atom(<<"%2F_", Q/binary>>).
+
+has_local_member(QName) ->
+    case rabbit_amqqueue:lookup(QName) of
+        {ok, Q} ->
+            #{name := StreamId} = amqqueue:get_type_state(Q),
+            case rabbit_stream_coordinator:local_pid(StreamId) of
+                {ok, Pid} ->
+                    is_process_alive(Pid);
+                {error, _} ->
+                    false
+            end;
+        {error, _} ->
+            false
+    end.
diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl
new file mode 100644
index 000000000000..b9f053db6ce9
--- /dev/null
+++ b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl
@@ -0,0 +1,221 @@
+%% 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) 2016-2023 VMware, Inc. or its affiliates.  All rights reserved.
+
+-module(amqp_credit_api_v2_SUITE).
+
+-compile([export_all, nowarn_export_all]).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("amqp_client/include/amqp_client.hrl").
+
+all() ->
+    [
+     {group, cluster_size_1}
+    ].
+
+groups() ->
+    [
+     {cluster_size_1, [],
+      [credit_api_v2]}
+    ].
+
+suite() ->
+    [
+     {timetrap, {minutes, 10}}
+    ].
+
+init_per_suite(Config) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    rabbit_ct_helpers:run_setup_steps(Config, []).
+
+end_per_suite(Config) ->
+    rabbit_ct_helpers:run_teardown_steps(Config).
+
+init_per_group(_Group, Config0) ->
+    Config = rabbit_ct_helpers:merge_app_env(
+               Config0, {rabbit, [{forced_feature_flags_on_init, []}]}),
+    rabbit_ct_helpers:run_steps(Config,
+                                rabbit_ct_broker_helpers:setup_steps() ++
+                                rabbit_ct_client_helpers:setup_steps()).
+
+end_per_group(_Group, Config) ->
+    rabbit_ct_helpers:run_steps(Config,
+                                rabbit_ct_client_helpers:teardown_steps() ++
+                                rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(TestCase, Config) ->
+    case rabbit_ct_broker_helpers:is_feature_flag_supported(Config, TestCase) of
+        true ->
+            ?assertNot(rabbit_ct_broker_helpers:is_feature_flag_enabled(Config, TestCase)),
+            Config;
+        false ->
+            {skip, io_lib:format("feature flag ~s is unsupported", [TestCase])}
+    end.
+
+end_per_testcase(_TestCase, Config) ->
+    Config.
+
+credit_api_v2(Config) ->
+    CQ = <<"classic queue">>,
+    QQ = <<"quorum queue">>,
+    CQAddr = <<"/amq/queue/", CQ/binary>>,
+    QQAddr = <<"/amq/queue/", QQ/binary>>,
+
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = CQ}),
+    #'queue.declare_ok'{} = amqp_channel:call(
+                              Ch, #'queue.declare'{
+                                     queue = QQ,
+                                     durable = true,
+                                     arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
+    ok = rabbit_ct_client_helpers:close_channel(Ch),
+
+    Host = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    OpnConf = #{address => Host,
+                port => Port,
+                container_id => <<"my container">>,
+                sasl => {plain, <<"guest">>, <<"guest">>}},
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+
+    {ok, CQSender} = amqp10_client:attach_sender_link(Session, <<"cq sender">>, CQAddr),
+    {ok, QQSender} = amqp10_client:attach_sender_link(Session, <<"qq sender">>, QQAddr),
+    receive {amqp10_event, {link, CQSender, credited}} -> ok
+    after 5000 -> ct:fail(credited_timeout)
+    end,
+    receive {amqp10_event, {link, QQSender, credited}} -> ok
+    after 5000 -> ct:fail(credited_timeout)
+    end,
+
+    %% Send 40 messages to each queue.
+    NumMsgs = 40,
+    [begin
+         Bin = integer_to_binary(N),
+         ok = amqp10_client:send_msg(CQSender, amqp10_msg:new(Bin, Bin, true)),
+         ok = amqp10_client:send_msg(QQSender, amqp10_msg:new(Bin, Bin, true))
+     end || N <- lists:seq(1, NumMsgs)],
+    ok = amqp10_client:detach_link(CQSender),
+    ok = amqp10_client:detach_link(QQSender),
+
+    %% Consume with credit API v1
+    CQAttachArgs = #{handle => 300,
+                     name => <<"cq receiver 1">>,
+                     role => {receiver, #{address => CQAddr,
+                                          durable => configuration}, self()},
+                     snd_settle_mode => unsettled,
+                     rcv_settle_mode => first,
+                     filter => #{}},
+    {ok, CQReceiver1} = amqp10_client:attach_link(Session, CQAttachArgs),
+    QQAttachArgs = #{handle => 400,
+                     name => <<"qq receiver 1">>,
+                     role => {receiver, #{address => QQAddr,
+                                          durable => configuration}, self()},
+                     snd_settle_mode => unsettled,
+                     rcv_settle_mode => first,
+                     filter => #{}},
+    {ok, QQReceiver1} = amqp10_client:attach_link(Session, QQAttachArgs),
+
+    ok = consume_and_accept(10, CQReceiver1, Session),
+    ok = consume_and_accept(10, QQReceiver1, Session),
+
+    ?assertEqual(ok,
+                 rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FUNCTION_NAME)),
+    flush(enabled_feature_flag),
+
+    %% Consume with credit API v2
+    {ok, CQReceiver2} = amqp10_client:attach_receiver_link(
+                          Session, <<"cq receiver 2">>, CQAddr, unsettled),
+    {ok, QQReceiver2} = amqp10_client:attach_receiver_link(
+                          Session, <<"qq receiver 2">>, QQAddr, unsettled),
+    ok = consume_and_accept(10, CQReceiver2, Session),
+    ok = consume_and_accept(10, QQReceiver2, Session),
+
+    %% Consume via with credit API v1
+    ok = consume_and_accept(10, CQReceiver1, Session),
+    ok = consume_and_accept(10, QQReceiver1, Session),
+
+    %% Detach the credit API v1 links and attach with the same output handle.
+    ok = detach_sync(CQReceiver1),
+    ok = detach_sync(QQReceiver1),
+    {ok, CQReceiver3} = amqp10_client:attach_link(Session, CQAttachArgs),
+    {ok, QQReceiver3} = amqp10_client:attach_link(Session, QQAttachArgs),
+
+    %% The new links should use credit API v2
+    ok = consume_and_accept(10, CQReceiver3, Session),
+    ok = consume_and_accept(10, QQReceiver3, Session),
+
+    flush(pre_drain),
+    %% Draining should also work.
+    ok = amqp10_client:flow_link_credit(CQReceiver3, 10, never, true),
+    receive {amqp10_event, {link, CQReceiver3, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_credit_exhausted, ?LINE})
+    end,
+    receive Unexpected1 -> ct:fail({unexpected, ?LINE, Unexpected1})
+    after 20 -> ok
+    end,
+
+    ok = amqp10_client:flow_link_credit(QQReceiver3, 10, never, true),
+    receive {amqp10_event, {link, QQReceiver3, credit_exhausted}} -> ok
+    after 5000 -> ct:fail({missing_credit_exhausted, ?LINE})
+    end,
+    receive Unexpected2 -> ct:fail({unexpected, ?LINE, Unexpected2})
+    after 20 -> ok
+    end,
+
+    ok = detach_sync(CQReceiver2),
+    ok = detach_sync(QQReceiver2),
+    ok = detach_sync(CQReceiver3),
+    ok = detach_sync(QQReceiver3),
+    ok = amqp10_client:end_session(Session),
+    receive {amqp10_event, {session, Session, {ended, _}}} -> ok
+    after 5000 -> ct:fail(missing_ended)
+    end,
+    ok = amqp10_client:close_connection(Connection),
+    receive {amqp10_event, {connection, Connection, {closed, normal}}} -> ok
+    after 5000 -> ct:fail(missing_closed)
+    end.
+
+consume_and_accept(NumMsgs, Receiver, Session) ->
+    ok = amqp10_client:flow_link_credit(Receiver, NumMsgs, never),
+    Msgs = receive_messages(Receiver, NumMsgs),
+    ok = amqp10_client_session:disposition(
+           Session,
+           receiver,
+           amqp10_msg:delivery_id(hd(Msgs)),
+           amqp10_msg:delivery_id(lists:last(Msgs)),
+           true,
+           accepted).
+
+receive_messages(Receiver, N) ->
+    receive_messages0(Receiver, N, []).
+
+receive_messages0(_Receiver, 0, Acc) ->
+    lists:reverse(Acc);
+receive_messages0(Receiver, N, Acc) ->
+    receive
+        {amqp10_msg, Receiver, Msg} ->
+            receive_messages0(Receiver, N - 1, [Msg | Acc])
+    after 5000 ->
+              exit({timeout, {num_received, length(Acc)}, {num_missing, N}})
+    end.
+
+detach_sync(Receiver) ->
+    ok = amqp10_client:detach_link(Receiver),
+    receive {amqp10_event, {link, Receiver, {detached, normal}}} -> ok
+    after 5000 -> ct:fail({missing_detached, Receiver})
+    end.
+
+flush(Prefix) ->
+    receive
+        Msg ->
+            ct:pal("~ts flushed: ~p~n", [Prefix, Msg]),
+            flush(Prefix)
+    after 1 ->
+              ok
+    end.
diff --git a/deps/rabbitmq_amqp1_0/test/proxy_protocol_SUITE.erl b/deps/rabbit/test/amqp_proxy_protocol_SUITE.erl
similarity index 63%
rename from deps/rabbitmq_amqp1_0/test/proxy_protocol_SUITE.erl
rename to deps/rabbit/test/amqp_proxy_protocol_SUITE.erl
index 22d0c835be28..7743af325ffb 100644
--- a/deps/rabbitmq_amqp1_0/test/proxy_protocol_SUITE.erl
+++ b/deps/rabbit/test/amqp_proxy_protocol_SUITE.erl
@@ -5,88 +5,83 @@
 %% Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
 %%
 
--module(proxy_protocol_SUITE).
+-module(amqp_proxy_protocol_SUITE).
 
--include_lib("common_test/include/ct.hrl").
+-compile([export_all, nowarn_export_all]).
 
--compile(export_all).
+-include_lib("eunit/include/eunit.hrl").
+
+-import(rabbit_ct_helpers, [eventually/3]).
+-import(rabbit_ct_broker_helpers, [rpc/4]).
 
 -define(TIMEOUT, 5000).
 
 all() ->
-    [
-        {group, sequential_tests}
-    ].
-
-groups() -> [
-        {sequential_tests, [], [
-            proxy_protocol_v1,
-            proxy_protocol_v1_tls,
-            proxy_protocol_v2_local
-        ]}
+    [{group, tests}].
+
+groups() ->
+    [{tests, [shuffle],
+      [
+       v1,
+       v1_tls,
+       v2_local
+      ]}
     ].
 
 init_per_suite(Config) ->
     rabbit_ct_helpers:log_environment(),
-    Config1 = rabbit_ct_helpers:set_config(Config, [
-        {rmq_nodename_suffix, ?MODULE}
-    ]),
-    Config2 = rabbit_ct_helpers:merge_app_env(Config1, [
-        {rabbit, [
-            {proxy_protocol, true}
-        ]}
-    ]),
-    Config3 = rabbit_ct_helpers:set_config(Config2, {rabbitmq_ct_tls_verify, verify_none}),
-    rabbit_ct_helpers:run_setup_steps(Config3,
-        rabbit_ct_broker_helpers:setup_steps() ++
-        rabbit_ct_client_helpers:setup_steps()).
+    Config1 = rabbit_ct_helpers:set_config(
+                Config,
+                [{rmq_nodename_suffix, ?MODULE},
+                 {rabbitmq_ct_tls_verify, verify_none}]),
+    Config2 = rabbit_ct_helpers:merge_app_env(
+                Config1,
+                [{rabbit, [{proxy_protocol, true}]}]),
+    rabbit_ct_helpers:run_setup_steps(
+      Config2,
+      rabbit_ct_broker_helpers:setup_steps() ++
+      rabbit_ct_client_helpers:setup_steps()).
 
 end_per_suite(Config) ->
     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) ->
+    eventually(?_assertEqual(0, rpc(Config, ets, info, [connection_created, size])), 1000, 10),
     rabbit_ct_helpers:testcase_finished(Config, Testcase).
 
-proxy_protocol_v1(Config) ->
+v1(Config) ->
     Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
     {ok, Socket} = gen_tcp:connect({127,0,0,1}, Port,
-        [binary, {active, false}, {packet, raw}]),
+                                   [binary, {active, false}, {packet, raw}]),
     ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
     [ok = inet:send(Socket, amqp_1_0_frame(FrameType))
-        || FrameType <- [header_sasl, sasl_init, header_amqp, open, 'begin']],
+     || FrameType <- [header_sasl, sasl_init, header_amqp, open]],
     {ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT),
-    ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
-        ?MODULE, connection_name, []),
-    match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81 \\(\\d\\)">>, [{capture, none}]),
-    gen_tcp:close(Socket),
-    ok.
+    ConnectionName = rpc(Config, ?MODULE, connection_name, []),
+    match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81$">>, [{capture, none}]),
+    ok = gen_tcp:close(Socket).
 
-proxy_protocol_v1_tls(Config) ->
+v1_tls(Config) ->
     app_utils:start_applications([asn1, crypto, public_key, ssl]),
     Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp_tls),
     {ok, Socket} = gen_tcp:connect({127,0,0,1}, Port,
-        [binary, {active, false}, {packet, raw}]),
-    ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 81\r\n"),
+                                   [binary, {active, false}, {packet, raw}]),
+    ok = inet:send(Socket, "PROXY TCP4 192.168.1.1 192.168.1.2 80 82\r\n"),
     {ok, SslSocket} = ssl:connect(Socket, [{verify, verify_none}], ?TIMEOUT),
     [ok = ssl:send(SslSocket, amqp_1_0_frame(FrameType))
-        || FrameType <- [header_sasl, sasl_init, header_amqp, open, 'begin']],
+     || FrameType <- [header_sasl, sasl_init, header_amqp, open]],
     {ok, _Packet} = ssl:recv(SslSocket, 0, ?TIMEOUT),
     timer:sleep(1000),
-    ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
-        ?MODULE, connection_name, []),
-    match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:81 \\(\\d\\)$">>, [{capture, none}]),
-    gen_tcp:close(Socket),
-    ok.
+    ConnectionName = rpc(Config, ?MODULE, connection_name, []),
+    match = re:run(ConnectionName, <<"^192.168.1.1:80 -> 192.168.1.2:82$">>, [{capture, none}]),
+    ok = gen_tcp:close(Socket).
 
-proxy_protocol_v2_local(Config) ->
+v2_local(Config) ->
     ProxyInfo = #{
         command => local,
         version => 2
@@ -96,14 +91,11 @@ proxy_protocol_v2_local(Config) ->
         [binary, {active, false}, {packet, raw}]),
     ok = inet:send(Socket, ranch_proxy_header:header(ProxyInfo)),
     [ok = inet:send(Socket, amqp_1_0_frame(FrameType))
-        || FrameType <- [header_sasl, sasl_init, header_amqp, open, 'begin']],
+        || FrameType <- [header_sasl, sasl_init, header_amqp, open]],
     {ok, _Packet} = gen_tcp:recv(Socket, 0, ?TIMEOUT),
-    ConnectionName = rabbit_ct_broker_helpers:rpc(Config, 0,
-        ?MODULE, connection_name, []),
-    match = re:run(ConnectionName, <<"^127.0.0.1:\\d+ -> 127.0.0.1:\\d+ \\(\\d\\)$">>, [{capture, none}]),
-    gen_tcp:close(Socket),
-    ok.
-
+    ConnectionName = rpc(Config, ?MODULE, connection_name, []),
+    match = re:run(ConnectionName, <<"^127.0.0.1:\\d+ -> 127.0.0.1:\\d+$">>, [{capture, none}]),
+    ok = gen_tcp:close(Socket).
 
 %% hex frames to send to have the connection recorded in RabbitMQ
 %% use wireshark with one of the Java tests to record those
@@ -114,9 +106,7 @@ amqp_1_0_frame(header_amqp) ->
 amqp_1_0_frame(sasl_init) ->
     hex_frame_to_binary("0000001902010000005341c00c01a309414e4f4e594d4f5553");
 amqp_1_0_frame(open) ->
-    hex_frame_to_binary("0000003f02000000005310c03202a12438306335323662332d653530662d343835352d613564302d336466643738623537633730a1096c6f63616c686f7374");
-amqp_1_0_frame('begin') ->
-    hex_frame_to_binary("0000002002000000005311c01305405201707fffffff707fffffff700000ffff").
+    hex_frame_to_binary("0000003f02000000005310c03202a12438306335323662332d653530662d343835352d613564302d336466643738623537633730a1096c6f63616c686f7374").
 
 hex_frame_to_binary(HexsString) ->
     Hexs = split(HexsString, []),
@@ -135,18 +125,16 @@ connection_name() ->
     %% hence the retry
     case retry(fun connection_registered/0, 20) of
         true ->
-            Connections = ets:tab2list(connection_created),
-            {_Key, Values} = lists:nth(1, Connections),
+            [{_Key, Values}] = ets:tab2list(connection_created),
             {_, Name} = lists:keyfind(name, 1, Values),
             Name;
         false ->
-            error
+            ct:fail("not 1 connection registered")
     end.
 
 connection_registered() ->
-    I = ets:info(connection_created),
-    Size = proplists:get_value(size, I),
-    Size > 0.
+    Size = ets:info(connection_created, size),
+    Size =:= 1.
 
 retry(_Function, 0) ->
     false;
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE.erl b/deps/rabbit/test/amqp_system_SUITE.erl
similarity index 71%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE.erl
rename to deps/rabbit/test/amqp_system_SUITE.erl
index 138ef0247671..1f1889f582f4 100644
--- a/deps/rabbitmq_amqp1_0/test/system_SUITE.erl
+++ b/deps/rabbit/test/amqp_system_SUITE.erl
@@ -5,7 +5,7 @@
 %% Copyright (c) 2007-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
 %%
 
--module(system_SUITE).
+-module(amqp_system_SUITE).
 
 -include_lib("common_test/include/ct.hrl").
 -include_lib("rabbit_common/include/rabbit_framing.hrl").
@@ -58,27 +58,11 @@ init_per_suite(Config) ->
 end_per_suite(Config) ->
     Config.
 
-init_per_group(streams, Config) ->
-    case rabbit_ct_helpers:is_mixed_versions() of
-        false ->
-            Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
-            Config1 = rabbit_ct_helpers:set_config(Config, [
-                {rmq_nodename_suffix, Suffix},
-                {amqp10_client_library, dotnet}
-              ]),
-            rabbit_ct_helpers:run_setup_steps(Config1, [
-                fun build_dotnet_test_project/1
-            ] ++
-            rabbit_ct_broker_helpers:setup_steps() ++
-            rabbit_ct_client_helpers:setup_steps());
-        _     ->
-            {skip, "stream tests are skipped in mixed mode"}
-    end;
 init_per_group(Group, Config) ->
     Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
     Config1 = rabbit_ct_helpers:set_config(Config, [
         {rmq_nodename_suffix, Suffix},
-        {amqp10_client_library, Group}
+        {amqp_client_library, Group}
       ]),
     GroupSetupStep = case Group of
         dotnet -> fun build_dotnet_test_project/1;
@@ -131,76 +115,51 @@ build_maven_test_project(Config) ->
 %% -------------------------------------------------------------------
 
 roundtrip(Config) ->
-    run(Config, [
-        {dotnet, "roundtrip"},
-        {java, "RoundTripTest"}
-      ]).
+    run(Config, [{dotnet, "roundtrip"},
+                 {java, "RoundTripTest"}]).
 
 streams(Config) ->
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    #'queue.declare_ok'{} =
-        amqp_channel:call(Ch, #'queue.declare'{queue = <<"stream_q2">>,
-                                               durable = true,
-                                               arguments = [{<<"x-queue-type">>, longstr, "stream"}]}),
-    run(Config, [
-        {dotnet, "streams"}
-      ]).
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
+    amqp_channel:call(Ch, #'queue.declare'{queue = <<"stream_q2">>,
+                                           durable = true,
+                                           arguments = [{<<"x-queue-type">>, longstr, "stream"}]}),
+    run(Config, [{dotnet, "streams"}]).
 
 roundtrip_to_amqp_091(Config) ->
-    run(Config, [
-        {dotnet, "roundtrip_to_amqp_091"}
-      ]).
+    run(Config, [{dotnet, "roundtrip_to_amqp_091"}]).
 
 default_outcome(Config) ->
-    run(Config, [
-        {dotnet, "default_outcome"}
-      ]).
+    run(Config, [{dotnet, "default_outcome"}]).
 
 no_routes_is_released(Config) ->
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
     amqp_channel:call(Ch, #'exchange.declare'{exchange = <<"no_routes_is_released">>,
                                               durable = true}),
-    run(Config, [
-        {dotnet, "no_routes_is_released"}
-      ]).
+    run(Config, [{dotnet, "no_routes_is_released"}]).
 
 outcomes(Config) ->
-    run(Config, [
-        {dotnet, "outcomes"}
-      ]).
+    run(Config, [{dotnet, "outcomes"}]).
 
 fragmentation(Config) ->
-    run(Config, [
-        {dotnet, "fragmentation"}
-      ]).
+    run(Config, [{dotnet, "fragmentation"}]).
 
 message_annotations(Config) ->
-    run(Config, [
-        {dotnet, "message_annotations"}
-      ]).
+    run(Config, [{dotnet, "message_annotations"}]).
 
 footer(Config) ->
-    run(Config, [
-        {dotnet, "footer"}
-      ]).
+    run(Config, [{dotnet, "footer"}]).
 
 data_types(Config) ->
-    run(Config, [
-        {dotnet, "data_types"}
-      ]).
+    run(Config, [{dotnet, "data_types"}]).
 
 reject(Config) ->
-    run(Config, [
-        {dotnet, "reject"}
-      ]).
+    run(Config, [{dotnet, "reject"}]).
 
 redelivery(Config) ->
-    run(Config, [
-        {dotnet, "redelivery"}
-      ]).
+    run(Config, [{dotnet, "redelivery"}]).
 
 routing(Config) ->
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
+    Ch = rabbit_ct_client_helpers:open_channel(Config),
     amqp_channel:call(Ch, #'queue.declare'{queue = <<"transient_q">>,
                                            durable = false}),
     amqp_channel:call(Ch, #'queue.declare'{queue = <<"durable_q">>,
@@ -217,18 +176,6 @@ routing(Config) ->
         {dotnet, "routing"}
       ]).
 
-%% TODO: this tests doesn't test anything that the standard routing test
-%% already does. We should test stream specific things here like attaching
-%% to a given offset
-stream_interop_basics(Config) ->
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    amqp_channel:call(Ch, #'queue.declare'{queue   = <<"stream_q">>,
-                                           durable = true,
-                                           arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}),
-    run(Config, [
-        {dotnet, "routing"}
-      ]).
-
 invalid_routes(Config) ->
     run(Config, [
         {dotnet, "invalid_routes"}
@@ -238,7 +185,7 @@ auth_failure(Config) ->
     run(Config, [ {dotnet, "auth_failure"} ]).
 
 access_failure(Config) ->
-    User = <<"access_failure">>,
+    User = atom_to_binary(?FUNCTION_NAME),
     rabbit_ct_broker_helpers:add_user(Config, User, <<"boo">>),
     rabbit_ct_broker_helpers:set_permissions(Config, User, <<"/">>,
                                              <<".*">>, %% configure
@@ -248,12 +195,12 @@ access_failure(Config) ->
     run(Config, [ {dotnet, "access_failure"} ]).
 
 access_failure_not_allowed(Config) ->
-    User = <<"access_failure_not_allowed">>,
+    User = atom_to_binary(?FUNCTION_NAME),
     rabbit_ct_broker_helpers:add_user(Config, User, <<"boo">>),
     run(Config, [ {dotnet, "access_failure_not_allowed"} ]).
 
 access_failure_send(Config) ->
-    User = <<"access_failure_send">>,
+    User = atom_to_binary(?FUNCTION_NAME),
     rabbit_ct_broker_helpers:add_user(Config, User, <<"boo">>),
     rabbit_ct_broker_helpers:set_permissions(Config, User, <<"/">>,
                                              <<".*">>, %% configure
@@ -263,15 +210,13 @@ access_failure_send(Config) ->
     run(Config, [ {dotnet, "access_failure_send"} ]).
 
 run(Config, Flavors) ->
-    ClientLibrary = ?config(amqp10_client_library, Config),
+    ClientLibrary = ?config(amqp_client_library, Config),
     Fun = case ClientLibrary of
-        dotnet -> fun run_dotnet_test/2;
-        java   -> fun run_java_test/2
-    end,
-    case proplists:get_value(ClientLibrary, Flavors) of
-        false    -> ok;
-        TestName -> Fun(Config, TestName)
-    end.
+              dotnet -> fun run_dotnet_test/2;
+              java   -> fun run_java_test/2
+          end,
+    {ClientLibrary, TestName} = proplists:lookup(ClientLibrary, Flavors),
+    Fun(Config, TestName).
 
 run_dotnet_test(Config, Method) ->
     TestProjectDir = ?config(dotnet_test_project_dir, Config),
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/Program.cs b/deps/rabbit/test/amqp_system_SUITE_data/console/Program.cs
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/Program.cs
rename to deps/rabbit/test/amqp_system_SUITE_data/console/Program.cs
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/README.md b/deps/rabbit/test/amqp_system_SUITE_data/console/README.md
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/README.md
rename to deps/rabbit/test/amqp_system_SUITE_data/console/README.md
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/standalone.csproj b/deps/rabbit/test/amqp_system_SUITE_data/console/standalone.csproj
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/console/standalone.csproj
rename to deps/rabbit/test/amqp_system_SUITE_data/console/standalone.csproj
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/Program.fs b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
similarity index 96%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/Program.fs
rename to deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
index cf8fb2e6d9d5..67910bfaf148 100755
--- a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/Program.fs
+++ b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
@@ -203,9 +203,11 @@ module Test =
             receiver.SetCredit(100, true)
             let rtd = receiver.Receive()
             assertNotNull rtd
-            assertTrue (rtd.MessageAnnotations.Map.Count = 1)
-            let (result, _) =  rtd.MessageAnnotations.Map.TryGetValue("x-stream-offset")
-            assertTrue result
+            assertEqual 3 rtd.MessageAnnotations.Map.Count
+            assertTrue (rtd.MessageAnnotations.Map.ContainsKey(Symbol "x-stream-offset"))
+            assertTrue (rtd.MessageAnnotations.Map.ContainsKey(Symbol "x-exchange"))
+            assertTrue (rtd.MessageAnnotations.Map.ContainsKey(Symbol "x-routing-key"))
+
             assertEqual body rtd.Body
             assertEqual rtd.Properties.CorrelationId  corr
             receiver.Close()
@@ -216,7 +218,7 @@ module Test =
     let roundtrip_to_amqp_091 uri =
         use c = connect uri
         let q = "roundtrip-091-q"
-        let corr = "corrlation"
+        let corr = "correlation"
         let sender = SenderLink(c.Session, q + "-sender" , q)
 
         new Message("hi"B, Header = Header(),
@@ -300,7 +302,8 @@ module Test =
 
         assertEqual m.Body m'.Body
         assertEqual (m.MessageAnnotations.Descriptor) (m'.MessageAnnotations.Descriptor)
-        assertEqual 2 (m'.MessageAnnotations.Map.Count)
+        // our 2 custom annotations + x-exchange + x-routing-key = 4
+        assertEqual 4 (m'.MessageAnnotations.Map.Count)
         assertTrue (m.MessageAnnotations.[k1] = m'.MessageAnnotations.[k1])
         assertTrue (m.MessageAnnotations.[k2] = m'.MessageAnnotations.[k2])
 
@@ -312,7 +315,7 @@ module Test =
         let k2 = Symbol "key2"
         footer.[Symbol "key1"] <- "value1"
         footer.[Symbol "key2"] <- "value2"
-        let m = new Message("testing annotations", Footer = footer)
+        let m = new Message("testing footer", Footer = footer)
         sender.Send m
         let m' = receive receiver
 
@@ -432,7 +435,7 @@ module Test =
                 receiver.Close()
             with
             | :? Amqp.AmqpException as ae ->
-                assertEqual (ae.Error.Condition) (Symbol cond)
+                assertEqual (Symbol cond) (ae.Error.Condition)
             | _ -> failwith "invalid expection thrown"
 
     let authFailure uri =
@@ -456,8 +459,6 @@ module Test =
                 ))
             let sender = new SenderLink(ac.Session, "test-sender", dest)
             sender.Send(new Message "hi", TimeSpan.FromSeconds 15.)
-
-
             failwith "expected exception not received"
         with
         | :? Amqp.AmqpException as ex ->
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/fsharp-tests.fsproj b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/fsharp-tests.fsproj
similarity index 83%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/fsharp-tests.fsproj
rename to deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/fsharp-tests.fsproj
index 157790aa61e0..e8817002e0b4 100755
--- a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/fsharp-tests.fsproj
+++ b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/fsharp-tests.fsproj
@@ -8,7 +8,7 @@
   
   
     
-    
-    
+    
+    
   
 
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/global.json b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/global.json
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/fsharp-tests/global.json
rename to deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/global.json
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.gitignore b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/.gitignore
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.gitignore
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/.gitignore
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/MavenWrapperDownloader.java b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/MavenWrapperDownloader.java
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/MavenWrapperDownloader.java
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/MavenWrapperDownloader.java
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.jar b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.jar
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.jar
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.jar
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.properties b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.properties
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.properties
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/.mvn/wrapper/maven-wrapper.properties
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/mvnw b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/mvnw
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/mvnw
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/mvnw
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/mvnw.cmd b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/mvnw.cmd
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/mvnw.cmd
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/mvnw.cmd
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/pom.xml b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/pom.xml
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/pom.xml
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/pom.xml
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/src/test/java/com/rabbitmq/amqp1_0/tests/jms/RoundTripTest.java b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/src/test/java/com/rabbitmq/amqp1_0/tests/jms/RoundTripTest.java
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/src/test/java/com/rabbitmq/amqp1_0/tests/jms/RoundTripTest.java
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/src/test/java/com/rabbitmq/amqp1_0/tests/jms/RoundTripTest.java
diff --git a/deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/src/test/resources/logback-test.xml b/deps/rabbit/test/amqp_system_SUITE_data/java-tests/src/test/resources/logback-test.xml
similarity index 100%
rename from deps/rabbitmq_amqp1_0/test/system_SUITE_data/java-tests/src/test/resources/logback-test.xml
rename to deps/rabbit/test/amqp_system_SUITE_data/java-tests/src/test/resources/logback-test.xml
diff --git a/deps/rabbit/test/event_recorder.erl b/deps/rabbit/test/event_recorder.erl
new file mode 100644
index 000000000000..08a621ddcd1c
--- /dev/null
+++ b/deps/rabbit/test/event_recorder.erl
@@ -0,0 +1,71 @@
+%% 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-2023 VMware, Inc. or its affiliates.  All rights reserved.
+%%
+
+-module(event_recorder).
+-behaviour(gen_event).
+
+-include_lib("stdlib/include/assert.hrl").
+-include_lib("rabbit_common/include/rabbit.hrl").
+
+%% gen_event callbacks
+-export([init/1,
+         handle_event/2,
+         handle_call/2]).
+%% client API
+-export([start/1,
+         stop/1,
+         get_events/1]).
+-export([assert_event_type/2,
+         assert_event_prop/2]).
+
+-import(rabbit_ct_broker_helpers,
+        [get_node_config/3]).
+
+-define(INIT_STATE, []).
+
+init(_) ->
+    {ok, ?INIT_STATE}.
+
+handle_event(#event{type = T}, State)
+  when T =:= node_stats orelse
+       T =:= node_node_stats orelse
+       T =:= node_node_deleted ->
+    {ok, State};
+handle_event(Event, State) ->
+    {ok, [Event | State]}.
+
+handle_call(take_state, State) ->
+    {ok, lists:reverse(State), ?INIT_STATE}.
+
+start(Config) ->
+    ok = rabbit_ct_broker_helpers:add_code_path_to_all_nodes(Config, ?MODULE),
+    ok = gen_event:add_handler(event_manager_ref(Config), ?MODULE, []).
+
+stop(Config) ->
+    ok = gen_event:delete_handler(event_manager_ref(Config), ?MODULE, []).
+
+get_events(Config) ->
+    %% events are sent and processed asynchronously
+    timer:sleep(500),
+    Result = gen_event:call(event_manager_ref(Config), ?MODULE, take_state),
+    ?assert(is_list(Result)),
+    Result.
+
+event_manager_ref(Config) ->
+    Node = get_node_config(Config, 0, nodename),
+    {rabbit_event, Node}.
+
+assert_event_type(ExpectedType, #event{type = ActualType}) ->
+    ?assertEqual(ExpectedType, ActualType).
+
+assert_event_prop(ExpectedProp = {Key, _Value}, #event{props = Props}) ->
+    ?assertEqual(ExpectedProp, lists:keyfind(Key, 1, Props));
+assert_event_prop(ExpectedProps, Event)
+  when is_list(ExpectedProps) ->
+    lists:foreach(fun(P) ->
+                          assert_event_prop(P, Event)
+                  end, ExpectedProps).
diff --git a/deps/rabbit/test/message_size_limit_SUITE.erl b/deps/rabbit/test/message_size_limit_SUITE.erl
index 5b2a852c7c5d..b3e0fb4210ec 100644
--- a/deps/rabbit/test/message_size_limit_SUITE.erl
+++ b/deps/rabbit/test/message_size_limit_SUITE.erl
@@ -7,29 +7,21 @@
 
 -module(message_size_limit_SUITE).
 
--include_lib("common_test/include/ct.hrl").
--include_lib("kernel/include/file.hrl").
+-compile([export_all, nowarn_export_all]).
 -include_lib("amqp_client/include/amqp_client.hrl").
 -include_lib("eunit/include/eunit.hrl").
-
--compile(export_all).
-
--define(TIMEOUT_LIST_OPS_PASS, 5000).
--define(TIMEOUT, 30000).
 -define(TIMEOUT_CHANNEL_EXCEPTION, 5000).
 
--define(CLEANUP_QUEUE_NAME, <<"cleanup-queue">>).
-
 all() ->
     [
-      {group, parallel_tests}
+     {group, tests}
     ].
 
 groups() ->
     [
-      {parallel_tests, [parallel], [
-          max_message_size
-       ]}
+     {tests, [], [
+                  max_message_size
+                 ]}
     ].
 
 suite() ->
@@ -81,8 +73,7 @@ max_message_size(Config) ->
     Size2Mb = 1024 * 1024 * 2,
     Size2Mb = byte_size(Binary2M),
 
-    rabbit_ct_broker_helpers:rpc(Config, 0,
-                                 application, set_env, [rabbit, max_message_size, 1024 * 1024 * 3]),
+    ok = rabbit_ct_broker_helpers:rpc(Config, persistent_term, put, [max_message_size, 1024 * 1024 * 3]),
 
     {_, Ch} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
 
@@ -96,8 +87,7 @@ max_message_size(Config) ->
     assert_channel_fail_max_size(Ch, Monitor),
 
     %% increase the limit
-    rabbit_ct_broker_helpers:rpc(Config, 0,
-                                 application, set_env, [rabbit, max_message_size, 1024 * 1024 * 8]),
+    ok = rabbit_ct_broker_helpers:rpc(Config, persistent_term, put, [max_message_size, 1024 * 1024 * 8]),
 
     {_, Ch1} = rabbit_ct_client_helpers:open_connection_and_channel(Config, 0),
 
@@ -112,15 +102,7 @@ max_message_size(Config) ->
 
     Monitor1 = monitor(process, Ch1),
     amqp_channel:call(Ch1, #'basic.publish'{routing_key = <<"none">>}, #amqp_msg{payload = Binary10M}),
-    assert_channel_fail_max_size(Ch1, Monitor1),
-
-    %% increase beyond the hard limit
-    rabbit_ct_broker_helpers:rpc(Config, 0,
-                                 application, set_env, [rabbit, max_message_size, 1024 * 1024 * 600]),
-    Val = rabbit_ct_broker_helpers:rpc(Config, 0,
-                                       rabbit_channel, get_max_message_size, []),
-
-    ?assertEqual(?MAX_MSG_SIZE, Val).
+    assert_channel_fail_max_size(Ch1, Monitor1).
 
 %% -------------------------------------------------------------------
 %% Implementation
diff --git a/deps/rabbit/test/rabbit_confirms_SUITE.erl b/deps/rabbit/test/rabbit_confirms_SUITE.erl
index 7252a9102a17..0bd3ef00cee8 100644
--- a/deps/rabbit/test/rabbit_confirms_SUITE.erl
+++ b/deps/rabbit/test/rabbit_confirms_SUITE.erl
@@ -64,8 +64,8 @@ confirm(_Config) ->
     ?assertEqual(undefined, rabbit_confirms:smallest(U7)),
 
     U8 = rabbit_confirms:insert(2, [QName], XName, U1),
-    {[{1, XName}, {2, XName}], _U9} = rabbit_confirms:confirm([1, 2], QName, U8),
-    ok.
+    {[{Seq1, XName}, {Seq2, XName}], _U9} = rabbit_confirms:confirm([1, 2], QName, U8),
+    ?assertEqual([1, 2], lists:sort([Seq1, Seq2])).
 
 
 reject(_Config) ->
@@ -94,8 +94,7 @@ reject(_Config) ->
     {ok, {2, XName}, U5} = rabbit_confirms:reject(2, U3),
     {error, not_found} = rabbit_confirms:reject(2, U5),
     ?assertEqual(1, rabbit_confirms:size(U5)),
-    ?assertEqual(1, rabbit_confirms:smallest(U5)),
-    ok.
+    ?assertEqual(1, rabbit_confirms:smallest(U5)).
 
 remove_queue(_Config) ->
     XName = rabbit_misc:r(<<"/">>, exchange, <<"X">>),
@@ -114,5 +113,5 @@ remove_queue(_Config) ->
 
     U5 = rabbit_confirms:insert(1, [QName], XName, U0),
     U6 = rabbit_confirms:insert(2, [QName], XName, U5),
-    {[{1, XName}, {2, XName}], _U} = rabbit_confirms:remove_queue(QName, U6),
-    ok.
+    {[{Seq1, XName}, {Seq2, XName}], _U} = rabbit_confirms:remove_queue(QName, U6),
+    ?assertEqual([1, 2], lists:sort([Seq1, Seq2])).
diff --git a/deps/rabbit/test/rabbit_fifo_SUITE.erl b/deps/rabbit/test/rabbit_fifo_SUITE.erl
index 8362833b336e..7eeb31daaa6a 100644
--- a/deps/rabbit/test/rabbit_fifo_SUITE.erl
+++ b/deps/rabbit/test/rabbit_fifo_SUITE.erl
@@ -34,17 +34,11 @@ all_tests() ->
 
 groups() ->
     [
-     {machine_version_2, [], all_tests()},
-     {machine_version_3, [], all_tests()},
-     {machine_version_conversion, [], [convert_v2_to_v3]}
+     {machine_version_2, [shuffle], all_tests()},
+     {machine_version_3, [shuffle], all_tests()},
+     {machine_version_conversion, [shuffle], [convert_v2_to_v3]}
     ].
 
-init_per_suite(Config) ->
-    Config.
-
-end_per_suite(_Config) ->
-    ok.
-
 init_per_group(machine_version_2, Config) ->
     [{machine_version, 2} | Config];
 init_per_group(machine_version_3, Config) ->
@@ -55,12 +49,6 @@ init_per_group(machine_version_conversion, Config) ->
 end_per_group(_Group, _Config) ->
     ok.
 
-init_per_testcase(_TestCase, Config) ->
-    Config.
-
-end_per_testcase(_TestCase, _Config) ->
-    ok.
-
 %%%===================================================================
 %%% Test cases
 %%%===================================================================
@@ -91,8 +79,7 @@ end_per_testcase(_TestCase, _Config) ->
 test_init(Name) ->
     init(#{name => Name,
            max_in_memory_length => 0,
-           queue_resource => rabbit_misc:r("/", queue,
-                                           atom_to_binary(Name, utf8)),
+           queue_resource => rabbit_misc:r("/", queue, atom_to_binary(Name)),
            release_cursor_interval => 0}).
 
 enq_enq_checkout_test(C) ->
@@ -109,7 +96,7 @@ enq_enq_checkout_test(C) ->
     ?ASSERT_EFF({log, [1,2], _Fun, _Local}, Effects),
     ok.
 
-credit_enq_enq_checkout_settled_credit_test(C) ->
+credit_enq_enq_checkout_settled_credit_v1_test(C) ->
     Cid = {?FUNCTION_NAME, self()},
     {State1, _} = enq(C, 1, 1, first, test_init(test)),
     {State2, _} = enq(C, 2, 2, second, State1),
@@ -122,7 +109,8 @@ credit_enq_enq_checkout_settled_credit_test(C) ->
     {State4, SettledEffects} = settle(C, Cid, 4, 1, State3),
     ?assertEqual(false, lists:any(fun ({log, _, _, _}) ->
                                           true;
-                                      (_) -> false
+                                      (_) ->
+                                          false
                                   end, SettledEffects)),
     %% granting credit (3) should deliver the second msg if the receivers
     %% delivery count is (1)
@@ -136,8 +124,43 @@ credit_enq_enq_checkout_settled_credit_test(C) ->
                                   end, FinalEffects)),
     ok.
 
-credit_with_drained_test(C) ->
-    Cid = {?FUNCTION_NAME, self()},
+credit_enq_enq_checkout_settled_credit_v2_test(C) ->
+    Ctag = ?FUNCTION_NAME,
+    Cid = {Ctag, self()},
+    {State1, _} = enq(C, 1, 1, first, test_init(test)),
+    {State2, _} = enq(C, 2, 2, second, State1),
+    {State3, _, Effects} = apply(meta(C, 3),
+                                 rabbit_fifo:make_checkout(
+                                   Cid,
+                                   {auto, 1, credited},
+                                   %% denotes that credit API v2 is used
+                                   #{initial_delivery_count => 16#ff_ff_ff_ff}),
+                                 State2),
+    ?ASSERT_EFF({monitor, _, _}, Effects),
+    ?ASSERT_EFF({log, [1], _Fun, _Local}, Effects),
+    %% Settling the delivery should not grant new credit.
+    {State4, SettledEffects} = settle(C, Cid, 4, 1, State3),
+    ?assertEqual(false, lists:any(fun ({log, _, _, _}) ->
+                                          true;
+                                      (_) ->
+                                          false
+                                  end, SettledEffects)),
+    {State5, CreditEffects} = credit(C, Cid, 5, 1, 0, false, State4),
+    ?ASSERT_EFF({log, [2], _, _}, CreditEffects),
+    %% The credit_reply should be sent **after** the delivery.
+    ?assertEqual({send_msg, self(),
+                  {credit_reply, Ctag, _DeliveryCount = 1, _Credit = 0, _Available = 0, _Drain = false},
+                  ?DELIVERY_SEND_MSG_OPTS},
+                 lists:last(CreditEffects)),
+    {_State6, FinalEffects} = enq(C, 6, 3, third, State5),
+    ?assertEqual(false, lists:any(fun ({log, _, _, _}) ->
+                                          true;
+                                      (_) -> false
+                                  end, FinalEffects)).
+
+credit_with_drained_v1_test(C) ->
+    Ctag = ?FUNCTION_NAME,
+    Cid = {Ctag, self()},
     State0 = test_init(test),
     %% checkout with a single credit
     {State1, _, _} =
@@ -147,17 +170,42 @@ credit_with_drained_test(C) ->
                                                        delivery_count = 0}}},
                  State1),
     {State, Result, _} =
-         apply(meta(C, 3), rabbit_fifo:make_credit(Cid, 0, 5, true), State1),
-    ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0,
-                                                       delivery_count = 5}}},
+         apply(meta(C, 3), rabbit_fifo:make_credit(Cid, 5, 0, true), State1),
+         ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0,
+                                                                  delivery_count = 5}}},
                  State),
     ?assertEqual({multi, [{send_credit_reply, 0},
-                          {send_drained, {?FUNCTION_NAME, 5}}]},
+                          {send_drained, {Ctag, 5}}]},
                            Result),
     ok.
 
-credit_and_drain_test(C) ->
-    Cid = {?FUNCTION_NAME, self()},
+credit_with_drained_v2_test(C) ->
+    Ctag = ?FUNCTION_NAME,
+    Cid = {Ctag, self()},
+    State0 = test_init(test),
+    %% checkout with a single credit
+    {State1, _, _} = apply(meta(C, 1),
+                           rabbit_fifo:make_checkout(
+                             Cid,
+                             {auto, 1, credited},
+                             %% denotes that credit API v2 is used
+                             #{initial_delivery_count => 0}),
+                           State0),
+    ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 1,
+                                                             delivery_count = 0}}},
+                 State1),
+    {State, ok, Effects} = apply(meta(C, 3), rabbit_fifo:make_credit(Cid, 5, 0, true), State1),
+    ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0,
+                                                             delivery_count = 5}}},
+                 State),
+    ?assertEqual([{send_msg, self(),
+                   {credit_reply, Ctag, _DeliveryCount = 5, _Credit = 0, _Available = 0, _Drain = true},
+                   ?DELIVERY_SEND_MSG_OPTS}],
+                 Effects).
+
+credit_and_drain_v1_test(C) ->
+    Ctag = ?FUNCTION_NAME,
+    Cid = {Ctag, self()},
     {State1, _} = enq(C, 1, 1, first, test_init(test)),
     {State2, _} = enq(C, 2, 2, second, State1),
     %% checkout without any initial credit (like AMQP 1.0 would)
@@ -167,7 +215,7 @@ credit_and_drain_test(C) ->
 
     ?ASSERT_NO_EFF({log, _, _, _}, CheckEffs),
     {State4, {multi, [{send_credit_reply, 0},
-                      {send_drained, {?FUNCTION_NAME, 2}}]},
+                      {send_drained, {Ctag, 2}}]},
     Effects} = apply(meta(C, 4), rabbit_fifo:make_credit(Cid, 4, 0, true), State3),
     ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0,
                                                              delivery_count = 4}}},
@@ -178,7 +226,36 @@ credit_and_drain_test(C) ->
     ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs),
     ok.
 
+credit_and_drain_v2_test(C) ->
+    Ctag = ?FUNCTION_NAME,
+    Cid = {Ctag, self()},
+    {State1, _} = enq(C, 1, 1, first, test_init(test)),
+    {State2, _} = enq(C, 2, 2, second, State1),
+    {State3, _, CheckEffs} = apply(meta(C, 3),
+                                   rabbit_fifo:make_checkout(
+                                     Cid,
+                                     %% checkout without any initial credit (like AMQP 1.0 would)
+                                     {auto, 0, credited},
+                                     %% denotes that credit API v2 is used
+                                     #{initial_delivery_count => 16#ff_ff_ff_ff - 1}),
+                                   State2),
+    ?ASSERT_NO_EFF({log, _, _, _}, CheckEffs),
+
+    {State4, ok, Effects} = apply(meta(C, 4),
+                                  rabbit_fifo:make_credit(Cid, 4, 16#ff_ff_ff_ff - 1, true),
+                                  State3),
+    ?assertMatch(#rabbit_fifo{consumers = #{Cid := #consumer{credit = 0,
+                                                             delivery_count = 2}}},
+                 State4),
+    ?ASSERT_EFF({log, [1, 2], _, _}, Effects),
+    %% The credit_reply should be sent **after** the deliveries.
+    ?assertEqual({send_msg, self(),
+                  {credit_reply, Ctag, _DeliveryCount = 2, _Credit = 0, _Available = 0, _Drain = true},
+                  ?DELIVERY_SEND_MSG_OPTS},
+                 lists:last(Effects)),
 
+    {_State5, EnqEffs} = enq(C, 5, 2, third, State4),
+    ?ASSERT_NO_EFF({log, _, _, _}, EnqEffs).
 
 enq_enq_deq_test(C) ->
     Cid = {?FUNCTION_NAME, self()},
@@ -1402,10 +1479,9 @@ single_active_cancelled_with_unacked_test(C) ->
     ?assertMatch([], rabbit_fifo:query_waiting_consumers(State6)),
     ok.
 
-single_active_with_credited_test(C) ->
+single_active_with_credited_v1_test(C) ->
     State0 = init(#{name => ?FUNCTION_NAME,
-                    queue_resource => rabbit_misc:r("/", queue,
-                        atom_to_binary(?FUNCTION_NAME, utf8)),
+                    queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME)),
                     release_cursor_interval => 0,
                     single_active_consumer_on => true}),
 
@@ -1435,6 +1511,45 @@ single_active_with_credited_test(C) ->
                  rabbit_fifo:query_waiting_consumers(State3)),
     ok.
 
+single_active_with_credited_v2_test(C) ->
+    State0 = init(#{name => ?FUNCTION_NAME,
+                    queue_resource => rabbit_misc:r("/", queue, atom_to_binary(?FUNCTION_NAME)),
+                    release_cursor_interval => 0,
+                    single_active_consumer_on => true}),
+    C1 = {<<"ctag1">>, self()},
+    {State1, _, _} = apply(meta(C, 1),
+                           make_checkout(C1,
+                                         {auto, 0, credited},
+                                         %% denotes that credit API v2 is used
+                                         #{initial_delivery_count => 0}),
+                           State0),
+    C2 = {<<"ctag2">>, self()},
+    {State2, _, _} = apply(meta(C, 2),
+                           make_checkout(C2,
+                                         {auto, 0, credited},
+                                         %% denotes that credit API v2 is used
+                                         #{initial_delivery_count => 0}),
+                           State1),
+    %% add some credit
+    C1Cred = rabbit_fifo:make_credit(C1, 5, 0, false),
+    {State3, ok, Effects1} = apply(meta(C, 3), C1Cred, State2),
+    ?assertEqual([{send_msg, self(),
+                   {credit_reply, <<"ctag1">>, _DeliveryCount = 0, _Credit = 5, _Available = 0, _Drain = false},
+                   ?DELIVERY_SEND_MSG_OPTS}],
+                 Effects1),
+
+    C2Cred = rabbit_fifo:make_credit(C2, 4, 0, false),
+    {State, ok, Effects2} = apply(meta(C, 4), C2Cred, State3),
+    ?assertEqual({send_msg, self(),
+                  {credit_reply, <<"ctag2">>, _DeliveryCount = 0, _Credit = 4, _Available = 0, _Drain = false},
+                  ?DELIVERY_SEND_MSG_OPTS},
+                 Effects2),
+
+    %% both consumers should have credit
+    ?assertMatch(#{C1 := #consumer{credit = 5}},
+                 State#rabbit_fifo.consumers),
+    ?assertMatch([{C2, #consumer{credit = 4}}],
+                 rabbit_fifo:query_waiting_consumers(State)).
 
 register_enqueuer_test(C) ->
     State0 = init(#{name => ?FUNCTION_NAME,
diff --git a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
index 39f4d52b01c0..e2df8e370e0c 100644
--- a/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
+++ b/deps/rabbit/test/rabbit_fifo_int_SUITE.erl
@@ -32,7 +32,8 @@ all_tests() ->
      discard,
      cancel_checkout,
      lost_delivery,
-     credit,
+     credit_api_v1,
+     credit_api_v2,
      untracked_enqueue,
      flow,
      test_queries,
@@ -42,7 +43,7 @@ all_tests() ->
 
 groups() ->
     [
-     {tests, [], all_tests()}
+     {tests, [shuffle], all_tests()}
     ].
 
 init_per_group(_, Config) ->
@@ -441,7 +442,7 @@ lost_delivery(Config) ->
                     end),
     ok.
 
-credit(Config) ->
+credit_api_v1(Config) ->
     ClusterName = ?config(cluster_name, Config),
     ServerId = ?config(node_id, Config),
     ok = start_cluster(ClusterName, [ServerId]),
@@ -450,21 +451,27 @@ credit(Config) ->
     {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1),
     {_, _, F3} = process_ra_events(receive_ra_events(2, 0), ClusterName, F2),
     %% checkout with 0 prefetch
-    {ok, F4} = rabbit_fifo_client:checkout(<<"tag">>, 0, credited, #{}, F3),
+    CTag = <<"my-tag">>,
+    {ok, F4} = rabbit_fifo_client:checkout(CTag, 0, credited, #{}, F3),
     %% assert no deliveries
     {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [],
                                    fun
                                        (D, _) -> error({unexpected_delivery, D})
                                    end),
     %% provide some credit
-    {F6, []} = rabbit_fifo_client:credit(<<"tag">>, 1, false, F5),
-    {[{_, _, _, _, m1}], [{send_credit_reply, _}], F7} =
-        process_ra_events(receive_ra_events(1, 1), ClusterName, F6),
+    {F6, []} = rabbit_fifo_client:credit_v1(CTag, 1, false, F5),
+    {[{_, _, _, _, m1}], [{send_credit_reply, 1}], F7} =
+    process_ra_events(receive_ra_events(1, 1), ClusterName, F6),
 
     %% credit and drain
-    {F8, []} = rabbit_fifo_client:credit(<<"tag">>, 4, true, F7),
-    {[{_, _, _, _, m2}], [{send_credit_reply, _}, {send_drained, _}], F9} =
-        process_ra_events(receive_ra_events(2, 1), ClusterName, F8),
+    Drain = true,
+    {F8, []} = rabbit_fifo_client:credit_v1(CTag, 4, Drain, F7),
+    AvailableAfterCheckout = 0,
+    {[{_, _, _, _, m2}],
+     [{send_credit_reply, AvailableAfterCheckout},
+      {credit_reply_v1, CTag, _CreditAfterCheckout = 3,
+       AvailableAfterCheckout, Drain}],
+     F9} = process_ra_events(receive_ra_events(2, 1), ClusterName, F8),
     flush(),
 
     %% enqueue another message - at this point the consumer credit should be
@@ -476,10 +483,78 @@ credit(Config) ->
                                         (D, _) -> error({unexpected_delivery, D})
                                     end),
     %% credit again and receive the last message
-    {F12, []} = rabbit_fifo_client:credit(<<"tag">>, 10, false, F11),
+    {F12, []} = rabbit_fifo_client:credit_v1(CTag, 10, false, F11),
     {[{_, _, _, _, m3}], _, _} = process_ra_events(receive_ra_events(1, 1), ClusterName, F12),
     ok.
 
+credit_api_v2(Config) ->
+    ClusterName = ?config(cluster_name, Config),
+    ServerId = ?config(node_id, Config),
+    ok = start_cluster(ClusterName, [ServerId]),
+    F0 = rabbit_fifo_client:init([ServerId], 4),
+    %% Enqueue 2 messages.
+    {ok, F1, []} = rabbit_fifo_client:enqueue(ClusterName, m1, F0),
+    {ok, F2, []} = rabbit_fifo_client:enqueue(ClusterName, m2, F1),
+    {_, _, F3} = process_ra_events(receive_ra_events(2, 0), ClusterName, F2),
+    CTag = <<"my-tag">>,
+    DC0 = 16#ff_ff_ff_ff,
+    DC1 = 0, %% = DC0 + 1 using 32 bit serial number arithmetic
+    {ok, F4} = rabbit_fifo_client:checkout(
+                 %% initial_delivery_count in consumer meta means credit API v2.
+                 CTag, 0, credited, #{initial_delivery_count => DC0}, F3),
+    %% assert no deliveries
+    {_, _, F5} = process_ra_events(receive_ra_events(), ClusterName, F4, [], [],
+                                   fun
+                                       (D, _) -> error({unexpected_delivery, D})
+                                   end),
+    %% Grant 1 credit.
+    {F6, []} = rabbit_fifo_client:credit(CTag, DC0, 1, false, _Echo0 = true, F5),
+    %% We expect exactly 1 message due to 1 credit being granted.
+    {[{_, _, _, _, m1}],
+     %% We expect a credit_reply action due to echo=true
+     [{credit_reply, CTag, DC1, _Credit0 = 0, _Available0 = 1, _Drain0 = false}],
+     F7} = process_ra_events(receive_ra_events(), ClusterName, F6),
+
+    %% Again, grant 1 credit.
+    %% However, because we still use the initial delivery count DC0, rabbit_fifo
+    %% wont' send us a new message since it already sent us m1 for that old delivery-count.
+    %% In other words, this credit top up simulates in-flight deliveries.
+    {F8, []} = rabbit_fifo_client:credit(CTag, DC0, 1, false, _Echo1 = true, F7),
+    {_NoMessages = [],
+     %% We still expect a credit_reply action due to echo=true
+     [{credit_reply, CTag, DC1, _Credit1 = 0, _Available1 = 1, _Drain1 = false}],
+     F9} = process_ra_events(receive_ra_events(), ClusterName, F8),
+
+    %% Grant 4 credits and drain.
+    {F10, []} = rabbit_fifo_client:credit(CTag, DC1, 4, true, _Echo2 = false, F9),
+    %% rabbit_fifo should advance the delivery-count as much as possible
+    %% consuming all credits due to drain=true and insufficient messages in the queue.
+    DC2 = DC1 + 4,
+    %% We expect to receive m2 which is the only message in the queue.
+    {[{_, _, _, _, m2}],
+     %% Even though echo=false, we still expect a credit_reply action due
+     %% drain=true and insufficient messages in the queue.
+     [{credit_reply, CTag, DC2, _Credit2 = 0, _Available2 = 0, _Drain2 = true}],
+     F11} = process_ra_events(receive_ra_events(), ClusterName, F10),
+    flush(),
+
+    %% Enqueue another message.
+    %% At this point the consumer credit should be all used up due to the drain.
+    {ok, F12, []} = rabbit_fifo_client:enqueue(ClusterName, m3, F11),
+    %% assert no deliveries
+    {_, _, F13} = process_ra_events(receive_ra_events(), ClusterName, F12, [], [],
+                                    fun
+                                        (D, _) -> error({unexpected_delivery, D})
+                                    end),
+
+    %% Grant 10 credits and receive the last message.
+    {F14, []} = rabbit_fifo_client:credit(CTag, DC2, 10, false, _Echo = false, F13),
+    ?assertMatch(
+       {[{_, _, _, _, m3}],
+        %% Due to echo=false, we don't expect a credit_reply action.
+        _NoCreditReplyAction = [],
+        _F15}, process_ra_events(receive_ra_events(), ClusterName, F14)).
+
 untracked_enqueue(Config) ->
     ClusterName = ?config(cluster_name, Config),
     ServerId = ?config(node_id, Config),
diff --git a/deps/rabbit/test/rabbit_stream_queue_SUITE.erl b/deps/rabbit/test/rabbit_stream_queue_SUITE.erl
index 2203632f6ec9..b50fb4537307 100644
--- a/deps/rabbit/test/rabbit_stream_queue_SUITE.erl
+++ b/deps/rabbit/test/rabbit_stream_queue_SUITE.erl
@@ -19,7 +19,7 @@
 -define(WAIT, 5000).
 
 suite() ->
-    [{timetrap, 15 * 60000}].
+    [{timetrap, 15 * 60_000}].
 
 all() ->
     [
@@ -1712,11 +1712,6 @@ consume_from_replica(Config) ->
     rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
 
 consume_credit(Config) ->
-    %% Because osiris provides one chunk on every read and we don't want to buffer
-    %% messages in the broker to avoid memory penalties, the credit value won't
-    %% be strict - we allow it into the negative values.
-    %% We can test that after receiving a chunk, no more messages are delivered until
-    %% the credit goes back to a positive value.
     [Server | _] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
 
     Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
@@ -1736,39 +1731,54 @@ consume_credit(Config) ->
     qos(Ch1, Credit, false),
     subscribe(Ch1, Q, false, 0),
 
-    %% Receive everything
-    DeliveryTags = receive_batch(),
+    %% We expect to receive exactly 2 messages.
+    DTag1 = receive {#'basic.deliver'{delivery_tag = Tag1}, _} -> Tag1
+            after 5000 -> ct:fail({missing_delivery, ?LINE})
+            end,
+    _DTag2 = receive {#'basic.deliver'{delivery_tag = Tag2}, _} -> Tag2
+             after 5000 -> ct:fail({missing_delivery, ?LINE})
+             end,
+    receive {#'basic.deliver'{}, _} -> ct:fail({unexpected_delivery, ?LINE})
+    after 100 -> ok
+    end,
 
-    %% We receive at least the given credit as we know there are 100 messages in the queue
-    ?assert(length(DeliveryTags) >= Credit),
+    %% When we ack the 1st message, we should receive exactly 1 more message
+    ok = amqp_channel:cast(Ch1, #'basic.ack'{delivery_tag = DTag1,
+                                             multiple = false}),
+    DTag3 = receive {#'basic.deliver'{delivery_tag = Tag3}, _} -> Tag3
+            after 5000 -> ct:fail({missing_delivery, ?LINE})
+            end,
+    receive {#'basic.deliver'{}, _} ->
+                ct:fail({unexpected_delivery, ?LINE})
+    after 100 -> ok
+    end,
 
-    %% Let's ack as many messages as we can while avoiding a positive credit for new deliveries
-    {ToAck, Pending} = lists:split(length(DeliveryTags) - Credit, DeliveryTags),
+    %% Whenever we ack 2 messages, we should receive exactly 2 more messages.
+    ok = consume_credit0(Ch1, DTag3),
 
-    [ok = amqp_channel:cast(Ch1, #'basic.ack'{delivery_tag = DeliveryTag,
-                                              multiple     = false})
-     || DeliveryTag <- ToAck],
+    rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
 
-    %% Nothing here, this is good
-    receive
-        {#'basic.deliver'{}, _} ->
-            exit(unexpected_delivery)
-    after 1000 ->
-            ok
+consume_credit0(_Ch, DTag)
+  when DTag > 50 ->
+    %% sufficiently tested
+    ok;
+consume_credit0(Ch, DTagPrev) ->
+    %% Ack 2 messages.
+    ok = amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DTagPrev,
+                                            multiple = true}),
+    %% Receive 1st message.
+    receive {#'basic.deliver'{}, _} -> ok
+    after 5000 -> ct:fail({missing_delivery, ?LINE})
     end,
-
-    %% Let's ack one more, we should receive a new chunk
-    ok = amqp_channel:cast(Ch1, #'basic.ack'{delivery_tag = hd(Pending),
-                                             multiple     = false}),
-
-    %% Yeah, here is the new chunk!
-    receive
-        {#'basic.deliver'{}, _} ->
-            ok
-    after 5000 ->
-            exit(timeout)
+    %% Receive 2nd message.
+    DTag = receive {#'basic.deliver'{delivery_tag = T}, _} -> T
+           after 5000 -> ct:fail({missing_delivery, ?LINE})
+           end,
+    %% We shouldn't receive more messages given that AMQP 0.9.1 prefetch count is 2.
+    receive {#'basic.deliver'{}, _} -> ct:fail({unexpected_delivery, ?LINE})
+    after 10 -> ok
     end,
-    rabbit_ct_broker_helpers:rpc(Config, 0, ?MODULE, delete_testcase_queue, [Q]).
+    consume_credit0(Ch, DTag).
 
 consume_credit_out_of_order_ack(Config) ->
     %% Like consume_credit but acknowledging the messages out of order.
diff --git a/deps/rabbit/test/single_active_consumer_SUITE.erl b/deps/rabbit/test/single_active_consumer_SUITE.erl
index b2b265bc2332..6945d213b85a 100644
--- a/deps/rabbit/test/single_active_consumer_SUITE.erl
+++ b/deps/rabbit/test/single_active_consumer_SUITE.erl
@@ -25,6 +25,7 @@ groups() ->
         {classic_queue, [], [
             all_messages_go_to_one_consumer,
             fallback_to_another_consumer_when_first_one_is_cancelled,
+            fallback_to_another_consumer_when_first_one_is_cancelled_qos1,
             fallback_to_another_consumer_when_exclusive_consumer_channel_is_cancelled,
             fallback_to_another_consumer_when_first_one_is_cancelled_manual_acks,
             amqp_exclusive_consume_fails_on_exclusive_consumer_queue
@@ -32,6 +33,7 @@ groups() ->
         {quorum_queue, [], [
             all_messages_go_to_one_consumer,
             fallback_to_another_consumer_when_first_one_is_cancelled,
+            fallback_to_another_consumer_when_first_one_is_cancelled_qos1,
             fallback_to_another_consumer_when_exclusive_consumer_channel_is_cancelled,
             fallback_to_another_consumer_when_first_one_is_cancelled_manual_acks,
             basic_get_is_unsupported
@@ -165,6 +167,49 @@ fallback_to_another_consumer_when_first_one_is_cancelled(Config) ->
     amqp_connection:close(C),
     ok.
 
+fallback_to_another_consumer_when_first_one_is_cancelled_qos1(Config) ->
+    {C, Ch} = connection_and_channel(Config),
+    Q = queue_declare(Ch, Config),
+    ?assertEqual(#'basic.qos_ok'{},
+                 amqp_channel:call(Ch, #'basic.qos'{prefetch_count = 1})),
+    CTag1 = <<"tag1">>,
+    CTag2 = <<"tag2">>,
+    amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q,
+                                                consumer_tag = CTag1}, self()),
+    receive #'basic.consume_ok'{consumer_tag = CTag1} -> ok
+    after 5000 -> ct:fail(timeout_ctag1)
+    end,
+
+    amqp_channel:subscribe(Ch, #'basic.consume'{queue = Q,
+                                                consumer_tag = CTag2}, self()),
+    receive #'basic.consume_ok'{consumer_tag = CTag2} -> ok
+    after 5000 -> ct:fail(timeout_ctag2)
+    end,
+
+    Publish = #'basic.publish'{exchange = <<>>, routing_key = Q},
+    amqp_channel:cast(Ch, Publish, #amqp_msg{payload = <<"m1">>}),
+    amqp_channel:cast(Ch, Publish, #amqp_msg{payload = <<"m2">>}),
+
+    DTag1 = receive {#'basic.deliver'{consumer_tag = CTag1,
+                                      delivery_tag = DTag},
+                     #amqp_msg{payload = <<"m1">>}} -> DTag
+            after 5000 -> ct:fail(timeout_m1)
+            end,
+
+    #'basic.cancel_ok'{consumer_tag = CTag1} = amqp_channel:call(Ch, #'basic.cancel'{consumer_tag = CTag1}),
+    receive #'basic.cancel_ok'{consumer_tag = CTag1} -> ok
+    after 5000 -> ct:fail(missing_cancel)
+    end,
+
+    amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DTag1}),
+
+    receive {#'basic.deliver'{consumer_tag = CTag2},
+             #amqp_msg{payload = <<"m2">>}} -> ok;
+            Unexpected -> ct:fail({unexpected, Unexpected})
+    after 5000 -> ct:fail(timeout_m2)
+    end,
+    amqp_connection:close(C).
+
 fallback_to_another_consumer_when_first_one_is_cancelled_manual_acks(Config) ->
     %% Let's ensure that although the consumer is cancelled we still keep the unacked
     %% messages and accept acknowledgments on them.
@@ -292,7 +337,7 @@ queue_declare(Channel, Config) ->
 
 consume({Parent, State, 0}) ->
     Parent ! {consumer_done, State};
-consume({Parent, {MessagesPerConsumer, MessageCount}, CountDown}) ->
+consume({Parent, {MessagesPerConsumer, MessageCount}, CountDown} = Arg) ->
     receive
         #'basic.consume_ok'{consumer_tag = CTag} ->
             consume({Parent, {maps:put(CTag, 0, MessagesPerConsumer), MessageCount}, CountDown});
@@ -307,9 +352,9 @@ consume({Parent, {MessagesPerConsumer, MessageCount}, CountDown}) ->
             consume({Parent, NewState, CountDown - 1});
         #'basic.cancel_ok'{consumer_tag = CTag} ->
             Parent ! {cancel_ok, CTag},
-            consume({Parent, {MessagesPerConsumer, MessageCount}, CountDown});
+            consume(Arg);
         _ ->
-            consume({Parent, {MessagesPerConsumer, MessageCount}, CountDown})
+            consume(Arg)
     after ?TIMEOUT ->
               Parent ! {consumer_timeout, {MessagesPerConsumer, MessageCount}},
               flush(),
diff --git a/deps/rabbit/test/unit_access_control_SUITE.erl b/deps/rabbit/test/unit_access_control_SUITE.erl
index 6650a5789fef..3bab2d7bb416 100644
--- a/deps/rabbit/test/unit_access_control_SUITE.erl
+++ b/deps/rabbit/test/unit_access_control_SUITE.erl
@@ -8,11 +8,10 @@
 -module(unit_access_control_SUITE).
 
 -include_lib("common_test/include/ct.hrl").
--include_lib("kernel/include/file.hrl").
 -include_lib("amqp_client/include/amqp_client.hrl").
 -include_lib("eunit/include/eunit.hrl").
 
--compile(export_all).
+-compile([export_all, nowarn_export_all]).
 
 all() ->
     [
@@ -24,7 +23,7 @@ groups() ->
     [
       {parallel_tests, [parallel], [
           password_hashing,
-          unsupported_connection_refusal
+          version_negotiation
       ]},
       {sequential_tests, [], [
           login_with_credentials_but_no_password,
@@ -278,20 +277,37 @@ auth_backend_internal_expand_topic_permission(_Config) ->
         ),
     ok.
 
-unsupported_connection_refusal(Config) ->
-    passed = rabbit_ct_broker_helpers:rpc(Config, 0,
-      ?MODULE, unsupported_connection_refusal1, [Config]).
+%% Test AMQP 1.0 §2.2
+version_negotiation(Config) ->
+    ok = rabbit_ct_broker_helpers:rpc(Config, ?MODULE, version_negotiation1, [Config]).
 
-unsupported_connection_refusal1(Config) ->
+version_negotiation1(Config) ->
     H = ?config(rmq_hostname, Config),
     P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    [passed = test_unsupported_connection_refusal(H, P, V) ||
-        V <- [<<"AMQP",9,9,9,9>>, <<"AMQP",0,1,0,0>>, <<"XXXX",0,0,9,1>>]],
-    passed.
 
-test_unsupported_connection_refusal(H, P, Header) ->
+    [?assertEqual(<<"AMQP",0,1,0,0>>, version_negotiation2(H, P, Vsn)) ||
+     Vsn <- [<<"AMQP",0,1,0,0>>,
+             <<"AMQP",0,1,0,1>>,
+             <<"AMQP",0,1,1,0>>,
+             <<"AMQP",0,9,1,0>>,
+             <<"AMQP",0,0,8,0>>,
+             <<"XXXX",0,1,0,0>>,
+             <<"XXXX",0,0,9,1>>]],
+
+    [?assertEqual(<<"AMQP",3,1,0,0>>, version_negotiation2(H, P, Vsn)) ||
+     Vsn <- [<<"AMQP",1,1,0,0>>,
+             <<"AMQP",4,1,0,0>>,
+             <<"AMQP",9,1,0,0>>]],
+
+    [?assertEqual(<<"AMQP",0,0,9,1>>, version_negotiation2(H, P, Vsn)) ||
+     Vsn <- [<<"AMQP",0,0,9,2>>,
+             <<"AMQP",0,0,10,0>>,
+             <<"AMQP",0,0,10,1>>]],
+    ok.
+
+version_negotiation2(H, P, Header) ->
     {ok, C} = gen_tcp:connect(H, P, [binary, {active, false}]),
     ok = gen_tcp:send(C, Header),
-    {ok, <<"AMQP",0,0,9,1>>} = gen_tcp:recv(C, 8, 100),
+    {ok, ServerVersion} = gen_tcp:recv(C, 8, 100),
     ok = gen_tcp:close(C),
-    passed.
+    ServerVersion.
diff --git a/deps/rabbit_common/app.bzl b/deps/rabbit_common/app.bzl
index ac112ef6043c..e81949b8b189 100644
--- a/deps/rabbit_common/app.bzl
+++ b/deps/rabbit_common/app.bzl
@@ -73,6 +73,7 @@ def all_beam_files(name = "all_beam_files"):
             "src/rabbit_queue_collector.erl",
             "src/rabbit_registry.erl",
             "src/rabbit_resource_monitor_misc.erl",
+            "src/rabbit_routing_parser.erl",
             "src/rabbit_runtime.erl",
             "src/rabbit_runtime_parameter.erl",
             "src/rabbit_semver.erl",
@@ -168,6 +169,7 @@ def all_test_beam_files(name = "all_test_beam_files"):
             "src/rabbit_queue_collector.erl",
             "src/rabbit_registry.erl",
             "src/rabbit_resource_monitor_misc.erl",
+            "src/rabbit_routing_parser.erl",
             "src/rabbit_runtime.erl",
             "src/rabbit_runtime_parameter.erl",
             "src/rabbit_semver.erl",
@@ -260,6 +262,7 @@ def all_srcs(name = "all_srcs"):
             "src/rabbit_registry.erl",
             "src/rabbit_registry_class.erl",
             "src/rabbit_resource_monitor_misc.erl",
+            "src/rabbit_routing_parser.erl",
             "src/rabbit_runtime.erl",
             "src/rabbit_runtime_parameter.erl",
             "src/rabbit_semver.erl",
diff --git a/deps/rabbit_common/include/rabbit.hrl b/deps/rabbit_common/include/rabbit.hrl
index 30459e5ccdb7..a3837106787a 100644
--- a/deps/rabbit_common/include/rabbit.hrl
+++ b/deps/rabbit_common/include/rabbit.hrl
@@ -8,7 +8,7 @@
 -include("resource.hrl").
 
 %% Passed around most places
--record(user, {username,
+-record(user, {username :: rabbit_types:option(rabbit_types:username()),
                tags,
                authz_backends}). %% List of {Module, AuthUserImpl} pairs
 
@@ -254,7 +254,7 @@
 %% Max message size is hard limited to 512 MiB.
 %% If user configures a greater rabbit.max_message_size,
 %% this value is used instead.
--define(MAX_MSG_SIZE, 536870912).
+-define(MAX_MSG_SIZE, 536_870_912).
 
 -define(store_proc_name(N), rabbit_misc:store_proc_name(?MODULE, N)).
 
diff --git a/deps/rabbit_common/src/rabbit_core_metrics.erl b/deps/rabbit_common/src/rabbit_core_metrics.erl
index 104bef73abe3..0c46b41db456 100644
--- a/deps/rabbit_common/src/rabbit_core_metrics.erl
+++ b/deps/rabbit_common/src/rabbit_core_metrics.erl
@@ -120,9 +120,8 @@ terminate() ->
      || {Table, _Type} <- ?CORE_TABLES ++ ?CORE_EXTRA_TABLES],
     ok.
 
-connection_created(Pid, Infos0) ->
-    Infos1 = maybe_cleanup_infos(Infos0),
-    ets:insert(connection_created, {Pid, Infos1}),
+connection_created(Pid, Infos) ->
+    ets:insert(connection_created, {Pid, Infos}),
     ets:update_counter(connection_churn_metrics, node(), {2, 1},
                        ?CONNECTION_CHURN_METRICS),
     ok.
@@ -447,14 +446,3 @@ format_auth_attempt({{RemoteAddress, Username, Protocol}, Total, Succeeded, Fail
 format_auth_attempt({Protocol, Total, Succeeded, Failed}) ->
     [{protocol, atom_to_binary(Protocol, utf8)}, {auth_attempts, Total},
      {auth_attempts_failed, Failed}, {auth_attempts_succeeded, Succeeded}].
-
-maybe_cleanup_infos(Infos0) when is_list(Infos0) ->
-    %% Note: authz_backends is added in rabbit_amqp1_0_session_sup:adapter_info/3
-    %% We delete it here, if present, because it should not be stored in the
-    %% connection_created table.
-    %%
-    %% TODO @ansd this will no longer be necessary once this PR is merged:
-    %% https://github.com/rabbitmq/rabbitmq-server/pull/9022
-    proplists:delete(authz_backends, Infos0);
-maybe_cleanup_infos(Infos) ->
-    Infos.
diff --git a/deps/rabbit_common/src/rabbit_data_coercion.erl b/deps/rabbit_common/src/rabbit_data_coercion.erl
index ac631751b22d..da498b1afac8 100644
--- a/deps/rabbit_common/src/rabbit_data_coercion.erl
+++ b/deps/rabbit_common/src/rabbit_data_coercion.erl
@@ -65,9 +65,9 @@ to_map(Val) when is_list(Val) -> maps:from_list(Val).
 
 -spec atomize_keys(Val :: map() | list()) -> map() | list().
 atomize_keys(Val) when is_list(Val) ->
-  [{to_atom(K), V} || {K, V} <- Val];
+    [{to_atom(K), V} || {K, V} <- Val];
 atomize_keys(Val) when is_map(Val) ->
-  maps:from_list(atomize_keys(maps:to_list(Val))).
+    #{to_atom(K) => V || K := V <- Val}.
 
 -spec to_list_of_binaries(Val :: undefined | [atom() | list() | binary() | integer()]) -> [binary()].
 to_list_of_binaries(Value) ->
diff --git a/deps/rabbit_common/src/rabbit_net.erl b/deps/rabbit_common/src/rabbit_net.erl
index 88ff58b8ac38..494126795740 100644
--- a/deps/rabbit_common/src/rabbit_net.erl
+++ b/deps/rabbit_common/src/rabbit_net.erl
@@ -32,11 +32,11 @@
                  {raw, non_neg_integer(), non_neg_integer(), binary()}].
 -type hostname() :: inet:hostname().
 -type ip_port() :: inet:port_number().
--type rabbit_proxy_socket() :: {'rabbit_proxy_socket', ranch_transport:socket(), ranch_proxy_header:proxy_info()}.
+-type proxy_socket() :: {'rabbit_proxy_socket', ranch_transport:socket(), ranch_proxy_header:proxy_info()}.
 % -type host_or_ip() :: binary() | inet:ip_address().
 -spec is_ssl(socket()) -> boolean().
 -spec ssl_info(socket()) -> 'nossl' | ok_val_or_error([{atom(), any()}]).
--spec proxy_ssl_info(socket(), rabbit_proxy_socket() | 'undefined') -> 'nossl' | ok_val_or_error([{atom(), any()}]).
+-spec proxy_ssl_info(socket(), proxy_socket() | 'undefined') -> 'nossl' | ok_val_or_error([{atom(), any()}]).
 -spec controlling_process(socket(), pid()) -> ok_or_any_error().
 -spec getstat(socket(), [stat_option()]) ->
           ok_val_or_error([{stat_option(), integer()}]).
diff --git a/deps/rabbit_common/src/rabbit_routing_parser.erl b/deps/rabbit_common/src/rabbit_routing_parser.erl
new file mode 100644
index 000000000000..59512c3785a9
--- /dev/null
+++ b/deps/rabbit_common/src/rabbit_routing_parser.erl
@@ -0,0 +1,76 @@
+%% 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) 2013-2023 VMware, Inc. or its affiliates.  All rights reserved.
+%%
+
+-module(rabbit_routing_parser).
+
+-export([parse_endpoint/1,
+         parse_endpoint/2,
+         parse_routing/1]).
+
+parse_endpoint(Destination) ->
+    parse_endpoint(Destination, false).
+
+parse_endpoint(undefined, AllowAnonymousQueue) ->
+    parse_endpoint("/queue", AllowAnonymousQueue);
+
+parse_endpoint(Destination, AllowAnonymousQueue) when is_binary(Destination) ->
+    parse_endpoint(unicode:characters_to_list(Destination),
+                                              AllowAnonymousQueue);
+parse_endpoint(Destination, AllowAnonymousQueue) when is_list(Destination) ->
+    case re:split(Destination, "/", [{return, list}]) of
+        [Name] ->
+            {ok, {queue, unescape(Name)}};
+        ["", Type | Rest]
+            when Type =:= "exchange" orelse Type =:= "queue" orelse
+                 Type =:= "topic"    orelse Type =:= "temp-queue" ->
+            parse_endpoint0(atomise(Type), Rest, AllowAnonymousQueue);
+        ["", "amq", "queue" | Rest] ->
+            parse_endpoint0(amqqueue, Rest, AllowAnonymousQueue);
+        ["", "reply-queue" = Prefix | [_|_]] ->
+            parse_endpoint0(reply_queue,
+                            [lists:nthtail(2 + length(Prefix), Destination)],
+                            AllowAnonymousQueue);
+        _ ->
+            {error, {unknown_destination, Destination}}
+    end.
+
+parse_endpoint0(exchange, ["" | _] = Rest,    _) ->
+    {error, {invalid_destination, exchange, to_url(Rest)}};
+parse_endpoint0(exchange, [Name],             _) ->
+    {ok, {exchange, {unescape(Name), undefined}}};
+parse_endpoint0(exchange, [Name, Pattern],    _) ->
+    {ok, {exchange, {unescape(Name), unescape(Pattern)}}};
+parse_endpoint0(queue,    [],                 false) ->
+    {error, {invalid_destination, queue, []}};
+parse_endpoint0(queue,    [],                 true) ->
+    {ok, {queue, undefined}};
+parse_endpoint0(Type,     [[_|_]] = [Name],   _) ->
+    {ok, {Type, unescape(Name)}};
+parse_endpoint0(Type,     Rest,               _) ->
+    {error, {invalid_destination, Type, to_url(Rest)}}.
+
+parse_routing({exchange, {Name, undefined}}) ->
+    {Name, ""};
+parse_routing({exchange, {Name, Pattern}}) ->
+    {Name, Pattern};
+parse_routing({topic, Name}) ->
+    {"amq.topic", Name};
+parse_routing({Type, Name})
+  when Type =:= queue orelse Type =:= reply_queue orelse Type =:= amqqueue ->
+    {"", Name}.
+
+atomise(Name) when is_list(Name) ->
+    list_to_atom(re:replace(Name, "-", "_", [{return,list}, global])).
+
+to_url([])  -> [];
+to_url(Lol) -> "/" ++ string:join(Lol, "/").
+
+unescape(Str) -> unescape(Str, []).
+
+unescape("%2F" ++ Str, Acc) -> unescape(Str, [$/ | Acc]);
+unescape([C | Str],    Acc) -> unescape(Str, [C | Acc]);
+unescape([],           Acc) -> lists:reverse(Acc).
diff --git a/deps/rabbit_common/src/rabbit_types.erl b/deps/rabbit_common/src/rabbit_types.erl
index 2f8ae90b1965..fb4f9aa2d356 100644
--- a/deps/rabbit_common/src/rabbit_types.erl
+++ b/deps/rabbit_common/src/rabbit_types.erl
@@ -39,6 +39,8 @@
 -type(timestamp() :: {non_neg_integer(), non_neg_integer(), non_neg_integer()}).
 
 -type(vhost() :: binary()).
+%% An arbitrary binary tag used to distinguish between different consumers
+%% set up by the same process.
 -type(ctag() :: binary()).
 
 %% TODO: make this more precise by tying specific class_ids to
diff --git a/deps/rabbit_common/src/rabbit_writer.erl b/deps/rabbit_common/src/rabbit_writer.erl
index c04b66cc4d6c..d24ae7e38361 100644
--- a/deps/rabbit_common/src/rabbit_writer.erl
+++ b/deps/rabbit_common/src/rabbit_writer.erl
@@ -105,10 +105,7 @@
 
 -spec send_command(pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
 -spec send_command
-        (pid(), rabbit_framing:amqp_method_record(),
-         rabbit_types:content() |
-         {integer(), rabbit_types:content()} %% publishing sequence for AMQP 1.0 return callback
-        ) ->
+        (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content()) ->
             'ok'.
 -spec send_command_sync(pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
 -spec send_command_sync
diff --git a/deps/rabbitmq_amqp1_0/.gitignore b/deps/rabbitmq_amqp1_0/.gitignore
index faa711d4f6eb..513ad3137ccb 100644
--- a/deps/rabbitmq_amqp1_0/.gitignore
+++ b/deps/rabbitmq_amqp1_0/.gitignore
@@ -1,7 +1,6 @@
 .sw?
 .*.sw?
 *.beam
-*.plt
 /.erlang.mk/
 /cover/
 /deps/
@@ -14,22 +13,5 @@
 /plugins.lock
 /sbin/
 /sbin.lock
-/xrefr
 
-rabbitmq_amqp1_0.d
-
-[Dd]ebug/
-[Rr]elease/
-x64/
-build/
-[Bb]in/
-[Oo]bj/
-*.lock.json
-
-[Dd]ebug/
-[Rr]elease/
-x64/
-build/
-[Bb]in/
-[Oo]bj/
-*.lock.json
+/rabbitmq_amqp1_0.d
diff --git a/deps/rabbitmq_amqp1_0/BUILD.bazel b/deps/rabbitmq_amqp1_0/BUILD.bazel
index 0b7218ab6c20..5a661f56fc69 100644
--- a/deps/rabbitmq_amqp1_0/BUILD.bazel
+++ b/deps/rabbitmq_amqp1_0/BUILD.bazel
@@ -1,148 +1,61 @@
-load("@rules_erlang//:eunit2.bzl", "eunit")
 load("@rules_erlang//:xref2.bzl", "xref")
 load("@rules_erlang//:dialyze.bzl", "dialyze", "plt")
 load(
     "//:rabbitmq.bzl",
     "BROKER_VERSION_REQUIREMENTS_ANY",
-    "RABBITMQ_DIALYZER_OPTS",
     "assert_suites",
-    "broker_for_integration_suites",
     "rabbitmq_app",
-    "rabbitmq_integration_suite",
-    "rabbitmq_suite",
 )
 load(
     ":app.bzl",
     "all_beam_files",
     "all_srcs",
-    "all_test_beam_files",
-    "test_suite_beam_files",
 )
 
-APP_ENV = """[
-	    {default_user, "guest"},
-	    {default_vhost, <<"/">>},
-	    {protocol_strict_mode, false}
-	  ]"""
-
 APP_NAME = "rabbitmq_amqp1_0"
 
-APP_DESCRIPTION = "AMQP 1.0 support for RabbitMQ"
+APP_DESCRIPTION = "Deprecated no-op AMQP 1.0 plugin"
 
 all_beam_files(name = "all_beam_files")
 
-all_test_beam_files(name = "all_test_beam_files")
-
-all_srcs(name = "all_srcs")
-
-test_suite_beam_files(name = "test_suite_beam_files")
-
 rabbitmq_app(
     name = "erlang_app",
     srcs = [":all_srcs"],
     hdrs = [":public_hdrs"],
     app_description = APP_DESCRIPTION,
-    app_env = APP_ENV,
     app_extra_keys = BROKER_VERSION_REQUIREMENTS_ANY,
     app_name = APP_NAME,
     beam_files = [":beam_files"],
     license_files = [":license_files"],
     priv = [":priv"],
     deps = [
-        "//deps/amqp10_common:erlang_app",
-        "//deps/amqp_client:erlang_app",
-        "//deps/rabbit:erlang_app",
-        "//deps/rabbit_common:erlang_app",
+        "//deps/rabbit:erlang_app",  # keep
     ],
 )
 
+all_srcs(name = "all_srcs")
+
+alias(
+    name = "rabbitmq_amqp1_0",
+    actual = ":erlang_app",
+    visibility = ["//visibility:public"],
+)
+
 xref(
     name = "xref",
-    additional_libs = [
-        "//deps/rabbitmq_cli:erlang_app",  # keep
-    ],
     target = ":erlang_app",
 )
 
 plt(
     name = "deps_plt",
-    apps = [
-        "ssl",  # keep
-    ],
     for_target = ":erlang_app",
-    ignore_warnings = True,
-    libs = ["//deps/rabbitmq_cli:elixir"],  # keep
     plt = "//:base_plt",
-    deps = ["//deps/rabbitmq_cli:erlang_app"],  # keep
 )
 
 dialyze(
     name = "dialyze",
-    size = "medium",
-    dialyzer_opts = RABBITMQ_DIALYZER_OPTS,
     plt = ":deps_plt",
     target = ":erlang_app",
 )
 
-eunit(
-    name = "eunit",
-    target = ":test_erlang_app",
-)
-
-broker_for_integration_suites()
-
-rabbitmq_integration_suite(
-    name = "amqp10_client_SUITE",
-    size = "medium",
-    runtime_deps = [
-        "//deps/amqp10_client:erlang_app",
-    ],
-)
-
-rabbitmq_integration_suite(
-    name = "command_SUITE",
-    size = "medium",
-    runtime_deps = [
-        "//deps/amqp10_client:erlang_app",
-    ],
-    deps = [
-        "//deps/amqp10_common:erlang_app",
-    ],
-)
-
-rabbitmq_integration_suite(
-    name = "proxy_protocol_SUITE",
-    size = "medium",
-)
-
-rabbitmq_integration_suite(
-    name = "system_SUITE",
-    flaky = True,
-    shard_count = 2,
-    tags = [
-        "dotnet",
-    ],
-    test_env = {
-        "TMPDIR": "$TEST_TMPDIR",
-    },
-)
-
-rabbitmq_integration_suite(
-    name = "config_schema_SUITE",
-)
-
-rabbitmq_suite(
-    name = "unit_SUITE",
-    size = "small",
-    deps = [
-        "//deps/amqp10_common:erlang_app",
-    ],
-)
-
 assert_suites()
-
-alias(
-    name = "rabbitmq_amqp1_0",
-    actual = ":erlang_app",
-    visibility = ["//visibility:public"],
-)
diff --git a/deps/rabbitmq_amqp1_0/CODE_OF_CONDUCT.md b/deps/rabbitmq_amqp1_0/CODE_OF_CONDUCT.md
deleted file mode 120000
index a3613c99f0b0..000000000000
--- a/deps/rabbitmq_amqp1_0/CODE_OF_CONDUCT.md
+++ /dev/null
@@ -1 +0,0 @@
-../../CODE_OF_CONDUCT.md
\ No newline at end of file
diff --git a/deps/rabbitmq_amqp1_0/CONTRIBUTING.md b/deps/rabbitmq_amqp1_0/CONTRIBUTING.md
deleted file mode 120000
index f939e75f21a8..000000000000
--- a/deps/rabbitmq_amqp1_0/CONTRIBUTING.md
+++ /dev/null
@@ -1 +0,0 @@
-../../CONTRIBUTING.md
\ No newline at end of file
diff --git a/deps/rabbitmq_amqp1_0/LICENSE b/deps/rabbitmq_amqp1_0/LICENSE
deleted file mode 100644
index e75136bfb5f8..000000000000
--- a/deps/rabbitmq_amqp1_0/LICENSE
+++ /dev/null
@@ -1,3 +0,0 @@
-This package is licensed under the MPL 2.0. For the MPL 2.0, please see LICENSE-MPL-RabbitMQ.
-
-If you have any questions regarding licensing, please contact us at rabbitmq-core@groups.vmware.com.
diff --git a/deps/rabbitmq_amqp1_0/LICENSE-MPL-RabbitMQ b/deps/rabbitmq_amqp1_0/LICENSE-MPL-RabbitMQ
deleted file mode 100644
index 14e2f777f6c3..000000000000
--- a/deps/rabbitmq_amqp1_0/LICENSE-MPL-RabbitMQ
+++ /dev/null
@@ -1,373 +0,0 @@
-Mozilla Public License Version 2.0
-==================================
-
-1. Definitions
---------------
-
-1.1. "Contributor"
-    means each individual or legal entity that creates, contributes to
-    the creation of, or owns Covered Software.
-
-1.2. "Contributor Version"
-    means the combination of the Contributions of others (if any) used
-    by a Contributor and that particular Contributor's Contribution.
-
-1.3. "Contribution"
-    means Covered Software of a particular Contributor.
-
-1.4. "Covered Software"
-    means Source Code Form to which the initial Contributor has attached
-    the notice in Exhibit A, the Executable Form of such Source Code
-    Form, and Modifications of such Source Code Form, in each case
-    including portions thereof.
-
-1.5. "Incompatible With Secondary Licenses"
-    means
-
-    (a) that the initial Contributor has attached the notice described
-        in Exhibit B to the Covered Software; or
-
-    (b) that the Covered Software was made available under the terms of
-        version 1.1 or earlier of the License, but not also under the
-        terms of a Secondary License.
-
-1.6. "Executable Form"
-    means any form of the work other than Source Code Form.
-
-1.7. "Larger Work"
-    means a work that combines Covered Software with other material, in 
-    a separate file or files, that is not Covered Software.
-
-1.8. "License"
-    means this document.
-
-1.9. "Licensable"
-    means having the right to grant, to the maximum extent possible,
-    whether at the time of the initial grant or subsequently, any and
-    all of the rights conveyed by this License.
-
-1.10. "Modifications"
-    means any of the following:
-
-    (a) any file in Source Code Form that results from an addition to,
-        deletion from, or modification of the contents of Covered
-        Software; or
-
-    (b) any new file in Source Code Form that contains any Covered
-        Software.
-
-1.11. "Patent Claims" of a Contributor
-    means any patent claim(s), including without limitation, method,
-    process, and apparatus claims, in any patent Licensable by such
-    Contributor that would be infringed, but for the grant of the
-    License, by the making, using, selling, offering for sale, having
-    made, import, or transfer of either its Contributions or its
-    Contributor Version.
-
-1.12. "Secondary License"
-    means either the GNU General Public License, Version 2.0, the GNU
-    Lesser General Public License, Version 2.1, the GNU Affero General
-    Public License, Version 3.0, or any later versions of those
-    licenses.
-
-1.13. "Source Code Form"
-    means the form of the work preferred for making modifications.
-
-1.14. "You" (or "Your")
-    means an individual or a legal entity exercising rights under this
-    License. For legal entities, "You" includes any entity that
-    controls, is controlled by, or is under common control with You. For
-    purposes of this definition, "control" means (a) the power, direct
-    or indirect, to cause the direction or management of such entity,
-    whether by contract or otherwise, or (b) ownership of more than
-    fifty percent (50%) of the outstanding shares or beneficial
-    ownership of such entity.
-
-2. License Grants and Conditions
---------------------------------
-
-2.1. Grants
-
-Each Contributor hereby grants You a world-wide, royalty-free,
-non-exclusive license:
-
-(a) under intellectual property rights (other than patent or trademark)
-    Licensable by such Contributor to use, reproduce, make available,
-    modify, display, perform, distribute, and otherwise exploit its
-    Contributions, either on an unmodified basis, with Modifications, or
-    as part of a Larger Work; and
-
-(b) under Patent Claims of such Contributor to make, use, sell, offer
-    for sale, have made, import, and otherwise transfer either its
-    Contributions or its Contributor Version.
-
-2.2. Effective Date
-
-The licenses granted in Section 2.1 with respect to any Contribution
-become effective for each Contribution on the date the Contributor first
-distributes such Contribution.
-
-2.3. Limitations on Grant Scope
-
-The licenses granted in this Section 2 are the only rights granted under
-this License. No additional rights or licenses will be implied from the
-distribution or licensing of Covered Software under this License.
-Notwithstanding Section 2.1(b) above, no patent license is granted by a
-Contributor:
-
-(a) for any code that a Contributor has removed from Covered Software;
-    or
-
-(b) for infringements caused by: (i) Your and any other third party's
-    modifications of Covered Software, or (ii) the combination of its
-    Contributions with other software (except as part of its Contributor
-    Version); or
-
-(c) under Patent Claims infringed by Covered Software in the absence of
-    its Contributions.
-
-This License does not grant any rights in the trademarks, service marks,
-or logos of any Contributor (except as may be necessary to comply with
-the notice requirements in Section 3.4).
-
-2.4. Subsequent Licenses
-
-No Contributor makes additional grants as a result of Your choice to
-distribute the Covered Software under a subsequent version of this
-License (see Section 10.2) or under the terms of a Secondary License (if
-permitted under the terms of Section 3.3).
-
-2.5. Representation
-
-Each Contributor represents that the Contributor believes its
-Contributions are its original creation(s) or it has sufficient rights
-to grant the rights to its Contributions conveyed by this License.
-
-2.6. Fair Use
-
-This License is not intended to limit any rights You have under
-applicable copyright doctrines of fair use, fair dealing, or other
-equivalents.
-
-2.7. Conditions
-
-Sections 3.1, 3.2, 3.3, and 3.4 are conditions of the licenses granted
-in Section 2.1.
-
-3. Responsibilities
--------------------
-
-3.1. Distribution of Source Form
-
-All distribution of Covered Software in Source Code Form, including any
-Modifications that You create or to which You contribute, must be under
-the terms of this License. You must inform recipients that the Source
-Code Form of the Covered Software is governed by the terms of this
-License, and how they can obtain a copy of this License. You may not
-attempt to alter or restrict the recipients' rights in the Source Code
-Form.
-
-3.2. Distribution of Executable Form
-
-If You distribute Covered Software in Executable Form then:
-
-(a) such Covered Software must also be made available in Source Code
-    Form, as described in Section 3.1, and You must inform recipients of
-    the Executable Form how they can obtain a copy of such Source Code
-    Form by reasonable means in a timely manner, at a charge no more
-    than the cost of distribution to the recipient; and
-
-(b) You may distribute such Executable Form under the terms of this
-    License, or sublicense it under different terms, provided that the
-    license for the Executable Form does not attempt to limit or alter
-    the recipients' rights in the Source Code Form under this License.
-
-3.3. Distribution of a Larger Work
-
-You may create and distribute a Larger Work under terms of Your choice,
-provided that You also comply with the requirements of this License for
-the Covered Software. If the Larger Work is a combination of Covered
-Software with a work governed by one or more Secondary Licenses, and the
-Covered Software is not Incompatible With Secondary Licenses, this
-License permits You to additionally distribute such Covered Software
-under the terms of such Secondary License(s), so that the recipient of
-the Larger Work may, at their option, further distribute the Covered
-Software under the terms of either this License or such Secondary
-License(s).
-
-3.4. Notices
-
-You may not remove or alter the substance of any license notices
-(including copyright notices, patent notices, disclaimers of warranty,
-or limitations of liability) contained within the Source Code Form of
-the Covered Software, except that You may alter any license notices to
-the extent required to remedy known factual inaccuracies.
-
-3.5. Application of Additional Terms
-
-You may choose to offer, and to charge a fee for, warranty, support,
-indemnity or liability obligations to one or more recipients of Covered
-Software. However, You may do so only on Your own behalf, and not on
-behalf of any Contributor. You must make it absolutely clear that any
-such warranty, support, indemnity, or liability obligation is offered by
-You alone, and You hereby agree to indemnify every Contributor for any
-liability incurred by such Contributor as a result of warranty, support,
-indemnity or liability terms You offer. You may include additional
-disclaimers of warranty and limitations of liability specific to any
-jurisdiction.
-
-4. Inability to Comply Due to Statute or Regulation
----------------------------------------------------
-
-If it is impossible for You to comply with any of the terms of this
-License with respect to some or all of the Covered Software due to
-statute, judicial order, or regulation then You must: (a) comply with
-the terms of this License to the maximum extent possible; and (b)
-describe the limitations and the code they affect. Such description must
-be placed in a text file included with all distributions of the Covered
-Software under this License. Except to the extent prohibited by statute
-or regulation, such description must be sufficiently detailed for a
-recipient of ordinary skill to be able to understand it.
-
-5. Termination
---------------
-
-5.1. The rights granted under this License will terminate automatically
-if You fail to comply with any of its terms. However, if You become
-compliant, then the rights granted under this License from a particular
-Contributor are reinstated (a) provisionally, unless and until such
-Contributor explicitly and finally terminates Your grants, and (b) on an
-ongoing basis, if such Contributor fails to notify You of the
-non-compliance by some reasonable means prior to 60 days after You have
-come back into compliance. Moreover, Your grants from a particular
-Contributor are reinstated on an ongoing basis if such Contributor
-notifies You of the non-compliance by some reasonable means, this is the
-first time You have received notice of non-compliance with this License
-from such Contributor, and You become compliant prior to 30 days after
-Your receipt of the notice.
-
-5.2. If You initiate litigation against any entity by asserting a patent
-infringement claim (excluding declaratory judgment actions,
-counter-claims, and cross-claims) alleging that a Contributor Version
-directly or indirectly infringes any patent, then the rights granted to
-You by any and all Contributors for the Covered Software under Section
-2.1 of this License shall terminate.
-
-5.3. In the event of termination under Sections 5.1 or 5.2 above, all
-end user license agreements (excluding distributors and resellers) which
-have been validly granted by You or Your distributors under this License
-prior to termination shall survive termination.
-
-************************************************************************
-*                                                                      *
-*  6. Disclaimer of Warranty                                           *
-*  -------------------------                                           *
-*                                                                      *
-*  Covered Software is provided under this License on an "as is"       *
-*  basis, without warranty of any kind, either expressed, implied, or  *
-*  statutory, including, without limitation, warranties that the       *
-*  Covered Software is free of defects, merchantable, fit for a        *
-*  particular purpose or non-infringing. The entire risk as to the     *
-*  quality and performance of the Covered Software is with You.        *
-*  Should any Covered Software prove defective in any respect, You     *
-*  (not any Contributor) assume the cost of any necessary servicing,   *
-*  repair, or correction. This disclaimer of warranty constitutes an   *
-*  essential part of this License. No use of any Covered Software is   *
-*  authorized under this License except under this disclaimer.         *
-*                                                                      *
-************************************************************************
-
-************************************************************************
-*                                                                      *
-*  7. Limitation of Liability                                          *
-*  --------------------------                                          *
-*                                                                      *
-*  Under no circumstances and under no legal theory, whether tort      *
-*  (including negligence), contract, or otherwise, shall any           *
-*  Contributor, or anyone who distributes Covered Software as          *
-*  permitted above, be liable to You for any direct, indirect,         *
-*  special, incidental, or consequential damages of any character      *
-*  including, without limitation, damages for lost profits, loss of    *
-*  goodwill, work stoppage, computer failure or malfunction, or any    *
-*  and all other commercial damages or losses, even if such party      *
-*  shall have been informed of the possibility of such damages. This   *
-*  limitation of liability shall not apply to liability for death or   *
-*  personal injury resulting from such party's negligence to the       *
-*  extent applicable law prohibits such limitation. Some               *
-*  jurisdictions do not allow the exclusion or limitation of           *
-*  incidental or consequential damages, so this exclusion and          *
-*  limitation may not apply to You.                                    *
-*                                                                      *
-************************************************************************
-
-8. Litigation
--------------
-
-Any litigation relating to this License may be brought only in the
-courts of a jurisdiction where the defendant maintains its principal
-place of business and such litigation shall be governed by laws of that
-jurisdiction, without reference to its conflict-of-law provisions.
-Nothing in this Section shall prevent a party's ability to bring
-cross-claims or counter-claims.
-
-9. Miscellaneous
-----------------
-
-This License represents the complete agreement concerning the subject
-matter hereof. If any provision of this License is held to be
-unenforceable, such provision shall be reformed only to the extent
-necessary to make it enforceable. Any law or regulation which provides
-that the language of a contract shall be construed against the drafter
-shall not be used to construe this License against a Contributor.
-
-10. Versions of the License
----------------------------
-
-10.1. New Versions
-
-Mozilla Foundation is the license steward. Except as provided in Section
-10.3, no one other than the license steward has the right to modify or
-publish new versions of this License. Each version will be given a
-distinguishing version number.
-
-10.2. Effect of New Versions
-
-You may distribute the Covered Software under the terms of the version
-of the License under which You originally received the Covered Software,
-or under the terms of any subsequent version published by the license
-steward.
-
-10.3. Modified Versions
-
-If you create software not governed by this License, and you want to
-create a new license for such software, you may create and use a
-modified version of this License if you rename the license and remove
-any references to the name of the license steward (except to note that
-such modified license differs from this License).
-
-10.4. Distributing Source Code Form that is Incompatible With Secondary
-Licenses
-
-If You choose to distribute Source Code Form that is Incompatible With
-Secondary Licenses under the terms of this version of the License, the
-notice described in Exhibit B of this License must be attached.
-
-Exhibit A - Source Code Form License Notice
--------------------------------------------
-
-  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 http://mozilla.org/MPL/2.0/.
-
-If it is not possible or desirable to put the notice in a particular
-file, then You may include the notice in a location (such as a LICENSE
-file in a relevant directory) where a recipient would be likely to look
-for such a notice.
-
-You may add additional accurate notices of copyright ownership.
-
-Exhibit B - "Incompatible With Secondary Licenses" Notice
----------------------------------------------------------
-
-  This Source Code Form is "Incompatible With Secondary Licenses", as
-  defined by the Mozilla Public License, v. 2.0.
diff --git a/deps/rabbitmq_amqp1_0/Makefile b/deps/rabbitmq_amqp1_0/Makefile
index 11f08ca22b93..ffbfaf90da06 100644
--- a/deps/rabbitmq_amqp1_0/Makefile
+++ b/deps/rabbitmq_amqp1_0/Makefile
@@ -1,46 +1,12 @@
 PROJECT = rabbitmq_amqp1_0
-PROJECT_DESCRIPTION = AMQP 1.0 support for RabbitMQ
+PROJECT_DESCRIPTION = Deprecated no-op AMQP 1.0 plugin
 
-define PROJECT_ENV
-[
-	    {default_user, "guest"},
-	    {default_vhost, <<"/">>},
-	    {protocol_strict_mode, false}
-	  ]
-endef
-
-define PROJECT_APP_EXTRA_KEYS
-	{broker_version_requirements, []}
-endef
-
-BUILD_DEPS = rabbitmq_codegen
-DEPS = rabbit_common rabbit amqp_client amqp10_common
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp10_client
+DEPS = rabbit
 
 DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
 DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk
 
 .DEFAULT_GOAL = all
-$(PROJECT).d:: $(EXTRA_SOURCES)
 
 include ../../rabbitmq-components.mk
 include ../../erlang.mk
-
-# --------------------------------------------------------------------
-# Framing sources generation.
-# --------------------------------------------------------------------
-
-clean:: clean-extra-sources
-
-clean-extra-sources:
-	$(gen_verbose) rm -f $(EXTRA_SOURCES)
-
-distclean:: distclean-dotnet-tests distclean-java-tests
-
-distclean-dotnet-tests:
-	$(gen_verbose) cd test/system_SUITE_data/dotnet-tests && \
-		rm -rf bin obj && \
-		rm -f project.lock.json TestResult.xml
-
-distclean-java-tests:
-	$(gen_verbose) cd test/system_SUITE_data/java-tests && mvn clean
diff --git a/deps/rabbitmq_amqp1_0/README.md b/deps/rabbitmq_amqp1_0/README.md
index 495c2674e5cb..14f65b94c0e0 100644
--- a/deps/rabbitmq_amqp1_0/README.md
+++ b/deps/rabbitmq_amqp1_0/README.md
@@ -1,237 +1,11 @@
-# AMQP 1.0 support for RabbitMQ
+This AMQP 1.0 plugin is deprecated and exists only for backward compatibility.
 
-This plugin adds AMQP 1.0 support to RabbitMQ.
-
-Despite the name,
-AMQP 0-9-1 and 1.0 are very much different protocols and thus
-1.0 is treated as a separate protocol supported by RabbitMQ,
-not a revision of the original protocol that will eventually supersede it.
-
-This plugin is several years old and is moderately mature. It may have certain
-limitations with its current architecture but most major AMQP 1.0 features should be in place.
-
-This plugin supports 0-9-1 and 1.0 client interoperability with certain limitations.
-
-# Configuration
-
-This plugin ships with modern versions of RabbitMQ.
-
-It will listen on the standard AMQP port, 5672. To reconfigure this,
-do so [as you would for 0-9-1](http://www.rabbitmq.com/configure.html). Clients connecting with 0-9-1
-will continue to work on the same port.
-
-The following two configuration options (which are specific to the AMQP 1.0 adapter)
-are accepted in the `rabbitmq_amqp1_0` section of the configuration file.
-
-AMQP 1.0 conceptually allows connections that are not authenticated
-with SASL (i.e. where no username and password is supplied). By
-default these will connect as the "guest" user. To change this, set
-`default_user` to a string with the name of the user to use, or the
-atom `none` to prevent unauthenticated connections.
-
-    {default_user, "guest"}
-
-The default virtual host can be specified using the `default_vhost` setting.
-See the "Virtual Hosts" section below for a description.
-
-    {default_vhost, <<"/">>}
-
-The `protocol_strict_mode` setting controls how strictly peers must conform
-to the specification. The default is not to enforce strictness, which allows
-non-fatal byte-counts in frames and inaccuracies in flow-control from peers.
-
-    {protocol_strict_mode, false}
-
-
-Configuration example using [sysctl config format](https://rabbitmq.com/configure.html#config-file-formats):
-
-    amqp1_0.default_user  = guest
-    amqp1_0.default_vhost = /
-    amqp1_0.protocol_strict_mode = false
-
-
-Configuration for interoperability between AMQP 0.9.1 and AMQP 1.0. 
+From RabbitMQ `v4.x` onwards, AMQP 1.0 is supported natively by RabbitMQ and all AMQP 1.0 code was moved from this directory to the core [rabbit](../rabbit/) app.
 
+This no-op plugin exists only such that deployment tools can continue to enable and disable this plugin without erroring:
 ```
-# Conversion only handles simple types, such as strings, ints and booleans.
-# Convert AMQP 0.9.1 message headers to application properties for an AMQP 1.0 consumer
-amqp1_0.convert_amqp091_headers_to_app_props = false | true (default false)
-# Convert AMQP 1.0 Application Properties to AMQP 0.9.1 headers
-amqp1_0.convert_app_props_to_amqp091_headers = false | true (default false)
-
+rabbitmq-plugins enable rabbitmq_amqp1_0
+rabbitmq-plugins disable rabbitmq_amqp1_0
 ```
-
-## Clients we have tested
-
-The current field of AMQP 1.0 clients is somewhat limited. Therefore
-we have not achieved as much interoperability as we might like.
-
-We have tested against:
-
- * SwiftMQ Java client [1]
-   We have done most of our testing against this client and things seem
-   to work.
-
- * QPid / Proton C client [2]
-   We have successfully tested against the "proton" command line tool
-   this client ships with.
-
- * QPid / Proton Java client [2]
-   We have not been able to get this client to get as far as opening a
-   network connection (tested against 0.2 and 0.4).
-
- * Windows Azure Service Bus [3]
-   It seems that the URI scheme used by this client assumes that it is
-   connecting to Azure; it does not seem to be possible to get it to
-   connect to another server.
-
-[1] http://www.swiftmq.com/products/router/swiftlets/sys_amqp/client/index.html
-
-[2] http://qpid.apache.org/proton/
-
-[3] http://www.windowsazure.com/en-us/develop/net/how-to-guides/service-bus-amqp/
-
-As new clients appear we will of course work on interoperability with them.
-
-# Interoperability with AMQP 0-9-1
-
-## Message payloads
-
-This implementation as a plugin aims for useful interoperability with
-AMQP 0-9-1 clients. AMQP 1.0 messages can be far more structured than
-AMQP 0-9-1 messages, which simply have a payload of bytes.
-
-The way we deal with this is that an AMQP 1.0 message with a single
-data section will be transcoded to an AMQP 0-9-1 message with just the
-bytes from that section, and vice versa. An AMQP 1.0 with any other
-payload will keep exactly that payload (i.e., encoded AMQP 1.0
-sections, concatenated), and for AMQP 0-9-1 clients the `type` field
-of the `basic.properties` will contain the value `"amqp-1.0"`.
-
-Thus, AMQP 0-9-1 clients may receive messages that they cannot
-understand (if they don't have an AMQP 1.0 codec handy, anyway);
-however, these will at least be labelled. AMQP 1.0 clients shall
-receive exactly what they expect.
-
-## Message properties, annotations, headers, etc.
-
-The headers and properties map as follows:
-
-    AMQP 1.0                                 AMQP 0-9-1
-    Header                                   Properties
-      durable              <--------------->   delivery-mode   [1]
-      priority             <--------------->   priority
-      ttl                  <--------------->   expiration      [2]
-      first-acquirer                                           [3]
-      delivery-count                                           [4]
-    Properties
-      message-id           <--------------->   message-id      [5]
-      user-id              <--------------->   user-id
-      to                                                       [6]
-      subject                                                  [6]
-      reply-to             <--------------->   reply-to        [6]
-      correlation-id       <--------------->   correlation-id
-      content-type         <--------------->   content-type
-      content-encoding     <--------------->   content-encoding
-      absolute-expiry-time                                     [7]
-      creation-time        <--------------->   timestamp
-    Application Properties <-------/------->   headers         [8]
-
-[1] `durable` is `true` if and only if `delivery-mode` is `2`.
-
-[2] `expiration` is a shortstr; since RabbitMQ will expect this to be
-an encoded string, we translate a `ttl` to the string representation
-of its integer value.
-
-[3] `first-acquirer` is true if and only if the `basic.deliver` field
-`redelivered` is false.
-
-[4] `delivery-count` is left null.
-
-[5] AMQP 0-9-1 expects this to be a shortstr.
-
-[6] See Routing and Addressing below.
-
-[7] `absolute-expiry-time` has no corresponding field in AMQP 0-9-1,
-and is not supported in RabbitMQ in any case.
-
-[8] The application properties section and the `basic.properties` field
-`headers` are natural analogues. However, rather than try to transcode
-an AMQP 1.0 map to an AMQP 0-9-1 field-table, currently we discard
-application properties (of AMQP 1.0 messages) and headers (of AMQP 0-9-1
-messages sent through to AMQP 1.0). In other words, the (AMQP 1.0)
-application properties section is only available to AMQP 1.0 clients, and
-the (AMQP 0-9-1) headers field is only available to AMQP 0-9-1
-clients.
-
-Note that properties (in both AMQP 1.0 and AMQP 0-9-1) and application
-properties (in AMQP 1.0) are immutable; however, this can only apply
-when the sending and receiving clients are using the same protocol.
-
-## Routing and Addressing
-
-In AMQP 1.0 source and destination addresses are opaque values, and
-each message may have a `subject` field value.
-
-For targets, addresses are:
-
-    = "/exchange/"  X "/" RK  Publish to exchange X with routing key RK
-    | "/exchange/"  X         Publish to exchange X with message subject as routing key
-    | "/topic/"     RK        Publish to amq.topic with routing key RK
-    | "/amq/queue/" Q         Publish to default exchange with routing key Q
-    | "/queue/"     Q         Publish to default exchange with routing key Q
-    | Q (no leading slash)    Publish to default exchange with routing key Q
-    | "/queue"                Publish to default exchange with message subj as routing key
-
-For sources, addresses are:
-
-    = "/exchange/"  X "/" BK  Consume from temp queue bound to X with binding key BK
-    | "/topic/"     BK        Consume from temp queue bound to amq.topic with binding key BK
-    | "/amq/queue/" Q         Consume from Q
-    | "/queue/"     Q         Consume from Q
-    | Q (no leading slash)    Consume from Q
-
-The intent is that the source and destination address formats should be
-mostly the same as those supported by the STOMP plugin, to the extent
-permitted by AMQP 1.0 semantics.
-
-Address format `"/amq/queue/" Q` refers to existing queues, i.e. queues created outside the RabbitMQ AMQP 1.0 plugin.
-Other address formats, e.g. `"/queue/" Q` refer to queues managed by the RabbitMQ AMQP 1.0 plugin, i.e. the plugin will declare such queues.
-
-## Virtual Hosts
-
-AMQP 1.0 has no equivalent of AMQP 0-9-1 virtual hosts. A virtual host
-on the broker may be addressed when opening an AMQP 1.0 connection by setting
-the `hostname` field, prefixing with "vhost:". Setting the `hostname` field
-to "vhost:/" addresses the default virtual host. If the `hostname` field
-does not start with "vhost:" then the `default_vhost` configuration
-setting will be consulted.
-
-# Limitations and unsupported features
-
-At the minute, the RabbitMQ AMQP 1.0 adapter does not support:
-
- - "Exactly once" delivery [9]
- - Link recovery [9]
- - Full message fragmentation [10]
- - Resuming messages
- - "Modified" outcome
- - Filters [11]
- - Transactions
- - Source/target expiry-policy other than link-detach and timeout
-   other than 0
- - Max message size for links
- - Aborted transfers
- - TLS negotiation via the AMQP2100 handshake (although SSL is supported)
-
-[9] We do not deduplicate as a target, though we may resend as a
-source (messages that have no settled outcome when an outgoing link is
-detached will be requeued).
-
-[10] We do fragment messages over multiple frames; however, if this
-would overflow the session window we may discard or requeue messages.
-
-[11] In principle, filters for consuming from an exchange could
-translate to AMQP 0-9-1 bindings. This is not implemented, so
-effectively only consuming from fanout exchanges and queues is useful
-currently.
+Enabling or disabling this plugin has no effect.
+RabbitMQ `v4.x` supports AMQP 1.0 by default.
diff --git a/deps/rabbitmq_amqp1_0/app.bzl b/deps/rabbitmq_amqp1_0/app.bzl
index 2191da43104b..78f6ada247e1 100644
--- a/deps/rabbitmq_amqp1_0/app.bzl
+++ b/deps/rabbitmq_amqp1_0/app.bzl
@@ -8,171 +8,46 @@ def all_beam_files(name = "all_beam_files"):
     )
     erlang_bytecode(
         name = "other_beam",
-        srcs = [
-            "src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl",
-            "src/rabbit_amqp1_0.erl",
-            "src/rabbit_amqp1_0_channel.erl",
-            "src/rabbit_amqp1_0_incoming_link.erl",
-            "src/rabbit_amqp1_0_link_util.erl",
-            "src/rabbit_amqp1_0_message.erl",
-            "src/rabbit_amqp1_0_outgoing_link.erl",
-            "src/rabbit_amqp1_0_reader.erl",
-            "src/rabbit_amqp1_0_session.erl",
-            "src/rabbit_amqp1_0_session_process.erl",
-            "src/rabbit_amqp1_0_session_sup.erl",
-            "src/rabbit_amqp1_0_session_sup_sup.erl",
-            "src/rabbit_amqp1_0_util.erl",
-            "src/rabbit_amqp1_0_writer.erl",
-        ],
+        srcs = ["src/rabbitmq_amqp1_0_noop.erl"],
         hdrs = [":public_and_private_hdrs"],
         app_name = "rabbitmq_amqp1_0",
         dest = "ebin",
         erlc_opts = "//:erlc_opts",
-        deps = [
-            "//deps/amqp10_common:erlang_app",
-            "//deps/amqp_client:erlang_app",
-            "//deps/rabbit_common:erlang_app",
-            "//deps/rabbitmq_cli:erlang_app",
-        ],
-    )
-
-def all_test_beam_files(name = "all_test_beam_files"):
-    filegroup(
-        name = "test_beam_files",
-        testonly = True,
-        srcs = [":test_other_beam"],
-    )
-    erlang_bytecode(
-        name = "test_other_beam",
-        testonly = True,
-        srcs = [
-            "src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl",
-            "src/rabbit_amqp1_0.erl",
-            "src/rabbit_amqp1_0_channel.erl",
-            "src/rabbit_amqp1_0_incoming_link.erl",
-            "src/rabbit_amqp1_0_link_util.erl",
-            "src/rabbit_amqp1_0_message.erl",
-            "src/rabbit_amqp1_0_outgoing_link.erl",
-            "src/rabbit_amqp1_0_reader.erl",
-            "src/rabbit_amqp1_0_session.erl",
-            "src/rabbit_amqp1_0_session_process.erl",
-            "src/rabbit_amqp1_0_session_sup.erl",
-            "src/rabbit_amqp1_0_session_sup_sup.erl",
-            "src/rabbit_amqp1_0_util.erl",
-            "src/rabbit_amqp1_0_writer.erl",
-        ],
-        hdrs = [":public_and_private_hdrs"],
-        app_name = "rabbitmq_amqp1_0",
-        dest = "test",
-        erlc_opts = "//:test_erlc_opts",
-        deps = [
-            "//deps/amqp10_common:erlang_app",
-            "//deps/amqp_client:erlang_app",
-            "//deps/rabbit_common:erlang_app",
-            "//deps/rabbitmq_cli:erlang_app",
-        ],
     )
 
 def all_srcs(name = "all_srcs"):
     filegroup(
-        name = "all_srcs",
-        srcs = [":public_and_private_hdrs", ":srcs"],
+        name = "srcs",
+        srcs = ["src/rabbitmq_amqp1_0_noop.erl"],
     )
+    filegroup(name = "private_hdrs")
+    filegroup(name = "public_hdrs")
+    filegroup(name = "priv")
+    filegroup(name = "license_files")
     filegroup(
         name = "public_and_private_hdrs",
         srcs = [":private_hdrs", ":public_hdrs"],
     )
-
-    filegroup(
-        name = "priv",
-        srcs = ["priv/schema/rabbitmq_amqp1_0.schema"],
-    )
-    filegroup(
-        name = "private_hdrs",
-    )
-    filegroup(
-        name = "srcs",
-        srcs = [
-            "src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl",
-            "src/rabbit_amqp1_0.erl",
-            "src/rabbit_amqp1_0_channel.erl",
-            "src/rabbit_amqp1_0_incoming_link.erl",
-            "src/rabbit_amqp1_0_link_util.erl",
-            "src/rabbit_amqp1_0_message.erl",
-            "src/rabbit_amqp1_0_outgoing_link.erl",
-            "src/rabbit_amqp1_0_reader.erl",
-            "src/rabbit_amqp1_0_session.erl",
-            "src/rabbit_amqp1_0_session_process.erl",
-            "src/rabbit_amqp1_0_session_sup.erl",
-            "src/rabbit_amqp1_0_session_sup_sup.erl",
-            "src/rabbit_amqp1_0_util.erl",
-            "src/rabbit_amqp1_0_writer.erl",
-        ],
-    )
-    filegroup(
-        name = "public_hdrs",
-        srcs = ["include/rabbit_amqp1_0.hrl"],
-    )
     filegroup(
-        name = "license_files",
-        srcs = [
-            "LICENSE",
-            "LICENSE-MPL-RabbitMQ",
-        ],
+        name = "all_srcs",
+        srcs = [":public_and_private_hdrs", ":srcs"],
     )
 
-def test_suite_beam_files(name = "test_suite_beam_files"):
-    erlang_bytecode(
-        name = "amqp10_client_SUITE_beam_files",
-        testonly = True,
-        srcs = ["test/amqp10_client_SUITE.erl"],
-        outs = ["test/amqp10_client_SUITE.beam"],
-        app_name = "rabbitmq_amqp1_0",
-        erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/amqp_client:erlang_app"],
-    )
-    erlang_bytecode(
-        name = "command_SUITE_beam_files",
-        testonly = True,
-        srcs = ["test/command_SUITE.erl"],
-        outs = ["test/command_SUITE.beam"],
-        hdrs = ["include/rabbit_amqp1_0.hrl"],
-        app_name = "rabbitmq_amqp1_0",
-        erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/amqp10_common:erlang_app", "//deps/amqp_client:erlang_app"],
-    )
-    erlang_bytecode(
-        name = "config_schema_SUITE_beam_files",
-        testonly = True,
-        srcs = ["test/config_schema_SUITE.erl"],
-        outs = ["test/config_schema_SUITE.beam"],
-        app_name = "rabbitmq_amqp1_0",
-        erlc_opts = "//:test_erlc_opts",
-    )
-    erlang_bytecode(
-        name = "proxy_protocol_SUITE_beam_files",
-        testonly = True,
-        srcs = ["test/proxy_protocol_SUITE.erl"],
-        outs = ["test/proxy_protocol_SUITE.beam"],
-        app_name = "rabbitmq_amqp1_0",
-        erlc_opts = "//:test_erlc_opts",
-    )
+def all_test_beam_files(name = "all_test_beam_files"):
     erlang_bytecode(
-        name = "system_SUITE_beam_files",
+        name = "test_other_beam",
         testonly = True,
-        srcs = ["test/system_SUITE.erl"],
-        outs = ["test/system_SUITE.beam"],
+        srcs = ["src/rabbitmq_amqp1_0_noop.erl"],
+        hdrs = [":public_and_private_hdrs"],
         app_name = "rabbitmq_amqp1_0",
+        dest = "test",
         erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/rabbit_common:erlang_app"],
     )
-    erlang_bytecode(
-        name = "unit_SUITE_beam_files",
+    filegroup(
+        name = "test_beam_files",
         testonly = True,
-        srcs = ["test/unit_SUITE.erl"],
-        outs = ["test/unit_SUITE.beam"],
-        hdrs = ["include/rabbit_amqp1_0.hrl"],
-        app_name = "rabbitmq_amqp1_0",
-        erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/amqp10_common:erlang_app"],
+        srcs = [":test_other_beam"],
     )
+
+def test_suite_beam_files(name = "test_suite_beam_files"):
+    pass
diff --git a/deps/rabbitmq_amqp1_0/include/rabbit_amqp1_0.hrl b/deps/rabbitmq_amqp1_0/include/rabbit_amqp1_0.hrl
deleted file mode 100644
index eb361683b985..000000000000
--- a/deps/rabbitmq_amqp1_0/include/rabbit_amqp1_0.hrl
+++ /dev/null
@@ -1,52 +0,0 @@
-%%-define(debug, true).
-
--ifdef(debug).
--define(DEBUG0(F), ?SAFE(io:format(F, []))).
--define(DEBUG(F, A), ?SAFE(io:format(F, A))).
--else.
--define(DEBUG0(F), ok).
--define(DEBUG(F, A), ok).
--endif.
-
--define(pprint(F), io:format("~p~n", [amqp10_framing:pprint(F)])).
-
--define(SAFE(F),
-        ((fun() ->
-                  try F
-                  catch __T:__E:__ST ->
-                          io:format("~p:~p thrown debugging~n~p~n",
-                                    [__T, __E, __ST])
-                  end
-          end)())).
-
-%% General consts
-
--define(FRAME_1_0_MIN_SIZE, 512).
-
--define(SEND_ROLE, false).
--define(RECV_ROLE, true).
-
-%% Encoding
-
--include_lib("amqp10_common/include/amqp10_framing.hrl").
-
--define(INFO_ITEMS, [pid,
-                     auth_mechanism,
-                     host,
-                     frame_max,
-                     timeout,
-                     user,
-                     state,
-                     recv_oct,
-                     recv_cnt,
-                     send_oct,
-                     send_cnt,
-                     ssl,
-                     ssl_protocol,
-                     ssl_key_exchange,
-                     ssl_cipher,
-                     ssl_hash,
-                     peer_cert_issuer,
-                     peer_cert_subject,
-                     peer_cert_validity,
-                     node]).
diff --git a/deps/rabbitmq_amqp1_0/priv/schema/rabbitmq_amqp1_0.schema b/deps/rabbitmq_amqp1_0/priv/schema/rabbitmq_amqp1_0.schema
deleted file mode 100644
index 09d2cd06b224..000000000000
--- a/deps/rabbitmq_amqp1_0/priv/schema/rabbitmq_amqp1_0.schema
+++ /dev/null
@@ -1,37 +0,0 @@
-%% ----------------------------------------------------------------------------
-%% RabbitMQ AMQP 1.0 Support
-%%
-%% See https://github.com/rabbitmq/rabbitmq-amqp1.0/blob/stable/README.md
-%% for details
-%% ----------------------------------------------------------------------------
-
-%  {rabbitmq_amqp1_0,[
-%% Connections that are not authenticated with SASL will connect as this
-%% account. See the README for more information.
-%%
-%% Please note that setting this will allow clients to connect without
-%% authenticating!
-%%
-%% {default_user, "guest"},
-{mapping, "amqp1_0.default_user", "rabbitmq_amqp1_0.default_user", 
-    [{datatype, [{enum, [none]}, string]}]}.
-%% Enable protocol strict mode. See the README for more information.
-%%
-%% {protocol_strict_mode, false}
-%   ]},
-{mapping, "amqp1_0.protocol_strict_mode", "rabbitmq_amqp1_0.protocol_strict_mode", 
-    [{datatype, {enum, [true, false]}}]}.
-
-{mapping, "amqp1_0.default_vhost", "rabbitmq_amqp1_0.default_vhost", 
-    [{datatype, string}]}.
-
-{translation , "rabbitmq_amqp1_0.default_vhost",
-fun(Conf) ->
-    list_to_binary(cuttlefish:conf_get("amqp1_0.default_vhost", Conf))
-end}.
-
-{mapping, "amqp1_0.convert_amqp091_headers_to_app_props", "rabbitmq_amqp1_0.convert_amqp091_headers_to_app_props", 
-    [{datatype, {enum, [true, false]}}]}.
-
-{mapping, "amqp1_0.convert_app_props_to_amqp091_headers", "rabbitmq_amqp1_0.convert_app_props_to_amqp091_headers", 
-    [{datatype, {enum, [true, false]}}]}.
diff --git a/deps/rabbitmq_amqp1_0/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl b/deps/rabbitmq_amqp1_0/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl
deleted file mode 100644
index 04959a632b96..000000000000
--- a/deps/rabbitmq_amqp1_0/src/Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand.erl
+++ /dev/null
@@ -1,80 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-
--module('Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand').
-
--behaviour('Elixir.RabbitMQ.CLI.CommandBehaviour').
--include("rabbit_amqp1_0.hrl").
-
--export([formatter/0,
-         scopes/0,
-         switches/0,
-         aliases/0,
-         usage/0,
-         usage_additional/0,
-         banner/2,
-         validate/2,
-         merge_defaults/2,
-         run/2,
-         output/2,
-         help_section/0,
-         description/0]).
-
-formatter() -> 'Elixir.RabbitMQ.CLI.Formatters.Table'.
-scopes() -> [ctl, diagnostics].
-switches() -> [{verbose, boolean}].
-aliases() -> [{'V', verbose}].
-
-validate(Args, _) ->
-    ValidKeys = lists:map(fun atom_to_list/1, ?INFO_ITEMS),
-    case 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':validate_info_keys(Args,
-                                                               ValidKeys) of
-        {ok, _} -> ok;
-        Error   -> Error
-    end.
-
-merge_defaults([], Opts) ->
-    merge_defaults([<<"pid">>], Opts);
-merge_defaults(Args, Opts) ->
-    {Args, maps:merge(#{verbose => false}, Opts)}.
-
-usage() ->
-    <<"list_amqp10_connections [ ...]">>.
-
-usage_additional() ->
-    Prefix = <<" must be one of ">>,
-    InfoItems = 'Elixir.Enum':join(lists:usort(?INFO_ITEMS), <<", ">>),
-    [
-      {<<"">>, <>}
-    ].
-
-description() -> <<"Lists AMQP 1.0 connections on the target node">>.
-
-help_section() ->
-    {plugin, 'amqp1.0'}.
-
-run(Args, #{node := NodeName,
-            timeout := Timeout,
-            verbose := Verbose}) ->
-    InfoKeys = case Verbose of
-        true  -> ?INFO_ITEMS;
-        false -> 'Elixir.RabbitMQ.CLI.Ctl.InfoKeys':prepare_info_keys(Args)
-    end,
-    Nodes = 'Elixir.RabbitMQ.CLI.Core.Helpers':nodes_in_cluster(NodeName),
-
-    'Elixir.RabbitMQ.CLI.Ctl.RpcStream':receive_list_items(
-        NodeName,
-        rabbit_amqp1_0,
-        emit_connection_info_all,
-        [Nodes, InfoKeys],
-        Timeout,
-        InfoKeys,
-        length(Nodes)).
-
-banner(_, _) -> <<"Listing AMQP 1.0 connections ...">>.
-
-output(Result, _Opts) ->
-    'Elixir.RabbitMQ.CLI.DefaultOutput':output(Result).
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_channel.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_channel.erl
deleted file mode 100644
index e1874d85f592..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_channel.erl
+++ /dev/null
@@ -1,61 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_channel).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--export([call/2, call/3, cast/2, cast/3, cast_flow/3, subscribe/3]).
--export([convert_code/1, convert_error/1]).
-
--import(rabbit_amqp1_0_util, [protocol_error/3]).
-
-call(Ch, Method) ->
-    convert_error(fun () -> amqp_channel:call(Ch, Method) end).
-
-call(Ch, Method, Content) ->
-    convert_error(fun () -> amqp_channel:call(Ch, Method, Content) end).
-
-cast(Ch, Method) ->
-    convert_error(fun () -> amqp_channel:cast(Ch, Method) end).
-
-cast(Ch, Method, Content) ->
-    convert_error(fun () -> amqp_channel:cast(Ch, Method, Content) end).
-
-cast_flow(Ch, Method, Content) ->
-    convert_error(fun () -> amqp_channel:cast_flow(Ch, Method, Content) end).
-
-subscribe(Ch, Method, Subscriber) ->
-    convert_error(fun () -> amqp_channel:subscribe(Ch, Method, Subscriber) end).
-
-convert_error(Fun) ->
-    try
-        Fun()
-        catch exit:{{shutdown, {server_initiated_close, Code, Msg}}, _} ->
-            protocol_error(convert_code(Code), Msg, [])
-    end.
-
-%% TODO this was completely off the top of my head. Check these make sense.
-convert_code(?CONTENT_TOO_LARGE)   -> ?V_1_0_AMQP_ERROR_FRAME_SIZE_TOO_SMALL;
-convert_code(?NO_ROUTE)            -> ?V_1_0_AMQP_ERROR_PRECONDITION_FAILED;
-convert_code(?NO_CONSUMERS)        -> ?V_1_0_AMQP_ERROR_PRECONDITION_FAILED;
-convert_code(?ACCESS_REFUSED)      -> ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS;
-convert_code(?NOT_FOUND)           -> ?V_1_0_AMQP_ERROR_NOT_FOUND;
-convert_code(?RESOURCE_LOCKED)     -> ?V_1_0_AMQP_ERROR_RESOURCE_LOCKED;
-convert_code(?PRECONDITION_FAILED) -> ?V_1_0_AMQP_ERROR_PRECONDITION_FAILED;
-convert_code(?CONNECTION_FORCED)   -> ?V_1_0_CONNECTION_ERROR_CONNECTION_FORCED;
-convert_code(?INVALID_PATH)        -> ?V_1_0_AMQP_ERROR_INVALID_FIELD;
-convert_code(?FRAME_ERROR)         -> ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR;
-convert_code(?SYNTAX_ERROR)        -> ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR;
-convert_code(?COMMAND_INVALID)     -> ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR;
-convert_code(?CHANNEL_ERROR)       -> ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR;
-convert_code(?UNEXPECTED_FRAME)    -> ?V_1_0_CONNECTION_ERROR_FRAMING_ERROR;
-convert_code(?RESOURCE_ERROR)      -> ?V_1_0_AMQP_ERROR_RESOURCE_LIMIT_EXCEEDED;
-convert_code(?NOT_ALLOWED)         -> ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS;
-convert_code(?NOT_IMPLEMENTED)     -> ?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED;
-convert_code(?INTERNAL_ERROR)      -> ?V_1_0_AMQP_ERROR_INTERNAL_ERROR.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_incoming_link.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_incoming_link.erl
deleted file mode 100644
index 89861315f484..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_incoming_link.erl
+++ /dev/null
@@ -1,246 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_incoming_link).
-
--export([attach/3, transfer/4]).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--import(rabbit_amqp1_0_util, [protocol_error/3]).
-
-%% Just make these constant for the time being.
--define(INCOMING_CREDIT, 65536).
-
--record(incoming_link, {name, exchange, routing_key, mandatory,
-                        delivery_id = undefined,
-                        delivery_count = 0,
-                        send_settle_mode = undefined,
-                        recv_settle_mode = undefined,
-                        credit_used = ?INCOMING_CREDIT div 2,
-                        msg_acc = [],
-                        route_state}).
-
-attach(#'v1_0.attach'{name = Name,
-                      handle = Handle,
-                      source = Source,
-                      snd_settle_mode = SndSettleMode,
-                      rcv_settle_mode = RcvSettleMode,
-                      target = Target,
-                      initial_delivery_count = {uint, InitTransfer}},
-       BCh, DCh) ->
-    %% TODO associate link name with target
-    case ensure_target(Target,
-                       #incoming_link{
-                         name        = Name,
-                         route_state = rabbit_routing_util:init_state(),
-                         delivery_count = InitTransfer },
-                       DCh) of
-        {ok, ServerTarget, IncomingLink} ->
-            {_, _Outcomes} = rabbit_amqp1_0_link_util:outcomes(Source),
-            %% Default is mixed
-            Confirm =
-                case SndSettleMode of
-                    ?V_1_0_SENDER_SETTLE_MODE_SETTLED ->
-                        false;
-                    _ when SndSettleMode == undefined;
-                           SndSettleMode == ?V_1_0_SENDER_SETTLE_MODE_UNSETTLED;
-                           SndSettleMode == ?V_1_0_SENDER_SETTLE_MODE_MIXED ->
-                        amqp_channel:register_confirm_handler(BCh, self()),
-                        rabbit_amqp1_0_channel:call(BCh, #'confirm.select'{}),
-                        amqp_channel:register_return_handler(BCh, self()),
-                        true
-                end,
-            Flow = #'v1_0.flow'{ handle = Handle,
-                                 link_credit = {uint, ?INCOMING_CREDIT},
-                                 drain = false,
-                                 echo = false },
-            Attach = #'v1_0.attach'{
-              name = Name,
-              handle = Handle,
-              source = Source,
-              snd_settle_mode = SndSettleMode,
-              rcv_settle_mode = RcvSettleMode,
-              target = ServerTarget,
-              initial_delivery_count = undefined, % must be, I am the receiver
-              role = ?RECV_ROLE}, %% server is receiver
-            IncomingLink1 =
-                IncomingLink#incoming_link{recv_settle_mode = RcvSettleMode,
-                                           mandatory = Confirm},
-            {ok, [Attach, Flow], IncomingLink1, Confirm};
-        {error, Reason} ->
-            %% TODO proper link establishment protocol here?
-            protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
-                               "Attach rejected: ~tp", [Reason])
-    end.
-
-set_delivery_id({uint, D},
-                #incoming_link{delivery_id = undefined} = Link) ->
-    Link#incoming_link{delivery_id = D};
-set_delivery_id(DeliveryId,
-                #incoming_link{delivery_id = D} = Link)
-  when DeliveryId == {uint, D} orelse DeliveryId == undefined ->
-    Link.
-
-effective_send_settle_mode(undefined, undefined) ->
-    false;
-effective_send_settle_mode(undefined, SettleMode)
-  when is_boolean(SettleMode) ->
-    SettleMode;
-effective_send_settle_mode(SettleMode, undefined)
-  when is_boolean(SettleMode) ->
-    SettleMode;
-effective_send_settle_mode(SettleMode, SettleMode)
-  when is_boolean(SettleMode) ->
-    SettleMode.
-
-effective_recv_settle_mode(undefined, undefined) ->
-    ?V_1_0_RECEIVER_SETTLE_MODE_FIRST;
-effective_recv_settle_mode(undefined, Mode) ->
-    Mode;
-effective_recv_settle_mode(Mode, _) ->
-    Mode.
-
-% TODO: validate effective settle modes against
-%       those declared during attach
-
-% TODO: handle aborted transfers
-
-transfer(#'v1_0.transfer'{delivery_id = DeliveryId,
-                          more        = true,
-                          settled     = Settled}, MsgPart,
-         #incoming_link{msg_acc = MsgAcc,
-                        send_settle_mode = SSM} = Link, _BCh) ->
-    {ok, set_delivery_id(
-           DeliveryId,
-           Link#incoming_link{msg_acc = [MsgPart | MsgAcc],
-                              send_settle_mode =
-                                effective_send_settle_mode(Settled, SSM)})};
-transfer(#'v1_0.transfer'{delivery_id     = DeliveryId0,
-                          settled         = Settled,
-                          rcv_settle_mode = RcvSettleMode,
-                          handle          = Handle},
-         MsgPart,
-         #incoming_link{exchange         = X,
-                        routing_key      = LinkRKey,
-                        delivery_count   = Count,
-                        credit_used      = CreditUsed,
-                        msg_acc          = MsgAcc,
-                        send_settle_mode = SSM,
-                        recv_settle_mode = RSM} = Link, BCh) ->
-    MsgBin = iolist_to_binary(lists:reverse([MsgPart | MsgAcc])),
-    ?DEBUG("Inbound content:~n  ~tp",
-           [[amqp10_framing:pprint(Section) ||
-                Section <- amqp10_framing:decode_bin(MsgBin)]]),
-    {MsgRKey, Msg} = rabbit_amqp1_0_message:assemble(MsgBin),
-    RKey = case LinkRKey of
-               undefined -> MsgRKey;
-               _         -> LinkRKey
-           end,
-    rabbit_amqp1_0_channel:cast_flow(
-      BCh, #'basic.publish'{exchange    = X,
-                            routing_key = RKey,
-                            mandatory = true}, Msg),
-    {SendFlow, CreditUsed1} = case CreditUsed - 1 of
-                                  C when C =< 0 ->
-                                      {true,  ?INCOMING_CREDIT div 2};
-                                  D ->
-                                      {false, D}
-                              end,
-    #incoming_link{delivery_id = DeliveryId} =
-      set_delivery_id(DeliveryId0, Link),
-    NewLink = Link#incoming_link{
-                delivery_id      = undefined,
-                send_settle_mode = undefined,
-                delivery_count   = rabbit_amqp1_0_util:serial_add(Count, 1),
-                credit_used      = CreditUsed1,
-                msg_acc          = []},
-    Reply = case SendFlow of
-                true  -> ?DEBUG("sending flow for incoming ~tp", [NewLink]),
-                         [incoming_flow(NewLink, Handle)];
-                false -> []
-            end,
-    EffectiveSendSettleMode = effective_send_settle_mode(Settled, SSM),
-    EffectiveRecvSettleMode = effective_recv_settle_mode(RcvSettleMode, RSM),
-    case not EffectiveSendSettleMode andalso
-         EffectiveRecvSettleMode =:= ?V_1_0_RECEIVER_SETTLE_MODE_SECOND of
-        false -> ok;
-        true  -> protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                                "rcv-settle-mode second not supported", [])
-    end,
-    {message, Reply, NewLink, DeliveryId,
-     EffectiveSendSettleMode}.
-
-%% TODO default-outcome and outcomes, dynamic lifetimes
-
-ensure_target(Target = #'v1_0.target'{address       = Address,
-                                      dynamic       = Dynamic,
-                                      durable       = Durable,
-                                      %% TODO
-                                      expiry_policy = _ExpiryPolicy,
-                                      %% TODO
-                                      timeout       = _Timeout},
-              Link = #incoming_link{ route_state = RouteState }, DCh) ->
-    DeclareParams = [{durable, rabbit_amqp1_0_link_util:durable(Durable)},
-                     {exclusive, false},
-                     {auto_delete, false},
-                     {check_exchange, true},
-                     {nowait, false}],
-    case Dynamic of
-        true ->
-            protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                           "Dynamic targets not supported", []);
-        _ ->
-            ok
-    end,
-    case Address of
-        {utf8, Destination} ->
-            case rabbit_routing_util:parse_endpoint(Destination, true) of
-                {ok, Dest} ->
-                    {ok, _Queue, RouteState1} =
-                        rabbit_amqp1_0_channel:convert_error(
-                          fun () ->
-                                  rabbit_routing_util:ensure_endpoint(
-                                    dest, DCh, Dest, DeclareParams,
-                                    RouteState)
-                          end),
-                    maybe_ensure_queue(Dest, DCh),
-                    {XName, RK} = rabbit_routing_util:parse_routing(Dest),
-                    {ok, Target, Link#incoming_link{
-                                   route_state = RouteState1,
-                                   exchange    = list_to_binary(XName),
-                                   routing_key = case RK of
-                                                     undefined -> undefined;
-                                                     []        -> undefined;
-                                                     _         -> list_to_binary(RK)
-                                                 end}};
-                {error, _} = E ->
-                    E
-            end;
-        _Else ->
-            {error, {address_not_utf8_string, Address}}
-    end.
-
-maybe_ensure_queue({amqqueue, Q}, Ch) ->
-    try
-        rabbit_amqp1_0_channel:convert_error(
-          fun () ->
-                  Method = #'queue.declare'{queue = list_to_binary(Q),
-                                            passive = true},
-                  amqp_channel:call(Ch, Method)
-          end)
-    catch exit:#'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_PRECONDITION_FAILED} ->
-              ok
-    end;
-maybe_ensure_queue(_, _) ->
-    ok.
-
-incoming_flow(#incoming_link{ delivery_count = Count }, Handle) ->
-    #'v1_0.flow'{handle         = Handle,
-                 delivery_count = {uint, Count},
-                 link_credit    = {uint, ?INCOMING_CREDIT}}.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_link_util.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_link_util.erl
deleted file mode 100644
index ed3bd991794c..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_link_util.erl
+++ /dev/null
@@ -1,64 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_link_util).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--export([outcomes/1, ctag_to_handle/1, handle_to_ctag/1, durable/1]).
-
--define(EXCHANGE_SUB_LIFETIME, "delete-on-close").
--define(DEFAULT_OUTCOME, #'v1_0.released'{}).
--define(OUTCOMES, [?V_1_0_SYMBOL_ACCEPTED,
-                   ?V_1_0_SYMBOL_REJECTED,
-                   ?V_1_0_SYMBOL_RELEASED,
-                   ?V_1_0_SYMBOL_MODIFIED]).
--define(SUPPORTED_OUTCOMES, ?OUTCOMES).
-
-outcomes(Source) ->
-    {DefaultOutcome, Outcomes} =
-        case Source of
-            #'v1_0.source' {
-                      default_outcome = DO,
-                      outcomes = Os
-                     } ->
-                DO1 = case DO of
-                          undefined -> ?DEFAULT_OUTCOME;
-                          _         -> DO
-                      end,
-                Os1 = case Os of
-                          undefined    -> ?SUPPORTED_OUTCOMES;
-                          {array, symbol, Syms} -> Syms;
-                          Bad1         -> rabbit_amqp1_0_util:protocol_error(
-                                            ?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                                            "Outcomes not supported: ~tp",
-                                            [Bad1])
-                      end,
-                {DO1, Os1};
-            _ ->
-                {?DEFAULT_OUTCOME, ?SUPPORTED_OUTCOMES}
-        end,
-    case [O || O <- Outcomes, not lists:member(O, ?OUTCOMES)] of
-        []  -> {DefaultOutcome, {array, symbol, Outcomes}};
-        Bad -> rabbit_amqp1_0_util:protocol_error(
-                 ?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                 "Outcomes not supported: ~tp", [Bad])
-    end.
-
-handle_to_ctag({uint, H}) ->
-    <<"ctag-", H:32/integer>>.
-
-ctag_to_handle(<<"ctag-", H:32/integer>>) ->
-    {uint, H}.
-
-durable(undefined)                                  -> false; %% default: none
-durable(?V_1_0_TERMINUS_DURABILITY_NONE)            -> false;
-%% This one means "existence of the thing is durable, but unacked msgs
-%% aren't". We choose to upgrade that.
-durable(?V_1_0_TERMINUS_DURABILITY_CONFIGURATION)   -> true;
-durable(?V_1_0_TERMINUS_DURABILITY_UNSETTLED_STATE) -> true.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_message.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_message.erl
deleted file mode 100644
index 2f1d30c03f30..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_message.erl
+++ /dev/null
@@ -1,371 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_message).
-
--export([assemble/1, annotated_message/3]).
-
--define(PROPERTIES_HEADER, <<"x-amqp-1.0-properties">>).
--define(APP_PROPERTIES_HEADER, <<"x-amqp-1.0-app-properties">>).
--define(MESSAGE_ANNOTATIONS_HEADER, <<"x-amqp-1.0-message-annotations">>).
--define(STREAM_OFFSET_HEADER, <<"x-stream-offset">>).
--define(FOOTER, <<"x-amqp-1.0-footer">>).
--define(X_DELIVERY_COUNT, <<"x-delivery-count">>).
--define(CONVERT_AMQP091_HEADERS_TO_APP_PROPS, application:get_env(rabbitmq_amqp1_0, convert_amqp091_headers_to_app_props, false)).
--define(CONVERT_APP_PROPS_TO_AMQP091_HEADERS, application:get_env(rabbitmq_amqp1_0, convert_app_props_to_amqp091_headers, false)).
-
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
-assemble(MsgBin) ->
-    {RKey, Props, Content0} = assemble(header, {<<"">>, #'P_basic'{}, []},
-                                      decode_section(MsgBin), MsgBin),
-
-    Content1 = case Content0 of
-                   Sections when is_list(Content0) ->
-                       lists:reverse(Sections);
-                   _ ->
-                       Content0
-               end,
-    {RKey, #amqp_msg{props = Props, payload = Content1}}.
-
-assemble(header, {R, P, C}, {H = #'v1_0.header'{}, Rest}, _Uneaten) ->
-    assemble(message_annotations, {R, translate_header(H, P), C},
-             decode_section(Rest), Rest);
-assemble(header, {R, P, C}, Else, Uneaten) ->
-    assemble(message_annotations, {R, P, C}, Else, Uneaten);
-
-%% This clause doesn't get called, and is commented out as not to confuse dialyzer.
-%%
-%% assemble(delivery_annotations, RPC, {#'v1_0.delivery_annotations'{}, Rest},
-%%          Uneaten) ->
-%%     %% ignore delivery annotations for now
-%%     %% TODO: handle "rejected" error
-%%     assemble(message_annotations, RPC, Rest, Uneaten);
-%% assemble(delivery_annotations, RPC, Else, Uneaten) ->
-%%     assemble(message_annotations, RPC, Else, Uneaten);
-
-assemble(message_annotations, {R, P = #'P_basic'{headers = Headers}, C},
-         {#'v1_0.message_annotations'{}, Rest}, Uneaten) ->
-    MsgAnnoBin = chunk(Rest, Uneaten),
-    assemble(properties, {R, P#'P_basic'{
-                               headers = set_header(?MESSAGE_ANNOTATIONS_HEADER,
-                                                    MsgAnnoBin, Headers)}, C},
-             decode_section(Rest), Rest);
-assemble(message_annotations, {R, P, C}, Else, Uneaten) ->
-    assemble(properties, {R, P, C}, Else, Uneaten);
-
-assemble(properties, {_R, P, C}, {X = #'v1_0.properties'{}, Rest}, Uneaten) ->
-    PropsBin = chunk(Rest, Uneaten),
-    assemble(app_properties, {routing_key(X),
-                    translate_properties(X, PropsBin, P), C},
-             decode_section(Rest), Rest);
-assemble(properties, {R, P, C}, Else, Uneaten) ->
-    assemble(app_properties, {R, P, C}, Else, Uneaten);
-
-assemble(app_properties, {R, P = #'P_basic'{headers = Headers}, C},
-         {#'v1_0.application_properties'{}, Rest}, Uneaten) ->
-    AppPropsBin = chunk(Rest, Uneaten),
-    Amqp091Headers = case ?CONVERT_APP_PROPS_TO_AMQP091_HEADERS of 
-                        true -> 
-                            amqp10_app_props_to_amqp091_headers(Headers, AppPropsBin);
-                        _ -> 
-                            Headers
-                     end,
-    AppPropsAdded = set_header(
-        ?APP_PROPERTIES_HEADER,
-        AppPropsBin, Amqp091Headers),
-    assemble(body, {R, P#'P_basic'{
-                         headers = AppPropsAdded}, C},
-             decode_section(Rest), Rest);
-assemble(app_properties, {R, P, C}, Else, Uneaten) ->
-    assemble(body, {R, P, C}, Else, Uneaten);
-
-%% The only 'interoperable' content is a single amqp-data section.
-%% Everything else we will leave as-is. We still have to parse the
-%% sections one-by-one, however, to see when we hit the footer or
-%% whatever comes next.
-
-%% NB we do not strictly enforce the (slightly random) rules
-%% pertaining to body sections, that is:
-%%  - one amqp-value; OR
-%%  - one or more amqp-sequence; OR
-%%  - one or more amqp-data.
-%% We allow any number of each kind, in any permutation.
-
-assemble(body, {R, P, _}, {#'v1_0.data'{content = Content}, Rest}, Uneaten) ->
-    Chunk = chunk(Rest, Uneaten),
-    assemble(amqp10body, {R, set_1_0_type(<<"binary">>, P),
-                          {data, Content, Chunk}},
-             decode_section(Rest), Rest);
-assemble(body, {R, P, C}, Else, Uneaten) ->
-    assemble(amqp10body, {R, P, C}, Else, Uneaten);
-
-assemble(amqp10body, {R, P, C}, {{Type, _}, Rest}, Uneaten)
-  when Type =:= 'v1_0.data' orelse
-       Type =:= 'v1_0.amqp_sequence' orelse
-       Type =:= 'v1_0.amqp_value' ->
-    Encoded = chunk(Rest, Uneaten),
-    assemble(amqp10body,
-             {R, set_1_0_type(<<"amqp-1.0">>, P), add_body_section(Encoded, C)},
-             decode_section(Rest), Rest);
-assemble(amqp10body, {R, P, C}, Else, Uneaten) ->
-    assemble(footer, {R, P, compile_body(C)}, Else, Uneaten);
-
-assemble(footer, {R, P = #'P_basic'{headers = Headers}, C},
-         {#'v1_0.footer'{}, <<>>}, Uneaten) ->
-    {R, P#'P_basic'{headers = set_header(?FOOTER, Uneaten, Headers)}, C};
-assemble(footer, {R, P, C}, none, _) ->
-    {R, P, C};
-assemble(footer, _, Else, _) ->
-    exit({unexpected_trailing_sections, Else}).
-
-%% Catch-all clause, not needed according to dialyzer
-%% assemble(Expected, _, Actual, _) ->
-%%     exit({expected_section, Expected, Actual}).
-
-decode_section(<<>>) ->
-    none;
-decode_section(MsgBin) ->
-    {AmqpValue, Rest} = amqp10_binary_parser:parse(MsgBin),
-    {amqp10_framing:decode(AmqpValue), Rest}.
-
-chunk(Rest, Uneaten) ->
-    ChunkLen = size(Uneaten) - size(Rest),
-    <> = Uneaten,
-    Chunk.
-
-add_body_section(C, {data, _, Bin}) ->
-    [C, Bin];
-add_body_section(C, Cs) ->
-    [C | Cs].
-
-compile_body({data, Content, _}) ->
-    Content;
-compile_body(Sections) ->
-    lists:reverse(Sections).
-
-translate_header(Header10, Props) ->
-    Props#'P_basic'{
-      delivery_mode = case Header10#'v1_0.header'.durable of
-                          true -> 2;
-                          _    -> 1
-                      end,
-      priority = unwrap(Header10#'v1_0.header'.priority),
-      expiration = to_expiration(Header10#'v1_0.header'.ttl),
-      type = undefined,
-      app_id = undefined,
-      cluster_id = undefined}.
-
-translate_properties(Props10, Props10Bin,
-                     Props = #'P_basic'{headers = Headers}) ->
-    Props#'P_basic'{
-      headers          = set_header(?PROPERTIES_HEADER, Props10Bin,
-                                     Headers),
-      content_type     = unwrap(Props10#'v1_0.properties'.content_type),
-      content_encoding = unwrap(Props10#'v1_0.properties'.content_encoding),
-      correlation_id   = unwrap(Props10#'v1_0.properties'.correlation_id),
-      reply_to         = case unwrap(Props10#'v1_0.properties'.reply_to) of
-                             <<"/queue/", Q/binary>> -> Q;
-                             Else                    -> Else
-                         end,
-      message_id       = unwrap(Props10#'v1_0.properties'.message_id),
-      user_id          = unwrap(Props10#'v1_0.properties'.user_id),
-      timestamp        = unwrap(Props10#'v1_0.properties'.creation_time)}.
-
-routing_key(Props10) ->
-    unwrap(Props10#'v1_0.properties'.subject).
-
-unwrap(undefined)      -> undefined;
-unwrap({_Type, Thing}) -> Thing.
-
-to_expiration(undefined) ->
-    undefined;
-to_expiration({uint, Num}) ->
-    list_to_binary(integer_to_list(Num)).
-
-from_expiration(PBasic) ->
-    case rabbit_basic:parse_expiration(PBasic) of
-        {ok, undefined} -> undefined;
-        {ok, N} -> {uint, N};
-        _ -> undefined
-    end.
-
-set_header(Header, Value, undefined) ->
-    set_header(Header, Value, []);
-set_header(Header, Value, Headers) ->
-    rabbit_misc:set_table_value(Headers, Header, longstr, Value).
-
-set_1_0_type(Type, Props = #'P_basic'{}) ->
-    Props#'P_basic'{type = Type}.
-
-%%--------------------------------------------------------------------
-
-%% TODO create delivery-annotations
-
-annotated_message(RKey, #'basic.deliver'{redelivered = Redelivered},
-                  #amqp_msg{props = Props,
-                            payload = Content}) ->
-    #'P_basic'{ headers = Headers } = Props,
-    Header10 = #'v1_0.header'
-      {durable = case Props#'P_basic'.delivery_mode of
-                     2 -> true;
-                     _ -> false
-                 end,
-       priority = wrap(ubyte, Props#'P_basic'.priority),
-       ttl = from_expiration(Props),
-       first_acquirer = not Redelivered,
-       delivery_count = case Redelivered of
-                            true -> deliverycount_from_headers(Headers);
-                            false -> undefined
-                        end},
-    HeadersBin = amqp10_framing:encode_bin(Header10),
-    MsgAnnoBin0 =
-        case table_lookup(Headers, ?MESSAGE_ANNOTATIONS_HEADER) of
-            undefined  -> <<>>;
-            {_, MABin} -> MABin
-        end,
-    MsgAnnoBin =
-        case table_lookup(Headers, ?STREAM_OFFSET_HEADER) of
-            undefined ->
-                MsgAnnoBin0;
-            {_, StreamOffset} when is_integer(StreamOffset) ->
-                case amqp10_framing:decode_bin(MsgAnnoBin0) of
-                    [#'v1_0.message_annotations'{content = C0} = MA] ->
-                        Contents = map_add(utf8, ?STREAM_OFFSET_HEADER,
-                                           ulong, StreamOffset, C0),
-                        amqp10_framing:encode_bin(
-                          MA#'v1_0.message_annotations'{content = Contents});
-                    [] ->
-                        Contents = map_add(utf8, ?STREAM_OFFSET_HEADER,
-                                           ulong, StreamOffset, []),
-                        amqp10_framing:encode_bin(
-                          #'v1_0.message_annotations'{content = Contents})
-                end
-        end,
-    PropsBin =
-        case table_lookup(Headers, ?PROPERTIES_HEADER) of
-            {_, Props10Bin} ->
-                Props10Bin;
-            undefined ->
-                Props10 = #'v1_0.properties'{
-                  message_id = wrap(utf8, Props#'P_basic'.message_id),
-                  user_id = wrap(utf8, Props#'P_basic'.user_id),
-                  to = undefined,
-                  subject = wrap(utf8, RKey),
-                  reply_to = case Props#'P_basic'.reply_to of
-                                 undefined ->
-                                     undefined;
-                                 _ ->
-                                     wrap(utf8,
-                                          <<"/queue/",
-                                            (Props#'P_basic'.reply_to)/binary>>)
-                             end,
-                  correlation_id = wrap(utf8, Props#'P_basic'.correlation_id),
-                  content_type = wrap(symbol, Props#'P_basic'.content_type),
-                  content_encoding = wrap(symbol, Props#'P_basic'.content_encoding),
-                  creation_time = wrap(timestamp, Props#'P_basic'.timestamp)},
-                amqp10_framing:encode_bin(Props10)
-        end,
-    AppPropsBin =
-        case table_lookup(Headers, ?APP_PROPERTIES_HEADER) of
-            {_, AppProps10Bin} ->
-                AppProps10Bin;
-            undefined ->
-                case ?CONVERT_AMQP091_HEADERS_TO_APP_PROPS of 
-                    true -> 
-                        case amqp091_headers_to_amqp10_app_props(Headers) of
-                            undefined -> [];
-                            Other -> 
-                                amqp10_framing:encode_bin(Other)
-                        end;
-                    _ -> 
-                        []
-                end
-        end,
-    DataBin = case Props#'P_basic'.type of
-                  <<"amqp-1.0">> ->
-                      Content;
-                  _Else -> % e.g., <<"binary">> if originally from 1.0
-                      amqp10_framing:encode_bin(
-                        #'v1_0.data'{content = Content})
-              end,
-    FooterBin =
-        case table_lookup(Headers, ?FOOTER) of
-            undefined -> <<>>;
-            {_, FBin} -> FBin
-    end,
-    [HeadersBin, MsgAnnoBin, PropsBin, AppPropsBin, DataBin, FooterBin].
-
-wrap(_Type, undefined) ->
-    undefined;
-wrap(Type, Val) ->
-    {Type, Val}.
-
-table_lookup(undefined, _)    -> undefined;
-table_lookup(Headers, Header) -> rabbit_misc:table_lookup(Headers, Header).
-
-map_add(KeyType, Key, Type, Value, Acc) ->
-    [{wrap(KeyType, Key), wrap(Type, Value)} | Acc].
-
-amqp10_app_props_to_amqp091_headers(CurrentHeaders, AppPropsBin) -> 
-    case amqp10_framing:decode_bin(AppPropsBin) of 
-        [#'v1_0.application_properties'{ content = AppProps}] when is_list(AppProps) -> 
-            Hs = case CurrentHeaders of 
-                undefined -> [];
-                Headers -> Headers
-            end,
-            lists:foldl(fun(Prop, Acc) -> 
-                            case Prop of 
-                                {{utf8, Key}, TypeVal} ->
-                                    case type10_to_type091(Key, TypeVal) of
-                                        undefined -> Acc;
-                                        Typed -> [Typed |Acc]
-                                    end;
-                                _ ->
-                                    Acc
-                            end
-                         end, Hs, AppProps);
-        _ -> CurrentHeaders
-    end.
-type10_to_type091(Key, TypeVal) ->
-    try
-        rabbit_msg_record:to_091(Key, TypeVal)
-    catch
-        _:function_clause -> undefined
-    end.
-
-amqp091_headers_to_amqp10_app_props(undefined) -> undefined;
-amqp091_headers_to_amqp10_app_props(Headers) when is_list(Headers) -> 
-    AppPropsOut = lists:foldl(fun(H, Acc) -> 
-        case H of 
-            {Key, Type, Value} -> 
-                case type091_to_type10(Type, Value) of 
-                    undefined -> Acc;
-                    Typed -> 
-                        [{{utf8, Key}, Typed}|Acc]
-                end;
-            _ -> Acc
-        end
-    end, [], Headers),
-    #'v1_0.application_properties'{content = AppPropsOut}.
-    
-type091_to_type10(Type, Value) ->
-    try
-        rabbit_msg_record:from_091(Type, Value)
-    catch
-        _:function_clause -> undefined
-    end.
-
-deliverycount_from_headers(Headers) -> 
-    case table_lookup(Headers, ?X_DELIVERY_COUNT) of
-            undefined -> undefined;
-            {_, Value} when is_integer(Value) -> wrap(uint,Value);
-            _ -> undefined
-    end.
-
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_outgoing_link.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_outgoing_link.erl
deleted file mode 100644
index 68f0d4b21c6e..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_outgoing_link.erl
+++ /dev/null
@@ -1,278 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_outgoing_link).
-
--export([attach/3, detach/3, delivery/6, transferred/3, credit_drained/3, flow/3]).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--import(rabbit_amqp1_0_util, [protocol_error/3, serial_add/2]).
--import(rabbit_amqp1_0_link_util, [handle_to_ctag/1]).
-
--define(INIT_TXFR_COUNT, 0).
--define(DEFAULT_SEND_SETTLED, false).
-
--record(outgoing_link, {queue,
-                        delivery_count = 0,
-                        send_settled,
-                        default_outcome,
-                        route_state}).
-
-detach(#'v1_0.detach'{handle = Handle}, BCh,_Link) ->
-    CTag = handle_to_ctag(Handle),
-    rabbit_amqp1_0_channel:call(BCh, #'basic.cancel'{consumer_tag = CTag}),
-    ok.
-
-attach(#'v1_0.attach'{name = Name,
-                      handle = Handle,
-                      source = Source,
-                      snd_settle_mode = SndSettleMode,
-                      rcv_settle_mode = RcvSettleMode}, BCh, DCh) ->
-    {DefaultOutcome, Outcomes} = rabbit_amqp1_0_link_util:outcomes(Source),
-    SndSettled =
-        case SndSettleMode of
-            ?V_1_0_SENDER_SETTLE_MODE_SETTLED   -> true;
-            ?V_1_0_SENDER_SETTLE_MODE_UNSETTLED -> false;
-            _                                   -> ?DEFAULT_SEND_SETTLED
-        end,
-    DOSym = amqp10_framing:symbol_for(DefaultOutcome),
-    case ensure_source(Source,
-                       #outgoing_link{delivery_count  = ?INIT_TXFR_COUNT,
-                                      send_settled    = SndSettled,
-                                      default_outcome = DOSym,
-                                      route_state     =
-                                        rabbit_routing_util:init_state()},
-                       DCh) of
-        {ok, Source1, OutgoingLink = #outgoing_link{queue = QueueName}} ->
-            CTag = handle_to_ctag(Handle),
-            Args = source_filters_to_consumer_args(Source1),
-
-            case rabbit_amqp1_0_channel:subscribe(
-                   BCh, #'basic.consume'{
-                     queue = QueueName,
-                     consumer_tag = CTag,
-                     %% we will ack when we've transferred
-                     %% a message, or when we get an ack
-                     %% from the client.
-                     no_ack = false,
-                     %% TODO exclusive?
-                     exclusive = false,
-                     arguments = Args ++
-                                  [{<<"x-credit">>, table,
-                                   [{<<"credit">>, long,    0},
-                                    {<<"drain">>,  bool, false}]}]},
-                   self()) of
-                #'basic.consume_ok'{} ->
-                    %% TODO we should avoid the race by getting the queue to send
-                    %% attach back, but a.t.m. it would use the wrong codec.
-                    {ok, [#'v1_0.attach'{
-                       name = Name,
-                       handle = Handle,
-                       initial_delivery_count = {uint, ?INIT_TXFR_COUNT},
-                       snd_settle_mode =
-                           case SndSettled of
-                               true  -> ?V_1_0_SENDER_SETTLE_MODE_SETTLED;
-                               false -> ?V_1_0_SENDER_SETTLE_MODE_UNSETTLED
-                           end,
-                       rcv_settle_mode = RcvSettleMode,
-                       source = Source1#'v1_0.source'{
-                                  default_outcome = DefaultOutcome,
-                                  outcomes        = Outcomes
-                                 },
-                       role = ?SEND_ROLE}], OutgoingLink};
-                Fail ->
-                    protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
-                                   "Consume failed: ~tp", [Fail])
-            end;
-        {error, Reason} ->
-            %% TODO proper link establishment protocol here?
-            protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
-                               "Attach rejected: ~tp", [Reason])
-    end.
-
-credit_drained(#'basic.credit_drained'{credit_drained = CreditDrained},
-               Handle, Link = #outgoing_link{delivery_count = Count0}) ->
-    Count = Count0 + CreditDrained,
-    %% The transfer count that is given by the queue should be at
-    %% least that we have locally, since we will either have received
-    %% all the deliveries and transferred them, or the queue will have
-    %% advanced it due to drain. So we adopt the queue's idea of the
-    %% count.
-    %% TODO account for it not being there any more
-    F = #'v1_0.flow'{ handle      = Handle,
-                      delivery_count = {uint, Count},
-                      link_credit = {uint, 0},
-                      available   = {uint, 0},
-                      drain       = true },
-    {F, Link#outgoing_link{delivery_count = Count}}.
-
-flow(#outgoing_link{delivery_count = LocalCount},
-     #'v1_0.flow'{handle         = Handle,
-                  delivery_count = Count0,
-                  link_credit    = {uint, RemoteCredit},
-                  drain          = Drain0}, BCh) ->
-    {uint, RemoteCount} = default(Count0, {uint, LocalCount}),
-    Drain = default(Drain0, false),
-    %% See section 2.6.7
-    LocalCredit = RemoteCount + RemoteCredit - LocalCount,
-    CTag = handle_to_ctag(Handle),
-    #'basic.credit_ok'{available = Available} =
-        rabbit_amqp1_0_channel:call(
-          BCh, #'basic.credit'{consumer_tag = CTag,
-                               credit       = LocalCredit,
-                               drain        = Drain}),
-    case Available of
-        -1 ->
-            {ok, []};
-        %% We don't know - probably because this flow relates
-        %% to a handle that does not yet exist
-        %% TODO is this an error?
-        _  ->
-            {ok, [#'v1_0.flow'{
-                    handle         = Handle,
-                    delivery_count = {uint, LocalCount},
-                    link_credit    = {uint, LocalCredit},
-                    available      = {uint, Available},
-                    drain          = Drain}]}
-    end.
-
-default(undefined, Default) -> Default;
-default(Thing,    _Default) -> Thing.
-
-ensure_source(Source = #'v1_0.source'{address       = Address,
-                                      dynamic       = Dynamic,
-                                      durable       = Durable,
-                                      filter = _Filters,
-                                      %% TODO
-                                      expiry_policy = _ExpiryPolicy,
-                                      %% TODO
-                                      timeout       = _Timeout},
-              Link = #outgoing_link{ route_state = RouteState }, DCh) ->
-    DeclareParams = [{durable, rabbit_amqp1_0_link_util:durable(Durable)},
-                     {exclusive, false},
-                     {auto_delete, false},
-                     {check_exchange, true},
-                     {nowait, false}],
-    case Dynamic of
-        true -> protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                               "Dynamic sources not supported", []);
-        _    -> ok
-    end,
-    case Address of
-        {utf8, Destination} ->
-            case rabbit_routing_util:parse_endpoint(Destination, false) of
-                {ok, Dest} ->
-                    {ok, Queue, RouteState1} =
-                        rabbit_amqp1_0_channel:convert_error(
-                          fun() ->
-                                  rabbit_routing_util:ensure_endpoint(
-                                    source, DCh, Dest, DeclareParams,
-                                    RouteState)
-                          end),
-                    ER = rabbit_routing_util:parse_routing(Dest),
-                    ok = rabbit_routing_util:ensure_binding(Queue, ER, DCh),
-                    {ok, Source, Link#outgoing_link{route_state = RouteState1,
-                                                    queue       = Queue}};
-                {error, _} = E ->
-                    E
-            end;
-        _ ->
-            {error, {address_not_utf8_string, Address}}
-    end.
-
-delivery(Deliver = #'basic.deliver'{delivery_tag = DeliveryTag,
-                                    routing_key  = RKey},
-                Msg, FrameMax, Handle, Session,
-                #outgoing_link{send_settled = SendSettled,
-                               default_outcome = DefaultOutcome}) ->
-    DeliveryId = rabbit_amqp1_0_session:next_delivery_id(Session),
-    Session1 = rabbit_amqp1_0_session:record_outgoing(
-                 DeliveryTag, SendSettled, DefaultOutcome, Session),
-    Txfr = #'v1_0.transfer'{handle = Handle,
-                            delivery_tag = {binary, <>},
-                            delivery_id = {uint, DeliveryId},
-                            %% The only one in AMQP 1-0
-                            message_format = {uint, 0},
-                            settled = SendSettled,
-                            resume = false,
-                            more = false,
-                            aborted = false,
-                            %% TODO: actually batchable would be fine,
-                            %% but in any case it's only a hint
-                            batchable = false},
-    Msg1_0 = rabbit_amqp1_0_message:annotated_message(
-               RKey, Deliver, Msg),
-    ?DEBUG("Outbound content:~n  ~tp",
-           [[amqp10_framing:pprint(Section) ||
-                Section <- amqp10_framing:decode_bin(
-                             iolist_to_binary(Msg1_0))]]),
-    %% TODO Ugh
-    TLen = iolist_size(amqp10_framing:encode_bin(Txfr)),
-    Frames = case FrameMax of
-                 unlimited ->
-                     [[Txfr, Msg1_0]];
-                 _ ->
-                     encode_frames(Txfr, Msg1_0, FrameMax - TLen, [])
-             end,
-    {ok, Frames, Session1}.
-
-encode_frames(_T, _Msg, MaxContentLen, _Transfers) when MaxContentLen =< 0 ->
-    protocol_error(?V_1_0_AMQP_ERROR_FRAME_SIZE_TOO_SMALL,
-                   "Frame size is too small by ~tp bytes", [-MaxContentLen]);
-encode_frames(T, Msg, MaxContentLen, Transfers) ->
-    case iolist_size(Msg) > MaxContentLen of
-        true  ->
-            <> =
-                iolist_to_binary(Msg),
-            T1 = T#'v1_0.transfer'{more = true},
-            encode_frames(T, Rest, MaxContentLen,
-                          [[T1, Chunk] | Transfers]);
-        false ->
-            lists:reverse([[T, Msg] | Transfers])
-    end.
-
-transferred(DeliveryTag, Channel,
-            Link = #outgoing_link{ delivery_count = Count,
-                                   send_settled   = SendSettled }) ->
-    if SendSettled ->
-            rabbit_amqp1_0_channel:cast(
-              Channel, #'basic.ack'{ delivery_tag = DeliveryTag });
-       true ->
-            ok
-    end,
-    Link#outgoing_link{delivery_count = serial_add(Count, 1)}.
-
-source_filters_to_consumer_args(#'v1_0.source'{filter = {map, KVList}}) ->
-    Key = {symbol, <<"rabbitmq:stream-offset-spec">>},
-    case keyfind_unpack_described(Key, KVList) of
-        {_, {timestamp, Ts}} ->
-            [{<<"x-stream-offset">>, timestamp, Ts div 1000}]; %% 0.9.1 uses second based timestamps
-        {_, {utf8, Spec}} ->
-            [{<<"x-stream-offset">>, longstr, Spec}]; %% next, last, first and "10m" etc
-        {_, {_, Offset}} when is_integer(Offset) ->
-            [{<<"x-stream-offset">>, long, Offset}]; %% integer offset
-        _ ->
-            []
-    end;
-source_filters_to_consumer_args(_Source) ->
-    [].
-
-keyfind_unpack_described(Key, KvList) ->
-    %% filterset values _should_ be described values
-    %% they aren't always however for historical reasons so we need this bit of
-    %% code to return a plain value for the given filter key
-    case lists:keyfind(Key, 1, KvList) of
-        {Key, {described, Key, Value}} ->
-            {Key, Value};
-        {Key, _} = Kv ->
-            Kv;
-        false ->
-            false
-    end.
-
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl
deleted file mode 100644
index a2f6d69d309a..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_reader.erl
+++ /dev/null
@@ -1,815 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_reader).
-
--include_lib("rabbit_common/include/rabbit.hrl").
--include_lib("rabbit_common/include/rabbit_framing.hrl").
--include_lib("kernel/include/inet.hrl").
--include("rabbit_amqp1_0.hrl").
-
--export([init/2, mainloop/2]).
--export([info/2]).
-
-%% TODO which of these are needed?
--export([shutdown/2]).
--export([system_continue/3, system_terminate/4, system_code_change/4]).
--export([conserve_resources/3]).
-
--import(rabbit_amqp1_0_util, [protocol_error/3]).
-
--define(HANDSHAKE_TIMEOUT, 10).
--define(NORMAL_TIMEOUT, 3).
--define(CLOSING_TIMEOUT, 30).
--define(SILENT_CLOSE_DELAY, 3).
-
-%%--------------------------------------------------------------------------
-
--record(v1, {parent, sock, connection, callback, recv_len, pending_recv,
-             connection_state, queue_collector, heartbeater, helper_sup,
-             channel_sup_sup_pid, buf, buf_len, throttle, proxy_socket,
-             tracked_channels}).
-
--record(v1_connection, {user, timeout_sec, frame_max, auth_mechanism, auth_state,
-                        hostname}).
-
--record(throttle, {alarmed_by, last_blocked_by, last_blocked_at}).
-
--define(IS_RUNNING(State),
-        (State#v1.connection_state =:= running orelse
-         State#v1.connection_state =:= blocking orelse
-         State#v1.connection_state =:= blocked)).
-
-%%--------------------------------------------------------------------------
-
-unpack_from_0_9_1({Parent, Sock,RecvLen, PendingRecv,
-                   HelperSupPid, Buf, BufLen, ProxySocket}) ->
-    #v1{parent              = Parent,
-        sock                = Sock,
-        callback            = handshake,
-        recv_len            = RecvLen,
-        pending_recv        = PendingRecv,
-        connection_state    = pre_init,
-        queue_collector     = undefined,
-        heartbeater         = none,
-        helper_sup          = HelperSupPid,
-        buf                 = Buf,
-        buf_len             = BufLen,
-        throttle = #throttle{alarmed_by      = [],
-                             last_blocked_by = none,
-                             last_blocked_at = never},
-        connection = #v1_connection{user           = none,
-                                    timeout_sec    = ?HANDSHAKE_TIMEOUT,
-                                    frame_max      = ?FRAME_MIN_SIZE,
-                                    auth_mechanism = none,
-                                    auth_state     = none},
-        proxy_socket        = ProxySocket,
-        tracked_channels    = maps:new()}.
-
-shutdown(Pid, Explanation) ->
-    gen_server:call(Pid, {shutdown, Explanation}, infinity).
-
-system_continue(Parent, Deb, State) ->
-    ?MODULE:mainloop(Deb, State#v1{parent = Parent}).
-
--spec system_terminate(term(), term(), term(), term()) -> no_return().
-system_terminate(Reason, _Parent, _Deb, _State) ->
-    exit(Reason).
-
-system_code_change(Misc, _Module, _OldVsn, _Extra) ->
-    {ok, Misc}.
-
--spec conserve_resources(pid(),
-                         rabbit_alarm:resource_alarm_source(),
-                         rabbit_alarm:resource_alert()) -> ok.
-conserve_resources(Pid, Source, {_, Conserve, _}) ->
-    Pid ! {conserve_resources, Source, Conserve},
-    ok.
-
-server_properties() ->
-    %% The atom doesn't match anything, it's just "not 0-9-1".
-    Raw = lists:keydelete(
-          <<"capabilities">>, 1, rabbit_reader:server_properties(amqp_1_0)),
-    {map, [{{symbol, K}, {utf8, V}} || {K, longstr, V}  <- Raw]}.
-
-%%--------------------------------------------------------------------------
-
-inet_op(F) -> rabbit_misc:throw_on_error(inet_error, F).
-
-recvloop(Deb, State = #v1{pending_recv = true}) ->
-    mainloop(Deb, State);
-recvloop(Deb, State = #v1{connection_state = blocked}) ->
-    mainloop(Deb, State);
-recvloop(Deb, State = #v1{sock = Sock, recv_len = RecvLen, buf_len = BufLen})
-  when BufLen < RecvLen ->
-    case rabbit_net:setopts(Sock, [{active, once}]) of
-        ok ->
-            mainloop(Deb, State#v1{pending_recv = true});
-        {error, Reason} ->
-            throw({inet_error, Reason})
-    end;
-recvloop(Deb, State = #v1{recv_len = RecvLen, buf = Buf, buf_len = BufLen}) ->
-    {Data, Rest} = split_binary(case Buf of
-                                    [B] -> B;
-                                    _   -> list_to_binary(lists:reverse(Buf))
-                                end, RecvLen),
-    recvloop(Deb, handle_input(State#v1.callback, Data,
-                               State#v1{buf = [Rest],
-                                        buf_len = BufLen - RecvLen})).
-
-mainloop(Deb, State = #v1{sock = Sock, buf = Buf, buf_len = BufLen}) ->
-    case rabbit_net:recv(Sock) of
-        {data, Data} ->
-            recvloop(Deb, State#v1{buf = [Data | Buf],
-                                   buf_len = BufLen + size(Data),
-                                   pending_recv = false});
-        closed when State#v1.connection_state =:= closed ->
-            ok;
-        closed ->
-            throw(connection_closed_abruptly);
-        {error, Reason} ->
-            throw({inet_error, Reason});
-        {other, {system, From, Request}} ->
-            sys:handle_system_msg(Request, From, State#v1.parent,
-                                  ?MODULE, Deb, State);
-        {other, Other} ->
-            case handle_other(Other, State) of
-                stop     -> ok;
-                NewState -> recvloop(Deb, NewState)
-            end
-    end.
-
-handle_other({conserve_resources, Source, Conserve},
-             State = #v1{throttle = Throttle =
-                             #throttle{alarmed_by = CR}}) ->
-    CR1 = case Conserve of
-              true  -> lists:usort([Source | CR]);
-              false -> CR -- [Source]
-          end,
-    Throttle1 = Throttle#throttle{alarmed_by = CR1},
-    control_throttle(State#v1{throttle = Throttle1});
-handle_other({'EXIT', Parent, Reason}, State = #v1{parent = Parent}) ->
-    _ = terminate(io_lib:format("broker forced connection closure "
-                                "with reason '~w'", [Reason]), State),
-    %% this is what we are expected to do according to
-    %% http://www.erlang.org/doc/man/sys.html
-    %%
-    %% If we wanted to be *really* nice we should wait for a while for
-    %% clients to close the socket at their end, just as we do in the
-    %% ordinary error case. However, since this termination is
-    %% initiated by our parent it is probably more important to exit
-    %% quickly.
-    exit(Reason);
-handle_other({'DOWN', _MRef, process, ChPid, Reason}, State) ->
-    handle_dependent_exit(ChPid, Reason, State);
-handle_other(handshake_timeout, State)
-  when ?IS_RUNNING(State) orelse
-       State#v1.connection_state =:= closing orelse
-       State#v1.connection_state =:= closed ->
-    State;
-handle_other(handshake_timeout, State) ->
-    throw({handshake_timeout, State#v1.callback});
-handle_other(heartbeat_timeout, State = #v1{connection_state = closed}) ->
-    State;
-handle_other(heartbeat_timeout, #v1{connection_state = S}) ->
-    throw({heartbeat_timeout, S});
-handle_other({'$gen_call', From, {shutdown, Explanation}}, State) ->
-    {ForceTermination, NewState} = terminate(Explanation, State),
-    gen_server:reply(From, ok),
-    case ForceTermination of
-        force  -> stop;
-        normal -> NewState
-    end;
-handle_other({'$gen_cast', force_event_refresh}, State) ->
-    %% Ignore, the broker sent us this as it thinks we are a 0-9-1 connection
-    State;
-handle_other({bump_credit, Msg}, State) ->
-    credit_flow:handle_bump_msg(Msg),
-    control_throttle(State);
-handle_other(terminate_connection, State) ->
-    State;
-handle_other({info, InfoItems, Pid}, State) ->
-    Infos = lists:map(
-              fun(InfoItem) ->
-                      {InfoItem, info_internal(InfoItem, State)}
-              end,
-              InfoItems),
-    Pid ! {info_reply, Infos},
-    State;
-handle_other(Other, _State) ->
-    %% internal error -> something worth dying for
-    exit({unexpected_message, Other}).
-
-switch_callback(State, Callback, Length) ->
-    State#v1{callback = Callback, recv_len = Length}.
-
-terminate(Reason, State) when ?IS_RUNNING(State) ->
-    {normal, handle_exception(State, 0,
-                              error_frame(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
-                                          "Connection forced: ~tp", [Reason]))};
-terminate(_Reason, State) ->
-    {force, State}.
-
-control_throttle(State = #v1{connection_state = CS, throttle = Throttle}) ->
-    IsThrottled = ((Throttle#throttle.alarmed_by =/= []) orelse
-               credit_flow:blocked()),
-    case {CS, IsThrottled} of
-        {running,   true} -> State#v1{connection_state = blocking};
-        {blocking, false} -> State#v1{connection_state = running};
-        {blocked,  false} -> ok = rabbit_heartbeat:resume_monitor(
-                                    State#v1.heartbeater),
-                             State#v1{connection_state = running};
-        {blocked,   true} -> State#v1{throttle = update_last_blocked_by(
-                                                   Throttle)};
-        {_,            _} -> State
-    end.
-
-update_last_blocked_by(Throttle = #throttle{alarmed_by = []}) ->
-    Throttle#throttle{last_blocked_by = flow};
-update_last_blocked_by(Throttle) ->
-    Throttle#throttle{last_blocked_by = resource}.
-
-%%--------------------------------------------------------------------------
-%% error handling / termination
-
-close_connection(State = #v1{connection = #v1_connection{
-                                             timeout_sec = TimeoutSec}}) ->
-    Pid = self(),
-    erlang:send_after((if TimeoutSec > 0 andalso
-                          TimeoutSec < ?CLOSING_TIMEOUT -> TimeoutSec;
-                          true                          -> ?CLOSING_TIMEOUT
-                       end) * 1000, Pid, terminate_connection),
-    rabbit_amqp1_0:unregister_connection(Pid),
-    State#v1{connection_state = closed}.
-
-handle_dependent_exit(ChPid, Reason, State) ->
-    credit_flow:peer_down(ChPid),
-
-    case {ChPid, termination_kind(Reason)} of
-        {_Channel, controlled} ->
-            maybe_close(control_throttle(State));
-        {Channel, uncontrolled} ->
-            {RealReason, Trace} = Reason,
-            R = error_frame(?V_1_0_AMQP_ERROR_INTERNAL_ERROR, "Session error: ~tp~n~tp", [RealReason, Trace]),
-            maybe_close(handle_exception(control_throttle(State), Channel, R))
-    end.
-
-termination_kind(normal) -> controlled;
-termination_kind(_)      -> uncontrolled.
-
-maybe_close(State = #v1{connection_state = closing,
-                        sock = Sock}) ->
-    NewState = close_connection(State),
-    ok = send_on_channel0(Sock, #'v1_0.close'{}),
-    % Perform an rpc call to each session process to allow it time to
-    % process it's internal message buffer before the supervision tree
-    % shuts everything down and in flight messages such as dispositions
-    % could be lost
-    _ = [ _ = rabbit_amqp1_0_session:get_info(SessionPid)
-          || {{channel, _}, {ch_fr_pid, SessionPid}} <- get()],
-    NewState;
-maybe_close(State) ->
-    State.
-
-error_frame(Condition, Fmt, Args) ->
-    #'v1_0.error'{condition = Condition,
-                  description = {utf8, list_to_binary(
-                                         rabbit_misc:format(Fmt, Args))}}.
-
-handle_exception(State = #v1{connection_state = closed}, Channel,
-                 #'v1_0.error'{description = {utf8, Desc}}) ->
-    rabbit_log_connection:error("Error on AMQP 1.0 connection ~tp (~tp), channel ~tp:~n~tp",
-        [self(), closed, Channel, Desc]),
-    State;
-handle_exception(State = #v1{connection_state = CS}, Channel,
-                 ErrorFrame = #'v1_0.error'{description = {utf8, Desc}})
-  when ?IS_RUNNING(State) orelse CS =:= closing ->
-    rabbit_log_connection:error("Error on AMQP 1.0 connection ~tp (~tp), channel ~tp:~n~tp",
-        [self(), CS, Channel, Desc]),
-    %% TODO: session errors shouldn't force the connection to close
-    State1 = close_connection(State),
-    ok = send_on_channel0(State#v1.sock, #'v1_0.close'{error = ErrorFrame}),
-    State1;
-handle_exception(State, Channel, Error) ->
-    %% We don't trust the client at this point - force them to wait
-    %% for a bit so they can't DOS us with repeated failed logins etc.
-    timer:sleep(?SILENT_CLOSE_DELAY * 1000),
-    throw({handshake_error, State#v1.connection_state, Channel, Error}).
-
-%%--------------------------------------------------------------------------
-
-%% Begin 1-0
-
-%% ----------------------------------------
-%% AMQP 1.0 frame handlers
-
-is_connection_frame(#'v1_0.open'{})  -> true;
-is_connection_frame(#'v1_0.close'{}) -> true;
-is_connection_frame(_)               -> false.
-
-%% TODO Handle depending on connection state
-%% TODO It'd be nice to only decode up to the descriptor
-
-handle_1_0_frame(Mode, Channel, Payload, State) ->
-    try
-        handle_1_0_frame0(Mode, Channel, Payload, State)
-    catch
-        _:#'v1_0.error'{} = Reason ->
-            handle_exception(State, 0, Reason);
-        _:{error, {not_allowed, Username}} ->
-            %% section 2.8.15 in http://docs.oasis-open.org/amqp/core/v1.0/os/amqp-core-complete-v1.0-os.pdf
-            handle_exception(State, 0, error_frame(
-                                         ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
-                                         "Access for user '~ts' was refused: insufficient permissions", [Username]));
-        _:Reason:Trace ->
-            handle_exception(State, 0, error_frame(
-                                         ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
-                                         "Reader error: ~tp~n~tp",
-                                         [Reason, Trace]))
-    end.
-
-%% Nothing specifies that connection methods have to be on a
-%% particular channel.
-handle_1_0_frame0(_Mode, Channel, Payload,
-                 State = #v1{ connection_state = CS}) when
-      CS =:= closing; CS =:= closed ->
-    Sections = parse_1_0_frame(Payload, Channel),
-    case is_connection_frame(Sections) of
-        true  -> handle_1_0_connection_frame(Sections, State);
-        false -> State
-    end;
-handle_1_0_frame0(Mode, Channel, Payload, State) ->
-    Sections = parse_1_0_frame(Payload, Channel),
-    case {Mode, is_connection_frame(Sections)} of
-        {amqp, true}  -> handle_1_0_connection_frame(Sections, State);
-        {amqp, false} -> handle_1_0_session_frame(Channel, Sections, State);
-        {sasl, false} -> handle_1_0_sasl_frame(Sections, State)
-    end.
-
-parse_1_0_frame(Payload, _Channel) ->
-    {PerfDesc, Rest} = amqp10_binary_parser:parse(Payload),
-    Perf = amqp10_framing:decode(PerfDesc),
-    ?DEBUG("Channel ~tp ->~n~tp~n~ts",
-           [_Channel, amqp10_framing:pprint(Perf),
-            case Rest of
-                <<>> -> <<>>;
-                _    -> rabbit_misc:format(
-                          "  followed by ~tp bytes of content", [size(Rest)])
-            end]),
-    case Rest of
-        <<>> -> Perf;
-        _    -> {Perf, Rest}
-    end.
-
-handle_1_0_connection_frame(#'v1_0.open'{ max_frame_size = ClientFrameMax,
-                                          channel_max = ClientChannelMax,
-                                          idle_time_out = IdleTimeout,
-                                          hostname = Hostname },
-                            State = #v1{
-                              connection_state = starting,
-                              connection = Connection,
-                              throttle   = Throttle,
-                              helper_sup = HelperSupPid,
-                              sock = Sock}) ->
-    ClientHeartbeatSec = case IdleTimeout of
-                             undefined        -> 0;
-                             {uint, Interval} -> Interval div 1000
-                         end,
-    FrameMax           = case ClientFrameMax of
-                             undefined -> unlimited;
-                             {_, FM}   -> FM
-                         end,
-    {ok, HeartbeatSec} = application:get_env(rabbit, heartbeat),
-    State1 =
-        if (FrameMax =/= unlimited) and (FrameMax < ?FRAME_1_0_MIN_SIZE) ->
-                protocol_error(?V_1_0_AMQP_ERROR_FRAME_SIZE_TOO_SMALL,
-                               "frame_max=~w < ~w min size",
-                               [FrameMax, ?FRAME_1_0_MIN_SIZE]);
-           true ->
-                {ok, Collector} =
-                    rabbit_connection_helper_sup:start_queue_collector(
-                      HelperSupPid, <<"AMQP 1.0">>), %% TODO describe the connection
-                SendFun =
-                    fun() ->
-                            Frame =
-                                amqp10_binary_generator:build_heartbeat_frame(),
-                            catch rabbit_net:send(Sock, Frame)
-                    end,
-
-                Parent = self(),
-                ReceiveFun =
-                    fun() ->
-                            Parent ! heartbeat_timeout
-                    end,
-                %% [2.4.5] the value in idle-time-out SHOULD be half the peer's
-                %%         actual timeout threshold
-                ReceiverHeartbeatSec = lists:min([HeartbeatSec * 2, 4294967]),
-                %% TODO: only start heartbeat receive timer at next next frame
-                Heartbeater =
-                    rabbit_heartbeat:start(HelperSupPid, Sock,
-                                           ClientHeartbeatSec, SendFun,
-                                           ReceiverHeartbeatSec, ReceiveFun),
-                State#v1{connection_state = running,
-                         connection = Connection#v1_connection{
-                                                   frame_max = FrameMax,
-                                                   hostname  = Hostname},
-                         heartbeater = Heartbeater,
-                         queue_collector = Collector}
-        end,
-    HostnameVal = case Hostname of
-                    undefined -> undefined;
-                    null -> undefined;
-                    {utf8, Val} -> Val
-                  end,
-    rabbit_log:debug("AMQP 1.0 connection.open frame: hostname = ~ts, extracted vhost = ~ts, idle_timeout = ~tp" ,
-                    [HostnameVal, vhost(Hostname), HeartbeatSec * 1000]),
-    %% TODO enforce channel_max
-    ok = send_on_channel0(
-           Sock,
-           #'v1_0.open'{channel_max    = ClientChannelMax,
-                        max_frame_size = ClientFrameMax,
-                        idle_time_out  = {uint, HeartbeatSec * 1000},
-                        container_id   = {utf8, rabbit_nodes:cluster_name()},
-                        properties     = server_properties()}),
-    Conserve = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
-    rabbit_amqp1_0:register_connection(self()),
-    control_throttle(
-      State1#v1{throttle = Throttle#throttle{alarmed_by = Conserve}});
-
-handle_1_0_connection_frame(_Frame, State) ->
-    maybe_close(State#v1{connection_state = closing}).
-
-handle_1_0_session_frame(Channel, Frame, State) ->
-    case maps:get(Channel, State#v1.tracked_channels, undefined) of
-        undefined ->
-            case ?IS_RUNNING(State) of
-                true ->
-                    send_to_new_1_0_session(Channel, Frame, State);
-                false ->
-                    throw({channel_frame_while_starting,
-                           Channel, State#v1.connection_state,
-                           Frame})
-            end;
-        SessionPid ->
-            ok = rabbit_amqp1_0_session:process_frame(SessionPid, Frame),
-            case Frame of
-                #'v1_0.end'{} ->
-                    untrack_channel(Channel, State);
-                {#'v1_0.transfer'{}, _MsgPart} ->
-                    case (State#v1.connection_state =:= blocking) of
-                        true ->
-                            ok = rabbit_heartbeat:pause_monitor(
-                                   State#v1.heartbeater),
-                            State#v1{connection_state = blocked};
-                        false ->
-                            State
-                    end;
-                _ ->
-                    State
-            end
-    end.
-
-%% TODO: write a proper ANONYMOUS plugin and unify with STOMP
-handle_1_0_sasl_frame(#'v1_0.sasl_init'{mechanism = {symbol, <<"ANONYMOUS">>},
-                                        hostname = _Hostname},
-                      State = #v1{connection_state = starting,
-                                  sock             = Sock}) ->
-    case application:get_env(rabbitmq_amqp1_0, default_user) of
-        {ok, none} ->
-            %% No need to do anything, we will blow up in start_connection
-            ok;
-        {ok, _} ->
-            %% We only need to send the frame, again start_connection
-            %% will set up the default user.
-            Outcome = #'v1_0.sasl_outcome'{code = {ubyte, 0}},
-            ok = send_on_channel0(Sock, Outcome, rabbit_amqp1_0_sasl),
-            switch_callback(State#v1{connection_state = waiting_amqp0100},
-                            handshake, 8)
-    end;
-handle_1_0_sasl_frame(#'v1_0.sasl_init'{mechanism        = {symbol, Mechanism},
-                                        initial_response = {binary, Response},
-                                        hostname         = _Hostname},
-                      State0 = #v1{connection_state = starting,
-                                   connection       = Connection,
-                                   sock             = Sock}) ->
-    AuthMechanism = auth_mechanism_to_module(Mechanism, Sock),
-    State = State0#v1{connection       =
-                          Connection#v1_connection{
-                            auth_mechanism    = {Mechanism, AuthMechanism},
-                            auth_state        = AuthMechanism:init(Sock)},
-                      connection_state = securing},
-    auth_phase_1_0(Response, State);
-handle_1_0_sasl_frame(#'v1_0.sasl_response'{response = {binary, Response}},
-                      State = #v1{connection_state = securing}) ->
-    auth_phase_1_0(Response, State);
-handle_1_0_sasl_frame(Frame, State) ->
-    throw({unexpected_1_0_sasl_frame, Frame, State}).
-
-%% We need to handle restarts...
-handle_input(handshake, <<"AMQP", 0, 1, 0, 0>>, State) ->
-    start_1_0_connection(amqp, State);
-
-%% 3 stands for "SASL" (keeping this here for when we do TLS)
-handle_input(handshake, <<"AMQP", 3, 1, 0, 0>>, State) ->
-    start_1_0_connection(sasl, State);
-
-handle_input({frame_header_1_0, Mode},
-             Header = <>,
-             State) when DOff >= 2 ->
-    case {Mode, Type} of
-        {amqp, 0} -> ok;
-        {sasl, 1} -> ok;
-        _         -> throw({bad_1_0_header_type, Header, Mode})
-    end,
-    case Size of
-        8 -> % length inclusive
-            State; %% heartbeat
-        _ ->
-            switch_callback(State, {frame_payload_1_0, Mode, DOff, Channel}, Size - 8)
-    end;
-handle_input({frame_header_1_0, _Mode}, Malformed, _State) ->
-    throw({bad_1_0_header, Malformed});
-handle_input({frame_payload_1_0, Mode, DOff, Channel},
-            FrameBin, State) ->
-    SkipBits = (DOff * 32 - 64), % DOff = 4-byte words, we've read 8 already
-    <> = FrameBin,
-    Skip = Skip, %% hide warning when debug is off
-    handle_1_0_frame(Mode, Channel, FramePayload,
-                     switch_callback(State, {frame_header_1_0, Mode}, 8));
-
-handle_input(Callback, Data, _State) ->
-    throw({bad_input, Callback, Data}).
-
-init(Mode, PackedState) ->
-    %% By invoking recvloop here we become 1.0.
-    recvloop(sys:debug_options([]),
-             start_1_0_connection(Mode, unpack_from_0_9_1(PackedState))).
-
-start_1_0_connection(sasl, State = #v1{sock = Sock}) ->
-    send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
-    Ms = {array, symbol,
-          case application:get_env(rabbitmq_amqp1_0, default_user)  of
-              {ok, none} -> [];
-              {ok, _}    -> [{symbol, <<"ANONYMOUS">>}]
-          end ++
-              [{symbol, list_to_binary(atom_to_list(M))} || M <- auth_mechanisms(Sock)]},
-    Mechanisms = #'v1_0.sasl_mechanisms'{sasl_server_mechanisms = Ms},
-    ok = send_on_channel0(Sock, Mechanisms, rabbit_amqp1_0_sasl),
-    start_1_0_connection0(sasl, State);
-
-start_1_0_connection(amqp,
-                     State = #v1{sock       = Sock,
-                                 connection = C = #v1_connection{user = User}}) ->
-    {ok, NoAuthUsername} = application:get_env(rabbitmq_amqp1_0, default_user),
-    case {User, NoAuthUsername} of
-        {none, none} ->
-            send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
-            throw(banned_unauthenticated_connection);
-        {none, Username} ->
-            case rabbit_access_control:check_user_login(
-                   list_to_binary(Username), []) of
-                {ok, NoAuthUser} ->
-                    State1 = State#v1{
-                               connection = C#v1_connection{user = NoAuthUser}},
-                    send_1_0_handshake(Sock, <<"AMQP",0,1,0,0>>),
-                    start_1_0_connection0(amqp, State1);
-                _ ->
-                    send_1_0_handshake(Sock, <<"AMQP",3,1,0,0>>),
-                    throw(default_user_missing)
-            end;
-        _ ->
-            send_1_0_handshake(Sock, <<"AMQP",0,1,0,0>>),
-            start_1_0_connection0(amqp, State)
-    end.
-
-start_1_0_connection0(Mode, State = #v1{connection = Connection,
-                                        helper_sup = HelperSup}) ->
-    ChannelSupSupPid =
-        case Mode of
-            sasl ->
-                undefined;
-            amqp ->
-                StartMFA = {rabbit_amqp1_0_session_sup_sup, start_link, []},
-                ChildSpec = #{id => channel_sup_sup,
-                              start => StartMFA,
-                              restart => transient,
-                              significant => true,
-                              shutdown => infinity,
-                              type => supervisor,
-                              modules => [rabbit_amqp1_0_session_sup_sup]},
-                {ok, Pid} = supervisor:start_child(HelperSup, ChildSpec),
-                Pid
-        end,
-    switch_callback(State#v1{connection = Connection#v1_connection{
-                                            timeout_sec = ?NORMAL_TIMEOUT},
-                             channel_sup_sup_pid = ChannelSupSupPid,
-                             connection_state = starting},
-                    {frame_header_1_0, Mode}, 8).
-
-send_1_0_handshake(Sock, Handshake) ->
-    ok = inet_op(fun () -> rabbit_net:send(Sock, Handshake) end).
-
-send_on_channel0(Sock, Method) ->
-    send_on_channel0(Sock, Method, amqp10_framing).
-
-send_on_channel0(Sock, Method, Framing) ->
-    ok = rabbit_amqp1_0_writer:internal_send_command(
-           Sock, 0, Method, Framing).
-
-%% End 1-0
-
-auth_mechanism_to_module(TypeBin, Sock) ->
-    case rabbit_registry:binary_to_type(TypeBin) of
-        {error, not_found} ->
-            protocol_error(?V_1_0_AMQP_ERROR_NOT_FOUND,
-                           "unknown authentication mechanism '~ts'", [TypeBin]);
-        T ->
-            case {lists:member(T, auth_mechanisms(Sock)),
-                  rabbit_registry:lookup_module(auth_mechanism, T)} of
-                {true, {ok, Module}} ->
-                    Module;
-                _ ->
-                    protocol_error(?V_1_0_AMQP_ERROR_NOT_FOUND,
-                                   "invalid authentication mechanism '~ts'", [T])
-            end
-    end.
-
-auth_mechanisms(Sock) ->
-    {ok, Configured} = application:get_env(rabbit, auth_mechanisms),
-    [Name || {Name, Module} <- rabbit_registry:lookup_all(auth_mechanism),
-             Module:should_offer(Sock), lists:member(Name, Configured)].
-
-%% Begin 1-0
-
-auth_phase_1_0(Response,
-               State = #v1{connection = Connection =
-                               #v1_connection{auth_mechanism = {Name, AuthMechanism},
-                                              auth_state     = AuthState},
-                       sock = Sock}) ->
-    case AuthMechanism:handle_response(Response, AuthState) of
-        {refused, User, Msg, Args} ->
-            %% We don't trust the client at this point - force them to wait
-            %% for a bit before sending the sasl outcome frame
-            %% so they can't DOS us with repeated failed logins etc.
-            rabbit_core_metrics:auth_attempt_failed(<<>>, User, amqp10),
-            timer:sleep(?SILENT_CLOSE_DELAY * 1000),
-            Outcome = #'v1_0.sasl_outcome'{code = {ubyte, 1}},
-            ok = send_on_channel0(Sock, Outcome, rabbit_amqp1_0_sasl),
-            protocol_error(
-              ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, "~ts login refused: ~ts",
-              [Name, io_lib:format(Msg, Args)]);
-        {protocol_error, Msg, Args} ->
-            rabbit_core_metrics:auth_attempt_failed(<<>>, <<>>, amqp10),
-            protocol_error(?V_1_0_AMQP_ERROR_DECODE_ERROR, Msg, Args);
-        {challenge, Challenge, AuthState1} ->
-            rabbit_core_metrics:auth_attempt_succeeded(<<>>, <<>>, amqp10),
-            Secure = #'v1_0.sasl_challenge'{challenge = {binary, Challenge}},
-            ok = send_on_channel0(Sock, Secure, rabbit_amqp1_0_sasl),
-            State#v1{connection = Connection#v1_connection{auth_state = AuthState1}};
-        {ok, User = #user{username = Username}} ->
-            case rabbit_access_control:check_user_loopback(Username, Sock) of
-                ok ->
-                    rabbit_log_connection:info(
-                        "AMQP 1.0 connection ~tp: user '~ts' authenticated",
-                        [self(), Username]),
-                    rabbit_core_metrics:auth_attempt_succeeded(<<>>, Username, amqp10),
-                    ok;
-                not_allowed ->
-                    rabbit_core_metrics:auth_attempt_failed(<<>>, Username, amqp10),
-                    protocol_error(
-                      ?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS,
-                      "user '~ts' can only connect via localhost",
-                      [Username])
-            end,
-            Outcome = #'v1_0.sasl_outcome'{code = {ubyte, 0}},
-            ok = send_on_channel0(Sock, Outcome, rabbit_amqp1_0_sasl),
-            switch_callback(
-              State#v1{connection_state = waiting_amqp0100,
-                       connection = Connection#v1_connection{user = User}},
-              handshake, 8)
-    end.
-
-track_channel(Channel, ChFrPid, State) ->
-    rabbit_log:debug("AMQP 1.0 opened channel = ~tp " , [{Channel, ChFrPid}]),
-    State#v1{tracked_channels = maps:put(Channel, ChFrPid, State#v1.tracked_channels)}.
-
-untrack_channel(Channel, State) ->
-    case maps:take(Channel, State#v1.tracked_channels) of
-        {Value, NewMap} ->
-            rabbit_log:debug("AMQP 1.0 closed channel = ~tp ", [{Channel, Value}]),
-            State#v1{tracked_channels = NewMap};
-        error -> State
-    end.
-
-send_to_new_1_0_session(Channel, Frame, State) ->
-    #v1{sock = Sock, queue_collector = Collector,
-        channel_sup_sup_pid = ChanSupSup,
-        connection = #v1_connection{frame_max = FrameMax,
-                                    hostname  = Hostname,
-                                    user      = User},
-        proxy_socket = ProxySocket} = State,
-    %% Note: the equivalent, start_channel is in channel_sup_sup
-
-    case rabbit_amqp1_0_session_sup_sup:start_session(
-           %% NB subtract fixed frame header size
-           ChanSupSup, {amqp10_framing, Sock, Channel,
-                        case FrameMax of
-                            unlimited -> unlimited;
-                            _         -> FrameMax - 8
-                        end,
-                        self(), User, vhost(Hostname), Collector, ProxySocket}) of
-        {ok, _ChSupPid, ChFrPid} ->
-            erlang:monitor(process, ChFrPid),
-            ModifiedState = track_channel(Channel, ChFrPid, State),
-            rabbit_log_connection:info(
-                        "AMQP 1.0 connection ~tp: "
-                        "user '~ts' authenticated and granted access to vhost '~ts'",
-                        [self(), User#user.username, vhost(Hostname)]),
-            ok = rabbit_amqp1_0_session:process_frame(ChFrPid, Frame),
-            ModifiedState;
-        {error, {not_allowed, _}} ->
-            rabbit_log:error("AMQP 1.0: user '~ts' is not allowed to access virtual host '~ts'",
-                [User#user.username, vhost(Hostname)]),
-            %% Let's skip the supervisor trace, this is an expected error
-            throw({error, {not_allowed, User#user.username}});
-        {error, _} = E ->
-            throw(E)
-    end.
-
-vhost({utf8, <<"vhost:", VHost/binary>>}) ->
-    VHost;
-vhost(_) ->
-    application:get_env(rabbitmq_amqp1_0, default_vhost,
-                        application:get_env(rabbit, default_vhost, <<"/">>)).
-
-%% End 1-0
-
-info(Pid, InfoItems) ->
-    case InfoItems -- ?INFO_ITEMS of
-        [] ->
-            Ref = erlang:monitor(process, Pid),
-            Pid ! {info, InfoItems, self()},
-            receive
-                {info_reply, Items} ->
-                    erlang:demonitor(Ref),
-                    Items;
-                {'DOWN', _, process, Pid, _} ->
-                    []
-            end;
-        UnknownItems -> throw({bad_argument, UnknownItems})
-    end.
-
-info_internal(pid, #v1{}) -> self();
-info_internal(connection, #v1{connection = Val}) ->
-    Val;
-info_internal(node, #v1{}) -> node();
-info_internal(auth_mechanism, #v1{connection = #v1_connection{auth_mechanism = none}}) ->
-    none;
-info_internal(auth_mechanism, #v1{connection = #v1_connection{auth_mechanism = {Name, _Mod}}}) ->
-    Name;
-info_internal(host, #v1{connection = #v1_connection{hostname = {utf8, Val}}}) ->
-    Val;
-info_internal(host, #v1{connection = #v1_connection{hostname = Val}}) ->
-    Val;
-info_internal(frame_max, #v1{connection = #v1_connection{frame_max = Val}}) ->
-    Val;
-info_internal(timeout, #v1{connection = #v1_connection{timeout_sec = Val}}) ->
-    Val;
-info_internal(user,
-              #v1{connection = #v1_connection{user = #user{username = Val}}}) ->
-    Val;
-info_internal(user,
-              #v1{connection = #v1_connection{user = none}}) ->
-    '';
-info_internal(state, #v1{connection_state = Val}) ->
-    Val;
-info_internal(SockStat, S) when SockStat =:= recv_oct;
-                                SockStat =:= recv_cnt;
-                                SockStat =:= send_oct;
-                                SockStat =:= send_cnt;
-                                SockStat =:= send_pend ->
-    socket_info(fun (Sock) -> rabbit_net:getstat(Sock, [SockStat]) end,
-                fun ([{_, I}]) -> I end, S);
-info_internal(ssl, #v1{sock = Sock}) -> rabbit_net:is_ssl(Sock);
-info_internal(SSL, #v1{sock = Sock, proxy_socket = ProxySock})
-  when SSL =:= ssl_protocol;
-       SSL =:= ssl_key_exchange;
-       SSL =:= ssl_cipher;
-       SSL =:= ssl_hash ->
-    rabbit_ssl:info(SSL, {Sock, ProxySock});
-info_internal(Cert, #v1{sock = Sock})
-  when Cert =:= peer_cert_issuer;
-       Cert =:= peer_cert_subject;
-       Cert =:= peer_cert_validity ->
-    rabbit_ssl:cert_info(Cert, Sock).
-
-%% From rabbit_reader
-socket_info(Get, Select, #v1{sock = Sock}) ->
-    case Get(Sock) of
-        {ok,    T} -> Select(T);
-        {error, _} -> ''
-    end.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session.erl
deleted file mode 100644
index a62433b1f107..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session.erl
+++ /dev/null
@@ -1,420 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_session).
-
--export([process_frame/2,
-         get_info/1]).
-
--export([init/1, begin_/2, maybe_init_publish_id/2, record_delivery/3,
-         incr_incoming_id/1, next_delivery_id/1, transfers_left/1,
-         record_transfers/2, bump_outgoing_window/1,
-         record_outgoing/4, settle/3, flow_fields/2, channel/1,
-         flow/2, ack/2, return/2, validate_attach/1]).
-
--import(rabbit_amqp1_0_util, [protocol_error/3,
-                              serial_add/2, serial_diff/2, serial_compare/2]).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--define(MAX_SESSION_WINDOW_SIZE, 65535).
--define(DEFAULT_MAX_HANDLE, 16#ffffffff).
--define(CALL_TIMEOUT, 30000). % 30s - matches CLOSE_TIMEOUT
-
--record(session, {channel_num, %% we just use the incoming (AMQP 1.0) channel number
-                  remote_incoming_window, % keep track of the window until we're told
-                  remote_outgoing_window,
-                  next_incoming_id, % just to keep a check
-                  incoming_window_max, % )
-                  incoming_window,     % ) so we know when to open the session window
-                  next_outgoing_id = 0, % arbitrary count of outgoing transfers
-                  outgoing_window,
-                  outgoing_window_max,
-                  next_publish_id, %% the 0-9-1-side counter for confirms
-                  next_delivery_id = 0,
-                  incoming_unsettled_map,
-                  outgoing_unsettled_map }).
-
-%% We record delivery_id -> #outgoing_delivery{}, so that we can
-%% respond to dispositions about messages we've sent. NB the
-%% delivery-tag doubles as the id we use when acking the rabbit
-%% delivery.
--record(outgoing_delivery, {delivery_tag, expected_outcome}).
-
-%% We record confirm_id -> #incoming_delivery{} so we can relay
-%% confirms from the broker back to the sending client. NB we have
-%% only one possible outcome, so there's no need to record it here.
--record(incoming_delivery, {delivery_id}).
-
-get_info(Pid) ->
-    gen_server2:call(Pid, info, ?CALL_TIMEOUT).
-
-process_frame(Pid, Frame) ->
-    credit_flow:send(Pid),
-    gen_server2:cast(Pid, {frame, Frame, self()}).
-
-init(Channel) ->
-    #session{channel_num            = Channel,
-             next_publish_id        = 0,
-             incoming_unsettled_map = gb_trees:empty(),
-             outgoing_unsettled_map = gb_trees:empty()}.
-
-%% Session window:
-%%
-%% Each session has two abstract[1] buffers, one to record the
-%% unsettled state of incoming messages, one to record the unsettled
-%% state of outgoing messages.  In general we want to bound these
-%% buffers; but if we bound them, and don't tell the other side, we
-%% may end up deadlocking the other party.
-%%
-%% Hence the flow frame contains a session window, expressed as the
-%% next-id and the window size for each of the buffers. The frame
-%% refers to the window of the sender of the frame, of course.
-%%
-%% The numbers work this way: for the outgoing window, the next-id
-%% counts the next transfer the session will send, and it will stop
-%% sending at next-id + window.  For the incoming window, the next-id
-%% counts the next transfer id expected, and it will not accept
-%% messages beyond next-id + window (in fact it will probably close
-%% the session, since sending outside the window is a transgression of
-%% the protocol).
-%%
-%% We may as well just pick a value for the incoming and outgoing
-%% windows; choosing based on what the client says may just stop
-%% things dead, if the value is zero for instance.
-%%
-%% [1] Abstract because there probably won't be a data structure with
-%% a size directly related to transfers; settlement is done with
-%% delivery-id, which may refer to one or more transfers.
-begin_(#'v1_0.begin'{next_outgoing_id = {uint, RemoteNextOut},
-                     incoming_window  = {uint, RemoteInWindow},
-                     outgoing_window  = {uint, RemoteOutWindow},
-                     handle_max       = HandleMax0},
-       Session = #session{next_outgoing_id = LocalNextOut,
-                          channel_num          = Channel}) ->
-    InWindow = ?MAX_SESSION_WINDOW_SIZE,
-    OutWindow = ?MAX_SESSION_WINDOW_SIZE,
-    HandleMax = case HandleMax0 of
-                    {uint, Max} -> Max;
-                    _ -> ?DEFAULT_MAX_HANDLE
-                end,
-    {ok, #'v1_0.begin'{remote_channel = {ushort, Channel},
-                       handle_max = {uint, HandleMax},
-                       next_outgoing_id = {uint, LocalNextOut},
-                       incoming_window = {uint, InWindow},
-                       outgoing_window = {uint, OutWindow}},
-     Session#session{
-       outgoing_window = OutWindow,
-       outgoing_window_max = OutWindow,
-       next_incoming_id = RemoteNextOut,
-       remote_incoming_window = RemoteInWindow,
-       remote_outgoing_window = RemoteOutWindow,
-       incoming_window  = InWindow,
-       incoming_window_max = InWindow},
-     OutWindow}.
-
-validate_attach(#'v1_0.attach'{target = #'v1_0.coordinator'{}}) ->
-    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                   "Transactions not supported", []);
-validate_attach(#'v1_0.attach'{unsettled = Unsettled,
-                               incomplete_unsettled = IncompleteSettled})
-  when Unsettled =/= undefined andalso Unsettled =/= {map, []} orelse
-       IncompleteSettled =:= true ->
-    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                   "Link recovery not supported", []);
-validate_attach(
-    #'v1_0.attach'{snd_settle_mode = SndSettleMode,
-                   rcv_settle_mode = ?V_1_0_RECEIVER_SETTLE_MODE_SECOND})
-  when SndSettleMode =/= ?V_1_0_SENDER_SETTLE_MODE_SETTLED ->
-    protocol_error(?V_1_0_AMQP_ERROR_NOT_IMPLEMENTED,
-                   "rcv-settle-mode second not supported", []);
-validate_attach(#'v1_0.attach'{}) ->
-    ok.
-
-maybe_init_publish_id(false, Session) ->
-    Session;
-maybe_init_publish_id(true, Session = #session{next_publish_id = Id}) ->
-    Session#session{next_publish_id = erlang:max(1, Id)}.
-
-record_delivery(DeliveryId, Settled,
-                Session = #session{next_publish_id = Id,
-                                   incoming_unsettled_map = Unsettled}) ->
-    Id1 = case Id of
-              0 -> 0;
-              _ -> Id + 1 % this ought to be a serial number in the broker, but isn't
-          end,
-    Unsettled1 = case Settled of
-                     true ->
-                         Unsettled;
-                     false ->
-                         gb_trees:insert(Id,
-                                         #incoming_delivery{
-                                           delivery_id = DeliveryId },
-                                         Unsettled)
-                 end,
-    Session#session{
-      next_publish_id = Id1,
-      incoming_unsettled_map = Unsettled1}.
-
-incr_incoming_id(Session = #session{ next_incoming_id = NextIn,
-                                     incoming_window = InWindow,
-                                     incoming_window_max = InWindowMax,
-                                     remote_outgoing_window = RemoteOut }) ->
-    NewOutWindow = RemoteOut - 1,
-    InWindow1 = InWindow - 1,
-    NewNextIn = serial_add(NextIn, 1),
-    %% If we've reached halfway, open the window
-    {Flows, NewInWindow} =
-        if InWindow1 =< (InWindowMax div 2) ->
-                {[#'v1_0.flow'{}], InWindowMax};
-               true ->
-                {[], InWindow1}
-        end,
-    {Flows, Session#session{ next_incoming_id = NewNextIn,
-                             incoming_window = NewInWindow,
-                             remote_outgoing_window = NewOutWindow}}.
-
-next_delivery_id(#session{next_delivery_id = Num}) -> Num.
-
-transfers_left(#session{remote_incoming_window = RemoteWindow,
-                        outgoing_window = LocalWindow}) ->
-    {LocalWindow, RemoteWindow}.
-
-record_outgoing(DeliveryTag, SendSettled, DefaultOutcome,
-                Session = #session{next_delivery_id = DeliveryId,
-                                   outgoing_unsettled_map = Unsettled}) ->
-    Unsettled1 = case SendSettled of
-                     true ->
-                         Unsettled;
-                     false ->
-                         gb_trees:insert(DeliveryId,
-                                         #outgoing_delivery{
-                                           delivery_tag     = DeliveryTag,
-                                           expected_outcome = DefaultOutcome },
-                                         Unsettled)
-                 end,
-    Session#session{outgoing_unsettled_map = Unsettled1,
-                    next_delivery_id = serial_add(DeliveryId, 1)}.
-
-record_transfers(NumTransfers,
-                 Session = #session{ remote_incoming_window = RemoteInWindow,
-                                     outgoing_window = OutWindow,
-                                     next_outgoing_id = NextOutId }) ->
-    Session#session{ remote_incoming_window = RemoteInWindow - NumTransfers,
-                     outgoing_window = OutWindow - NumTransfers,
-                     next_outgoing_id = serial_add(NextOutId, NumTransfers) }.
-
-%% Make sure we have "room" in our outgoing window by bumping the
-%% window if necessary. TODO this *could* be based on how much
-%% notional "room" there is in outgoing_unsettled.
-bump_outgoing_window(Session = #session{ outgoing_window_max = OutMax }) ->
-    {#'v1_0.flow'{}, Session#session{ outgoing_window = OutMax }}.
-
-%% We've been told that the fate of a delivery has been determined.
-%% Generally if the other side has not settled it, we will do so.  If
-%% the other side /has/ settled it, we don't need to reply -- it's
-%% already forgotten its state for the delivery anyway.
-settle(Disp = #'v1_0.disposition'{first   = First0,
-                                  last    = Last0,
-                                  state   = _Outcome,
-                                  settled = Settled},
-       Session = #session{outgoing_unsettled_map = Unsettled},
-       UpstreamAckFun) ->
-    {uint, First} = First0,
-    %% Last may be omitted, in which case it's the same as first
-    Last = case Last0 of
-               {uint, L} -> L;
-               undefined -> First
-           end,
-    %% The other party may be talking about something we've already
-    %% forgotten; this isn't a crime, we can just ignore it.
-    case gb_trees:is_empty(Unsettled) of
-        true ->
-            {none, Session};
-        false ->
-            {LWM, _} = gb_trees:smallest(Unsettled),
-            {HWM, _} = gb_trees:largest(Unsettled),
-            if Last < LWM ->
-                    {none, Session};
-               %% TODO this should probably be an error, rather than ignored.
-               First > HWM ->
-                    {none, Session};
-               true ->
-                    Unsettled1 =
-                        lists:foldl(
-                          fun (Delivery, Map) ->
-                                  case gb_trees:lookup(Delivery, Map) of
-                                      none ->
-                                          Map;
-                                      {value, Entry} ->
-                                          #outgoing_delivery{delivery_tag = DeliveryTag } = Entry,
-                                          ?DEBUG("Settling ~tp with ~tp", [Delivery, _Outcome]),
-                                          UpstreamAckFun(DeliveryTag),
-                                          gb_trees:delete(Delivery, Map)
-                                  end
-                          end,
-                          Unsettled, lists:seq(erlang:max(LWM, First),
-                                               erlang:min(HWM, Last))),
-                    {case Settled of
-                         true  -> none;
-                         false -> Disp#'v1_0.disposition'{ settled = true,
-                                                           role = ?SEND_ROLE }
-                     end,
-                     Session#session{outgoing_unsettled_map = Unsettled1}}
-            end
-    end.
-
-flow_fields(Frames, Session) when is_list(Frames) ->
-    [flow_fields(F, Session) || F <- Frames];
-
-flow_fields(Flow = #'v1_0.flow'{},
-             #session{next_outgoing_id = NextOut,
-                      next_incoming_id = NextIn,
-                      outgoing_window = OutWindow,
-                      incoming_window = InWindow}) ->
-    Flow#'v1_0.flow'{
-      next_outgoing_id = {uint, NextOut},
-      outgoing_window = {uint, OutWindow},
-      next_incoming_id = {uint, NextIn},
-      incoming_window = {uint, InWindow}};
-
-flow_fields(Frame, _Session) ->
-    Frame.
-
-channel(#session{channel_num = Channel}) -> Channel.
-
-%% We should already know the next outgoing transfer sequence number,
-%% because it's one more than the last transfer we saw; and, we don't
-%% need to know the next incoming transfer sequence number (although
-%% we might use it to detect congestion -- e.g., if it's lagging far
-%% behind our outgoing sequence number). We probably care about the
-%% outgoing window, since we want to keep it open by sending back
-%% settlements, but there's not much we can do to hurry things along.
-%%
-%% We do care about the incoming window, because we must not send
-%% beyond it. This may cause us problems, even in normal operation,
-%% since we want our unsettled transfers to be exactly those that are
-%% held as unacked by the backing channel; however, the far side may
-%% close the window while we still have messages pending transfer, and
-%% indeed, an individual message may take more than one 'slot'.
-%%
-%% Note that this isn't a race so far as AMQP 1.0 is concerned; it's
-%% only because AMQP 0-9-1 defines QoS in terms of the total number of
-%% unacked messages, whereas 1.0 has an explicit window.
-flow(#'v1_0.flow'{next_incoming_id = FlowNextIn0,
-                  incoming_window  = {uint, FlowInWindow},
-                  next_outgoing_id = {uint, FlowNextOut},
-                  outgoing_window  = {uint, FlowOutWindow}},
-     Session = #session{next_incoming_id     = LocalNextIn,
-                        next_outgoing_id     = LocalNextOut}) ->
-    %% The far side may not have our begin{} with our next-transfer-id
-    FlowNextIn = case FlowNextIn0 of
-                       {uint, Id} -> Id;
-                       undefined  -> LocalNextOut
-                   end,
-    case serial_compare(FlowNextOut, LocalNextIn) of
-        equal ->
-            case serial_compare(FlowNextIn, LocalNextOut) of
-                greater ->
-                    protocol_error(?V_1_0_SESSION_ERROR_WINDOW_VIOLATION,
-                                   "Remote incoming id (~tp) leads "
-                                   "local outgoing id (~tp)",
-                                   [FlowNextIn, LocalNextOut]);
-                equal ->
-                    Session#session{
-                      remote_outgoing_window = FlowOutWindow,
-                      remote_incoming_window = FlowInWindow};
-                less ->
-                    Session#session{
-                      remote_outgoing_window = FlowOutWindow,
-                      remote_incoming_window =
-                      serial_diff(serial_add(FlowNextIn, FlowInWindow),
-                                  LocalNextOut)}
-            end;
-        _ ->
-            case application:get_env(rabbitmq_amqp1_0, protocol_strict_mode) of
-                {ok, false} ->
-                    Session#session{next_incoming_id = FlowNextOut};
-                {ok, true} ->
-                    protocol_error(?V_1_0_SESSION_ERROR_WINDOW_VIOLATION,
-                                   "Remote outgoing id (~tp) not equal to "
-                                   "local incoming id (~tp)",
-                                   [FlowNextOut, LocalNextIn])
-            end
-    end.
-
-%% An acknowledgement from the queue, which we'll get if we are
-%% using confirms.
-ack(#'basic.ack'{delivery_tag = DTag, multiple = Multiple},
-    Session = #session{incoming_unsettled_map = Unsettled}) ->
-    {DeliveryIds, Unsettled1} =
-        case Multiple of
-            true  -> acknowledgement_range(DTag, Unsettled);
-            false -> case gb_trees:lookup(DTag, Unsettled) of
-                         {value, #incoming_delivery{ delivery_id = Id }} ->
-                             {[Id], gb_trees:delete(DTag, Unsettled)};
-                         none ->
-                             {[], Unsettled}
-                     end
-        end,
-    Disposition = case DeliveryIds of
-                      [] -> [];
-                      _  -> [acknowledgement(
-                               DeliveryIds,
-                               #'v1_0.disposition'{role = ?RECV_ROLE})]
-    end,
-    {Disposition,
-     Session#session{incoming_unsettled_map = Unsettled1}}.
-
-acknowledgement_range(DTag, Unsettled) ->
-    acknowledgement_range(DTag, Unsettled, []).
-
-acknowledgement_range(DTag, Unsettled, Acc) ->
-    case gb_trees:is_empty(Unsettled) of
-        true ->
-            {lists:reverse(Acc), Unsettled};
-        false ->
-            {DTag1, #incoming_delivery{ delivery_id = Id}} =
-                gb_trees:smallest(Unsettled),
-            case DTag1 =< DTag of
-                true ->
-                    {_K, _V, Unsettled1} = gb_trees:take_smallest(Unsettled),
-                    acknowledgement_range(DTag, Unsettled1,
-                                          [Id|Acc]);
-                false ->
-                    {lists:reverse(Acc), Unsettled}
-            end
-    end.
-
-acknowledgement(DeliveryIds, Disposition) ->
-    Disposition#'v1_0.disposition'{ first = {uint, hd(DeliveryIds)},
-                                    last = {uint, lists:last(DeliveryIds)},
-                                    settled = true,
-                                    state = #'v1_0.accepted'{} }.
-
-return(DTag, Session = #session{incoming_unsettled_map = Unsettled}) ->
-    {DeliveryId,
-     Unsettled1} = case gb_trees:lookup(DTag, Unsettled) of
-                       {value, #incoming_delivery{ delivery_id = Id }} ->
-                           {Id, gb_trees:delete(DTag, Unsettled)};
-                       none ->
-                           {undefined, Unsettled}
-                   end,
-    Disposition = case DeliveryId of
-                      undefined -> undefined;
-                      _  -> release(DeliveryId,
-                                    #'v1_0.disposition'{role = ?RECV_ROLE})
-    end,
-    {Disposition,
-     Session#session{incoming_unsettled_map = Unsettled1}}.
-
-release(DeliveryId, Disposition) ->
-    Disposition#'v1_0.disposition'{ first = {uint, DeliveryId},
-                                    last = {uint, DeliveryId},
-                                    settled = true,
-                                    state = #'v1_0.released'{} }.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_process.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_process.erl
deleted file mode 100644
index 92f22f659315..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_process.erl
+++ /dev/null
@@ -1,467 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_session_process).
-
--behaviour(gen_server2).
-
--export([init/1, terminate/2, code_change/3,
-         handle_call/3, handle_cast/2, handle_info/2]).
-
--export([start_link/1]).
--export([info/1]).
-
--record(state, {backing_connection, backing_channel, frame_max,
-                reader_pid, writer_pid, buffer, session}).
-
--record(pending, {delivery_tag, frames, link_handle }).
-
--include_lib("amqp_client/include/amqp_client.hrl").
--include("rabbit_amqp1_0.hrl").
-
--import(rabbit_amqp1_0_util, [protocol_error/3]).
--import(rabbit_amqp1_0_link_util, [ctag_to_handle/1]).
-
-start_link(Args) ->
-    gen_server2:start_link(?MODULE, Args, []).
-
-info(Pid) ->
-    gen_server2:call(Pid, info, infinity).
-%% ---------
-
-init({Channel, ReaderPid, WriterPid, #user{username = Username}, VHost,
-      FrameMax, AdapterInfo, _Collector}) ->
-    process_flag(trap_exit, true),
-    case amqp_connection:start(
-           #amqp_params_direct{username     = Username,
-                               virtual_host = VHost,
-                               adapter_info = AdapterInfo}) of
-        {ok, Conn}  ->
-            case amqp_connection:open_channel(Conn) of
-                {ok, Ch} ->
-                    monitor(process, Ch),
-                    {ok, #state{backing_connection = Conn,
-                                backing_channel    = Ch,
-                                reader_pid         = ReaderPid,
-                                writer_pid         = WriterPid,
-                                frame_max          = FrameMax,
-                                buffer             = queue:new(),
-                                session            = rabbit_amqp1_0_session:init(Channel)
-                               }};
-                {error, Reason} ->
-                    rabbit_log:warning("Closing session for connection ~tp:~n~tp",
-                                       [ReaderPid, Reason]),
-                    {stop, Reason}
-            end;
-        {error, Reason} ->
-            rabbit_log:warning("Closing session for connection ~tp:~n~tp",
-                               [ReaderPid, Reason]),
-            {stop, Reason}
-    end.
-
-terminate(_Reason, _State = #state{backing_connection = Conn}) ->
-    rabbit_misc:with_exit_handler(fun () -> ok end,
-                                  fun () -> amqp_connection:close(Conn) end).
-
-code_change(_OldVsn, State, _Extra) ->
-    {ok, State}.
-
-handle_call(info, _From, #state{reader_pid = ReaderPid,
-                                backing_connection = Conn} = State) ->
-    Info = [{reader, ReaderPid}, {connection, Conn}],
-    {reply, Info, State};
-handle_call(Msg, _From, State) ->
-    {reply, {error, not_understood, Msg}, State}.
-
-handle_info(#'basic.consume_ok'{}, State) ->
-    %% Handled above
-    {noreply, State};
-
-handle_info(#'basic.cancel_ok'{}, State) ->
-    %% just ignore this for now,
-    %% At some point we should send the detach here but then we'd need to track
-    %% consumer tags -> link handle somewhere
-    {noreply, State};
-handle_info({#'basic.deliver'{ consumer_tag = ConsumerTag,
-                               delivery_tag = DeliveryTag } = Deliver, Msg},
-            State = #state{frame_max       = FrameMax,
-                           buffer          = Buffer,
-                           session         = Session}) ->
-    Handle = ctag_to_handle(ConsumerTag),
-    case get({out, Handle}) of
-        undefined ->
-            %% TODO handle missing link -- why does the queue think it's there?
-            rabbit_log:warning("Delivery to non-existent consumer ~tp",
-                               [ConsumerTag]),
-            {noreply, State};
-        Link ->
-            {ok, Frames, Session1} =
-                rabbit_amqp1_0_outgoing_link:delivery(
-                  Deliver, Msg, FrameMax, Handle, Session, Link),
-            Pending = #pending{ delivery_tag = DeliveryTag,
-                                frames = Frames,
-                                link_handle = Handle },
-            Buffer1 = queue:in(Pending, Buffer),
-            {noreply, run_buffer(
-                        state(Session1, State#state{ buffer = Buffer1 }))}
-    end;
-
-%% A message from the queue saying that there are no more messages
-handle_info(#'basic.credit_drained'{consumer_tag = CTag} = CreditDrained,
-            State = #state{writer_pid = WriterPid,
-                           session = Session}) ->
-    Handle = ctag_to_handle(CTag),
-    Link = get({out, Handle}),
-    {Flow0, Link1} = rabbit_amqp1_0_outgoing_link:credit_drained(
-                      CreditDrained, Handle, Link),
-    Flow = rabbit_amqp1_0_session:flow_fields(Flow0, Session),
-    rabbit_amqp1_0_writer:send_command(WriterPid, Flow),
-    put({out, Handle}, Link1),
-    {noreply, State};
-
-handle_info(#'basic.ack'{} = Ack, State = #state{writer_pid = WriterPid,
-                                                 session    = Session}) ->
-    {Reply, Session1} = rabbit_amqp1_0_session:ack(Ack, Session),
-    [rabbit_amqp1_0_writer:send_command(WriterPid, F) ||
-        F <- rabbit_amqp1_0_session:flow_fields(Reply, Session)],
-    {noreply, state(Session1, State)};
-
-handle_info({#'basic.return'{}, {DTag, _Msg}}, State = #state{writer_pid = WriterPid,
-                                                              session    = Session}) ->
-    {Reply, Session1} = rabbit_amqp1_0_session:return(DTag, Session),
-    case Reply of
-        undefined ->
-            ok;
-        _ ->
-            rabbit_amqp1_0_writer:send_command(
-              WriterPid,
-              rabbit_amqp1_0_session:flow_fields(Reply, Session)
-             )
-    end,
-    {noreply, state(Session1, State)};
-
-handle_info({#'basic.return'{}, _Msg}, State = #state{session = Session}) ->
-    rabbit_log:warning("AMQP 1.0 message return without publishing sequence"),
-    {noreply, state(Session, State)};
-
-handle_info({bump_credit, Msg}, State) ->
-    credit_flow:handle_bump_msg(Msg),
-    {noreply, State};
-
-%% TODO these pretty much copied wholesale from rabbit_channel
-handle_info({'EXIT', WriterPid, Reason = {writer, send_failed, _Error}},
-            State = #state{writer_pid = WriterPid}) ->
-    State#state.reader_pid !
-        {channel_exit, rabbit_amqp1_0_session:channel(session(State)), Reason},
-    {stop, normal, State};
-handle_info({'EXIT', _Pid, Reason}, State) ->
-    {stop, Reason, State};
-handle_info({'DOWN', _MRef, process, Ch, Reason},
-            #state{reader_pid = ReaderPid,
-                   writer_pid = Sock,
-                   backing_channel = Ch} = State) ->
-    Error =
-    case Reason of
-        {shutdown, {server_initiated_close, Code, Msg}} ->
-            #'v1_0.error'{condition = rabbit_amqp1_0_channel:convert_code(Code),
-                          description = {utf8, Msg}};
-        _ ->
-            #'v1_0.error'{condition = ?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
-                          description = {utf8,
-                                         list_to_binary(
-                                           lists:flatten(
-                                             io_lib:format("~w", [Reason])))}}
-    end,
-    End = #'v1_0.end'{ error = Error },
-    rabbit_log:warning("Closing session for connection ~tp:~n~tp",
-                       [ReaderPid, Reason]),
-    ok = rabbit_amqp1_0_writer:send_command_sync(Sock, End),
-    {stop, normal, State};
-handle_info({'DOWN', _MRef, process, _QPid, _Reason}, State) ->
-    %% TODO do we care any more since we're using direct client?
-    {noreply, State}. % TODO rabbit_channel uses queue_blocked?
-
-handle_cast({frame, Frame, FlowPid},
-            State = #state{ reader_pid = ReaderPid,
-                            writer_pid = Sock }) ->
-    credit_flow:ack(FlowPid),
-    try handle_control(Frame, State) of
-        {reply, Replies, NewState} when is_list(Replies) ->
-            lists:foreach(fun (Reply) ->
-                                  rabbit_amqp1_0_writer:send_command(Sock, Reply)
-                          end, Replies),
-            noreply(NewState);
-        {reply, Reply, NewState} ->
-            rabbit_amqp1_0_writer:send_command(Sock, Reply),
-            noreply(NewState);
-        {noreply, NewState} ->
-            noreply(NewState);
-        stop ->
-            {stop, normal, State}
-    catch exit:Reason = #'v1_0.error'{} ->
-            %% TODO shut down nicely like rabbit_channel
-            End = #'v1_0.end'{ error = Reason },
-            rabbit_log:warning("Closing session for connection ~tp:~n~tp",
-                               [ReaderPid, Reason]),
-            ok = rabbit_amqp1_0_writer:send_command_sync(Sock, End),
-            {stop, normal, State};
-          exit:normal ->
-            {stop, normal, State};
-          _:Reason:Stacktrace ->
-            {stop, {Reason, Stacktrace}, State}
-    end.
-
-%% TODO rabbit_channel returns {noreply, State, hibernate}, but that
-%% appears to break things here (it stops the session responding to
-%% frames).
-noreply(State) ->
-    {noreply, State}.
-
-%% ------
-
-handle_control(#'v1_0.begin'{} = Begin,
-               State = #state{backing_channel = Ch,
-                              session         = Session}) ->
-    {ok, Reply, Session1, Prefetch} =
-        rabbit_amqp1_0_session:begin_(Begin, Session),
-    %% Attempt to limit the number of "at risk" messages we can have.
-    rabbit_amqp1_0_channel:cast(Ch, #'basic.qos'{prefetch_count = Prefetch}),
-    reply(Reply, state(Session1, State));
-
-handle_control(#'v1_0.attach'{handle = Handle,
-                              role   = ?SEND_ROLE} = Attach,
-               State = #state{backing_channel    = BCh,
-                              backing_connection = Conn}) ->
-    ok = rabbit_amqp1_0_session:validate_attach(Attach),
-    {ok, Reply, Link, Confirm} =
-        with_disposable_channel(
-          Conn, fun (DCh) ->
-                        rabbit_amqp1_0_incoming_link:attach(Attach, BCh, DCh)
-                end),
-    put({in, Handle}, Link),
-    reply(Reply, state(rabbit_amqp1_0_session:maybe_init_publish_id(
-                         Confirm, session(State)), State));
-
-handle_control(#'v1_0.attach'{handle = Handle,
-                              role   = ?RECV_ROLE} = Attach,
-               State = #state{backing_channel    = BCh,
-                              backing_connection = Conn}) ->
-    ok = rabbit_amqp1_0_session:validate_attach(Attach),
-    {ok, Reply, Link} =
-        with_disposable_channel(
-          Conn, fun (DCh) ->
-                        rabbit_amqp1_0_outgoing_link:attach(Attach, BCh, DCh)
-                end),
-    put({out, Handle}, Link),
-    reply(Reply, State);
-
-handle_control({Txfr = #'v1_0.transfer'{handle = Handle},
-                MsgPart},
-               State = #state{backing_channel = BCh,
-                              session         = Session}) ->
-    case get({in, Handle}) of
-        undefined ->
-            protocol_error(?V_1_0_AMQP_ERROR_ILLEGAL_STATE,
-                           "Unknown link handle ~tp", [Handle]);
-        Link ->
-            {Flows, Session1} = rabbit_amqp1_0_session:incr_incoming_id(Session),
-            case rabbit_amqp1_0_incoming_link:transfer(
-                   Txfr, MsgPart, Link, BCh) of
-                {message, Reply, Link1, DeliveryId, Settled} ->
-                    put({in, Handle}, Link1),
-                    Session2 = rabbit_amqp1_0_session:record_delivery(
-                                 DeliveryId, Settled, Session1),
-                    reply(Reply ++ Flows, state(Session2, State));
-                {ok, Link1} ->
-                    put({in, Handle}, Link1),
-                    reply(Flows, state(Session1, State))
-            end
-    end;
-
-%% Disposition: multiple deliveries may be settled at a time.
-%% TODO: should we send a flow after this, to indicate the state
-%% of the session window?
-handle_control(#'v1_0.disposition'{state = Outcome,
-                                   role = ?RECV_ROLE} = Disp,
-               State = #state{backing_channel = Ch}) ->
-    AckFun =
-        fun (DeliveryTag) ->
-                ok = rabbit_amqp1_0_channel:call(
-                       Ch, case Outcome of
-                               #'v1_0.accepted'{} ->
-                                   #'basic.ack'{delivery_tag = DeliveryTag,
-                                                multiple     = false};
-                               #'v1_0.modified'{delivery_failed = true,
-                                                undeliverable_here = UndelHere} ->
-                                   %% NB: this is not quite correct.
-                                   %% `undeliverable_here' refers to the link
-                                   %% not the message in general but we cannot
-                                   %% filter messages from being assigned to
-                                   %% individual consumers
-                                   %% so will have to reject it without requeue
-                                   %% in this case.
-                                   Requeue = case UndelHere of
-                                                 true ->
-                                                     false;
-                                                 _ ->
-                                                     true
-                                             end,
-                                   #'basic.reject'{delivery_tag = DeliveryTag,
-                                                   requeue = Requeue};
-                               #'v1_0.modified'{} ->
-                                   %% if delivery_failed is not true, treat we
-                                   %% can't increment its' delivery_count so
-                                   %% will have to reject without requeue
-                                   #'basic.reject'{delivery_tag = DeliveryTag,
-                                                   requeue = false};
-                               #'v1_0.rejected'{} ->
-                                   #'basic.reject'{delivery_tag = DeliveryTag,
-                                                   requeue      = false};
-                               #'v1_0.released'{} ->
-                                   #'basic.reject'{delivery_tag = DeliveryTag,
-                                                   requeue      = true};
-                               _ ->
-                                   protocol_error(
-                                     ?V_1_0_AMQP_ERROR_INVALID_FIELD,
-                                     "Unrecognised state: ~tp~n"
-                                     "Disposition was: ~tp", [Outcome, Disp])
-                           end)
-        end,
-    case rabbit_amqp1_0_session:settle(Disp, session(State), AckFun) of
-        {none,  Session1} -> {noreply,        state(Session1, State)};
-        {Reply, Session1} -> {reply,   Reply, state(Session1, State)}
-    end;
-
-handle_control(#'v1_0.detach'{handle = Handle} = Detach,
-               #state{backing_channel = BCh} = State) ->
-    %% TODO keep the state around depending on the lifetime
-    %% TODO outgoing links?
-    case get({out, Handle}) of
-        undefined ->
-            ok;
-        Link ->
-            erase({out, Handle}),
-            ok = rabbit_amqp1_0_outgoing_link:detach(Detach, BCh, Link)
-    end,
-    erase({in, Handle}),
-    {reply, #'v1_0.detach'{handle = Handle}, State};
-
-handle_control(#'v1_0.end'{}, _State = #state{ writer_pid = Sock }) ->
-    ok = rabbit_amqp1_0_writer:send_command(Sock, #'v1_0.end'{}),
-    stop;
-
-%% Flow control.  These frames come with two pieces of information:
-%% the session window, and optionally, credit for a particular link.
-%% We'll deal with each of them separately.
-handle_control(Flow = #'v1_0.flow'{},
-               State = #state{backing_channel = BCh,
-                              session         = Session}) ->
-    State1 = state(rabbit_amqp1_0_session:flow(Flow, Session), State),
-    State2 = run_buffer(State1),
-    case Flow#'v1_0.flow'.handle of
-        undefined ->
-            {noreply, State2};
-        Handle ->
-            case get({in, Handle}) of
-                undefined ->
-                    case get({out, Handle}) of
-                        undefined ->
-                            rabbit_log:warning("Flow for unknown link handle ~tp", [Flow]),
-                            protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
-                                           "Unattached handle: ~tp", [Handle]);
-                        Out ->
-                            {ok, Reply} = rabbit_amqp1_0_outgoing_link:flow(
-                                            Out, Flow, BCh),
-                            reply(Reply, State2)
-                    end;
-                _In ->
-                    %% We're being told about available messages at
-                    %% the sender.  Yawn.
-                    %% TODO at least check transfer-count?
-                    {noreply, State2}
-            end
-    end;
-
-handle_control(Frame, _State) ->
-    protocol_error(?V_1_0_AMQP_ERROR_INTERNAL_ERROR,
-                   "Unexpected frame ~tp",
-                   [amqp10_framing:pprint(Frame)]).
-
-run_buffer(State = #state{ writer_pid = WriterPid,
-                           session = Session,
-                           backing_channel = BCh,
-                           buffer = Buffer }) ->
-    {Session1, Buffer1} =
-        run_buffer1(WriterPid, BCh, Session, Buffer),
-    State#state{ buffer = Buffer1, session = Session1 }.
-
-run_buffer1(WriterPid, BCh, Session, Buffer) ->
-    case rabbit_amqp1_0_session:transfers_left(Session) of
-        {LocalSpace, RemoteSpace} when RemoteSpace > 0 andalso LocalSpace > 0 ->
-            Space = erlang:min(LocalSpace, RemoteSpace),
-            case queue:out(Buffer) of
-                {empty, Buffer} ->
-                    {Session, Buffer};
-                {{value, #pending{ delivery_tag = DeliveryTag,
-                                   frames = Frames,
-                                   link_handle = Handle } = Pending},
-                 BufferTail} ->
-                    Link = get({out, Handle}),
-                    case send_frames(WriterPid, Frames, Space) of
-                        {all, SpaceLeft} ->
-                            NewLink =
-                                rabbit_amqp1_0_outgoing_link:transferred(
-                                  DeliveryTag, BCh, Link),
-                            put({out, Handle}, NewLink),
-                            Session1 = rabbit_amqp1_0_session:record_transfers(
-                                         Space - SpaceLeft, Session),
-                            run_buffer1(WriterPid, BCh, Session1, BufferTail);
-                        {some, Rest} ->
-                            Session1 = rabbit_amqp1_0_session:record_transfers(
-                                         Space, Session),
-                            Buffer1 = queue:in_r(Pending#pending{ frames = Rest },
-                                                 BufferTail),
-                            run_buffer1(WriterPid, BCh, Session1, Buffer1)
-                    end
-            end;
-         {_, RemoteSpace} when RemoteSpace > 0 ->
-            case rabbit_amqp1_0_session:bump_outgoing_window(Session) of
-                {Flow = #'v1_0.flow'{}, Session1} ->
-                    rabbit_amqp1_0_writer:send_command(
-                      WriterPid,
-                      rabbit_amqp1_0_session:flow_fields(Flow, Session1)),
-                    run_buffer1(WriterPid, BCh, Session1, Buffer)
-            end;
-        _ ->
-            {Session, Buffer}
-    end.
-
-send_frames(_WriterPid, [], Left) ->
-    {all, Left};
-send_frames(_WriterPid, Rest, 0) ->
-    {some, Rest};
-send_frames(WriterPid, [[T, C] | Rest], Left) ->
-    rabbit_amqp1_0_writer:send_command(WriterPid, T, C),
-    send_frames(WriterPid, Rest, Left - 1).
-
-%% ------
-
-reply([], State) ->
-    {noreply, State};
-reply(Reply, State) ->
-    {reply, rabbit_amqp1_0_session:flow_fields(Reply, session(State)), State}.
-
-session(#state{session = Session}) -> Session.
-state(Session, State) -> State#state{session = Session}.
-
-with_disposable_channel(Conn, Fun) ->
-    {ok, Ch} = amqp_connection:open_channel(Conn),
-    try
-        Fun(Ch)
-    after
-        catch amqp_channel:close(Ch)
-    end.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup.erl
deleted file mode 100644
index a5422874de59..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup.erl
+++ /dev/null
@@ -1,104 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_session_sup).
-
--behaviour(supervisor).
-
--export([start_link/1]).
-
--export([init/1]).
-
--include_lib("amqp_client/include/amqp_client.hrl").
-
-%%----------------------------------------------------------------------------
-
--export_type([start_link_args/0]).
-
--type start_link_args() ::
-        {'amqp10_framing', rabbit_net:socket(),
-         rabbit_channel:channel_number(), non_neg_integer() | 'unlimited', pid(),
-         #user{}, rabbit_types:vhost(), pid(),
-         {'rabbit_proxy_socket', rabbit_net:socket(), term()} | 'undefined'}.
-
--spec start_link(start_link_args()) -> {'ok', pid(), pid()} | {'error', term()}.
-
-%%----------------------------------------------------------------------------
-start_link({amqp10_framing, Sock, Channel, FrameMax, ReaderPid,
-            User, VHost, Collector, ProxySocket}) ->
-    {ok, SupPid} = supervisor:start_link(?MODULE, []),
-    {ok, WriterPid} =
-        supervisor:start_child(
-            SupPid,
-            #{
-                id => writer,
-                start =>
-                    {rabbit_amqp1_0_writer, start_link, [
-                        Sock,
-                        Channel,
-                        FrameMax,
-                        amqp10_framing,
-                        ReaderPid
-                    ]},
-                restart => transient,
-                significant => true,
-                shutdown => ?WORKER_WAIT,
-                type => worker,
-                modules => [rabbit_amqp1_0_writer]
-            }
-        ),
-    SocketForAdapterInfo = case ProxySocket of
-        undefined -> Sock;
-        _         -> ProxySocket
-    end,
-    case supervisor:start_child(
-           SupPid,
-           #{
-               id => channel,
-               start =>
-                   {rabbit_amqp1_0_session_process, start_link, [
-                       {Channel, ReaderPid, WriterPid, User, VHost, FrameMax,
-                           adapter_info(User, SocketForAdapterInfo, Channel), Collector}
-                   ]},
-               restart => transient,
-               significant => true,
-               shutdown => ?WORKER_WAIT,
-               type => worker,
-               modules => [rabbit_amqp1_0_session_process]
-           }
-        ) of
-        {ok, ChannelPid} ->
-            {ok, SupPid, ChannelPid};
-        {error, Reason} ->
-            {error, Reason}
-    end.
-
-%%----------------------------------------------------------------------------
-
-init([]) ->
-    SupFlags = #{strategy => one_for_all,
-                intensity => 0,
-                period => 1,
-                auto_shutdown => any_significant},
-    {ok, {SupFlags, []}}.
-
-
-%% For each AMQP 1.0 session opened, an internal direct AMQP 0-9-1 connection is opened too.
-%% This direct connection will authenticate the user again. Again because at this point
-%% the SASL handshake has already taken place and this user has already been authenticated.
-%% However, we do not have the credentials the user presented. For that reason, the
-%% #amqp_adapter_info.additional_info carries an extra property called authz_backends
-%% which is initialized from the #user.authz_backends attribute. In other words, we
-%% propagate the outcome from the first authentication attempt to the subsequent attempts.
-
-%% See rabbit_direct.erl to see how `authz_bakends` is propagated from
-% amqp_adapter_info.additional_info to the rabbit_access_control module
-
-adapter_info(User, Sock, UniqueId) ->
-    AdapterInfo = amqp_connection:socket_adapter_info(Sock, {'AMQP', "1.0"}, UniqueId),
-    AdapterInfo#amqp_adapter_info{additional_info =
-        AdapterInfo#amqp_adapter_info.additional_info ++ [{authz_backends, User#user.authz_backends}]}.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup_sup.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup_sup.erl
deleted file mode 100644
index 71b24125629c..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_session_sup_sup.erl
+++ /dev/null
@@ -1,38 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_session_sup_sup).
-
--behaviour(supervisor).
-
--export([start_link/0, start_session/2]).
-
--export([init/1]).
-
-%% It would be much nicer if rabbit_channel_sup_sup was parameterised
-%% on the module.
-
-%%----------------------------------------------------------------------------
-
--spec start_link() -> rabbit_types:ok_pid_or_error().
--spec start_session(pid(), rabbit_amqp1_0_session_sup:start_link_args()) ->
-          supervisor:startchild_ret().
-
-%%----------------------------------------------------------------------------
-
-start_link() ->
-    supervisor:start_link(?MODULE, []).
-
-start_session(Pid, Args) ->
-    supervisor:start_child(Pid, [Args]).
-
-%%----------------------------------------------------------------------------
-
-init([]) ->
-    {ok, {{simple_one_for_one, 0, 1},
-          [{session_sup, {rabbit_amqp1_0_session_sup, start_link, []},
-            temporary, infinity, supervisor, [rabbit_amqp1_0_session_sup]}]}}.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_util.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_util.erl
deleted file mode 100644
index eb0278615ef7..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_util.erl
+++ /dev/null
@@ -1,73 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_util).
-
--include("rabbit_amqp1_0.hrl").
-
--export([protocol_error/3]).
--export([serial_add/2, serial_compare/2, serial_diff/2]).
-
--export_type([serial_number/0]).
--type serial_number() :: non_neg_integer().
--type serial_compare_result() :: 'equal' | 'less' | 'greater'.
-
--spec serial_add(serial_number(), non_neg_integer()) ->
-             serial_number().
--spec serial_compare(serial_number(), serial_number()) ->
-             serial_compare_result().
--spec serial_diff(serial_number(), serial_number()) ->
-             integer().
-
--spec protocol_error(term(), io:format(), [term()]) -> no_return().
-protocol_error(Condition, Msg, Args) ->
-    exit(#'v1_0.error'{
-        condition   = Condition,
-        description = {utf8, list_to_binary(
-                               lists:flatten(io_lib:format(Msg, Args)))}
-       }).
-
-%% Serial arithmetic for unsigned ints.
-%% http://www.faqs.org/rfcs/rfc1982.html
-%% SERIAL_BITS = 32
-
-%% 2 ^ SERIAL_BITS
--define(SERIAL_MAX, 16#100000000).
-%% 2 ^ (SERIAL_BITS - 1) - 1
--define(SERIAL_MAX_ADDEND, 16#7fffffff).
-
-serial_add(S, N) when N =< ?SERIAL_MAX_ADDEND ->
-    (S + N) rem ?SERIAL_MAX;
-serial_add(S, N) ->
-    exit({out_of_bound_serial_addition, S, N}).
-
-serial_compare(A, B) ->
-    if A =:= B ->
-            equal;
-       (A < B andalso B - A < ?SERIAL_MAX_ADDEND) orelse
-       (A > B andalso A - B > ?SERIAL_MAX_ADDEND) ->
-            less;
-       (A < B andalso B - A > ?SERIAL_MAX_ADDEND) orelse
-       (A > B andalso B - A < ?SERIAL_MAX_ADDEND) ->
-            greater;
-       true -> exit({indeterminate_serial_comparison, A, B})
-    end.
-
--define(SERIAL_DIFF_BOUND, 16#80000000).
-
-serial_diff(A, B) ->
-    Diff = A - B,
-    if Diff > (?SERIAL_DIFF_BOUND) ->
-            %% B is actually greater than A
-            - (?SERIAL_MAX - Diff);
-       Diff < - (?SERIAL_DIFF_BOUND) ->
-            ?SERIAL_MAX + Diff;
-       Diff < ?SERIAL_DIFF_BOUND andalso Diff > -?SERIAL_DIFF_BOUND ->
-            Diff;
-       true ->
-            exit({indeterminate_serial_diff, A, B})
-    end.
diff --git a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_writer.erl b/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_writer.erl
deleted file mode 100644
index 8873dd31e8f8..000000000000
--- a/deps/rabbitmq_amqp1_0/src/rabbit_amqp1_0_writer.erl
+++ /dev/null
@@ -1,258 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(rabbit_amqp1_0_writer).
--include_lib("rabbit_common/include/rabbit.hrl").
--include_lib("rabbit_common/include/rabbit_framing.hrl").
--include("rabbit_amqp1_0.hrl").
-
--export([start/5, start_link/5, start/6, start_link/6]).
--export([send_command/2, send_command/3,
-         send_command_sync/2, send_command_sync/3,
-         send_command_and_notify/4, send_command_and_notify/5]).
--export([internal_send_command/4]).
-
-%% internal
--export([mainloop/1, mainloop1/1]).
-
--record(wstate, {sock, channel, frame_max, protocol, reader,
-                 stats_timer, pending}).
-
--define(HIBERNATE_AFTER, 6_000).
--define(AMQP_SASL_FRAME_TYPE, 1).
-
-%%---------------------------------------------------------------------------
-
--spec start
-        (rabbit_net:socket(), rabbit_channel:channel_number(),
-         non_neg_integer(), rabbit_types:protocol(), pid())
-        -> rabbit_types:ok(pid()).
--spec start_link
-        (rabbit_net:socket(), rabbit_channel:channel_number(),
-         non_neg_integer(), rabbit_types:protocol(), pid())
-        -> rabbit_types:ok(pid()).
--spec start
-        (rabbit_net:socket(), rabbit_channel:channel_number(),
-         non_neg_integer(), rabbit_types:protocol(), pid(), boolean())
-        -> rabbit_types:ok(pid()).
--spec start_link
-        (rabbit_net:socket(), rabbit_channel:channel_number(),
-         non_neg_integer(), rabbit_types:protocol(), pid(), boolean())
-        -> rabbit_types:ok(pid()).
--spec send_command
-        (pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
--spec send_command
-        (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content())
-        -> 'ok'.
--spec send_command_sync
-        (pid(), rabbit_framing:amqp_method_record()) -> 'ok'.
--spec send_command_sync
-        (pid(), rabbit_framing:amqp_method_record(), rabbit_types:content())
-        -> 'ok'.
--spec send_command_and_notify
-        (pid(), pid(), pid(), rabbit_framing:amqp_method_record())
-        -> 'ok'.
--spec send_command_and_notify
-        (pid(), pid(), pid(), rabbit_framing:amqp_method_record(),
-         rabbit_types:content())
-        -> 'ok'.
--spec internal_send_command
-        (rabbit_net:socket(), rabbit_channel:channel_number(),
-         rabbit_framing:amqp_method_record(), 'amqp10_framing' | 'rabbit_amqp1_0_sasl')
-        -> 'ok'.
-
-%%---------------------------------------------------------------------------
-
-start(Sock, Channel, FrameMax, Protocol, ReaderPid) ->
-    start(Sock, Channel, FrameMax, Protocol, ReaderPid, false).
-
-start_link(Sock, Channel, FrameMax, Protocol, ReaderPid) ->
-    start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, false).
-
-start(Sock, Channel, FrameMax, Protocol, ReaderPid, ReaderWantsStats) ->
-    State = initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid,
-                          ReaderWantsStats),
-    {ok, proc_lib:spawn(?MODULE, mainloop, [State])}.
-
-start_link(Sock, Channel, FrameMax, Protocol, ReaderPid, ReaderWantsStats) ->
-    State = initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid,
-                          ReaderWantsStats),
-    {ok, proc_lib:spawn_link(?MODULE, mainloop, [State])}.
-
-initial_state(Sock, Channel, FrameMax, Protocol, ReaderPid, ReaderWantsStats) ->
-    (case ReaderWantsStats of
-         true  -> fun rabbit_event:init_stats_timer/2;
-         false -> fun rabbit_event:init_disabled_stats_timer/2
-     end)(#wstate{sock      = Sock,
-                  channel   = Channel,
-                  frame_max = FrameMax,
-                  protocol  = Protocol,
-                  reader    = ReaderPid,
-                  pending   = []},
-          #wstate.stats_timer).
-
-mainloop(State) ->
-    try
-        mainloop1(State)
-    catch
-        exit:Error -> #wstate{reader = ReaderPid, channel = Channel} = State,
-                      ReaderPid ! {channel_exit, Channel, Error}
-    end,
-    done.
-
-mainloop1(State = #wstate{pending = []}) ->
-    receive
-        Message -> ?MODULE:mainloop1(handle_message(Message, State))
-    after ?HIBERNATE_AFTER ->
-            erlang:hibernate(?MODULE, mainloop, [State])
-    end;
-mainloop1(State) ->
-    receive
-        Message -> ?MODULE:mainloop1(handle_message(Message, State))
-    after 0 ->
-            ?MODULE:mainloop1(flush(State))
-    end.
-
-handle_message({send_command, MethodRecord}, State) ->
-    internal_send_command_async(MethodRecord, State);
-handle_message({send_command, MethodRecord, Content}, State) ->
-    internal_send_command_async(MethodRecord, Content, State);
-handle_message({'$gen_call', From, {send_command_sync, MethodRecord}}, State) ->
-    State1 = flush(internal_send_command_async(MethodRecord, State)),
-    gen_server:reply(From, ok),
-    State1;
-handle_message({'$gen_call', From, {send_command_sync, MethodRecord, Content}},
-               State) ->
-    State1 = flush(internal_send_command_async(MethodRecord, Content, State)),
-    gen_server:reply(From, ok),
-    State1;
-handle_message({send_command_and_notify, QPid, ChPid, MethodRecord}, State) ->
-    State1 = internal_send_command_async(MethodRecord, State),
-    rabbit_amqqueue:notify_sent(QPid, ChPid),
-    State1;
-handle_message({send_command_and_notify, QPid, ChPid, MethodRecord, Content},
-               State) ->
-    State1 = internal_send_command_async(MethodRecord, Content, State),
-    rabbit_amqqueue:notify_sent(QPid, ChPid),
-    State1;
-handle_message({'DOWN', _MRef, process, QPid, _Reason}, State) ->
-    rabbit_amqqueue:notify_sent_queue_down(QPid),
-    State;
-handle_message(emit_stats, State = #wstate{reader = ReaderPid}) ->
-    ReaderPid ! ensure_stats,
-    rabbit_event:reset_stats_timer(State, #wstate.stats_timer);
-handle_message(Message, _State) ->
-    exit({writer, message_not_understood, Message}).
-
-%%---------------------------------------------------------------------------
-
-send_command(W, MethodRecord) ->
-    W ! {send_command, MethodRecord},
-    ok.
-
-send_command(W, MethodRecord, Content) ->
-    W ! {send_command, MethodRecord, Content},
-    ok.
-
-send_command_sync(W, MethodRecord) ->
-    call(W, {send_command_sync, MethodRecord}).
-
-send_command_sync(W, MethodRecord, Content) ->
-    call(W, {send_command_sync, MethodRecord, Content}).
-
-send_command_and_notify(W, Q, ChPid, MethodRecord) ->
-    W ! {send_command_and_notify, Q, ChPid, MethodRecord},
-    ok.
-
-send_command_and_notify(W, Q, ChPid, MethodRecord, Content) ->
-    W ! {send_command_and_notify, Q, ChPid, MethodRecord, Content},
-    ok.
-
-%%---------------------------------------------------------------------------
-
-call(Pid, Msg) ->
-    {ok, Res} = gen:call(Pid, '$gen_call', Msg, infinity),
-    Res.
-
-%%---------------------------------------------------------------------------
-
-%% Begin 1-0
-
-assemble_frame(Channel, Performative, amqp10_framing) ->
-    ?DEBUG("Channel ~tp <-~n~tp",
-           [Channel, amqp10_framing:pprint(Performative)]),
-    PerfBin = amqp10_framing:encode_bin(Performative),
-    amqp10_binary_generator:build_frame(Channel, PerfBin);
-
-assemble_frame(Channel, Performative, rabbit_amqp1_0_sasl) ->
-    ?DEBUG("Channel ~tp <-~n~tp",
-           [Channel, amqp10_framing:pprint(Performative)]),
-    PerfBin = amqp10_framing:encode_bin(Performative),
-    amqp10_binary_generator:build_frame(Channel,
-                                                ?AMQP_SASL_FRAME_TYPE, PerfBin).
-
-%% Note: a transfer record can be followed by a number of other
-%% records to make a complete frame but unlike 0-9-1 we may have many
-%% content records. However, that's already been handled for us, we're
-%% just sending a chunk, so from this perspective it's just a binary.
-
-assemble_frames(Channel, Performative, Content, _FrameMax,
-                amqp10_framing) ->
-    ?DEBUG("Channel ~tp <-~n~tp~n  followed by ~tp bytes of content",
-           [Channel, amqp10_framing:pprint(Performative),
-            iolist_size(Content)]),
-    PerfBin = amqp10_framing:encode_bin(Performative),
-    amqp10_binary_generator:build_frame(Channel, [PerfBin, Content]).
-
-%% End 1-0
-
-tcp_send(Sock, Data) ->
-    rabbit_misc:throw_on_error(inet_error,
-                               fun () -> rabbit_net:send(Sock, Data) end).
-
-internal_send_command(Sock, Channel, MethodRecord, Protocol) ->
-    ok = tcp_send(Sock, assemble_frame(Channel, MethodRecord, Protocol)).
-
-internal_send_command_async(MethodRecord,
-                            State = #wstate{channel   = Channel,
-                                            protocol  = Protocol,
-                                            pending   = Pending}) ->
-    Frame = assemble_frame(Channel, MethodRecord, Protocol),
-    maybe_flush(State#wstate{pending = [Frame | Pending]}).
-
-internal_send_command_async(MethodRecord, Content,
-                            State = #wstate{channel   = Channel,
-                                            frame_max = FrameMax,
-                                            protocol  = Protocol,
-                                            pending   = Pending}) ->
-    Frames = assemble_frames(Channel, MethodRecord, Content, FrameMax,
-                             Protocol),
-    maybe_flush(State#wstate{pending = [Frames | Pending]}).
-
-%% This magic number is the tcp-over-ethernet MSS (1460) minus the
-%% minimum size of a AMQP basic.deliver method frame (24) plus basic
-%% content header (22). The idea is that we want to flush just before
-%% exceeding the MSS.
--define(FLUSH_THRESHOLD, 1414).
-
-maybe_flush(State = #wstate{pending = Pending}) ->
-    case iolist_size(Pending) >= ?FLUSH_THRESHOLD of
-        true  -> flush(State);
-        false -> State
-    end.
-
-flush(State = #wstate{pending = []}) ->
-    State;
-flush(State0 = #wstate{sock = Sock, pending = Pending}) ->
-    case rabbit_net:send(Sock, lists:reverse(Pending)) of
-        ok ->
-            ok;
-        {error, Reason} ->
-            exit({writer, send_failed, Reason})
-    end,
-    State = State0#wstate{pending = []},
-    rabbit_event:ensure_stats_timer(State, #wstate.stats_timer, emit_stats).
diff --git a/deps/rabbitmq_amqp1_0/src/rabbitmq_amqp1_0_noop.erl b/deps/rabbitmq_amqp1_0/src/rabbitmq_amqp1_0_noop.erl
new file mode 100644
index 000000000000..3007ead09cb9
--- /dev/null
+++ b/deps/rabbitmq_amqp1_0/src/rabbitmq_amqp1_0_noop.erl
@@ -0,0 +1 @@
+-module(rabbitmq_amqp1_0_noop).
diff --git a/deps/rabbitmq_amqp1_0/test/amqp10_client_SUITE.erl b/deps/rabbitmq_amqp1_0/test/amqp10_client_SUITE.erl
deleted file mode 100644
index beeabe548e5b..000000000000
--- a/deps/rabbitmq_amqp1_0/test/amqp10_client_SUITE.erl
+++ /dev/null
@@ -1,624 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(amqp10_client_SUITE).
-
--include_lib("common_test/include/ct.hrl").
--include_lib("eunit/include/eunit.hrl").
--include_lib("amqp_client/include/amqp_client.hrl").
-
--compile(nowarn_export_all).
--compile(export_all).
-
-all() ->
-    [
-      {group, tests},
-      {group, metrics}
-    ].
-
-groups() ->
-    [
-     {tests, [], [
-                  reliable_send_receive_with_outcomes,
-                  publishing_to_non_existing_queue_should_settle_with_released,
-                  open_link_to_non_existing_destination_should_end_session,
-                  roundtrip_classic_queue_with_drain,
-                  roundtrip_quorum_queue_with_drain,
-                  roundtrip_stream_queue_with_drain,
-                  amqp_stream_amqpl,
-                  message_headers_conversion,
-                  resource_alarm
-                 ]},
-     {metrics, [], [
-                    auth_attempt_metrics
-                   ]}
-    ].
-
-%% -------------------------------------------------------------------
-%% Testsuite setup/teardown.
-%% -------------------------------------------------------------------
-
-init_per_suite(Config) ->
-    application:ensure_all_started(amqp10_client),
-    rabbit_ct_helpers:log_environment(),
-    Config.
-
-end_per_suite(Config) ->
-    Config.
-
-init_per_group(Group, Config) ->
-    Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
-    Config1 = rabbit_ct_helpers:set_config(
-                Config, [
-                         {rmq_nodename_suffix, Suffix},
-                         {amqp10_client_library, Group}
-                        ]),
-    rabbit_ct_helpers:run_setup_steps(
-      Config1,
-      rabbit_ct_broker_helpers:setup_steps() ++
-      rabbit_ct_client_helpers:setup_steps()).
-
-end_per_group(_, Config) ->
-    rabbit_ct_helpers:run_teardown_steps(Config,
-      rabbit_ct_client_helpers:teardown_steps() ++
-      rabbit_ct_broker_helpers:teardown_steps()).
-
-init_per_testcase(Testcase, Config) ->
-    rabbit_ct_helpers:testcase_started(Config, Testcase).
-
-end_per_testcase(Testcase, Config) ->
-    rabbit_ct_helpers:testcase_finished(Config, Testcase).
-
-%%% TESTS
-%%%
-
-reliable_send_receive_with_outcomes(Config) ->
-    Outcomes = [accepted,
-                modified,
-                {modified, true, false, #{<<"fruit">> => <<"banana">>}},
-                {modified, false, true, #{}},
-                rejected,
-                released],
-    [begin
-         reliable_send_receive(Config, Outcome)
-     end || Outcome <- Outcomes],
-    ok.
-
-reliable_send_receive(Config, Outcome) ->
-    Container = atom_to_binary(?FUNCTION_NAME, utf8),
-    OutcomeBin = case is_atom(Outcome) of
-                     true ->
-                         atom_to_binary(Outcome, utf8);
-                     false ->
-                         O1 = atom_to_binary(element(1, Outcome), utf8),
-                         O2 = atom_to_binary(element(2, Outcome), utf8),
-                         <>
-                 end,
-
-    ct:pal("~s testing ~s", [?FUNCTION_NAME, OutcomeBin]),
-    QName = <>,
-    %% declare a quorum queue
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    amqp_channel:call(Ch, #'queue.declare'{queue = QName,
-                                           durable = true,
-                                           arguments = [{<<"x-queue-type">>,
-                                                         longstr, <<"quorum">>}]}),
-    rabbit_ct_client_helpers:close_channel(Ch),
-    %% reliable send and consume
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    Address = <<"/amq/queue/", QName/binary>>,
-
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => Container,
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-    ok = wait_for_credit(Sender),
-    DTag1 = <<"dtag-1">>,
-    %% create an unsettled message,
-    %% link will be in "mixed" mode by default
-    Msg1 = amqp10_msg:new(DTag1, <<"body-1">>, false),
-    %% Use the 2 byte AMQP boolean encoding, see AMQP §1.6.2
-    True = {boolean, true},
-    Msg2 = amqp10_msg:set_headers(#{durable => True}, Msg1),
-    ok = amqp10_client:send_msg(Sender, Msg2),
-    ok = wait_for_settlement(DTag1),
-
-    ok = amqp10_client:detach_link(Sender),
-    ok = amqp10_client:close_connection(Connection),
-    flush("post sender close"),
-
-    {ok, Connection2} = amqp10_client:open_connection(OpnConf),
-    {ok, Session2} = amqp10_client:begin_session(Connection2),
-    ReceiverLinkName = <<"test-receiver">>,
-    {ok, Receiver} = amqp10_client:attach_receiver_link(Session2,
-                                                        ReceiverLinkName,
-                                                        Address,
-                                                        unsettled),
-    {ok, Msg} = amqp10_client:get_msg(Receiver),
-    ct:pal("got ~p", [amqp10_msg:body(Msg)]),
-    ?assertEqual(true, amqp10_msg:header(durable, Msg)),
-
-    ok = amqp10_client:settle_msg(Receiver, Msg, Outcome),
-
-    flush("post accept"),
-
-    ok = amqp10_client:detach_link(Receiver),
-    ok = amqp10_client:close_connection(Connection2),
-
-    ok.
-
-publishing_to_non_existing_queue_should_settle_with_released(Config) ->
-    Container = atom_to_binary(?FUNCTION_NAME, utf8),
-    Suffix = <<"foo">>,
-    %% does not exist
-    QName = <>,
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    Address = <<"/exchange/amq.direct/", QName/binary>>,
-
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => Container,
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-    ok = wait_for_credit(Sender),
-    DTag1 = <<"dtag-1">>,
-    %% create an unsettled message,
-    %% link will be in "mixed" mode by default
-    Msg1 = amqp10_msg:new(DTag1, <<"body-1">>, false),
-    ok = amqp10_client:send_msg(Sender, Msg1),
-    ok = wait_for_settlement(DTag1, released),
-
-    ok = amqp10_client:detach_link(Sender),
-    ok = amqp10_client:close_connection(Connection),
-    flush("post sender close"),
-    ok.
-
-open_link_to_non_existing_destination_should_end_session(Config) ->
-    Container = atom_to_list(?FUNCTION_NAME),
-    Name = Container ++ "foo",
-    Addresses = [
-                 "/exchange/" ++ Name ++ "/bar",
-                 "/amq/queue/" ++ Name
-                ],
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => list_to_binary(Container),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-
-    [begin
-         {ok, Connection} = amqp10_client:open_connection(OpnConf),
-         {ok, Session} = amqp10_client:begin_session(Connection),
-         SenderLinkName = <<"test-sender">>,
-         ct:pal("Address ~p", [Address]),
-         {ok, _} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    list_to_binary(Address)),
-
-         wait_for_session_end(Session),
-         ok = amqp10_client:close_connection(Connection),
-         flush("post sender close")
-
-     end || Address <- Addresses],
-    ok.
-
-roundtrip_classic_queue_with_drain(Config) ->
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    roundtrip_queue_with_drain(Config, <<"classic">>, QName).
-
-roundtrip_quorum_queue_with_drain(Config) ->
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    roundtrip_queue_with_drain(Config, <<"quorum">>, QName).
-
-roundtrip_stream_queue_with_drain(Config) ->
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    roundtrip_queue_with_drain(Config, <<"stream">>, QName).
-
-roundtrip_queue_with_drain(Config, QueueType, QName) when is_binary(QueueType) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    Address = <<"/amq/queue/", QName/binary>>,
-    %% declare a queue
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    Args = [{<<"x-queue-type">>, longstr, QueueType}],
-    amqp_channel:call(Ch, #'queue.declare'{queue = QName,
-                                           durable = true,
-                                           arguments = Args}),
-    % create a configuration map
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => atom_to_binary(?FUNCTION_NAME, utf8),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-
-    wait_for_credit(Sender),
-
-    Dtag = <<"my-tag">>,
-    % create a new message using a delivery-tag, body and indicate
-    % it's settlement status (true meaning no disposition confirmation
-    % will be sent by the receiver).
-    OutMsg = amqp10_msg:new(Dtag, <<"my-body">>, false),
-    ok = amqp10_client:send_msg(Sender, OutMsg),
-    ok = wait_for_settlement(Dtag),
-
-    flush("pre-receive"),
-    % create a receiver link
-
-    TerminusDurability = none,
-    Filter = case QueueType of
-                 <<"stream">> ->
-                     #{<<"rabbitmq:stream-offset-spec">> => <<"first">>};
-                 _ ->
-                     #{}
-             end,
-    Properties = #{},
-    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"test-receiver">>,
-                                                        Address, unsettled,
-                                                        TerminusDurability,
-                                                        Filter, Properties),
-
-    % grant credit and drain
-    ok = amqp10_client:flow_link_credit(Receiver, 1, never, true),
-
-    % wait for a delivery
-    receive
-        {amqp10_msg, Receiver, InMsg} ->
-            ok = amqp10_client:accept_msg(Receiver, InMsg),
-            wait_for_accepts(1),
-            ok
-    after 2000 ->
-              flush("delivery_timeout"),
-              exit(delivery_timeout)
-    end,
-    Dtag = <<"my-tag">>,
-    OutMsg2 = amqp10_msg:new(Dtag, <<"my-body2">>, false),
-    ok = amqp10_client:send_msg(Sender, OutMsg2),
-    ok = wait_for_settlement(Dtag),
-
-    %% no delivery should be made at this point
-    receive
-        {amqp10_msg, _, _} ->
-            flush("unexpected_delivery"),
-            exit(unexpected_delivery)
-    after 500 ->
-              ok
-    end,
-
-    flush("final"),
-    ok = amqp10_client:detach_link(Sender),
-
-    ok = amqp10_client:close_connection(Connection),
-    ok.
-
-%% Send a message with a body containing a single AMQP 1.0 value section
-%% to a stream and consume via AMQP 0.9.1.
-amqp_stream_amqpl(Config) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    ContainerId = QName = atom_to_binary(?FUNCTION_NAME),
-
-    amqp_channel:call(Ch, #'queue.declare'{
-                             queue = QName,
-                             durable = true,
-                             arguments = [{<<"x-queue-type">>, longstr, <<"stream">>}]}),
-
-    Address = <<"/amq/queue/", QName/binary>>,
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => ContainerId,
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-    wait_for_credit(Sender),
-    OutMsg = amqp10_msg:new(<<"my-tag">>, {'v1_0.amqp_value', {binary, <<0, 255>>}}, true),
-    ok = amqp10_client:send_msg(Sender, OutMsg),
-    flush("final"),
-    ok = amqp10_client:detach_link(Sender),
-    ok = amqp10_client:close_connection(Connection),
-
-    #'basic.qos_ok'{} =  amqp_channel:call(Ch, #'basic.qos'{global = false,
-                                                            prefetch_count = 1}),
-    CTag = <<"my-tag">>,
-    #'basic.consume_ok'{} = amqp_channel:subscribe(
-                              Ch,
-                              #'basic.consume'{
-                                 queue = QName,
-                                 consumer_tag = CTag,
-                                 arguments = [{<<"x-stream-offset">>, longstr, <<"first">>}]},
-                              self()),
-    receive
-        {#'basic.deliver'{consumer_tag = CTag,
-                          redelivered  = false},
-         #amqp_msg{props = #'P_basic'{type = <<"amqp-1.0">>}}} ->
-            ok
-    after 5000 ->
-              exit(basic_deliver_timeout)
-    end,
-    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
-    ok = rabbit_ct_client_helpers:close_channel(Ch).
-
-message_headers_conversion(Config) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    Address = <<"/amq/queue/", QName/binary>>,
-    %% declare a quorum queue
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    amqp_channel:call(Ch, #'queue.declare'{queue = QName,
-                                            durable = true,
-                                            arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
-
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,[rabbitmq_amqp1_0, convert_amqp091_headers_to_app_props, true]),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,[rabbitmq_amqp1_0, convert_app_props_to_amqp091_headers, true]),
-
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => atom_to_binary(?FUNCTION_NAME, utf8),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-
-    amqp10_to_amqp091_header_conversion(Session, Ch, QName, Address),
-
-    amqp091_to_amqp10_header_conversion(Session, Ch, QName, Address),
-    delete_queue(Config, QName),
-    ok = amqp10_client:close_connection(Connection),
-    ok.
-
-resource_alarm(Config) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    Address = <<"/amq/queue/", QName/binary>>,
-    Ch = rabbit_ct_client_helpers:open_channel(Config),
-    #'queue.declare_ok'{} = amqp_channel:call(Ch, #'queue.declare'{queue = QName}),
-
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => atom_to_binary(?FUNCTION_NAME),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    {ok, Sender} = create_amqp10_sender(Session, Address),
-
-    M1 = amqp10_msg:new(<<"t1">>, <<"m1">>, false),
-    M2 = amqp10_msg:new(<<"t2">>, <<"m2">>, false),
-    M3 = amqp10_msg:new(<<"t3">>, <<"m3">>, false),
-    M4 = amqp10_msg:new(<<"t4">>, <<"m4">>, false),
-
-    ok = amqp10_client:send_msg(Sender, M1),
-    ok = wait_for_settlement(<<"t1">>),
-
-    ok = rabbit_ct_broker_helpers:rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
-    %% Let connection block.
-    timer:sleep(100),
-
-    ok = amqp10_client:send_msg(Sender, M2),
-    ok = amqp10_client:send_msg(Sender, M3),
-    ok = amqp10_client:send_msg(Sender, M4),
-
-    %% M2 still goes through, but M3 should get blocked. (Server is off by one message.)
-    receive {amqp10_disposition, {accepted, <<"t2">>}} -> ok
-    after 300 -> ct:fail({accepted_timeout, ?LINE})
-    end,
-    receive {amqp10_disposition, {accepted, <<"t3">>}} -> ct:fail("expected connection to be blocked")
-    after 300 -> ok
-    end,
-
-    %% Unblock connection.
-    ok = rabbit_ct_broker_helpers:rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
-
-    %% Previously sent M3 and M4 should now be processed on the server.
-    receive {amqp10_disposition, {accepted, <<"t3">>}} -> ok
-    after 5000 -> ct:fail({accepted_timeout, ?LINE})
-    end,
-    ok = wait_for_settlement(<<"t4">>),
-
-    delete_queue(Config, QName),
-    ok = amqp10_client:close_connection(Connection).
-
-amqp10_to_amqp091_header_conversion(Session,Ch, QName, Address) -> 
-    {ok, Sender} = create_amqp10_sender(Session, Address),
-
-    OutMsg = amqp10_msg:new(<<"my-tag">>, <<"my-body">>, true),
-    OutMsg2 = amqp10_msg:set_application_properties(#{
-        "x-string" => "string-value",
-        "x-int" => 3,
-        "x-bool" => true
-    }, OutMsg),
-    ok = amqp10_client:send_msg(Sender, OutMsg2),
-    wait_for_accepts(1),
-
-    {ok, Headers} = amqp091_get_msg_headers(Ch, QName),
-
-    ?assertEqual({bool, true}, rabbit_misc:table_lookup(Headers, <<"x-bool">>)),
-    ?assertEqual({unsignedint, 3}, rabbit_misc:table_lookup(Headers, <<"x-int">>)),
-    ?assertEqual({longstr, <<"string-value">>}, rabbit_misc:table_lookup(Headers, <<"x-string">>)).    
-
-
-amqp091_to_amqp10_header_conversion(Session, Ch, QName, Address) -> 
-    Amqp091Headers = [{<<"x-forwarding">>, array, 
-                        [{table, [{<<"uri">>, longstr,
-                                   <<"amqp://localhost/%2F/upstream">>}]}]},
-                      {<<"x-string">>, longstr, "my-string"},
-                      {<<"x-int">>, long, 92},
-                      {<<"x-bool">>, bool, true}],
-
-    amqp_channel:cast(Ch, 
-        #'basic.publish'{exchange = <<"">>, routing_key = QName},
-        #amqp_msg{props = #'P_basic'{
-            headers = Amqp091Headers}, 
-            payload = <<"foobar">> }
-        ),
-
-    {ok, [Msg]} = drain_queue(Session, Address, 1),
-    Amqp10Props = amqp10_msg:application_properties(Msg),
-    ?assertEqual(true, maps:get(<<"x-bool">>, Amqp10Props, undefined)),
-    ?assertEqual(92, maps:get(<<"x-int">>, Amqp10Props, undefined)),    
-    ?assertEqual(<<"my-string">>, maps:get(<<"x-string">>, Amqp10Props, undefined)).
-
-auth_attempt_metrics(Config) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    % create a configuration map
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => atom_to_binary(?FUNCTION_NAME, utf8),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-    open_and_close_connection(OpnConf),
-    [Attempt] =
-        rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts, []),
-    ?assertEqual(false, proplists:is_defined(remote_address, Attempt)),
-    ?assertEqual(false, proplists:is_defined(username, Attempt)),
-    ?assertEqual(proplists:get_value(protocol, Attempt), <<"amqp10">>),
-    ?assertEqual(proplists:get_value(auth_attempts, Attempt), 1),
-    ?assertEqual(proplists:get_value(auth_attempts_failed, Attempt), 0),
-    ?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt), 1),
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbit, track_auth_attempt_source, true]),
-    open_and_close_connection(OpnConf),
-    Attempts =
-        rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, get_auth_attempts_by_source, []),
-    [Attempt1] = lists:filter(fun(Props) ->
-                                      proplists:is_defined(remote_address, Props)
-                              end, Attempts),
-    ?assertEqual(proplists:get_value(remote_address, Attempt1), <<>>),
-    ?assertEqual(proplists:get_value(username, Attempt1), <<"guest">>),
-    ?assertEqual(proplists:get_value(protocol, Attempt), <<"amqp10">>),
-    ?assertEqual(proplists:get_value(auth_attempts, Attempt1), 1),
-    ?assertEqual(proplists:get_value(auth_attempts_failed, Attempt1), 0),
-    ?assertEqual(proplists:get_value(auth_attempts_succeeded, Attempt1), 1),
-    ok.
-
-%% internal
-%%
-
-flush(Prefix) ->
-    receive
-        Msg ->
-            ct:pal("~ts flushed: ~w~n", [Prefix, Msg]),
-            flush(Prefix)
-    after 1 ->
-              ok
-    end.
-
-open_and_close_connection(OpnConf) ->
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, _} = amqp10_client:begin_session(Connection),
-    ok = amqp10_client:close_connection(Connection).
-
-% before we can send messages we have to wait for credit from the server
-wait_for_credit(Sender) ->
-    receive
-        {amqp10_event, {link, Sender, credited}} ->
-            flush(?FUNCTION_NAME),
-            ok
-    after 5000 ->
-              flush("wait_for_credit timed out"),
-              ct:fail(credited_timeout)
-    end.
-
-wait_for_session_end(Session) ->
-    receive
-        {amqp10_event, {session, Session, {ended, _}}} ->
-            flush(?FUNCTION_NAME),
-            ok
-    after 5000 ->
-              flush("wait_for_session_end timed out"),
-              ct:fail(settled_timeout)
-    end.
-
-wait_for_settlement(Tag) ->
-    wait_for_settlement(Tag, accepted).
-
-wait_for_settlement(Tag, State) ->
-    receive
-        {amqp10_disposition, {State, Tag}} ->
-            flush(?FUNCTION_NAME),
-            ok
-    after 5000 ->
-              flush("wait_for_settlement timed out"),
-              ct:fail({settled_timeout, Tag})
-    end.
-
-wait_for_accepts(0) -> ok;
-wait_for_accepts(N) ->
-    receive
-        {amqp10_disposition,{accepted,_}} ->
-            wait_for_accepts(N -1)
-    after 250 ->
-              ok
-    end.
-
-delete_queue(Config, QName) -> 
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
-    rabbit_ct_client_helpers:close_channel(Ch).
-
-
-amqp091_get_msg_headers(Channel, QName) -> 
-    {#'basic.get_ok'{}, #amqp_msg{props = #'P_basic'{ headers= Headers}}}
-        = amqp_channel:call(Channel, #'basic.get'{queue = QName, no_ack = true}),
-    {ok, Headers}.
-
-create_amqp10_sender(Session, Address) -> 
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-    wait_for_credit(Sender),
-    {ok, Sender}.
-
-    drain_queue(Session, Address, N) -> 
-        flush("Before drain_queue"),
-        {ok, Receiver} = amqp10_client:attach_receiver_link(Session,
-                        <<"test-receiver">>,
-                        Address, 
-                        settled,
-                        configuration),
-    
-        ok = amqp10_client:flow_link_credit(Receiver, 1000, never, true),
-        Msgs = receive_message(Receiver, N, []),
-        flush("after drain"),
-        ok = amqp10_client:detach_link(Receiver),
-        {ok, Msgs}.
-    
-receive_message(_Receiver, 0, Acc) -> lists:reverse(Acc);
-receive_message(Receiver, N, Acc) ->
-    receive
-        {amqp10_msg, Receiver, Msg} -> 
-            receive_message(Receiver, N-1, [Msg | Acc])
-    after 5000  ->
-            exit(receive_timed_out)
-    end.
diff --git a/deps/rabbitmq_amqp1_0/test/command_SUITE.erl b/deps/rabbitmq_amqp1_0/test/command_SUITE.erl
deleted file mode 100644
index bdf61ccf55a7..000000000000
--- a/deps/rabbitmq_amqp1_0/test/command_SUITE.erl
+++ /dev/null
@@ -1,163 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-
-
--module(command_SUITE).
--compile([export_all]).
-
--include_lib("common_test/include/ct.hrl").
--include_lib("eunit/include/eunit.hrl").
--include("rabbit_amqp1_0.hrl").
--include_lib("amqp_client/include/amqp_client.hrl").
-
--define(COMMAND, 'Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand').
-
-all() ->
-    [
-     {group, non_parallel_tests}
-    ].
-
-groups() ->
-    [
-     {non_parallel_tests, [], [
-                               merge_defaults,
-                               validate,
-                               when_no_connections,
-                               when_one_connection
-                                ]}
-    ].
-
-init_per_suite(Config) ->
-    application:ensure_all_started(amqp10_client),
-    rabbit_ct_helpers:log_environment(),
-    Config.
-
-end_per_suite(Config) ->
-    Config.
-
-init_per_group(Group, Config) ->
-    Suffix = rabbit_ct_helpers:testcase_absname(Config, "", "-"),
-    Config1 = rabbit_ct_helpers:set_config(
-                Config, [
-                         {rmq_nodename_suffix, Suffix},
-                         {amqp10_client_library, Group}
-                        ]),
-    rabbit_ct_helpers:run_setup_steps(
-      Config1,
-      rabbit_ct_broker_helpers:setup_steps() ++
-      rabbit_ct_client_helpers:setup_steps()).
-
-end_per_group(_, Config) ->
-    rabbit_ct_helpers:run_teardown_steps(Config,
-      rabbit_ct_client_helpers:teardown_steps() ++
-      rabbit_ct_broker_helpers:teardown_steps()).
-
-init_per_testcase(Testcase, Config) ->
-    rabbit_ct_helpers:testcase_started(Config, Testcase).
-
-end_per_testcase(Testcase, Config) ->
-    rabbit_ct_helpers:testcase_finished(Config, Testcase).
-
-merge_defaults(_Config) ->
-    {[<<"pid">>], #{verbose := false}} =
-        ?COMMAND:merge_defaults([], #{}),
-
-    {[<<"other_key">>], #{verbose := true}} =
-        ?COMMAND:merge_defaults([<<"other_key">>], #{verbose => true}),
-
-    {[<<"other_key">>], #{verbose := false}} =
-        ?COMMAND:merge_defaults([<<"other_key">>], #{verbose => false}).
-
-validate(_Config) ->
-    ok = ?COMMAND:validate([], #{}),
-    ok = ?COMMAND:validate([<<"recv_oct">>, <<"ssl">>], #{}),
-    ok = ?COMMAND:validate([atom_to_binary(K, utf8) || K <- ?INFO_ITEMS], #{}),
-    {validation_failure,{bad_info_key,[other]}} =
-        ?COMMAND:validate([<<"other">>], #{}).
-
-when_no_connections(_Config) ->
-    [A] = rabbit_ct_broker_helpers:get_node_configs(_Config, nodename),
-    Opts = #{node => A, timeout => 2000, verbose => true},
-    [] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts)).
-
-when_one_connection(_Config) ->
-    [A] = rabbit_ct_broker_helpers:get_node_configs(_Config, nodename),
-    Opts = #{node => A, timeout => 2000, verbose => true},
-
-    [Connection,Sender] = open_amqp10_connection(_Config),
-
-    [_] = 'Elixir.Enum':to_list(?COMMAND:run([], Opts)),
-    close_amqp10_connection(Connection, Sender).
-
-open_amqp10_connection(Config) ->
-    Host = ?config(rmq_hostname, Config),
-    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    QName  = atom_to_binary(?FUNCTION_NAME, utf8),
-    Address = <<"/amq/queue/", QName/binary>>,
-    %% declare a quorum queue
-    Ch = rabbit_ct_client_helpers:open_channel(Config, 0),
-    amqp_channel:call(Ch, #'queue.declare'{queue = QName,
-                                           durable = true,
-                                           arguments = [{<<"x-queue-type">>, longstr, <<"quorum">>}]}),
-
-    % create a configuration map
-    OpnConf = #{address => Host,
-                port => Port,
-                container_id => atom_to_binary(?FUNCTION_NAME, utf8),
-                sasl => {plain, <<"guest">>, <<"guest">>}},
-
-    % ct:pal("opening connectoin with ~tp", [OpnConf]),
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session(Connection),
-    SenderLinkName = <<"test-sender">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(Session,
-                                                    SenderLinkName,
-                                                    Address),
-
-    % wait for credit to be received
-    receive
-        {amqp10_event, {link, Sender, credited}} -> ok
-    after 2000 ->
-              exit(credited_timeout)
-    end,
-
-    OutMsg = amqp10_msg:new(<<"my-tag">>, <<"my-body">>, true),
-    ok = amqp10_client:send_msg(Sender, OutMsg),
-
-    flush("pre-receive"),
-    {ok, Receiver} = amqp10_client:attach_receiver_link(Session,
-                                                        <<"test-receiver">>,
-                                                        Address),
-
-    % grant credit and drain
-    ok = amqp10_client:flow_link_credit(Receiver, 1, never, true),
-
-    % wait for a delivery
-    receive
-        {amqp10_msg, Receiver, _InMsg} -> ct:pal("Received amqp 1.0 message : ~w~n", [_InMsg]), ok
-    after 2000 ->
-              exit(delivery_timeout)
-    end,
-
-
-
-    [Connection, Sender].
-
-flush(Prefix) ->
-    receive
-        Msg ->
-            ct:pal("~ts flushed: ~w~n", [Prefix, Msg]),
-            flush(Prefix)
-    after 1 ->
-              ok
-    end.
-
-close_amqp10_connection(Connection, Sender) ->
-  flush("final"),
-  ct:pal("Closing connection ~w~n", [Connection]),
-  ok = amqp10_client:detach_link(Sender),
-  ok = amqp10_client:close_connection(Connection),
-  ok.
diff --git a/deps/rabbitmq_amqp1_0/test/config_schema_SUITE.erl b/deps/rabbitmq_amqp1_0/test/config_schema_SUITE.erl
deleted file mode 100644
index 34c2ad52ace7..000000000000
--- a/deps/rabbitmq_amqp1_0/test/config_schema_SUITE.erl
+++ /dev/null
@@ -1,54 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(config_schema_SUITE).
-
--compile(export_all).
-
-all() ->
-    [
-      run_snippets
-    ].
-
-%% -------------------------------------------------------------------
-%% Testsuite setup/teardown.
-%% -------------------------------------------------------------------
-
-init_per_suite(Config) ->
-    rabbit_ct_helpers:log_environment(),
-    Config1 = rabbit_ct_helpers:run_setup_steps(Config),
-    rabbit_ct_config_schema:init_schemas(rabbitmq_amqp1_0, Config1).
-
-
-end_per_suite(Config) ->
-    rabbit_ct_helpers:run_teardown_steps(Config).
-
-init_per_testcase(Testcase, Config) ->
-    rabbit_ct_helpers:testcase_started(Config, Testcase),
-    Config1 = rabbit_ct_helpers:set_config(Config, [
-        {rmq_nodename_suffix, Testcase}
-      ]),
-    rabbit_ct_helpers:run_steps(Config1,
-      rabbit_ct_broker_helpers:setup_steps() ++
-      rabbit_ct_client_helpers:setup_steps()).
-
-end_per_testcase(Testcase, Config) ->
-    Config1 = rabbit_ct_helpers:run_steps(Config,
-      rabbit_ct_client_helpers:teardown_steps() ++
-      rabbit_ct_broker_helpers:teardown_steps()),
-    rabbit_ct_helpers:testcase_finished(Config1, Testcase).
-
-%% -------------------------------------------------------------------
-%% Testcases.
-%% -------------------------------------------------------------------
-
-run_snippets(Config) ->
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0,
-      ?MODULE, run_snippets1, [Config]).
-
-run_snippets1(Config) ->
-    rabbit_ct_config_schema:run_snippets(Config).
diff --git a/deps/rabbitmq_amqp1_0/test/config_schema_SUITE_data/rabbitmq_amqp1_0.snippets b/deps/rabbitmq_amqp1_0/test/config_schema_SUITE_data/rabbitmq_amqp1_0.snippets
deleted file mode 100644
index 8786fa77fb77..000000000000
--- a/deps/rabbitmq_amqp1_0/test/config_schema_SUITE_data/rabbitmq_amqp1_0.snippets
+++ /dev/null
@@ -1,17 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
-[
-  {rabbitmq_amqp1_0_config,
-   "amqp1_0.convert_amqp091_headers_to_app_props = true
-    amqp1_0.convert_app_props_to_amqp091_headers = true",
-   [{rabbitmq_amqp1_0,[
-                          {convert_amqp091_headers_to_app_props, true},
-                          {convert_app_props_to_amqp091_headers, true}
-                         ]}],
-   [rabbitmq_amqp1_0]}
-].
diff --git a/deps/rabbitmq_amqp1_0/test/unit_SUITE.erl b/deps/rabbitmq_amqp1_0/test/unit_SUITE.erl
deleted file mode 100644
index 88a69d291e8e..000000000000
--- a/deps/rabbitmq_amqp1_0/test/unit_SUITE.erl
+++ /dev/null
@@ -1,39 +0,0 @@
-%% 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-2024 Broadcom. All Rights Reserved. The term “Broadcom” refers to Broadcom Inc. and/or its subsidiaries. All rights reserved.
-%%
-
--module(unit_SUITE).
-
--include_lib("common_test/include/ct.hrl").
--include_lib("eunit/include/eunit.hrl").
-
--include("rabbit_amqp1_0.hrl").
-
--import(rabbit_amqp1_0_util, [serial_add/2, serial_diff/2, serial_compare/2]).
-
--compile(export_all).
-
-all() ->
-    [
-      serial_arithmetic
-    ].
-
--include_lib("eunit/include/eunit.hrl").
-
-serial_arithmetic(_Config) ->
-    ?assertEqual(1, serial_add(0, 1)),
-    ?assertEqual(16#7fffffff, serial_add(0, 16#7fffffff)),
-    ?assertEqual(0, serial_add(16#ffffffff, 1)),
-    %% Cannot add more than 2 ^ 31 - 1
-    ?assertExit({out_of_bound_serial_addition, _, _},
-                serial_add(200, 16#80000000)),
-    ?assertEqual(1, serial_diff(1, 0)),
-    ?assertEqual(2, serial_diff(1, 16#ffffffff)),
-    ?assertEqual(-2, serial_diff(16#ffffffff, 1)),
-    ?assertExit({indeterminate_serial_diff, _, _},
-                serial_diff(0, 16#80000000)),
-    ?assertExit({indeterminate_serial_diff, _, _},
-                serial_diff(16#ffffffff, 16#7fffffff)).
diff --git a/deps/rabbitmq_cli/test/test_helper.exs b/deps/rabbitmq_cli/test/test_helper.exs
index 575fcdf7f7f8..1d605faa1f51 100644
--- a/deps/rabbitmq_cli/test/test_helper.exs
+++ b/deps/rabbitmq_cli/test/test_helper.exs
@@ -542,7 +542,7 @@ defmodule TestHelper do
   end
 
   def await_no_client_connections_with_iterations(node, n) when n > 0 do
-    case :rpc.call(node, :rabbit_networking, :connections_local, []) do
+    case :rpc.call(node, :rabbit_networking, :local_connections, []) do
       [] ->
         :ok
 
@@ -561,13 +561,13 @@ defmodule TestHelper do
   end
 
   def close_all_connections(node) do
-    # we intentionally use connections_local/0 here because connections/0,
+    # we intentionally use local_connections/0 here because connections/0,
     # the cluster-wide version, loads some bits around cluster membership
     # that are not normally ready with a single node. MK.
     #
     # when/if we decide to test
     # this project against a cluster of nodes this will need revisiting. MK.
-    for pid <- :rpc.call(node, :rabbit_networking, :connections_local, []) do
+    for pid <- :rpc.call(node, :rabbit_networking, :local_connections, []) do
       :rpc.call(node, :rabbit_networking, :close_connection, [pid, :force_closed])
     end
 
diff --git a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
index 27a9fc4a0190..f59ca3073335 100644
--- a/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
+++ b/deps/rabbitmq_ct_helpers/src/rabbit_ct_broker_helpers.erl
@@ -26,6 +26,7 @@
     cluster_nodes/1, cluster_nodes/2,
 
     setup_meck/1,
+    setup_meck/2,
 
     get_node_configs/1, get_node_configs/2,
     get_node_config/2, get_node_config/3, set_node_config/3,
@@ -160,6 +161,7 @@
     clear_permissions/5,
 
     set_vhost_limit/5,
+    clear_vhost_limit/3,
 
     set_user_limits/3,
     set_user_limits/4,
@@ -1711,6 +1713,11 @@ set_vhost_limit(Config, Node, VHost, Limit0, Value) ->
         rabbit_vhost_limit, set,
         [VHost, Limits, <<"ct-tests">>]).
 
+clear_vhost_limit(Config, Node, VHost) ->
+    rpc(Config, Node,
+        rabbit_vhost_limit, clear,
+        [VHost, <<"ct-tests">>]).
+
 set_user_limits(Config, Username, Limits) ->
     set_user_limits(Config, 0, Username, Limits).
 
@@ -2252,7 +2259,13 @@ if_cover(F) ->
     end.
 
 setup_meck(Config) ->
-    {Mod, Bin, File} = code:get_object_code(meck),
-    [true | _] = rpc_all(Config, code, add_path, [filename:dirname(File)]),
-    [{module, Mod} | _] = rpc_all(Config, code, load_binary, [Mod, File, Bin]),
-    ok.
+    setup_meck(Config, []).
+
+setup_meck(Config, LoadModules)
+  when is_list(LoadModules) ->
+    lists:foreach(
+      fun(Mod) ->
+              {Mod, Bin, File} = code:get_object_code(Mod),
+              [true | _] = rpc_all(Config, code, add_path, [filename:dirname(File)]),
+              [{module, Mod} | _] = rpc_all(Config, code, load_binary, [Mod, File, Bin])
+      end, [meck | LoadModules]).
diff --git a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl b/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl
index 99e0adb1ca49..209cbb2b3faa 100644
--- a/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl
+++ b/deps/rabbitmq_federation/test/rabbit_federation_test_util.erl
@@ -193,7 +193,7 @@ expect(Payloads, Timeout) ->
                 false -> ?assert(false, rabbit_misc:format("received an unexpected payload ~tp", [Payload]))
             end
     after Timeout ->
-        ?assert(false, rabbit_misc:format("Did not receive expected payloads ~tp in time", [Payloads]))
+              ct:fail("Did not receive expected payloads ~tp in time", [Payloads])
     end.
 
 expect_empty(Ch, Q) ->
diff --git a/deps/rabbitmq_management/BUILD.bazel b/deps/rabbitmq_management/BUILD.bazel
index 83572b39f305..f273eceb9915 100644
--- a/deps/rabbitmq_management/BUILD.bazel
+++ b/deps/rabbitmq_management/BUILD.bazel
@@ -168,6 +168,9 @@ rabbitmq_integration_suite(
         "test/rabbit_mgmt_runtime_parameters_util.beam",
     ],
     shard_count = 2,
+    runtime_deps = [
+        "//deps/amqp10_client:erlang_app",
+    ],
 )
 
 rabbitmq_integration_suite(
diff --git a/deps/rabbitmq_management/selenium/test/authnz-msg-protocols/enabled_plugins b/deps/rabbitmq_management/selenium/test/authnz-msg-protocols/enabled_plugins
index c91f7ba880c3..59b57cb3828f 100644
--- a/deps/rabbitmq_management/selenium/test/authnz-msg-protocols/enabled_plugins
+++ b/deps/rabbitmq_management/selenium/test/authnz-msg-protocols/enabled_plugins
@@ -1,5 +1,5 @@
 [accept,amqp10_client,amqp_client,base64url,cowboy,cowlib,eetcd,gun,jose,
- oauth2_client,prometheus,rabbitmq_amqp1_0,rabbitmq_auth_backend_cache,
+ oauth2_client,prometheus,rabbitmq_auth_backend_cache,
  rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap,
  rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws,
  rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange,
diff --git a/deps/rabbitmq_management/selenium/test/basic-auth/enabled_plugins b/deps/rabbitmq_management/selenium/test/basic-auth/enabled_plugins
index c91f7ba880c3..59b57cb3828f 100644
--- a/deps/rabbitmq_management/selenium/test/basic-auth/enabled_plugins
+++ b/deps/rabbitmq_management/selenium/test/basic-auth/enabled_plugins
@@ -1,5 +1,5 @@
 [accept,amqp10_client,amqp_client,base64url,cowboy,cowlib,eetcd,gun,jose,
- oauth2_client,prometheus,rabbitmq_amqp1_0,rabbitmq_auth_backend_cache,
+ oauth2_client,prometheus,rabbitmq_auth_backend_cache,
  rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap,
  rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws,
  rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange,
diff --git a/deps/rabbitmq_management/selenium/test/mgt-only/enabled_plugins b/deps/rabbitmq_management/selenium/test/mgt-only/enabled_plugins
index feaf7912d783..ea2a6a29ba53 100644
--- a/deps/rabbitmq_management/selenium/test/mgt-only/enabled_plugins
+++ b/deps/rabbitmq_management/selenium/test/mgt-only/enabled_plugins
@@ -1,5 +1,5 @@
 [accept,amqp10_client,amqp_client,base64url,cowboy,cowlib,eetcd,gun,jose,
- prometheus,rabbitmq_amqp1_0,rabbitmq_auth_backend_cache,
+ prometheus,rabbitmq_auth_backend_cache,
  rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap,
  rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws,
  rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange,
diff --git a/deps/rabbitmq_management/selenium/test/oauth/enabled_plugins b/deps/rabbitmq_management/selenium/test/oauth/enabled_plugins
index c91f7ba880c3..59b57cb3828f 100644
--- a/deps/rabbitmq_management/selenium/test/oauth/enabled_plugins
+++ b/deps/rabbitmq_management/selenium/test/oauth/enabled_plugins
@@ -1,5 +1,5 @@
 [accept,amqp10_client,amqp_client,base64url,cowboy,cowlib,eetcd,gun,jose,
- oauth2_client,prometheus,rabbitmq_amqp1_0,rabbitmq_auth_backend_cache,
+ oauth2_client,prometheus,rabbitmq_auth_backend_cache,
  rabbitmq_auth_backend_http,rabbitmq_auth_backend_ldap,
  rabbitmq_auth_backend_oauth2,rabbitmq_auth_mechanism_ssl,rabbitmq_aws,
  rabbitmq_consistent_hash_exchange,rabbitmq_event_exchange,
diff --git a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl
index 59882068d487..f64ad8cbf4d4 100644
--- a/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl
+++ b/deps/rabbitmq_management/test/rabbit_mgmt_http_SUITE.erl
@@ -15,6 +15,10 @@
 
 -import(rabbit_ct_client_helpers, [close_connection/1, close_channel/1,
                                    open_unmanaged_connection/1]).
+-import(rabbit_ct_broker_helpers, [rpc/4]).
+-import(rabbit_ct_helpers,
+        [eventually/3,
+         eventually/1]).
 -import(rabbit_mgmt_test_util, [assert_list/2, assert_item/2, test_item/2,
                                 assert_keys/2, assert_no_keys/2,
                                 http_get/2, http_get/3, http_get/5,
@@ -35,7 +39,7 @@
 -define(COLLECT_INTERVAL, 1000).
 -define(PATH_PREFIX, "/custom-prefix").
 
--compile(export_all).
+-compile([export_all, nowarn_export_all]).
 
 all() ->
     [
@@ -79,7 +83,8 @@ all_tests() -> [
     permissions_validation_test,
     permissions_list_test,
     permissions_test,
-    connections_test,
+    connections_test_amqpl,
+    connections_test_amqp,
     multiple_invalid_connections_test,
     exchanges_test,
     queues_test,
@@ -192,6 +197,13 @@ finish_init(Group, Config) ->
     Config1 = rabbit_ct_helpers:set_config(Config, NodeConf),
     merge_app_env(Config1).
 
+init_per_suite(Config) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    Config.
+
+end_per_suite(Config) ->
+    Config.
+
 init_per_group(all_tests_with_prefix=Group, Config0) ->
     PathConfig = {rabbitmq_management, [{path_prefix, ?PATH_PREFIX}]},
     Config1 = rabbit_ct_helpers:merge_app_env(Config0, PathConfig),
@@ -243,8 +255,7 @@ init_per_testcase(Testcase, Config) ->
 
 end_per_testcase(Testcase, Config) ->
     rabbit_ct_broker_helpers:close_all_connections(Config, 0, <<"rabbit_mgmt_SUITE:end_per_testcase">>),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, disable_basic_auth, false]),
+    rpc(Config, application, set_env, [rabbitmq_management, disable_basic_auth, false]),
     Config1 = end_per_testcase0(Testcase, Config),
     rabbit_ct_helpers:testcase_finished(Config1, Testcase).
 
@@ -255,9 +266,7 @@ end_per_testcase0(T, Config)
      || #{name := Name} <- Vhosts],
     Config;
 end_per_testcase0(definitions_password_test, Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0,
-                                 application, unset_env,
-                                 [rabbit, password_hashing_module]),
+    rpc(Config, application, unset_env, [rabbit, password_hashing_module]),
     Config;
 end_per_testcase0(queues_test, Config) ->
     rabbit_ct_broker_helpers:delete_vhost(Config, <<"downvhost">>),
@@ -291,23 +300,18 @@ end_per_testcase0(permissions_vhost_test, Config) ->
     rabbit_ct_broker_helpers:delete_user(Config, <<"myuser2">>),
     Config;
 end_per_testcase0(config_environment_test, Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, unset_env,
-                                 [rabbit, config_environment_test_env]),
+    rpc(Config, application, unset_env, [rabbit, config_environment_test_env]),
     Config;
 end_per_testcase0(disabled_operator_policy_test, Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, unset_env,
-                                 [rabbitmq_management, restrictions]),
+    rpc(Config, application, unset_env, [rabbitmq_management, restrictions]),
     Config;
 end_per_testcase0(disabled_qq_replica_opers_test, Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, unset_env,
-                                 [rabbitmq_management, restrictions]),
+    rpc(Config, application, unset_env, [rabbitmq_management, restrictions]),
     Config;
 end_per_testcase0(Testcase, Config)
   when Testcase == list_deprecated_features_test;
        Testcase == list_used_deprecated_features_test ->
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_feature_flags,
-                                      clear_injected_test_feature_flags,
-                                      []),
+    ok = rpc(Config, rabbit_feature_flags, clear_injected_test_feature_flags, []),
     Config;
 end_per_testcase0(_, Config) -> Config.
 
@@ -513,8 +517,7 @@ vhosts_trace_test(Config) ->
 users_test(Config) ->
     assert_item(#{name => <<"guest">>, tags => [<<"administrator">>]},
                 http_get(Config, "/whoami")),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, login_session_timeout, 100]),
+    rpc(Config, application, set_env, [rabbitmq_management, login_session_timeout, 100]),
     assert_item(#{name => <<"guest">>,
                   tags => [<<"administrator">>],
                   login_session_timeout => 100},
@@ -925,7 +928,7 @@ topic_permissions_test(Config) ->
     http_delete(Config, "/vhosts/myvhost2", {group, '2xx'}),
     passed.
 
-connections_test(Config) ->
+connections_test_amqpl(Config) ->
     {Conn, _Ch} = open_connection_and_channel(Config),
     LocalPort = local_port(Conn),
     Path = binary_to_list(
@@ -954,6 +957,72 @@ connections_test(Config) ->
     close_connection(Conn),
     passed.
 
+%% Test that AMQP 1.0 connection can be listed and closed via the rabbitmq_management plugin.
+connections_test_amqp(Config) ->
+    Node = atom_to_binary(rabbit_ct_broker_helpers:get_node_config(Config, 0, nodename)),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    User = <<"guest">>,
+    OpnConf = #{address => ?config(rmq_hostname, Config),
+                port => Port,
+                container_id => <<"my container">>,
+                sasl => {plain, User, <<"guest">>}},
+    {ok, C1} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, C1, opened}} -> ok
+    after 5000 -> ct:fail(opened_timeout)
+    end,
+    eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10),
+    ?assertEqual(1, length(rpc(Config, rabbit_amqp1_0, list_local, []))),
+    [Connection1] = http_get(Config, "/connections"),
+    ?assertMatch(#{node := Node,
+                   vhost := <<"/">>,
+                   user := User,
+                   auth_mechanism := <<"PLAIN">>,
+                   protocol := <<"AMQP 1-0">>,
+                   client_properties := #{version := _,
+                                          product := <<"AMQP 1.0 client from the RabbitMQ Project">>,
+                                          platform := _}},
+                 Connection1),
+    ConnectionName = maps:get(name, Connection1),
+    http_delete(Config,
+                "/connections/" ++ binary_to_list(uri_string:quote(ConnectionName)),
+                ?NO_CONTENT),
+    receive {amqp10_event,
+             {connection, C1,
+              {closed,
+               {internal_error,
+                <<"Connection forced: \"Closed via management plugin\"">>}}}} -> ok
+    after 5000 -> ct:fail(closed_timeout)
+    end,
+    eventually(?_assertNot(is_process_alive(C1))),
+    ?assertEqual([], http_get(Config, "/connections")),
+
+    {ok, C2} = amqp10_client:open_connection(OpnConf),
+    receive {amqp10_event, {connection, C2, opened}} -> ok
+    after 5000 -> ct:fail(opened_timeout)
+    end,
+    eventually(?_assertEqual(1, length(http_get(Config, "/connections"))), 1000, 10),
+    http_delete(Config,
+                "/connections/username/guest",
+                ?NO_CONTENT),
+    receive {amqp10_event,
+             {connection, C2,
+              {closed,
+               {internal_error,
+                <<"Connection forced: \"Closed via management plugin\"">>}}}} -> ok
+    after 5000 -> ct:fail(closed_timeout)
+    end,
+    ?assertEqual([], http_get(Config, "/connections")),
+    ?assertEqual(0, length(rpc(Config, rabbit_amqp1_0, list_local, []))).
+
+flush(Prefix) ->
+    receive
+        Msg ->
+            ct:pal("~ts flushed: ~p~n", [Prefix, Msg]),
+            flush(Prefix)
+    after 1 ->
+              ok
+    end.
+
 multiple_invalid_connections_test(Config) ->
     Count = 100,
     spawn_invalid(Config, Count),
@@ -1098,14 +1167,11 @@ crashed_queues_test(Config) ->
     QArgs = #{},
     http_put(Config, "/queues/%2F/crashingqueue", QArgs, {group, '2xx'}),
 
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0,
-            rabbit_amqqueue_control, await_state, [Node, Q, running]),
+    ok = rpc(Config, rabbit_amqqueue_control, await_state, [Node, Q, running]),
 
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0,
-        rabbit_amqqueue, kill_queue_hard, [Node, Q]),
+    ok = rpc(Config, rabbit_amqqueue, kill_queue_hard, [Node, Q]),
 
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0,
-            rabbit_amqqueue_control, await_state, [Node, Q, crashed]),
+    ok = rpc(Config, rabbit_amqqueue_control, await_state, [Node, Q, crashed]),
 
     CrashedQueue  = http_get(Config, "/queues/%2F/crashingqueue"),
 
@@ -1664,12 +1730,12 @@ defs(Config, Key, URI, CreateMethod, Args, DeleteFun0, DeleteFun1) ->
     passed.
 
 register_parameters_and_policy_validator(Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mgmt_runtime_parameters_util, register, []),
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mgmt_runtime_parameters_util, register_policy_validator, []).
+    rpc(Config, rabbit_mgmt_runtime_parameters_util, register, []),
+    rpc(Config, rabbit_mgmt_runtime_parameters_util, register_policy_validator, []).
 
 unregister_parameters_and_policy_validator(Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mgmt_runtime_parameters_util, unregister_policy_validator, []),
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_mgmt_runtime_parameters_util, unregister, []).
+    rpc(Config, rabbit_mgmt_runtime_parameters_util, unregister_policy_validator, []),
+    rpc(Config, rabbit_mgmt_runtime_parameters_util, unregister, []).
 
 definitions_test(Config) ->
     register_parameters_and_policy_validator(Config),
@@ -1933,9 +1999,7 @@ definitions_password_test(Config) ->
                                 password_hash => <<"WAbU0ZIcvjTpxM3Q3SbJhEAM2tQ=">>,
                                 tags          => <<"management">>}
                              ]},
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, [rabbit,
-                                                                   password_hashing_module,
-                                                                   rabbit_password_hashing_sha512]),
+    rpc(Config, application, set_env, [rabbit, password_hashing_module, rabbit_password_hashing_sha512]),
 
     ExpectedDefault = #{name              => <<"myuser">>,
                         password_hash     => <<"WAbU0ZIcvjTpxM3Q3SbJhEAM2tQ=">>,
@@ -2163,7 +2227,7 @@ exchanges_pagination_test(Config) ->
     %% for stats to update
     timer:sleep(1500),
 
-    Total     = length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_exchange, list_names, [])),
+    Total = length(rpc(Config, rabbit_exchange, list_names, [])),
 
     PageOfTwo = http_get(Config, "/exchanges?page=1&page_size=2", ?OK),
     ?assertEqual(Total, maps:get(total_count, PageOfTwo)),
@@ -2266,7 +2330,7 @@ queue_pagination_test(Config) ->
     %% for stats to update
     timer:sleep(1500),
 
-    Total     = length(rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_amqqueue, list_names, [])),
+    Total = length(rpc(Config, rabbit_amqqueue, list_names, [])),
 
     PageOfTwo = http_get(Config, "/queues?page=1&page_size=2", ?OK),
     ?assertEqual(Total, maps:get(total_count, PageOfTwo)),
@@ -3176,7 +3240,7 @@ cors_test(Config) ->
     %% The Vary header should include "Origin" regardless of CORS configuration.
     {_, "accept, accept-encoding, origin"} = lists:keyfind("vary", 1, HdNoCORS),
     %% Enable CORS.
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, [rabbitmq_management, cors_allow_origins, ["https://rabbitmq.com"]]),
+    rpc(Config, application, set_env, [rabbitmq_management, cors_allow_origins, ["https://rabbitmq.com"]]),
     %% We should only receive allow-origin and allow-credentials from GET.
     {ok, {_, HdGetCORS, _}} = req(Config, get, "/overview",
                                   [{"origin", "https://rabbitmq.com"}, auth_header("guest", "guest")]),
@@ -3202,7 +3266,7 @@ cors_test(Config) ->
                                             {"access-control-request-headers", "x-piggy-bank"}]),
     {_, "x-piggy-bank"} = lists:keyfind("access-control-allow-headers", 1, HdAllowHeadersCORS),
     %% Disable preflight request caching.
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, [rabbitmq_management, cors_max_age, undefined]),
+    rpc(Config, application, set_env, [rabbitmq_management, cors_max_age, undefined]),
     %% We shouldn't receive max-age anymore.
     {ok, {_, HdNoMaxAgeCORS, _}} = req(Config, options, "/overview",
                                        [{"origin", "https://rabbitmq.com"}, auth_header("guest", "guest")]),
@@ -3211,7 +3275,7 @@ cors_test(Config) ->
     %% Check OPTIONS method in all paths
     check_cors_all_endpoints(Config),
     %% Disable CORS again.
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env, [rabbitmq_management, cors_allow_origins, []]),
+    rpc(Config, application, set_env, [rabbitmq_management, cors_allow_origins, []]),
     passed.
 
 check_cors_all_endpoints(Config) ->
@@ -3546,21 +3610,16 @@ oauth_test(Config) ->
     ?assertEqual(false, maps:get(oauth_enabled, Map1)),
 
     %% Misconfiguration
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, oauth_enabled, true]),
+    rpc(Config, application, set_env, [rabbitmq_management, oauth_enabled, true]),
     Map2 = http_get(Config, "/auth", ?OK),
     ?assertEqual(false, maps:get(oauth_enabled, Map2)),
     ?assertEqual(<<>>, maps:get(oauth_client_id, Map2)),
     ?assertEqual(<<>>, maps:get(oauth_provider_url, Map2)),
     %% Valid config requires non empty OAuthClientId, OAuthClientSecret, OAuthResourceId, OAuthProviderUrl
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, oauth_client_id, "rabbit_user"]),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, oauth_client_secret, "rabbit_secret"]),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, oauth_provider_url, "http://localhost:8080/uaa"]),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_auth_backend_oauth2, resource_server_id, "rabbitmq"]),
+    rpc(Config, application, set_env, [rabbitmq_management, oauth_client_id, "rabbit_user"]),
+    rpc(Config, application, set_env, [rabbitmq_management, oauth_client_secret, "rabbit_secret"]),
+    rpc(Config, application, set_env, [rabbitmq_management, oauth_provider_url, "http://localhost:8080/uaa"]),
+    rpc(Config, application, set_env, [rabbitmq_auth_backend_oauth2, resource_server_id, "rabbitmq"]),
     Map3 = http_get(Config, "/auth", ?OK),
     println(Map3),
     ?assertEqual(true, maps:get(oauth_enabled, Map3)),
@@ -3569,8 +3628,7 @@ oauth_test(Config) ->
     ?assertEqual(<<"rabbitmq">>, maps:get(resource_server_id, Map3)),
     ?assertEqual(<<"http://localhost:8080/uaa">>, maps:get(oauth_provider_url, Map3)),
     %% cleanup
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, unset_env,
-                                 [rabbitmq_management, oauth_enabled]).
+    rpc(Config, application, unset_env, [rabbitmq_management, oauth_enabled]).
 
 login_test(Config) ->
     http_put(Config, "/users/myuser", [{password, <<"myuser">>},
@@ -3611,8 +3669,7 @@ csp_headers_test(Config) ->
     ?assert(lists:keymember("content-security-policy", 1, HdGetCsp1)).
 
 disable_basic_auth_test(Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, disable_basic_auth, true]),
+    rpc(Config, application, set_env, [rabbitmq_management, disable_basic_auth, true]),
     http_get(Config, "/overview", ?NOT_AUTHORISED),
 
     %% Ensure that a request without auth header does not return a basic auth prompt
@@ -3629,13 +3686,12 @@ disable_basic_auth_test(Config) ->
     http_delete(Config, "/queues/%2F/myqueue", ?NOT_AUTHORISED),
     http_get(Config, "/definitions", ?NOT_AUTHORISED),
     http_post(Config, "/definitions", [], ?NOT_AUTHORISED),
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management, disable_basic_auth, 50]),
+    rpc(Config, application, set_env, [rabbitmq_management, disable_basic_auth, 50]),
     %% Defaults to 'false' when config is invalid
     http_get(Config, "/overview", ?OK).
 
 auth_attempts_test(Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_core_metrics, reset_auth_attempt_metrics, []),
+    rpc(Config, rabbit_core_metrics, reset_auth_attempt_metrics, []),
     {Conn, _Ch} = open_connection_and_channel(Config),
     close_connection(Conn),
     [NodeData] = http_get(Config, "/nodes"),
@@ -3654,8 +3710,7 @@ auth_attempts_test(Config) ->
     ?assertEqual(2, maps:get(auth_attempts_succeeded, Http)),
     ?assertEqual(0, maps:get(auth_attempts_failed, Http)),
 
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbit, track_auth_attempt_source, true]),
+    rpc(Config, application, set_env, [rabbit, track_auth_attempt_source, true]),
     {Conn2, _Ch2} = open_connection_and_channel(Config),
     close_connection(Conn2),
     Map2 = http_get(Config, "/auth/attempts/" ++ atom_to_list(Node) ++ "/source", ?OK),
@@ -3692,10 +3747,9 @@ auth_attempts_test(Config) ->
 
 
 config_environment_test(Config) ->
-    rabbit_ct_broker_helpers:rpc(Config, 0, application, set_env,
-                                 [rabbitmq_management,
-                                  config_environment_test_env,
-                                  config_environment_test_value]),
+    rpc(Config, application, set_env, [rabbitmq_management,
+                                       config_environment_test_env,
+                                       config_environment_test_value]),
     ResultString = http_get_no_decode(Config, "/config/effective",
                                       "guest", "guest", ?OK),
     CleanString = re:replace(ResultString, "\\s+", "", [global,{return,list}]),
@@ -3724,9 +3778,7 @@ list_deprecated_features_test(Config) ->
                            deprecation_phase => permitted_by_default,
                            desc => Desc,
                            doc_url => DocUrl}},
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_feature_flags,
-                                      inject_test_feature_flags,
-                                      [FeatureFlags]),
+    ok = rpc(Config, rabbit_feature_flags, inject_test_feature_flags, [FeatureFlags]),
     Result = http_get(Config, "/deprecated-features", ?OK),
     Features = lists:filter(fun(Map) ->
                                     maps:get(name, Map) == atom_to_binary(?FUNCTION_NAME)
@@ -3747,9 +3799,7 @@ list_used_deprecated_features_test(Config) ->
                            desc => Desc,
                            doc_url => DocUrl,
                            callbacks => #{is_feature_used => {rabbit_mgmt_wm_deprecated_features, feature_is_used}}}},
-    ok = rabbit_ct_broker_helpers:rpc(Config, 0, rabbit_feature_flags,
-                                      inject_test_feature_flags,
-                                      [FeatureFlags]),
+    ok = rpc(Config, rabbit_feature_flags, inject_test_feature_flags, [FeatureFlags]),
     Result = http_get(Config, "/deprecated-features/used", ?OK),
     Features = lists:filter(fun(Map) ->
                                     maps:get(name, Map) == atom_to_binary(?FUNCTION_NAME)
diff --git a/deps/rabbitmq_management_agent/src/rabbit_mgmt_format.erl b/deps/rabbitmq_management_agent/src/rabbit_mgmt_format.erl
index e142db3e13f6..9b7e806bc8fc 100644
--- a/deps/rabbitmq_management_agent/src/rabbit_mgmt_format.erl
+++ b/deps/rabbitmq_management_agent/src/rabbit_mgmt_format.erl
@@ -214,6 +214,8 @@ protocol(unknown) ->
     unknown;
 protocol(Version = {_Major, _Minor, _Revision}) ->
     protocol({'AMQP', Version});
+protocol(Version = {1, 0}) ->
+    protocol({'AMQP', Version});
 protocol({Family, Version}) ->
     print("~ts ~ts", [Family, protocol_version(Version)]);
 protocol(Protocol) when is_binary(Protocol) ->
diff --git a/deps/rabbitmq_mqtt/BUILD.bazel b/deps/rabbitmq_mqtt/BUILD.bazel
index 05c38cac6ac2..332753152c01 100644
--- a/deps/rabbitmq_mqtt/BUILD.bazel
+++ b/deps/rabbitmq_mqtt/BUILD.bazel
@@ -128,7 +128,6 @@ broker_for_integration_suites(
         "//deps/rabbitmq_management:erlang_app",
         "//deps/rabbitmq_web_mqtt:erlang_app",
         "//deps/rabbitmq_consistent_hash_exchange:erlang_app",
-        "//deps/rabbitmq_amqp1_0:erlang_app",
         "//deps/rabbitmq_stomp:erlang_app",
         "//deps/rabbitmq_stream:erlang_app",
     ],
diff --git a/deps/rabbitmq_mqtt/Makefile b/deps/rabbitmq_mqtt/Makefile
index 0d362c237461..c6db3954d6d3 100644
--- a/deps/rabbitmq_mqtt/Makefile
+++ b/deps/rabbitmq_mqtt/Makefile
@@ -45,7 +45,7 @@ export BUILD_WITHOUT_QUIC
 
 LOCAL_DEPS = ssl
 DEPS = ranch rabbit_common rabbit ra amqp10_common
-TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management rabbitmq_web_mqtt amqp_client rabbitmq_consistent_hash_exchange rabbitmq_amqp1_0 amqp10_client rabbitmq_stomp rabbitmq_stream
+TEST_DEPS = emqtt ct_helper rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_management rabbitmq_web_mqtt amqp_client rabbitmq_consistent_hash_exchange amqp10_client rabbitmq_stomp rabbitmq_stream
 
 dep_ct_helper = git https://github.com/extend/ct_helper.git master
 dep_emqtt = git https://github.com/rabbitmq/emqtt.git master
diff --git a/deps/rabbitmq_mqtt/app.bzl b/deps/rabbitmq_mqtt/app.bzl
index 55573cd74b30..44db6fc9d2e1 100644
--- a/deps/rabbitmq_mqtt/app.bzl
+++ b/deps/rabbitmq_mqtt/app.bzl
@@ -360,5 +360,5 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
         outs = ["test/protocol_interop_SUITE.beam"],
         app_name = "rabbitmq_mqtt",
         erlc_opts = "//:test_erlc_opts",
-        deps = ["//deps/amqp_client:erlang_app", "//deps/rabbitmq_stomp:erlang_app"],
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/amqp_client:erlang_app", "//deps/rabbitmq_stomp:erlang_app"],
     )
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl
index c721ef169bea..cbcf6159d079 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_confirms.erl
@@ -18,12 +18,8 @@
          size/1,
          contains/2]).
 
-%% As done in OTP's sets module:
-%% Empty list is cheaper to serialize than atom.
--define(VALUE, []).
-
 -type queue_name() :: rabbit_amqqueue:name().
--opaque state() :: #{packet_id() => #{queue_name() => ?VALUE}}.
+-opaque state() :: #{packet_id() => #{queue_name() => ok}}.
 -export_type([state/0]).
 
 -spec init() -> state().
@@ -43,7 +39,7 @@ insert(PktId, QNames, State)
   when is_integer(PktId) andalso
        PktId > 0 andalso
        not is_map_key(PktId, State) ->
-    QMap = maps:from_keys(QNames, ?VALUE),
+    QMap = maps:from_keys(QNames, ok),
     maps:put(PktId, QMap, State).
 
 -spec confirm([packet_id()], queue_name(), state()) ->
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
index 1e1197c6548a..45cfc3676091 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_processor.erl
@@ -144,9 +144,9 @@ process_connect(
                "protocol version: ~p, keepalive: ~p, property names: ~p",
                [ClientId0, Username0, CleanStart, ProtoVer, KeepaliveSecs, maps:keys(ConnectProps)]),
     SslLoginName = ssl_login_name(Socket),
-    Flow = case rabbit_misc:get_env(rabbit, mirroring_flow_control, true) of
-               true   -> flow;
-               false  -> noflow
+    Flow = case application:get_env(rabbit, mirroring_flow_control) of
+               {ok, true} -> flow;
+               {ok, false} -> noflow
            end,
     MaxPacketSize = maps:get('Maximum-Packet-Size', ConnectProps, ?MAX_PACKET_SIZE),
     TopicAliasMax = persistent_term:get(?PERSISTENT_TERM_TOPIC_ALIAS_MAXIMUM),
@@ -1054,7 +1054,7 @@ check_vhost_connection_limit(VHost) ->
 
 check_vhost_alive(VHost) ->
     case rabbit_vhost_sup_sup:is_vhost_alive(VHost) of
-        true  ->
+        true ->
             ok;
         false ->
             ?LOG_ERROR("MQTT connection failed: vhost '~s' is down", [VHost]),
@@ -1445,7 +1445,7 @@ consume(Q, QoS, #state{
                              channel_pid => self(),
                              limiter_pid => none,
                              limiter_active => false,
-                             prefetch_count => Prefetch,
+                             mode => {simple_prefetch, Prefetch},
                              consumer_tag => ?CONSUMER_TAG,
                              exclusive_consume => false,
                              args => [],
@@ -1711,7 +1711,8 @@ send_disconnect(_, _) ->
     ok.
 
 -spec terminate(boolean(), rabbit_event:event_props(), state()) -> ok.
-terminate(SendWill, Infos, State) ->
+terminate(SendWill, Infos, State = #state{queue_states = QStates}) ->
+    rabbit_queue_type:close(QStates),
     rabbit_core_metrics:connection_closed(self()),
     rabbit_event:notify(connection_closed, Infos),
     rabbit_networking:unregister_non_amqp_connection(self()),
diff --git a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl
index aeb8f74d07ea..73885ddba2e2 100644
--- a/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl
+++ b/deps/rabbitmq_mqtt/src/rabbit_mqtt_qos0_queue.erl
@@ -51,7 +51,8 @@
          cancel/5,
          handle_event/3,
          settle/5,
-         credit/5,
+         credit_v1/5,
+         credit/7,
          dequeue/5,
          state_info/1
         ]).
@@ -277,9 +278,12 @@ handle_event(A1,A2,A3) ->
 settle(A1,A2,A3,A4,A5) ->
     ?UNSUPPORTED([A1,A2,A3,A4,A5]).
 
-credit(A1,A2,A3,A4,A5) ->
+credit_v1(A1,A2,A3,A4,A5) ->
     ?UNSUPPORTED([A1,A2,A3,A4,A5]).
 
+credit(A1,A2,A3,A4,A5,A6,A7) ->
+    ?UNSUPPORTED([A1,A2,A3,A4,A5,A6,A7]).
+
 dequeue(A1,A2,A3,A4,A5) ->
     ?UNSUPPORTED([A1,A2,A3,A4,A5]).
 
diff --git a/deps/rabbitmq_mqtt/test/auth_SUITE.erl b/deps/rabbitmq_mqtt/test/auth_SUITE.erl
index 28b3bc1c1261..b0dafad084e8 100644
--- a/deps/rabbitmq_mqtt/test/auth_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/auth_SUITE.erl
@@ -1092,7 +1092,8 @@ vhost_connection_limit(Config) ->
     unlink(C3),
     ?assertMatch({error, {ExpectedError, _}}, emqtt:connect(C3)),
     ok = emqtt:disconnect(C1),
-    ok = emqtt:disconnect(C2).
+    ok = emqtt:disconnect(C2),
+    ok = rabbit_ct_broker_helpers:clear_vhost_limit(Config, 0, <<"/">>).
 
 vhost_queue_limit(Config) ->
     ok = rabbit_ct_broker_helpers:set_vhost_limit(Config, 0, <<"/">>, max_queues, 1),
@@ -1105,7 +1106,8 @@ vhost_queue_limit(Config) ->
                  emqtt:subscribe(C, [{<<"topic1">>, qos0},
                                      {<<"topic2">>, qos1},
                                      {<<"topic3">>, qos1}])),
-    ok = assert_connection_closed(C).
+    ok = assert_connection_closed(C),
+    ok = rabbit_ct_broker_helpers:clear_vhost_limit(Config, 0, <<"/">>).
 
 user_connection_limit(Config) ->
     DefaultUser = <<"guest">>,
diff --git a/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl b/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl
index 2a50408692b2..230c3cc215d3 100644
--- a/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/protocol_interop_SUITE.erl
@@ -15,6 +15,7 @@
 -include_lib("common_test/include/ct.hrl").
 -include_lib("eunit/include/eunit.hrl").
 -include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("amqp10_common/include/amqp10_framing.hrl").
 -include_lib("rabbitmq_stomp/include/rabbit_stomp_frame.hrl").
 
 -import(util,
@@ -59,8 +60,7 @@ init_per_group(_Group, Config0) ->
                rabbit_ct_client_helpers:setup_steps()),
     ok = rabbit_ct_broker_helpers:enable_feature_flag(Config, mqtt_v5),
 
-    Plugins = [rabbitmq_amqp1_0,
-               rabbitmq_stomp,
+    Plugins = [rabbitmq_stomp,
                rabbitmq_stream],
     [ok = rabbit_ct_broker_helpers:enable_plugin(Config, 0, Plugin) || Plugin <- Plugins],
     Config.
@@ -171,8 +171,8 @@ amqp(Config) ->
     Correlation = <<"some correlation ID">>,
     ContentType = <<"text/plain">>,
     RequestPayload = <<"my request">>,
-    UserProperty = [{<<"rabbit🐇"/utf8>>, <<"carrot🥕"/utf8>>},
-                    {<<"x-rabbit🐇"/utf8>>, <<"carrot🥕"/utf8>>},
+    UserProperty = [{<<"🐇"/utf8>>, <<"🥕"/utf8>>},
+                    {<<"x-🐇"/utf8>>, <<"🥕"/utf8>>},
                     {<<"key">>, <<"val">>},
                     {<<"key">>, <<"val">>},
                     {<<"x-key">>, <<"val">>},
@@ -181,22 +181,38 @@ amqp(Config) ->
                             #{'Content-Type' => ContentType,
                               'Correlation-Data' => Correlation,
                               'Response-Topic' => MqttResponseTopic,
-                              'User-Property' => UserProperty},
+                              'User-Property' => UserProperty,
+                              'Payload-Format-Indicator' => 1},
                             RequestPayload, [{qos, 1}]),
 
-
-    %% As of 3.13, AMQP 1.0 is proxied via AMQP 0.9.1 and therefore the conversion from
-    %% mc_mqtt to mc_amqpl takes place. We therefore lose MQTT User Property and Response Topic
-    %% which gets converted to AMQP 0.9.1 headers. In the future, Native AMQP 1.0 will convert
-    %% from mc_mqtt to mc_amqp allowing us to do many more assertions here.
     {ok, Msg1} = amqp10_client:get_msg(Receiver),
     ct:pal("Received AMQP 1.0 message:~n~p", [Msg1]),
-    ?assertEqual([RequestPayload], amqp10_msg:body(Msg1)),
-    ?assertMatch(#{correlation_id := Correlation,
-                   content_type := ContentType}, amqp10_msg:properties(Msg1)),
+
     ?assert(amqp10_msg:header(durable, Msg1)),
     ?assert(amqp10_msg:header(first_acquirer, Msg1)),
 
+    %% We expect to receive x-headers in message annotations.
+    %% However, since annotation keys are symbols and symbols are only valid ASCII,
+    %% we expect header
+    %% {<<"x-🐇"/utf8>>, <<"🥕"/utf8>>}
+    %% to be dropped.
+    ?assertEqual(#{<<"x-key">> => <<"val">>,
+                   <<"x-exchange">> => <<"amq.topic">>,
+                   <<"x-routing-key">> => <<"topic.1">>},
+                 amqp10_msg:message_annotations(Msg1)),
+    %% In contrast, application property keys are of type string, and therefore UTF-8 encoded.
+    ?assertEqual(#{<<"🐇"/utf8>> => <<"🥕"/utf8>>,
+                   <<"key">> => <<"val">>},
+                 amqp10_msg:application_properties(Msg1)),
+
+    #{correlation_id := Correlation,
+      content_type := ContentType,
+      reply_to := ReplyToAddress} = amqp10_msg:properties(Msg1),
+    ?assertEqual(<<"/exchange/amq.topic/response.topic">>, ReplyToAddress),
+
+    %% Thanks to the 'Payload-Format-Indicator', we get a single utf8 value.
+    ?assertEqual(#'v1_0.amqp_value'{content = {utf8, RequestPayload}}, amqp10_msg:body(Msg1)),
+
     ok = amqp10_client:settle_msg(Receiver, Msg1, accepted),
     ok = amqp10_client:detach_link(Receiver),
     ok = amqp10_client:end_session(Session1),
@@ -207,15 +223,14 @@ amqp(Config) ->
     {ok, Session2} = amqp10_client:begin_session(Connection2),
     SenderLinkName = <<"test-sender">>,
     {ok, Sender} = amqp10_client:attach_sender_link(
-                     %% With Native AMQP 1.0, address should be read from received reply-to
-                     Session2, SenderLinkName, <<"/topic/response.topic">>, unsettled),
+                     Session2, SenderLinkName, ReplyToAddress, unsettled),
     receive {amqp10_event, {link, Sender, credited}} -> ok
     after 1000 -> ct:fail(credited_timeout)
     end,
 
     DTag = <<"my-dtag">>,
     ReplyPayload = <<"my response">>,
-    Msg2a = amqp10_msg:new(DTag, ReplyPayload),
+    Msg2a = amqp10_msg:new(DTag, #'v1_0.amqp_value'{content = {utf8, ReplyPayload}}),
     Msg2b = amqp10_msg:set_properties(
               #{correlation_id => Correlation,
                 content_type => ContentType},
@@ -241,7 +256,9 @@ amqp(Config) ->
                      payload := ReplyPayload,
                      properties := #{'Content-Type' := ContentType,
                                      'Correlation-Data' := Correlation,
-                                     'Subscription-Identifier' := 999}},
+                                     'Subscription-Identifier' := 999,
+                                     %% since the AMQP 1.0 client sent UTF-8
+                                     'Payload-Format-Indicator' := 1}},
                    MqttMsg)
     after 1000 -> ct:fail("did not receive reply")
     end,
diff --git a/deps/rabbitmq_mqtt/test/reader_SUITE.erl b/deps/rabbitmq_mqtt/test/reader_SUITE.erl
index 8e8067ddcdb6..59abae0fc149 100644
--- a/deps/rabbitmq_mqtt/test/reader_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/reader_SUITE.erl
@@ -96,6 +96,7 @@ block_connack_timeout(Config) ->
     P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
     Ports = rpc(Config, erlang, ports, []),
 
+    DefaultWatermark = rpc(Config, vm_memory_monitor, get_vm_memory_high_watermark, []),
     ok = rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0]),
     %% Let connection block.
     timer:sleep(100),
@@ -117,7 +118,7 @@ block_connack_timeout(Config) ->
     MqttReaderMRef = monitor(process, MqttReader),
 
     %% Unblock connection. CONNECT packet will be processed on the server.
-    rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [0.4]),
+    rpc(Config, vm_memory_monitor, set_vm_memory_high_watermark, [DefaultWatermark]),
 
     receive {'DOWN', MqttReaderMRef, process, MqttReader, {shutdown, {socket_ends, einval}}} ->
                 %% We expect that MQTT reader process exits (without crashing)
@@ -151,15 +152,13 @@ handle_invalid_packets(Config) ->
     ?assertEqual(N, rpc(Config, ets, info, [connection_metrics, size])).
 
 login_timeout(Config) ->
-    rpc(Config, application, set_env, [rabbitmq_mqtt, login_timeout, 400]),
-    P = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
-    {ok, C} = gen_tcp:connect("localhost", P, [{active, false}]),
-
-    try
-        {error, closed} = gen_tcp:recv(C, 0, 500)
-    after
-        rpc(Config, application, unset_env, [rabbitmq_mqtt, login_timeout])
-    end.
+    App = rabbitmq_mqtt,
+    Par = ?FUNCTION_NAME,
+    ok = rpc(Config, application, set_env, [App, Par, 400]),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_mqtt),
+    {ok, Socket} = gen_tcp:connect("localhost", Port, [{active, false}]),
+    ?assertEqual({error, closed}, gen_tcp:recv(Socket, 0, 500)),
+    ok = rpc(Config, application, unset_env, [App, Par]).
 
 stats(Config) ->
     C = connect(?FUNCTION_NAME, Config),
diff --git a/deps/rabbitmq_mqtt/test/shared_SUITE.erl b/deps/rabbitmq_mqtt/test/shared_SUITE.erl
index e3e82b9b3da9..5d0e3ee3f150 100644
--- a/deps/rabbitmq_mqtt/test/shared_SUITE.erl
+++ b/deps/rabbitmq_mqtt/test/shared_SUITE.erl
@@ -1268,7 +1268,7 @@ management_plugin_connection(Config) ->
        name := ConnectionName}] = http_get(Config, "/connections"),
     process_flag(trap_exit, true),
     http_delete(Config,
-                "/connections/" ++ binary_to_list(uri_string:quote((ConnectionName))),
+                "/connections/" ++ binary_to_list(uri_string:quote(ConnectionName)),
                 ?NO_CONTENT),
     await_exit(C1),
     ?assertEqual([], http_get(Config, "/connections")),
@@ -1386,7 +1386,7 @@ keepalive(Config) ->
 
     await_exit(C1),
     assert_v5_disconnect_reason_code(Config, ?RC_KEEP_ALIVE_TIMEOUT),
-    true = rpc(Config, meck, validate, [Mod]),
+    ?assert(rpc(Config, meck, validate, [Mod])),
     ok = rpc(Config, meck, unload, [Mod]),
 
     C2 = connect(<<"client2">>, Config),
@@ -1415,7 +1415,7 @@ keepalive_turned_off(Config) ->
 
     rabbit_ct_helpers:consistently(?_assert(erlang:is_process_alive(C))),
 
-    true = rpc(Config, meck, validate, [Mod]),
+    ?assert(rpc(Config, meck, validate, [Mod])),
     ok = rpc(Config, meck, unload, [Mod]),
     ok = emqtt:disconnect(C).
 
diff --git a/deps/rabbitmq_shovel/BUILD.bazel b/deps/rabbitmq_shovel/BUILD.bazel
index 4836f3017683..5a876af1b420 100644
--- a/deps/rabbitmq_shovel/BUILD.bazel
+++ b/deps/rabbitmq_shovel/BUILD.bazel
@@ -101,7 +101,6 @@ rabbitmq_home(
     name = "broker-for-tests-home",
     plugins = [
         "//deps/rabbit:erlang_app",
-        "//deps/rabbitmq_amqp1_0:erlang_app",
         ":erlang_app",
     ],
 )
diff --git a/deps/rabbitmq_shovel/Makefile b/deps/rabbitmq_shovel/Makefile
index df31371fb528..d21c0169982f 100644
--- a/deps/rabbitmq_shovel/Makefile
+++ b/deps/rabbitmq_shovel/Makefile
@@ -23,7 +23,7 @@ dep_amqp10_client = git https://github.com/rabbitmq/rabbitmq-amqp1.0-client.git
 
 LOCAL_DEPS = crypto
 
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_amqp1_0 meck
+TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers meck
 
 DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
 DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk elvis_mk
diff --git a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
index e979dce2d143..147626de7f72 100644
--- a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
@@ -101,7 +101,12 @@ connect_dest(State = #{name := Name,
                                      uri => Uri}}}.
 
 connect(Name, AckMode, Uri, Postfix, Addr, Map, AttachFun) ->
-    {ok, Config} = amqp10_client:parse_uri(Uri),
+    {ok, Config0} = amqp10_client:parse_uri(Uri),
+    %% As done for AMQP 0.9.1, exclude AMQP 1.0 shovel connections from maintenance mode
+    %% to prevent crashes and errors being logged by the shovel plugin when a node gets drained.
+    %% A better solution would be that the shovel plugin subscribes to event
+    %% maintenance_connections_closed to gracefully transfer shovels over to other live nodes.
+    Config = Config0#{properties => #{<<"ignore-maintenance">> => {boolean, true}}},
     {ok, Conn} = amqp10_client:open_connection(Config),
     {ok, Sess} = amqp10_client:begin_session(Conn),
     link(Conn),
@@ -125,13 +130,13 @@ connect(Name, AckMode, Uri, Postfix, Addr, Map, AttachFun) ->
 -spec init_source(state()) -> state().
 init_source(State = #{source := #{current := #{link := Link},
                                   prefetch_count := Prefetch} = Src}) ->
-    {Credit, RenewAfter} = case Src of
-                               #{delete_after := R} when is_integer(R) ->
-                                   {R, never};
-                               #{prefetch_count := Pre} ->
-                                   {Pre, round(Prefetch/10)}
-                           end,
-    ok = amqp10_client:flow_link_credit(Link, Credit, RenewAfter),
+    {Credit, RenewWhenBelow} = case Src of
+                                   #{delete_after := R} when is_integer(R) ->
+                                       {R, never};
+                                   #{prefetch_count := Pre} ->
+                                       {Pre, max(1, round(Prefetch/10))}
+                               end,
+    ok = amqp10_client:flow_link_credit(Link, Credit, RenewWhenBelow),
     Remaining = case Src of
                     #{delete_after := never} -> unlimited;
                     #{delete_after := Rem} -> Rem;
diff --git a/deps/rabbitmq_shovel_management/BUILD.bazel b/deps/rabbitmq_shovel_management/BUILD.bazel
index dafb102efb5e..a3b7b90b0232 100644
--- a/deps/rabbitmq_shovel_management/BUILD.bazel
+++ b/deps/rabbitmq_shovel_management/BUILD.bazel
@@ -85,7 +85,6 @@ rabbitmq_home(
     name = "broker-for-tests-home",
     plugins = [
         "//deps/rabbit:erlang_app",
-        "//deps/rabbitmq_amqp1_0:erlang_app",
         ":erlang_app",
     ],
 )
diff --git a/deps/rabbitmq_shovel_management/Makefile b/deps/rabbitmq_shovel_management/Makefile
index 8fc599c1e45a..1b5f98d02936 100644
--- a/deps/rabbitmq_shovel_management/Makefile
+++ b/deps/rabbitmq_shovel_management/Makefile
@@ -6,7 +6,7 @@ define PROJECT_APP_EXTRA_KEYS
 endef
 
 DEPS = rabbit_common rabbit rabbitmq_management rabbitmq_shovel
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers rabbitmq_amqp1_0 meck
+TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers meck
 
 DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
 DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk
diff --git a/deps/rabbitmq_stomp/src/rabbit_stomp_client_sup.erl b/deps/rabbitmq_stomp/src/rabbit_stomp_client_sup.erl
index f1cda7709814..b039030b11a1 100644
--- a/deps/rabbitmq_stomp/src/rabbit_stomp_client_sup.erl
+++ b/deps/rabbitmq_stomp/src/rabbit_stomp_client_sup.erl
@@ -15,12 +15,16 @@
 
 start_link(Ref, _Transport, Configuration) ->
     {ok, SupPid} = supervisor:start_link(?MODULE, []),
+    ConnectionHelperSupFlags = #{strategy => one_for_one,
+                                 intensity => 10,
+                                 period => 10,
+                                 auto_shutdown => any_significant},
     {ok, HelperPid} =
         supervisor:start_child(
             SupPid,
             #{
                 id => rabbit_stomp_heartbeat_sup,
-                start => {rabbit_connection_helper_sup, start_link, []},
+                start => {rabbit_connection_helper_sup, start_link, [ConnectionHelperSupFlags]},
                 restart => transient,
                 significant => true,
                 shutdown => infinity,
diff --git a/deps/rabbitmq_stomp/src/rabbit_stomp_processor.erl b/deps/rabbitmq_stomp/src/rabbit_stomp_processor.erl
index 28228974170e..9e4129ee8d2d 100644
--- a/deps/rabbitmq_stomp/src/rabbit_stomp_processor.erl
+++ b/deps/rabbitmq_stomp/src/rabbit_stomp_processor.erl
@@ -506,7 +506,7 @@ tidy_canceled_subscription(ConsumerTag, _Subscription,
 tidy_canceled_subscription(ConsumerTag, #subscription{dest_hdr = DestHdr},
                            Frame, State = #proc_state{subscriptions = Subs}) ->
     Subs1 = maps:remove(ConsumerTag, Subs),
-    {ok, Dest} = rabbit_routing_util:parse_endpoint(DestHdr),
+    {ok, Dest} = rabbit_routing_parser:parse_endpoint(DestHdr),
     maybe_delete_durable_sub(Dest, Frame, State#proc_state{subscriptions = Subs1}).
 
 maybe_delete_durable_sub({topic, Name}, Frame,
@@ -528,7 +528,7 @@ maybe_delete_durable_sub(_Destination, _Frame, State) ->
 with_destination(Command, Frame, State, Fun) ->
     case rabbit_stomp_frame:header(Frame, ?HEADER_DESTINATION) of
         {ok, DestHdr} ->
-            case rabbit_routing_util:parse_endpoint(DestHdr) of
+            case rabbit_routing_parser:parse_endpoint(DestHdr) of
                 {ok, Destination} ->
                     case Fun(Destination, DestHdr, Frame, State) of
                         {error, invalid_endpoint} ->
@@ -902,7 +902,7 @@ ensure_reply_to(Frame = #stomp_frame{headers = Headers}, State) ->
         not_found ->
             {Frame, State};
         {ok, ReplyTo} ->
-            {ok, Destination} = rabbit_routing_util:parse_endpoint(ReplyTo),
+            {ok, Destination} = rabbit_routing_parser:parse_endpoint(ReplyTo),
             case rabbit_routing_util:dest_temp_queue(Destination) of
                 none ->
                     {Frame, State};
@@ -1126,7 +1126,7 @@ ensure_endpoint(source, EndPoint, {_, _, Headers, _} = Frame, Channel, State) ->
               Id = build_subscription_id(Frame),
               % Note: we discard the exchange here so there's no need to use
               % the default_topic_exchange configuration key
-              {_, Name} = rabbit_routing_util:parse_routing(EndPoint),
+              {_, Name} = rabbit_routing_parser:parse_routing(EndPoint),
               list_to_binary(rabbit_stomp_util:subscription_queue_name(Name, Id, Frame))
           end
          }] ++ rabbit_stomp_util:build_params(EndPoint, Headers),
@@ -1230,7 +1230,7 @@ additional_info(Key,
     proplists:get_value(Key, AddInfo).
 
 parse_routing(Destination, DefaultTopicExchange) ->
-    {Exchange0, RoutingKey} = rabbit_routing_util:parse_routing(Destination),
+    {Exchange0, RoutingKey} = rabbit_routing_parser:parse_routing(Destination),
     Exchange1 = maybe_apply_default_topic_exchange(Exchange0, DefaultTopicExchange),
     {Exchange1, RoutingKey}.
 
diff --git a/deps/rabbitmq_stream/BUILD.bazel b/deps/rabbitmq_stream/BUILD.bazel
index a8edfc3393d7..a4ba9852a78c 100644
--- a/deps/rabbitmq_stream/BUILD.bazel
+++ b/deps/rabbitmq_stream/BUILD.bazel
@@ -100,7 +100,8 @@ eunit(
     target = ":test_erlang_app",
 )
 
-broker_for_integration_suites()
+broker_for_integration_suites(
+)
 
 rabbitmq_integration_suite(
     name = "commands_SUITE",
@@ -142,6 +143,14 @@ rabbitmq_integration_suite(
     ],
 )
 
+rabbitmq_integration_suite(
+    name = "protocol_interop_SUITE",
+    size = "medium",
+    runtime_deps = [
+        "//deps/amqp10_client:erlang_app",
+    ],
+)
+
 assert_suites()
 
 alias(
diff --git a/deps/rabbitmq_stream/Makefile b/deps/rabbitmq_stream/Makefile
index e63a7ab94733..549e590ff10a 100644
--- a/deps/rabbitmq_stream/Makefile
+++ b/deps/rabbitmq_stream/Makefile
@@ -23,7 +23,7 @@ endef
 
 LOCAL_DEPS = ssl
 DEPS = rabbit rabbitmq_stream_common osiris ranch
-TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client
+TEST_DEPS = rabbitmq_ct_helpers rabbitmq_ct_client_helpers amqp_client amqp10_client
 
 DEP_EARLY_PLUGINS = rabbit_common/mk/rabbitmq-early-plugin.mk
 DEP_PLUGINS = rabbit_common/mk/rabbitmq-plugin.mk
diff --git a/deps/rabbitmq_stream/app.bzl b/deps/rabbitmq_stream/app.bzl
index 38c9c7e228bf..0f0b0b5153b3 100644
--- a/deps/rabbitmq_stream/app.bzl
+++ b/deps/rabbitmq_stream/app.bzl
@@ -189,3 +189,12 @@ def test_suite_beam_files(name = "test_suite_beam_files"):
             "//deps/rabbitmq_stream_common:erlang_app",
         ],
     )
+    erlang_bytecode(
+        name = "protocol_interop_SUITE_beam_files",
+        testonly = True,
+        srcs = ["test/protocol_interop_SUITE.erl"],
+        outs = ["test/protocol_interop_SUITE.beam"],
+        app_name = "rabbitmq_stream",
+        erlc_opts = "//:test_erlc_opts",
+        deps = ["//deps/amqp10_common:erlang_app", "//deps/amqp_client:erlang_app"],
+    )
diff --git a/deps/rabbitmq_stream/src/rabbit_stream_reader.hrl b/deps/rabbitmq_stream/src/rabbit_stream_reader.hrl
index 59c4e0b48aa8..6f44c63a7c81 100644
--- a/deps/rabbitmq_stream/src/rabbit_stream_reader.hrl
+++ b/deps/rabbitmq_stream/src/rabbit_stream_reader.hrl
@@ -26,6 +26,8 @@
          stream :: stream(),
          reference :: undefined | publisher_reference(),
          leader :: pid(),
+         %% We do not use atomics here for concurrent access. Instead, we use atomics
+         %% to reduce memory copy overhead for record fields that change often.
          message_counters :: atomics:atomics_ref(),
          %% use to distinguish a stale publisher from a live publisher with the same ID
          %% used only for publishers without a reference (dedup off)
diff --git a/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl b/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl
new file mode 100644
index 000000000000..3ae1fbe603a5
--- /dev/null
+++ b/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl
@@ -0,0 +1,412 @@
+%% 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-2023 VMware, Inc. or its affiliates.  All rights reserved.
+%%
+
+-module(protocol_interop_SUITE).
+
+-compile([export_all,
+          nowarn_export_all]).
+
+-include_lib("common_test/include/ct.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("amqp_client/include/amqp_client.hrl").
+-include_lib("amqp10_common/include/amqp10_framing.hrl").
+
+all() ->
+    [{group, tests}].
+
+groups() ->
+    [{tests, [shuffle],
+      [
+       amqpl,
+       amqp_credit_multiple_grants,
+       amqp_credit_single_grant,
+       amqp_attach_sub_batch
+      ]
+     }].
+
+%% -------------------------------------------------------------------
+%% Testsuite setup/teardown.
+%% -------------------------------------------------------------------
+
+init_per_suite(Config) ->
+    {ok, _} = application:ensure_all_started(amqp10_client),
+    rabbit_ct_helpers:log_environment(),
+    rabbit_ct_helpers:run_setup_steps(Config).
+
+end_per_suite(Config) ->
+    rabbit_ct_helpers:run_teardown_steps(Config).
+
+init_per_group(_Group, Config) ->
+    rabbit_ct_helpers:run_steps(
+      Config,
+      rabbit_ct_broker_helpers:setup_steps() ++
+      rabbit_ct_client_helpers:setup_steps()).
+
+end_per_group(_Group, Config) ->
+    rabbit_ct_helpers:run_steps(
+      Config,
+      rabbit_ct_client_helpers:teardown_steps() ++
+      rabbit_ct_broker_helpers:teardown_steps()).
+
+init_per_testcase(Testcase, Config) ->
+    rabbit_ct_helpers:testcase_started(Config, Testcase).
+
+end_per_testcase(Testcase, Config) ->
+    %% Wait for exclusive or auto-delete queues being deleted.
+    timer:sleep(800),
+    rabbit_ct_broker_helpers:rpc(Config, ?MODULE, delete_queues, []),
+    rabbit_ct_helpers:testcase_finished(Config, Testcase).
+
+%% -------------------------------------------------------------------
+%% Testsuite cases
+%% -------------------------------------------------------------------
+
+amqpl(Config) ->
+    [Server] = rabbit_ct_broker_helpers:get_node_configs(Config, nodename),
+    Ch = rabbit_ct_client_helpers:open_channel(Config, Server),
+    Ctag = Stream = atom_to_binary(?FUNCTION_NAME),
+    publish_via_stream_protocol(Stream, Config),
+
+    #'basic.qos_ok'{} = amqp_channel:call(Ch, #'basic.qos'{prefetch_count = 2}),
+    amqp_channel:subscribe(Ch,
+                           #'basic.consume'{queue = Stream,
+                                            consumer_tag = Ctag,
+                                            arguments = [{<<"x-stream-offset">>, long, 0}]},
+                           self()),
+    receive #'basic.consume_ok'{consumer_tag = Ctag} -> ok
+    after 5000 -> ct:fail(consume_timeout)
+    end,
+
+    %% Since prefetch is 2, we expect to receive exactly 2 messages.
+    %% Whenever we ack both messages, we should receive exactly 2 more messages.
+    ExpectedPayloads = [{<<"m1">>, <<"m2">>},
+                        {<<"m3">>, <<"m4">>},
+                        {<<"m5">>, <<"m6">>},
+                        %% The broker skips delivery of compressed sub batches to non Stream protocol
+                        %% consumers, i.e. skips delivery of m7, m8, m9.
+                        {<<"m10">>, <<"m11">>}],
+    lists:foreach(
+      fun({P1, P2}) ->
+              ok = process_2_amqpl_messages(Ch, P1, P2)
+      end, ExpectedPayloads),
+
+    ok = amqp_channel:close(Ch).
+
+process_2_amqpl_messages(Ch, P1, P2) ->
+    %% We expect to receive exactly 2 messages.
+    receive {#'basic.deliver'{},
+             #amqp_msg{payload = P1}} -> ok
+    after 5000 -> ct:fail({missing_delivery, P1})
+    end,
+    DTag = receive {#'basic.deliver'{delivery_tag = Tag},
+                    #amqp_msg{payload = P2}} -> Tag
+           after 5000 -> ct:fail({missing_delivery, P2})
+           end,
+    receive Msg -> ct:fail({unexpected_message, Msg})
+    after 10 -> ok
+    end,
+    ok = amqp_channel:cast(Ch, #'basic.ack'{delivery_tag = DTag,
+                                            multiple = true}).
+
+amqp_credit_single_grant(Config) ->
+    Stream = atom_to_binary(?FUNCTION_NAME),
+    publish_via_stream_protocol(Stream, Config),
+
+    %% Consume from the stream via AMQP 1.0.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", Stream/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, settled,
+                       configuration, #{<<"rabbitmq:stream-offset-spec">> => <<"first">>}),
+
+    %% There are 8 uncompressed messages in the stream.
+    ok = amqp10_client:flow_link_credit(Receiver, 8, never),
+
+    Msgs = receive_amqp_messages(Receiver, 8),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(hd(Msgs))),
+    ?assertEqual([<<"m11">>], amqp10_msg:body(lists:last(Msgs))),
+    ok = amqp10_client:close_connection(Connection).
+
+amqp_credit_multiple_grants(Config) ->
+    Stream = atom_to_binary(?FUNCTION_NAME),
+    publish_via_stream_protocol(Stream, Config),
+
+    %% Consume from the stream via AMQP 1.0.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", Stream/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, unsettled,
+                       configuration, #{<<"rabbitmq:stream-offset-spec">> => <<"first">>}),
+
+    %% Granting 1 credit should deliver us exactly 1 message.
+    {ok, M1} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"m1">>], amqp10_msg:body(M1)),
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp1 -> ct:fail({unexpected_message, Unexp1})
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:flow_link_credit(Receiver, 3, never),
+    %% We expect to receive exactly 3 more messages
+    receive {amqp10_msg, Receiver, Msg2} ->
+                ?assertEqual([<<"m2">>], amqp10_msg:body(Msg2))
+    after 5000 -> ct:fail("missing m2")
+    end,
+    receive {amqp10_msg, Receiver, Msg3} ->
+                ?assertEqual([<<"m3">>], amqp10_msg:body(Msg3))
+    after 5000 -> ct:fail("missing m3")
+    end,
+    %% Messages in an uncompressed subbatch should be delivered individually.
+    M4 = receive {amqp10_msg, Receiver, Msg4} ->
+                     ?assertEqual([<<"m4">>], amqp10_msg:body(Msg4)),
+                     Msg4
+         after 5000 -> ct:fail("missing m4")
+         end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+
+    %% Let's ack all of them.
+    ok = amqp10_client_session:disposition(
+           Session,
+           receiver,
+           amqp10_msg:delivery_id(M1),
+           amqp10_msg:delivery_id(M4),
+           true,
+           accepted),
+    %% Acking shouldn't grant more credits.
+    receive {amqp10_msg, _, _} = Unexp2 -> ct:fail({unexpected_message, Unexp2})
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:flow_link_credit(Receiver, 3, never),
+    M5 = receive {amqp10_msg, Receiver, Msg5} ->
+                     ?assertEqual([<<"m5">>], amqp10_msg:body(Msg5)),
+                     Msg5
+         after 5000 -> ct:fail("missing m5")
+         end,
+    receive {amqp10_msg, Receiver, Msg6} ->
+                ?assertEqual([<<"m6">>], amqp10_msg:body(Msg6))
+    after 5000 -> ct:fail("missing m6")
+    end,
+    %% The broker skips delivery of compressed sub batches to non Stream protocol
+    %% consumers, i.e. skips delivery of m7, m8, m9.
+    receive {amqp10_msg, Receiver, Msg10} ->
+                ?assertEqual([<<"m10">>], amqp10_msg:body(Msg10))
+    after 5000 -> ct:fail("missing m10")
+    end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+    receive {amqp10_msg, _, _} = Unexp3 -> ct:fail({unexpected_message, Unexp3})
+    after 10 -> ok
+    end,
+
+    %% 1 message should be left in the stream.
+    %% Let's drain the stream.
+    ok = amqp10_client:flow_link_credit(Receiver, 1000, never, true),
+    M11 = receive {amqp10_msg, Receiver, Msg11} ->
+                      ?assertEqual([<<"m11">>], amqp10_msg:body(Msg11)),
+                      Msg11
+          after 5000 -> ct:fail("missing m11")
+          end,
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> ct:fail("expected credit_exhausted")
+    end,
+
+    %% Let's ack them all.
+    ok = amqp10_client_session:disposition(
+           Session,
+           receiver,
+           amqp10_msg:delivery_id(M5),
+           amqp10_msg:delivery_id(M11),
+           true,
+           accepted),
+
+    receive {amqp10_msg, _, _} = Unexp4 -> ct:fail({unexpected_message, Unexp4})
+    after 10 -> ok
+    end,
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:close_connection(Connection).
+
+amqp_attach_sub_batch(Config) ->
+    Stream = atom_to_binary(?FUNCTION_NAME),
+    publish_via_stream_protocol(Stream, Config),
+
+    %% Consume from the stream via AMQP 1.0.
+    OpnConf = connection_config(Config),
+    {ok, Connection} = amqp10_client:open_connection(OpnConf),
+    {ok, Session} = amqp10_client:begin_session_sync(Connection),
+    Address = <<"/amq/queue/", Stream/binary>>,
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"test-receiver">>, Address, settled, configuration,
+                       %% Attach in the middle of an uncompresssed sub batch.
+                       #{<<"rabbitmq:stream-offset-spec">> => 4}),
+
+    {ok, M5} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"m5">>], amqp10_msg:body(M5)),
+
+    {ok, M6} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"m6">>], amqp10_msg:body(M6)),
+
+    %% The broker skips delivery of compressed sub batches to non Stream protocol
+    %% consumers, i.e. skips delivery of m7, m8, m9.
+
+    {ok, M10} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"m10">>], amqp10_msg:body(M10)),
+
+    {ok, M11} = amqp10_client:get_msg(Receiver),
+    ?assertEqual([<<"m11">>], amqp10_msg:body(M11)),
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:close_connection(Connection).
+
+%% -------------------------------------------------------------------
+%% Helpers
+%% -------------------------------------------------------------------
+
+publish_via_stream_protocol(Stream, Config) ->
+    %% There is no open source Erlang RabbitMQ Stream client.
+    %% Therefore, we have to build the Stream protocol commands manually.
+
+    StreamPort = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_stream),
+    {ok, S} = gen_tcp:connect("localhost", StreamPort, [{active, false}, {mode, binary}]),
+
+    C0 = rabbit_stream_core:init(0),
+    PeerPropertiesFrame = rabbit_stream_core:frame({request, 1, {peer_properties, #{}}}),
+    ok = gen_tcp:send(S, PeerPropertiesFrame),
+    {{response, 1, {peer_properties, _, _}}, C1} = receive_stream_commands(S, C0),
+
+    ok = gen_tcp:send(S, rabbit_stream_core:frame({request, 1, sasl_handshake})),
+    {{response, _, {sasl_handshake, _, _}}, C2} = receive_stream_commands(S, C1),
+    Username = <<"guest">>,
+    Password = <<"guest">>,
+    Null = 0,
+    PlainSasl = <>,
+    ok = gen_tcp:send(S, rabbit_stream_core:frame({request, 2, {sasl_authenticate, <<"PLAIN">>, PlainSasl}})),
+    {{response, 2, {sasl_authenticate, _}}, C3} = receive_stream_commands(S, C2),
+    {{tune, DefaultFrameMax, _}, C4} = receive_stream_commands(S, C3),
+
+    ok = gen_tcp:send(S, rabbit_stream_core:frame({response, 0, {tune, DefaultFrameMax, 0}})),
+    ok = gen_tcp:send(S, rabbit_stream_core:frame({request, 3, {open, <<"/">>}})),
+    {{response, 3, {open, _, _ConnectionProperties}}, C5} = receive_stream_commands(S, C4),
+
+    CreateStreamFrame = rabbit_stream_core:frame({request, 1, {create_stream, Stream, #{}}}),
+    ok = gen_tcp:send(S, CreateStreamFrame),
+    {{response, 1, {create_stream, _}}, C6} = receive_stream_commands(S, C5),
+
+    PublisherId = 99,
+    DeclarePublisherFrame = rabbit_stream_core:frame({request, 1, {declare_publisher, PublisherId, <<>>, Stream}}),
+    ok = gen_tcp:send(S, DeclarePublisherFrame),
+    {{response, 1, {declare_publisher, _}}, C7} = receive_stream_commands(S, C6),
+
+    M1 = simple_entry(1, <<"m1">>),
+    M2 = simple_entry(2, <<"m2">>),
+    M3 = simple_entry(3, <<"m3">>),
+    Messages1 = [M1, M2, M3],
+    PublishFrame1 = rabbit_stream_core:frame({publish, PublisherId, length(Messages1), Messages1}),
+    ok = gen_tcp:send(S, PublishFrame1),
+    {{publish_confirm, PublisherId, _}, C8} = receive_stream_commands(S, C7),
+
+    UncompressedSubbatch = sub_batch_entry_uncompressed(4, [<<"m4">>, <<"m5">>, <<"m6">>]),
+    PublishFrame2 = rabbit_stream_core:frame({publish, PublisherId, 3, UncompressedSubbatch}),
+    ok = gen_tcp:send(S, PublishFrame2),
+    {{publish_confirm, PublisherId, _}, C9} = receive_stream_commands(S, C8),
+
+    CompressedSubbatch = sub_batch_entry_compressed(5, [<<"m7">>, <<"m8">>, <<"m9">>]),
+    PublishFrame3 = rabbit_stream_core:frame({publish, PublisherId, 3, CompressedSubbatch}),
+    ok = gen_tcp:send(S, PublishFrame3),
+    {{publish_confirm, PublisherId, _}, C10} = receive_stream_commands(S, C9),
+
+    M10 = simple_entry(6, <<"m10">>),
+    M11 = simple_entry(7, <<"m11">>),
+    Messages2 = [M10, M11],
+    PublishFrame4 = rabbit_stream_core:frame({publish, PublisherId, length(Messages2), Messages2}),
+    ok = gen_tcp:send(S, PublishFrame4),
+    {{publish_confirm, PublisherId, _}, _C11} = receive_stream_commands(S, C10).
+
+%% Streams contain AMQP 1.0 encoded messages.
+%% In this case, the AMQP 1.0 encoded message contains a single data section.
+simple_entry(Sequence, Body)
+  when is_binary(Body) ->
+    DataSect = iolist_to_binary(amqp10_framing:encode_bin(#'v1_0.data'{content = Body})),
+    DataSectSize = byte_size(DataSect),
+    <>.
+
+%% Here, each AMQP 1.0 encoded message contains a single data section.
+%% All data sections are delivered uncompressed in 1 batch.
+sub_batch_entry_uncompressed(Sequence, Bodies) ->
+    Batch = lists:foldl(fun(Body, Acc) ->
+                                Sect = iolist_to_binary(amqp10_framing:encode_bin(#'v1_0.data'{content = Body})),
+                                <>
+                        end, <<>>, Bodies),
+    Size = byte_size(Batch),
+    <>.
+
+%% Here, each AMQP 1.0 encoded message contains a single data section.
+%% All data sections are delivered in 1 gzip compressed batch.
+sub_batch_entry_compressed(Sequence, Bodies) ->
+    Uncompressed = lists:foldl(fun(Body, Acc) ->
+                                       Bin = iolist_to_binary(amqp10_framing:encode_bin(#'v1_0.data'{content = Body})),
+                                       <>
+                               end, <<>>, Bodies),
+    Compressed = zlib:gzip(Uncompressed),
+    CompressedLen = byte_size(Compressed),
+    <>.
+
+connection_config(Config) ->
+    Host = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    #{address => Host,
+      port => Port,
+      container_id => <<"my container">>,
+      sasl => {plain, <<"guest">>, <<"guest">>}}.
+
+receive_stream_commands(Sock, C0) ->
+    case rabbit_stream_core:next_command(C0) of
+        empty ->
+            case gen_tcp:recv(Sock, 0, 5000) of
+                {ok, Data} ->
+                    C1 = rabbit_stream_core:incoming_data(Data, C0),
+                    case rabbit_stream_core:next_command(C1) of
+                        empty ->
+                            {ok, Data2} = gen_tcp:recv(Sock, 0, 5000),
+                            rabbit_stream_core:next_command(
+                              rabbit_stream_core:incoming_data(Data2, C1));
+                        Res ->
+                            Res
+                    end;
+                {error, Err} ->
+                    ct:fail("error receiving stream data ~w", [Err])
+            end;
+        Res ->
+            Res
+    end.
+
+receive_amqp_messages(Receiver, N) ->
+    receive_amqp_messages0(Receiver, N, []).
+
+receive_amqp_messages0(_Receiver, 0, Acc) ->
+    lists:reverse(Acc);
+receive_amqp_messages0(Receiver, N, Acc) ->
+    receive
+        {amqp10_msg, Receiver, Msg} ->
+            receive_amqp_messages0(Receiver, N - 1, [Msg | Acc])
+    after 5000  ->
+              exit({timeout, {num_received, length(Acc)}, {num_missing, N}})
+    end.
+
+delete_queues() ->
+    [{ok, 0} = rabbit_amqqueue:delete(Q, false, false, <<"dummy">>) || Q <- rabbit_amqqueue:list()].
diff --git a/deps/rabbitmq_stream_management/BUILD.bazel b/deps/rabbitmq_stream_management/BUILD.bazel
index 85d04900c941..28844d70fe42 100644
--- a/deps/rabbitmq_stream_management/BUILD.bazel
+++ b/deps/rabbitmq_stream_management/BUILD.bazel
@@ -84,7 +84,6 @@ rabbitmq_home(
     name = "broker-for-tests-home",
     plugins = [
         "//deps/rabbit:erlang_app",
-        "//deps/rabbitmq_amqp1_0:erlang_app",
         ":erlang_app",
     ],
 )
diff --git a/moduleindex.yaml b/moduleindex.yaml
index 563825737b0f..19f69504b60d 100755
--- a/moduleindex.yaml
+++ b/moduleindex.yaml
@@ -33,7 +33,6 @@ amqp10_client:
 - amqp10_client_app
 - amqp10_client_connection
 - amqp10_client_connection_sup
-- amqp10_client_connections_sup
 - amqp10_client_frame_reader
 - amqp10_client_session
 - amqp10_client_sessions_sup
@@ -45,6 +44,7 @@ amqp10_common:
 - amqp10_binary_parser
 - amqp10_framing
 - amqp10_framing0
+- serial_number
 aten:
 - aten
 - aten_app
@@ -543,6 +543,12 @@ rabbit:
 - rabbit
 - rabbit_access_control
 - rabbit_alarm
+- rabbit_amqp1_0
+- rabbit_amqp_reader
+- rabbit_amqp_session
+- rabbit_amqp_session_sup
+- rabbit_amqp_util
+- rabbit_amqp_writer
 - rabbit_amqqueue
 - rabbit_amqqueue_control
 - rabbit_amqqueue_process
@@ -805,6 +811,7 @@ rabbit_common:
 - rabbit_registry
 - rabbit_registry_class
 - rabbit_resource_monitor_misc
+- rabbit_routing_parser
 - rabbit_runtime
 - rabbit_runtime_parameter
 - rabbit_semver
@@ -818,20 +825,7 @@ rabbit_common:
 - worker_pool_sup
 - worker_pool_worker
 rabbitmq_amqp1_0:
-- Elixir.RabbitMQ.CLI.Ctl.Commands.ListAmqp10ConnectionsCommand
-- rabbit_amqp1_0
-- rabbit_amqp1_0_channel
-- rabbit_amqp1_0_incoming_link
-- rabbit_amqp1_0_link_util
-- rabbit_amqp1_0_message
-- rabbit_amqp1_0_outgoing_link
-- rabbit_amqp1_0_reader
-- rabbit_amqp1_0_session
-- rabbit_amqp1_0_session_process
-- rabbit_amqp1_0_session_sup
-- rabbit_amqp1_0_session_sup_sup
-- rabbit_amqp1_0_util
-- rabbit_amqp1_0_writer
+- rabbitmq_amqp1_0_noop
 rabbitmq_auth_backend_cache:
 - rabbit_auth_backend_cache
 - rabbit_auth_backend_cache_app
diff --git a/plugins.mk b/plugins.mk
index 7536c6705ae1..38b946318f05 100644
--- a/plugins.mk
+++ b/plugins.mk
@@ -8,6 +8,7 @@
 PLUGINS := rabbitmq_amqp1_0 \
 	   rabbitmq_auth_backend_cache \
 	   rabbitmq_auth_backend_http \
+	   rabbitmq_auth_backend_http \
 	   rabbitmq_auth_backend_ldap \
 	   rabbitmq_auth_backend_oauth2 \
 	   rabbitmq_auth_mechanism_ssl \

From ab726a0241acbff7159ddfb92adac3d593bd973d Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Wed, 7 Feb 2024 17:38:47 +0100
Subject: [PATCH 02/16] Fix crash

```
./omq amqp -t /queue --amqp-subject foo -C 1
```
crashed with
```
reason: {badarg,
            [{erlang,list_to_binary,
                 [undefined],
             {rabbit_amqp_session,ensure_terminus,5,
                 [{file,"rabbit_amqp_session.erl"},{line,2046}]},
             {rabbit_amqp_session,ensure_target,3,
                 [{file,"rabbit_amqp_session.erl"},{line,1705}]},
             {rabbit_amqp_session,handle_control,2,
                 [{file,"rabbit_amqp_session.erl"},{line,715}]},
             {rabbit_amqp_session,handle_cast,2,
                 [{file,"rabbit_amqp_session.erl"},{line,331}]},
             {gen_server,try_handle_cast,3,
                 [{file,"gen_server.erl"},{line,1121}]},
             {gen_server,handle_msg,6,
                 [{file,"gen_server.erl"},{line,1183}]},
             {proc_lib,init_p_do_apply,3,
                 [{file,"proc_lib.erl"},{line,241}]}]}
```
---
 deps/rabbit/src/rabbit_amqp_session.erl                       | 4 ++++
 .../test/amqp_system_SUITE_data/fsharp-tests/Program.fs       | 1 +
 2 files changed, 5 insertions(+)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 4ffdcb2846ba..f3945d2f000b 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -2042,6 +2042,10 @@ ensure_terminus(target, {topic, _bindingkey}, _, _, _) ->
 ensure_terminus(source, {topic, _BindingKey}, Vhost, User, Durability) ->
     %% exchange amq.topic exists
     declare_queue(generate_queue_name(), Vhost, User, Durability);
+ensure_terminus(target, {queue, undefined}, _, _, _) ->
+    %% Target "/queue" means publish to default exchange with message subject as routing key.
+    %% Default exchange exists.
+    undefined;
 ensure_terminus(_, {queue, QNameList}, Vhost, User, Durability) ->
     declare_queue(list_to_binary(QNameList), Vhost, User, Durability);
 ensure_terminus(_, {amqqueue, QNameList}, Vhost, _, _) ->
diff --git a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
index 67910bfaf148..f61492450dc6 100755
--- a/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
+++ b/deps/rabbit/test/amqp_system_SUITE_data/fsharp-tests/Program.fs
@@ -374,6 +374,7 @@ module Test =
              "/exchange/amq.fanout",  "/exchange/amq.fanout/",  "",  true
              "/exchange/amq.direct",  "/exchange/amq.direct/",  "",  true
              "/exchange/amq.direct",  "/exchange/amq.direct/a", "a", true
+             "/queue",  "/queue/b", "b", true
 
              (* FIXME: The following three tests rely on the queue "test"
               * created by previous tests in this function. *)

From 769de60fafb480b141c1c6ab37b7afacb9d549e2 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Thu, 8 Feb 2024 11:15:05 +0100
Subject: [PATCH 03/16] Set delivery-count in 1st FLOW to client

"In the event that the receiving link endpoint has not yet seen the
initial attach frame from the sender this field MUST NOT be set."
[2.7.4]

Since we (the server / the receiving link endpoint), have already seen
the initial attach frame from the sender, set the delivery-count.
---
 deps/rabbit/src/rabbit_amqp_session.erl | 17 +++++++----------
 1 file changed, 7 insertions(+), 10 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index f3945d2f000b..4ef57b647f46 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -79,7 +79,7 @@
           %% queue_name_bin is only set if the link target address refers to a queue.
           queue_name_bin :: undefined | rabbit_misc:resource_name(),
           delivery_count :: sequence_no(),
-          credit = 0 :: non_neg_integer(),
+          credit :: non_neg_integer(),
           %% TRANSFER delivery IDs published to queues but not yet confirmed by queues
           incoming_unconfirmed_map = #{} :: #{delivery_number() =>
                                               {#{rabbit_amqqueue:name() := ok},
@@ -707,7 +707,8 @@ handle_control(#'v1_0.attach'{role = ?SEND_ROLE,
                               source = Source,
                               snd_settle_mode = SndSettleMode,
                               target = Target,
-                              initial_delivery_count = ?UINT(DeliveryCount)} = Attach,
+                              initial_delivery_count = DeliveryCount = ?UINT(DeliveryCountInt)
+                             } = Attach,
                State0 = #state{incoming_links = IncomingLinks0,
                                cfg = #cfg{vhost = Vhost,
                                           user = User}}) ->
@@ -718,7 +719,7 @@ handle_control(#'v1_0.attach'{role = ?SEND_ROLE,
                               exchange = XName,
                               routing_key = RoutingKey,
                               queue_name_bin = QNameBin,
-                              delivery_count = DeliveryCount,
+                              delivery_count = DeliveryCountInt,
                               credit = ?LINK_CREDIT_RCV},
             _Outcomes = outcomes(Source),
             OutputHandle = output_handle(InputHandle),
@@ -731,14 +732,10 @@ handle_control(#'v1_0.attach'{role = ?SEND_ROLE,
                        target = Target,
                        %% We are the receiver.
                        role = ?RECV_ROLE,
-                       %% "ignored if the role is receiver"
-                       initial_delivery_count = undefined,
                        max_message_size = {ulong, persistent_term:get(max_message_size)}},
-            Flow = #'v1_0.flow'{
-                      handle = OutputHandle,
-                      link_credit = ?UINT(?LINK_CREDIT_RCV),
-                      drain = false,
-                      echo = false},
+            Flow = #'v1_0.flow'{handle = OutputHandle,
+                                delivery_count = DeliveryCount,
+                                link_credit = ?UINT(?LINK_CREDIT_RCV)},
             %%TODO check that handle is not present in either incoming_links or outgoing_links:
             %%"The handle MUST NOT be used for other open links. An attempt to attach
             %% using a handle which is already associated with a link MUST be responded to

From 168b8945e250711948e80f0b1a9352e4a15cb829 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Thu, 8 Feb 2024 18:38:09 +0100
Subject: [PATCH 04/16] Allow no routing key

to be set in both target address and subject.
---
 deps/rabbit/src/rabbit_amqp_session.erl | 26 +++++++++++-------
 deps/rabbit/test/amqp_client_SUITE.erl  | 36 ++++---------------------
 2 files changed, 21 insertions(+), 41 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 4ef57b647f46..5c2419e9f53d 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -1592,10 +1592,9 @@ incoming_link_transfer(
                _ -> Anns0#{?ANN_ROUTING_KEYS => [LinkRKey]}
            end,
     Mc0 = mc:init(mc_amqp, Sections, Anns),
-    Mc = rabbit_message_interceptor:intercept(Mc0),
+    Mc1 = rabbit_message_interceptor:intercept(Mc0),
+    {Mc, RoutingKey} = ensure_routing_key(Mc1),
     check_user_id(Mc, User),
-    RoutingKeys = mc:routing_keys(Mc),
-    RoutingKey = routing_key(RoutingKeys, XName),
     messages_received(Settled),
     case rabbit_exchange:lookup(XName) of
         {ok, Exchange} ->
@@ -1642,6 +1641,20 @@ incoming_link_transfer(
             {error, [Disposition, Detach]}
     end.
 
+ensure_routing_key(Mc) ->
+    case mc:routing_keys(Mc) of
+        [RoutingKey] ->
+            {Mc, RoutingKey};
+        [] ->
+            %% Set the default routing key of AMQP 0.9.1 'basic.publish'{}.
+            %% For example, when the client attached to target /exchange/amq.fanout and sends a
+            %% message without setting a 'subject' in the message properties, the routing key is
+            %% ignored during routing, but receiving code paths still expect some routing key to be set.
+            DefaultRoutingKey = <<"">>,
+            Mc1 = mc:set_annotation(?ANN_ROUTING_KEYS, [DefaultRoutingKey], Mc),
+            {Mc1, DefaultRoutingKey}
+    end.
+
 process_routing_confirm([], _SenderSettles = true, _, U) ->
     rabbit_global_counters:messages_unroutable_dropped(?PROTOCOL, 1),
     {U, []};
@@ -2159,13 +2172,6 @@ remove_link_from_outgoing_unsettled_map(Ctag, Map)
                       Acc
               end, {Map, []}, Map).
 
-routing_key([RoutingKey], _XName) ->
-    RoutingKey;
-routing_key([], XName) ->
-    protocol_error(?V_1_0_AMQP_ERROR_INVALID_FIELD,
-                   "Publishing to ~ts failed since no routing key was provided",
-                   [rabbit_misc:rs(XName)]).
-
 messages_received(Settled) ->
     rabbit_global_counters:messages_received(?PROTOCOL, 1),
     case Settled of
diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl
index 98c67da26336..54a0d72c70d2 100644
--- a/deps/rabbit/test/amqp_client_SUITE.erl
+++ b/deps/rabbit/test/amqp_client_SUITE.erl
@@ -61,7 +61,6 @@ groups() ->
        link_target_classic_queue_deleted,
        link_target_quorum_queue_deleted,
        target_queues_deleted_accepted,
-       no_routing_key,
        events,
        sync_get_unsettled_classic_queue,
        sync_get_unsettled_quorum_queue,
@@ -320,9 +319,8 @@ sender_settle_mode_unsettled_fanout(Config) ->
     OpnConf = connection_config(Config),
     {ok, Connection} = amqp10_client:open_connection(OpnConf),
     {ok, Session} = amqp10_client:begin_session_sync(Connection),
-    Address = <<"/exchange/amq.fanout/ignored">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(
-                     Session, <<"test-sender">>, Address, unsettled),
+    Address = <<"/exchange/amq.fanout">>,
+    {ok, Sender} = amqp10_client:attach_sender_link(Session, <<"test-sender">>, Address, unsettled),
     ok = wait_for_credit(Sender),
 
     %% Send many messages aync.
@@ -796,7 +794,7 @@ multiple_sessions(Config) ->
     ok = rabbit_ct_client_helpers:close_channel(Ch),
 
     %% Send on each session.
-    TargetAddr = <<"/exchange/amq.fanout/ignored">>,
+    TargetAddr = <<"/exchange/amq.fanout">>,
     {ok, Sender1} = amqp10_client:attach_sender_link_sync(
                       Session1, <<"sender link 1">>, TargetAddr, settled, configuration),
     ok = wait_for_credit(Sender1),
@@ -1087,30 +1085,6 @@ rabbit_queue_type_deliver_to_q1(Qs, Msg, Opts, QTypeState) ->
     1 = length(Q1),
     meck:passthrough([Q1, Msg, Opts, QTypeState]).
 
-%% Set routing key neither in target address nor in message subject.
-no_routing_key(Config) ->
-    OpnConf = connection_config(Config),
-    {ok, Connection} = amqp10_client:open_connection(OpnConf),
-    {ok, Session} = amqp10_client:begin_session_sync(Connection),
-    Address = <<"/exchange/amq.direct">>,
-    {ok, Sender} = amqp10_client:attach_sender_link(
-                     Session, <<"test-sender">>, Address),
-    ok = wait_for_credit(Sender),
-    Msg = amqp10_msg:new(<<0>>, <<1>>, true),
-    ok = amqp10_client:send_msg(Sender, Msg),
-    receive
-        {amqp10_event,
-         {session, Session,
-          {ended,
-           #'v1_0.error'{
-              condition = ?V_1_0_AMQP_ERROR_INVALID_FIELD,
-              description = {utf8, <<"Publishing to exchange 'amq.direct' in vhost '/' "
-                                     "failed since no routing key was provided">>}}}}} -> ok
-    after 5000 -> flush(missing_ended),
-                  ct:fail("did not receive expected error")
-    end,
-    ok = amqp10_client:close_connection(Connection).
-
 events(Config) ->
     ok = event_recorder:start(Config),
 
@@ -2085,7 +2059,7 @@ last_queue_confirms(Config) ->
     {ok, Connection} = amqp10_client:open_connection(OpnConf),
     {ok, Session} = amqp10_client:begin_session_sync(Connection),
 
-    AddressFanout = <<"/exchange/amq.fanout/ignored">>,
+    AddressFanout = <<"/exchange/amq.fanout">>,
     {ok, SenderFanout} = amqp10_client:attach_sender_link(
                            Session, <<"sender-1">>, AddressFanout, unsettled),
     ok = wait_for_credit(SenderFanout),
@@ -2160,7 +2134,7 @@ target_queue_deleted(Config) ->
     {ok, Connection} = amqp10_client:open_connection(OpnConf),
     {ok, Session} = amqp10_client:begin_session_sync(Connection),
 
-    Address = <<"/exchange/amq.fanout/ignored">>,
+    Address = <<"/exchange/amq.fanout">>,
     {ok, Sender} = amqp10_client:attach_sender_link(
                      Session, <<"sender">>, Address, unsettled),
     ok = wait_for_credit(Sender),

From 0bfb1db86d88d09134c8fa1a3b6fca782b1583d2 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Sat, 10 Feb 2024 12:57:20 +0100
Subject: [PATCH 05/16] Bump outgoing delivery-count in handle_deliver

What?
For credit API v1, increase the outgoing delivery-count as soon as the
message is scheduled for delivery, that is before the message is queued
in the session's outgoing_pending queue.

Why?
1. More correct for credit API v1 in case a FLOW is received
   for an outgoing link topping up credit while an outgoing transfer on
   the same link is queued in outgoing_pending. For the server's credit
   calculation to be correct, it doesn't matter whether the outgoing
   in-flight message travels through the network, is queued in TCP
   buffers, processed by the writer, or just queued in the session's
   outgoing_pending queue.
2. Higher performance as no map update is performed for credit API v2
   in send_pending()
3. Simplifies code
---
 deps/rabbit/src/rabbit_amqp_session.erl | 54 ++++++++++++-------------
 1 file changed, 27 insertions(+), 27 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 5c2419e9f53d..58e2dc603d14 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -1127,9 +1127,8 @@ send_pending(#state{remote_incoming_window = Space,
         {{value, #pending_transfer{
                     frames = Frames,
                     queue_pid = QPid,
-                    outgoing_unsettled = #outgoing_unsettled{
-                                            consumer_tag = Ctag,
-                                            queue_name = QName}} = Pending}, Buf1}
+                    outgoing_unsettled = #outgoing_unsettled{queue_name = QName}
+                   } = Pending}, Buf1}
           when Space > 0 ->
             SendFun = case rabbit_queue_type:module(QName, State0#state.queue_states) of
                           {ok, rabbit_classic_queue} ->
@@ -1143,28 +1142,20 @@ send_pending(#state{remote_incoming_window = Space,
                                         WriterPid, Ch, Transfer, Sections)
                               end
                       end,
-            %% rabbit_basic:maybe_gc_large_msg(Content, GCThreshold)
+            {NumTransfersSent, Buf, State1} =
             case send_frames(SendFun, Frames, Space) of
                 {all, SpaceLeft} ->
-                    State1 = #state{outgoing_links = OutgoingLinks0} = session_flow_control_sent_transfers(
-                                                                         Space - SpaceLeft, State0),
-                    HandleInt = ctag_to_handle(Ctag),
-                    OutgoingLinks = maps:update_with(
-                                      HandleInt,
-                                      fun(#outgoing_link{delivery_count = {credit_api_v1, C}} = Link) ->
-                                              Link#outgoing_link{delivery_count = {credit_api_v1, add(C, 1)}};
-                                         (#outgoing_link{delivery_count = credit_api_v2} = Link) ->
-                                              Link
-                                      end,
-                                      OutgoingLinks0),
-                    State2 = State1#state{outgoing_links = OutgoingLinks},
-                    State = record_outgoing_unsettled(Pending, State2),
-                    send_pending(State#state{outgoing_pending = Buf1});
+                    {Space - SpaceLeft,
+                     Buf1,
+                     record_outgoing_unsettled(Pending, State0)};
                 {some, Rest} ->
-                    State = session_flow_control_sent_transfers(Space, State0),
-                    Buf = queue:in_r(Pending#pending_transfer{frames = Rest}, Buf1),
-                    send_pending(State#state{outgoing_pending = Buf})
-            end;
+                    {Space,
+                     queue:in_r(Pending#pending_transfer{frames = Rest}, Buf1),
+                     State0}
+            end,
+            State2 = session_flow_control_sent_transfers(NumTransfersSent, State1),
+            State = State2#state{outgoing_pending = Buf},
+            send_pending(State);
         {{value, #pending_transfer{}}, _}
           when Space =:= 0 ->
             State0
@@ -1415,17 +1406,18 @@ handle_deliver(ConsumerTag, AckRequired,
                Msg = {QName, QPid, MsgId, Redelivered, Mc0},
                State = #state{outgoing_pending = Pending,
                               outgoing_delivery_id = DeliveryId,
-                              outgoing_links = OutgoingLinks,
+                              outgoing_links = OutgoingLinks0,
                               cfg = #cfg{outgoing_max_frame_size = MaxFrameSize,
                                          conn_name = ConnName,
                                          channel_num = ChannelNum,
                                          user = #user{username = Username},
                                          trace_state = Trace}}) ->
     Handle = ctag_to_handle(ConsumerTag),
-    case OutgoingLinks of
+    case OutgoingLinks0 of
         #{Handle := #outgoing_link{queue_type = QType,
                                    send_settled = SendSettled,
-                                   max_message_size = MaxMessageSize}} ->
+                                   max_message_size = MaxMessageSize,
+                                   delivery_count = DelCount} = Link0} ->
             Dtag = delivery_tag(MsgId, SendSettled),
             Transfer = #'v1_0.transfer'{
                           handle = ?UINT(Handle),
@@ -1451,6 +1443,13 @@ handle_deliver(ConsumerTag, AckRequired,
                      end,
             messages_delivered(Redelivered, QType),
             rabbit_trace:tap_out(Msg, ConnName, ChannelNum, Username, Trace),
+            OutgoingLinks = case DelCount of
+                                credit_api_v2 ->
+                                    OutgoingLinks0;
+                                {credit_api_v1, C} ->
+                                    Link = Link0#outgoing_link{delivery_count = {credit_api_v1, add(C, 1)}},
+                                    maps:update(Handle, Link, OutgoingLinks0)
+                            end,
             Del = #outgoing_unsettled{
                      msg_id = MsgId,
                      consumer_tag = ConsumerTag,
@@ -1465,8 +1464,9 @@ handle_deliver(ConsumerTag, AckRequired,
                                  queue_pid = QPid,
                                  delivery_id = DeliveryId,
                                  outgoing_unsettled = Del},
-            State#state{outgoing_delivery_id = add(DeliveryId, 1),
-                        outgoing_pending = queue:in(PendingTransfer, Pending)};
+            State#state{outgoing_pending = queue:in(PendingTransfer, Pending),
+                        outgoing_delivery_id = add(DeliveryId, 1),
+                        outgoing_links = OutgoingLinks};
         _ ->
             %% TODO handle missing link -- why does the queue think it's there?
             rabbit_log:warning(

From d31fdb9abe22d50f95b1778047c3a8ad9deee74a Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Tue, 13 Feb 2024 18:48:43 +0100
Subject: [PATCH 06/16] Terminate channels and queue collector after reader

What?

To not risk any regressions, keep the behaviour of RabbitMQ 3.x
where channel processes and connection helper processes such as
rabbit_queue_collector and rabbit_heartbeat are terminated after
rabbit_reader process.

For example, when RabbitMQ terminates with SIGTERM, we want
exclusive queues being deleted synchronously (as in 3.x).

Prior to this commit:
1. java -jar target/perf-test.jar -x 0 -y 1
2. ./sbin/rabbitmqctl stop_app
resulted in the following crash:
```
crasher:
  initial call: rabbit_reader:init/2
  pid: <0.2389.0>
  registered_name: []
  exception exit: {noproc,
                      {gen_server,call,[<0.2391.0>,delete_all,infinity]}}
    in function  gen_server:call/3 (gen_server.erl, line 419)
    in call from rabbit_reader:close_connection/1 (rabbit_reader.erl, line 683)
    in call from rabbit_reader:send_error_on_channel0_and_close/4 (rabbit_reader.erl, line 1668)
    in call from rabbit_reader:handle_dependent_exit/3 (rabbit_reader.erl, line 710)
    in call from rabbit_reader:mainloop/4 (rabbit_reader.erl, line 530)
    in call from rabbit_reader:run/1 (rabbit_reader.erl, line 452)
    in call from rabbit_reader:start_connection/4 (rabbit_reader.erl, line 351)
```
because rabbit_queue_collector was terminated before rabbit_reader.
This commit fixes this crash.

How?

Any Erlang supervisor including the rabbit_connection_sup supervisor
terminates its children in the opposite of the start order.
Since we want channel and queue collector processes - children of
rabbit_connection_helper_sup - be terminated after the
reader process, we must start rabbit_connection_helper_sup before the
reader process.

Since rabbit_connection_sup - the ranch_protocol implementation - does
not know yet whether it will supervise an AMQP 0.9.1 or AMQP 1.0
connection, it creates rabbit_connection_helper_sup for each AMQP protocol
version removing the superfluous one as soon as the protocol version negotation is
completed. Spawning and deleting this addition process has a negligible
effect on performance.

The whole problem is that the rabbit_connection_helper_sup differs in
its supervisor flags for AMQP 0.9.1 and AMQP 1.0 when it is started
because for Native AMQP 1.0 in 4.0 we remove the unnecessary
rabbit_amqp1_0_session_sup_sup supervisor level.
Therefore, we achieve our goal:
* in Native AMQP 1.0, 1 additional Erlang process is created per session
* in AMQP 1.0 in 3.x, 15 additional Erlang processes are created per session
---
 deps/rabbit/src/rabbit_amqp_reader.erl    | 17 +++----
 deps/rabbit/src/rabbit_amqp_session.erl   |  2 +-
 deps/rabbit/src/rabbit_connection_sup.erl | 60 ++++++++++++++++-------
 deps/rabbit/src/rabbit_reader.erl         | 39 +++++++--------
 deps/rabbit/test/amqp_client_SUITE.erl    |  7 ++-
 5 files changed, 75 insertions(+), 50 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_reader.erl b/deps/rabbit/src/rabbit_amqp_reader.erl
index 3310320722f7..4b161bb0689c 100644
--- a/deps/rabbit/src/rabbit_amqp_reader.erl
+++ b/deps/rabbit/src/rabbit_amqp_reader.erl
@@ -78,9 +78,9 @@
 %%--------------------------------------------------------------------------
 
 unpack_from_0_9_1(
-  {Sock,RecvLen, PendingRecv, Buf, BufLen, ProxySocket,
+  {Sock,RecvLen, PendingRecv, SupPid, Buf, BufLen, ProxySocket,
    ConnectionName, Host, PeerHost, Port, PeerPort, ConnectedAt},
-  Parent, ConnectionHelperSupPid, HandshakeTimeout) ->
+  Parent, HandshakeTimeout) ->
     #v1{parent              = Parent,
         sock                = Sock,
         callback            = handshake,
@@ -88,7 +88,7 @@ unpack_from_0_9_1(
         pending_recv        = PendingRecv,
         connection_state    = pre_init,
         heartbeater         = none,
-        helper_sup          = ConnectionHelperSupPid,
+        helper_sup          = SupPid,
         buf                 = Buf,
         buf_len             = BufLen,
         proxy_socket        = ProxySocket,
@@ -612,13 +612,8 @@ handle_input(Callback, Data, _State) ->
 init(Mode, PackedState) ->
     {ok, HandshakeTimeout} = application:get_env(rabbit, handshake_timeout),
     {parent, Parent} = erlang:process_info(self(), parent),
-    ConnectionHelperSupFlags = #{strategy => one_for_all,
-                                 intensity => 0,
-                                 period => 1,
-                                 auto_shutdown => any_significant},
-    {ok, ConnectionHelperSupPid} = rabbit_connection_sup:start_connection_helper_sup(
-                                     Parent, ConnectionHelperSupFlags),
-    State0 = unpack_from_0_9_1(PackedState, Parent, ConnectionHelperSupPid, HandshakeTimeout),
+    ok = rabbit_connection_sup:remove_connection_helper_sup(Parent, helper_sup_amqp_091),
+    State0 = unpack_from_0_9_1(PackedState, Parent, HandshakeTimeout),
     State = start_1_0_connection(Mode, State0),
     %% By invoking recvloop here we become 1.0.
     recvloop(sys:debug_options([]), State).
@@ -1036,7 +1031,7 @@ socket_info(Get, Select, #v1{sock = Sock}) ->
 
 ignore_maintenance({map, Properties}) ->
     lists:member(
-      {{symbol, <<"ignore-maintenance">>}, {boolean, true}},
+      {{symbol, <<"ignore-maintenance">>}, true},
       Properties);
 ignore_maintenance(_) ->
     false.
diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 58e2dc603d14..6b937c4c2fee 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -1950,7 +1950,7 @@ source_filters_to_consumer_args([<<"rabbitmq:stream-filter">> = H | T], KVList,
 source_filters_to_consumer_args([<<"rabbitmq:stream-match-unfiltered">> = H | T], KVList, Acc) ->
     Key = {symbol, H},
     Arg = case keyfind_unpack_described(Key, KVList) of
-              {_, {boolean, MU}} ->
+              {_, MU} when is_boolean(MU) ->
                   [{<<"x-stream-match-unfiltered">>, bool, MU}];
               _ ->
                   []
diff --git a/deps/rabbit/src/rabbit_connection_sup.erl b/deps/rabbit/src/rabbit_connection_sup.erl
index 92ddaaf7784e..00c003c8d136 100644
--- a/deps/rabbit/src/rabbit_connection_sup.erl
+++ b/deps/rabbit/src/rabbit_connection_sup.erl
@@ -7,7 +7,7 @@
 
 -module(rabbit_connection_sup).
 
-%% Supervisor for a (network) AMQP 0-9-1 client connection.
+%% Supervisor for a (network) AMQP client connection.
 %%
 %% Supervises
 %%
@@ -21,7 +21,7 @@
 
 -export([start_link/3,
          reader/1,
-         start_connection_helper_sup/2
+         remove_connection_helper_sup/2
         ]).
 
 -export([init/1]).
@@ -35,12 +35,48 @@
 
 start_link(Ref, _Transport, _Opts) ->
     {ok, SupPid} = supervisor:start_link(?MODULE, []),
+    %% We need to get channels in the hierarchy here so they get shut
+    %% down after the reader, so the reader gets a chance to terminate
+    %% them cleanly. But for 1.0 readers we can't start the real
+    %% ch_sup_sup (because we don't know if we will be 0-9-1 or 1.0) -
+    %% so we add another supervisor into the hierarchy.
+    %%
+    %% This supervisor also acts as an intermediary for heartbeaters and
+    %% the queue collector process, since these must not be siblings of the
+    %% reader due to the potential for deadlock if they are added/restarted
+    %% whilst the supervision tree is shutting down.
+    ChildSpec = #{restart => transient,
+                  significant => true,
+                  shutdown => infinity,
+                  type => supervisor},
+    {ok, HelperSup091} =
+        supervisor:start_child(
+          SupPid,
+          ChildSpec#{
+            id => helper_sup_amqp_091,
+            start => {rabbit_connection_helper_sup, start_link,
+                      [#{strategy => one_for_one,
+                         intensity => 10,
+                         period => 10,
+                         auto_shutdown => any_significant}]}}
+         ),
+    {ok, HelperSup10} =
+        supervisor:start_child(
+          SupPid,
+          ChildSpec#{
+            id => helper_sup_amqp_10,
+            start => {rabbit_connection_helper_sup, start_link,
+                      [#{strategy => one_for_all,
+                         intensity => 0,
+                         period => 1,
+                         auto_shutdown => any_significant}]}}
+         ),
     {ok, ReaderPid} =
         supervisor:start_child(
             SupPid,
             #{
                 id => reader,
-                start => {rabbit_reader, start_link, [Ref]},
+                start => {rabbit_reader, start_link, [{HelperSup091, HelperSup10}, Ref]},
                 restart => transient,
                 significant => true,
                 shutdown => ?WORKER_WAIT,
@@ -51,23 +87,13 @@ start_link(Ref, _Transport, _Opts) ->
     {ok, SupPid, ReaderPid}.
 
 -spec reader(pid()) -> pid().
-
 reader(Pid) ->
     hd(rabbit_misc:find_child(Pid, reader)).
 
--spec start_connection_helper_sup(pid(), supervisor:sup_flags()) ->
-    supervisor:startchild_ret().
-start_connection_helper_sup(ConnectionSupPid, ConnectionHelperSupFlags) ->
-    supervisor:start_child(
-      ConnectionSupPid,
-      #{
-        id => helper_sup,
-        start => {rabbit_connection_helper_sup, start_link, [ConnectionHelperSupFlags]},
-        restart => transient,
-        significant => true,
-        shutdown => infinity,
-        type => supervisor
-       }).
+-spec remove_connection_helper_sup(pid(), helper_sup_amqp_091 | helper_sup_amqp_10) -> ok.
+remove_connection_helper_sup(ConnectionSupPid, ConnectionHelperId) ->
+    ok = supervisor:terminate_child(ConnectionSupPid, ConnectionHelperId),
+    ok = supervisor:delete_child(ConnectionSupPid, ConnectionHelperId).
 
 %%--------------------------------------------------------------------------
 
diff --git a/deps/rabbit/src/rabbit_reader.erl b/deps/rabbit/src/rabbit_reader.erl
index 4f8c9a61b6cd..bb5268450d7e 100644
--- a/deps/rabbit/src/rabbit_reader.erl
+++ b/deps/rabbit/src/rabbit_reader.erl
@@ -43,12 +43,12 @@
 -include_lib("rabbit_common/include/rabbit_framing.hrl").
 -include_lib("rabbit_common/include/rabbit.hrl").
 
--export([start_link/1, info_keys/0, info/1, info/2, force_event_refresh/2,
+-export([start_link/2, info_keys/0, info/1, info/2, force_event_refresh/2,
          shutdown/2]).
 
 -export([system_continue/3, system_terminate/4, system_code_change/4]).
 
--export([init/2, mainloop/4, recvloop/4]).
+-export([init/3, mainloop/4, recvloop/4]).
 
 -export([conserve_resources/3, server_properties/1]).
 
@@ -78,7 +78,9 @@
           %% pre_init | securing | running | blocking | blocked | closing | closed | {become, F}
           connection_state,
           %% see comment in rabbit_connection_sup:start_link/0
-          helper_sup,
+          helper_sup :: {HelperSupAmqp091 :: pid(),
+                         HelperSupAmqp10 :: pid()} % pre version negotiation
+                        | pid(), % post version negotiation
           %% takes care of cleaning up exclusive queues,
           %% see rabbit_queue_collector
           queue_collector,
@@ -145,10 +147,10 @@
 
 %%--------------------------------------------------------------------------
 
--spec start_link(ranch:ref()) ->
+-spec start_link({pid(), pid()}, ranch:ref()) ->
     rabbit_types:ok(pid()).
-start_link(Ref) ->
-    Pid = proc_lib:spawn_link(?MODULE, init, [self(), Ref]),
+start_link(HelperSups, Ref) ->
+    Pid = proc_lib:spawn_link(?MODULE, init, [self(), HelperSups, Ref]),
     {ok, Pid}.
 
 -spec shutdown(pid(), string()) -> 'ok'.
@@ -156,14 +158,14 @@ start_link(Ref) ->
 shutdown(Pid, Explanation) ->
     gen_server:call(Pid, {shutdown, Explanation}, infinity).
 
--spec init(pid(), ranch:ref()) ->
+-spec init(pid(), {pid(), pid()}, ranch:ref()) ->
     no_return().
-init(Parent, Ref) ->
+init(Parent, HelperSups, Ref) ->
     ?LG_PROCESS_TYPE(reader),
     {ok, Sock} = rabbit_networking:handshake(Ref,
         application:get_env(rabbit, proxy_protocol, false)),
     Deb = sys:debug_options([]),
-    start_connection(Parent, Ref, Deb, Sock).
+    start_connection(Parent, HelperSups, Ref, Deb, Sock).
 
 -spec system_continue(_,_,{[binary()], non_neg_integer(), #v1{}}) -> any().
 
@@ -290,10 +292,10 @@ socket_op(Sock, Fun) ->
                            exit(normal)
     end.
 
--spec start_connection(pid(), ranch:ref(), any(), rabbit_net:socket()) ->
+-spec start_connection(pid(), {pid(), pid()}, ranch:ref(), any(), rabbit_net:socket()) ->
           no_return().
 
-start_connection(Parent, RanchRef, Deb, Sock) ->
+start_connection(Parent, HelperSups, RanchRef, Deb, Sock) ->
     process_flag(trap_exit, true),
     RealSocket = rabbit_net:unwrap_socket(Sock),
     Name = case rabbit_net:connection_string(Sock, inbound) of
@@ -336,7 +338,7 @@ start_connection(Parent, RanchRef, Deb, Sock) ->
                 pending_recv        = false,
                 connection_state    = pre_init,
                 queue_collector     = undefined,  %% started on tune-ok
-                helper_sup          = none,
+                helper_sup          = HelperSups,
                 heartbeater         = none,
                 channel_sup_sup_pid = none,
                 channel_count       = 0,
@@ -1104,13 +1106,9 @@ start_091_connection({ProtocolMajor, ProtocolMinor, _ProtocolRevision},
                      Protocol,
                      #v1{parent = Parent,
                          sock = Sock,
+                         helper_sup = {HelperSup091, _HelperSup10},
                          connection = Connection} = State0) ->
-    ConnectionHelperSupFlags = #{strategy => one_for_one,
-                                 intensity => 10,
-                                 period => 10,
-                                 auto_shutdown => any_significant},
-    {ok, ConnectionHelperSupPid} = rabbit_connection_sup:start_connection_helper_sup(
-                                     Parent, ConnectionHelperSupFlags),
+    ok = rabbit_connection_sup:remove_connection_helper_sup(Parent, helper_sup_amqp_10),
     rabbit_networking:register_connection(self()),
     Start = #'connection.start'{
                version_major = ProtocolMajor,
@@ -1123,7 +1121,7 @@ start_091_connection({ProtocolMajor, ProtocolMinor, _ProtocolRevision},
                                      timeout_sec = ?NORMAL_TIMEOUT,
                                      protocol = Protocol},
                       connection_state = starting,
-                      helper_sup = ConnectionHelperSupPid},
+                      helper_sup = HelperSup091},
     switch_callback(State, frame_header, 7).
 
 -spec refuse_connection(rabbit_net:socket(), any()) -> no_return().
@@ -1647,6 +1645,7 @@ become_10(Id, State = #v1{sock = Sock}) ->
 pack_for_1_0(Buf, BufLen, #v1{sock         = Sock,
                               recv_len     = RecvLen,
                               pending_recv = PendingRecv,
+                              helper_sup = {_HelperSup091, HelperSup10},
                               proxy_socket = ProxySocket,
                               connection = #connection{
                                               name = Name,
@@ -1655,7 +1654,7 @@ pack_for_1_0(Buf, BufLen, #v1{sock         = Sock,
                                               port = Port,
                                               peer_port = PeerPort,
                                               connected_at = ConnectedAt}}) ->
-    {Sock, RecvLen, PendingRecv, Buf, BufLen, ProxySocket,
+    {Sock, RecvLen, PendingRecv, HelperSup10, Buf, BufLen, ProxySocket,
      Name, Host, PeerHost, Port, PeerPort, ConnectedAt}.
 
 respond_and_close(State, Channel, Protocol, Reason, LogErr) ->
diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl
index 54a0d72c70d2..884f54033a9f 100644
--- a/deps/rabbit/test/amqp_client_SUITE.erl
+++ b/deps/rabbit/test/amqp_client_SUITE.erl
@@ -244,7 +244,11 @@ reliable_send_receive(QType, Outcome, Config) ->
     %% link will be in "mixed" mode by default
     Body = <<"body-1">>,
     Msg1 = amqp10_msg:new(DTag1, Body, false),
-    ok = amqp10_client:send_msg(Sender, Msg1),
+
+    %% Use the 2 byte AMQP boolean encoding, see AMQP §1.6.2
+    True = {boolean, true},
+    Msg2 = amqp10_msg:set_headers(#{durable => True}, Msg1),
+    ok = amqp10_client:send_msg(Sender, Msg2),
     ok = wait_for_settlement(DTag1),
 
     ok = amqp10_client:detach_link(Sender),
@@ -258,6 +262,7 @@ reliable_send_receive(QType, Outcome, Config) ->
                        Session2, <<"test-receiver">>, Address, unsettled),
     {ok, Msg} = amqp10_client:get_msg(Receiver),
     ?assertEqual([Body], amqp10_msg:body(Msg)),
+    ?assertEqual(true, amqp10_msg:header(durable, Msg)),
 
     ok = amqp10_client:settle_msg(Receiver, Msg, Outcome),
     flush("post accept"),

From 3d67842db8f9912048f361b68a2614d45d7ff5ac Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Tue, 20 Feb 2024 11:22:21 +0100
Subject: [PATCH 07/16] Protect receiving app from being overloaded

What?

Protect receiving application from being overloaded with new messages
while still processing existing messages if the auto credit renewal
feature of the Erlang AMQP 1.0 client library is used.

This feature can therefore be thought of as a prefetch window equivalent
in AMQP 0.9.1 or MQTT 5.0 property Receive Maximum.

How?

The credit auto renewal feature in RabbitMQ 3.x was wrongly implemented.
This commit takes the same approach as done in the server:
The incoming_unsettled map is hold in the link instead of in the session
to accurately and quickly determine the number of unsettled messages for
a receiving link.

The amqp10_client lib will grant more credits to the sender when the sum
of remaining link credits and number of unsettled deliveries falls below
the threshold RenewWhenBelow.

This avoids maintaning additional state like the `link_credit_unsettled`
or an alternative delivery_count_settled sequence number which is more
complex to implement correctly.

This commit breaks the amqp10_client_session:disposition/6 API:
This commit forces the client application to only range settle for a
given link, i.e. not across multiple links on a given session at once.
The latter is allowed according to the AMQP spec.
---
 deps/amqp10_client/Makefile                   |   3 -
 deps/amqp10_client/src/amqp10_client.erl      |  34 +--
 deps/amqp10_client/src/amqp10_client.hrl      |   1 +
 .../src/amqp10_client_session.erl             | 115 +++++-----
 deps/amqp10_client/test/system_SUITE.erl      | 210 ++++++++++++++----
 deps/rabbit/test/amqp_client_SUITE.erl        |  60 +++--
 deps/rabbit/test/amqp_credit_api_v2_SUITE.erl |  21 +-
 .../src/rabbit_amqp10_shovel.erl              |  24 +-
 .../test/protocol_interop_SUITE.erl           |   6 +-
 moduleindex.yaml                              |  34 +--
 10 files changed, 315 insertions(+), 193 deletions(-)

diff --git a/deps/amqp10_client/Makefile b/deps/amqp10_client/Makefile
index 7dd0f4c657a5..b42ca3017e45 100644
--- a/deps/amqp10_client/Makefile
+++ b/deps/amqp10_client/Makefile
@@ -51,9 +51,6 @@ include erlang.mk
 HEX_TARBALL_FILES += rabbitmq-components.mk \
 		     git-revisions.txt
 
-# Dialyze the tests.
-DIALYZER_OPTS += --src -r test
-
 # --------------------------------------------------------------------
 # ActiveMQ for the testsuite.
 # --------------------------------------------------------------------
diff --git a/deps/amqp10_client/src/amqp10_client.erl b/deps/amqp10_client/src/amqp10_client.erl
index c9939bc263e4..32f91a5f7aea 100644
--- a/deps/amqp10_client/src/amqp10_client.erl
+++ b/deps/amqp10_client/src/amqp10_client.erl
@@ -301,16 +301,19 @@ attach_link(Session, AttachArgs) ->
 %% This is asynchronous and will notify completion of the attach request to the
 %% caller using an amqp10_event of the following format:
 %% {amqp10_event, {link, LinkRef, {detached, Why}}}
--spec detach_link(link_ref()) -> _.
+-spec detach_link(link_ref()) -> ok | {error, term()}.
 detach_link(#link_ref{link_handle = Handle, session = Session}) ->
     amqp10_client_session:detach(Session, Handle).
 
-%% @doc Grant credit to a sender.
-%% The amqp10_client will automatically grant Credit to the sender when
-%% the remaining link credit falls below the value of RenewWhenBelow.
-%% If RenewWhenBelow is 'never' the client will never grant more credit. Instead
-%% the caller will be notified when the link_credit reaches 0 with an
-%% amqp10_event of the following format:
+%% @doc Grant Credit to a sender.
+%%
+%% In addition, if RenewWhenBelow is an integer, the amqp10_client will automatically grant more
+%% Credit to the sender when the sum of the remaining link credit and the number of unsettled
+%% messages falls below the value of RenewWhenBelow.
+%% `Credit + RenewWhenBelow - 1` is the maximum number of in-flight unsettled messages.
+%%
+%% If RenewWhenBelow is `never` the amqp10_client will never grant more credit. Instead the caller
+%% will be notified when the link_credit reaches 0 with an amqp10_event of the following format:
 %% {amqp10_event, {link, LinkRef, credit_exhausted}}
 -spec flow_link_credit(link_ref(), Credit :: non_neg_integer(),
                        RenewWhenBelow :: never | pos_integer()) -> ok.
@@ -323,10 +326,16 @@ flow_link_credit(Ref, Credit, RenewWhenBelow) ->
 flow_link_credit(#link_ref{role = receiver, session = Session,
                            link_handle = Handle},
                  Credit, RenewWhenBelow, Drain)
-  when RenewWhenBelow =:= never orelse
+  when
+      %% Drain together with auto renewal doesn't make sense, so disallow it in the API.
+      ((Drain) andalso RenewWhenBelow =:= never
+       orelse not(Drain))
+      andalso
+      %% Check that the RenewWhenBelow value make sense.
+      (RenewWhenBelow =:= never orelse
        is_integer(RenewWhenBelow) andalso
        RenewWhenBelow > 0 andalso
-       RenewWhenBelow =< Credit ->
+       RenewWhenBelow =< Credit) ->
     Flow = #'v1_0.flow'{link_credit = {uint, Credit},
                         drain = Drain},
     ok = amqp10_client_session:flow(Session, Handle, Flow, RenewWhenBelow).
@@ -359,11 +368,10 @@ accept_msg(LinkRef, Msg) ->
 %% the chosen delivery state.
 -spec settle_msg(link_ref(), amqp10_msg:amqp10_msg(),
                  amqp10_client_types:delivery_state()) -> ok.
-settle_msg(#link_ref{role = receiver,
-                     session = Session}, Msg, Settlement) ->
+settle_msg(LinkRef, Msg, Settlement) ->
     DeliveryId = amqp10_msg:delivery_id(Msg),
-    amqp10_client_session:disposition(Session, receiver, DeliveryId,
-                                      DeliveryId, true, Settlement).
+    amqp10_client_session:disposition(LinkRef, DeliveryId, DeliveryId, true, Settlement).
+
 %% @doc Get a single message from a link.
 %% Flows a single link credit then awaits delivery or timeout.
 -spec get_msg(link_ref()) -> {ok, amqp10_msg:amqp10_msg()} | {error, timeout}.
diff --git a/deps/amqp10_client/src/amqp10_client.hrl b/deps/amqp10_client/src/amqp10_client.hrl
index 137e82552199..99cad7578300 100644
--- a/deps/amqp10_client/src/amqp10_client.hrl
+++ b/deps/amqp10_client/src/amqp10_client.hrl
@@ -20,4 +20,5 @@
 
 -record(link_ref, {role :: sender | receiver,
                    session :: pid(),
+                   %% locally chosen output handle
                    link_handle :: non_neg_integer()}).
diff --git a/deps/amqp10_client/src/amqp10_client_session.erl b/deps/amqp10_client/src/amqp10_client_session.erl
index e728f4f5ce05..7b1cba641d76 100644
--- a/deps/amqp10_client/src/amqp10_client_session.erl
+++ b/deps/amqp10_client/src/amqp10_client_session.erl
@@ -21,7 +21,7 @@
          detach/2,
          transfer/3,
          flow/4,
-         disposition/6
+         disposition/5
         ]).
 
 %% Private API
@@ -131,8 +131,9 @@
          available = 0 :: non_neg_integer(),
          drain = false :: boolean(),
          partial_transfers :: undefined | {#'v1_0.transfer'{}, [binary()]},
-         auto_flow :: never | {auto, RenewWhenBelow :: pos_integer(), Credit :: pos_integer()}
-         }).
+         auto_flow :: never | {auto, RenewWhenBelow :: pos_integer(), Credit :: pos_integer()},
+         incoming_unsettled = #{} :: #{delivery_number() => ok}
+        }).
 
 -record(state,
         {channel :: pos_integer(),
@@ -155,7 +156,6 @@
          connection_config :: amqp10_client_connection:connection_config(),
          outgoing_delivery_id = ?INITIAL_OUTGOING_DELIVERY_ID :: delivery_number(),
          outgoing_unsettled = #{} :: #{delivery_number() => {amqp10_msg:delivery_tag(), Notify :: pid()}},
-         incoming_unsettled = #{} :: #{delivery_number() => output_handle()},
          notify :: pid()
         }).
 
@@ -204,14 +204,18 @@ transfer(Session, Amqp10Msg, Timeout) ->
 flow(Session, Handle, Flow, RenewWhenBelow) ->
     gen_statem:cast(Session, {flow_link, Handle, Flow, RenewWhenBelow}).
 
--spec disposition(pid(), link_role(), delivery_number(), delivery_number(), boolean(),
+%% Sending a disposition on a sender link (with receiver-settle-mode = second)
+%% is currently unsupported.
+-spec disposition(link_ref(), delivery_number(), delivery_number(), boolean(),
                   amqp10_client_types:delivery_state()) -> ok.
-disposition(Session, Role, First, Last, Settled, DeliveryState) ->
-    gen_statem:call(Session, {disposition, Role, First, Last, Settled,
+disposition(#link_ref{role = receiver,
+                      session = Session,
+                      link_handle = Handle},
+            First, Last, Settled, DeliveryState) ->
+    gen_statem:call(Session, {disposition, Handle, First, Last, Settled,
                               DeliveryState}, ?TIMEOUT).
 
 
-
 %% -------------------------------------------------------------------
 %% Private API.
 %% -------------------------------------------------------------------
@@ -277,7 +281,7 @@ mapped(cast, 'end', State) ->
     send_end(State),
     {next_state, end_sent, State};
 mapped(cast, {flow_link, OutHandle, Flow0, RenewWhenBelow}, State0) ->
-    State = send_flow_link(fun send/2, OutHandle, Flow0, RenewWhenBelow, State0),
+    State = send_flow_link(OutHandle, Flow0, RenewWhenBelow, State0),
     {keep_state, State};
 mapped(cast, {flow_session, Flow0 = #'v1_0.flow'{incoming_window = {uint, IncomingWindow}}},
        #state{next_incoming_id = NII,
@@ -367,45 +371,43 @@ mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
     State = book_partial_transfer_received(
               State0#state{links = Links#{OutHandle => Link1}}),
     {keep_state, State};
-mapped(cast, {#'v1_0.transfer'{handle = {uint, InHandle},
-                               delivery_id = MaybeDeliveryId,
-                               settled = Settled} = Transfer0, Payload0},
-       #state{incoming_unsettled = Unsettled0} = State0) ->
-
+mapped(cast, {Transfer0 = #'v1_0.transfer'{handle = {uint, InHandle}},
+              Payload0}, State0) ->
     {ok, #link{target = {pid, TargetPid},
-               output_handle = OutHandle,
-               ref = LinkRef} = Link0} =
-        find_link_by_input_handle(InHandle, State0),
+               ref = LinkRef,
+               incoming_unsettled = Unsettled
+              } = Link0} = find_link_by_input_handle(InHandle, State0),
 
-    {Transfer, Payload, Link1} = complete_partial_transfer(Transfer0, Payload0, Link0),
-    Msg = decode_as_msg(Transfer, Payload),
-
-    % stash the DeliveryId - not sure for what yet
-    Unsettled = case MaybeDeliveryId of
-                    {uint, DeliveryId} when Settled =/= true ->
-                        Unsettled0#{DeliveryId => OutHandle};
-                    _ ->
-                        Unsettled0
-                end,
+    {Transfer = #'v1_0.transfer'{settled = Settled,
+                                 delivery_id = {uint, DeliveryId}},
+     Payload, Link1} = complete_partial_transfer(Transfer0, Payload0, Link0),
 
+    Msg = decode_as_msg(Transfer, Payload),
+    Link2 = case Settled of
+                true ->
+                    Link1;
+                _ ->
+                    %% "If not set on the first (or only) transfer for a (multi-transfer) delivery,
+                    %% then the settled flag MUST be interpreted as being false." [2.7.5]
+                    Link1#link{incoming_unsettled = Unsettled#{DeliveryId => ok}}
+            end,
     % link bookkeeping
     % notify when credit is exhausted (link_credit = 0)
     % detach the Link with a transfer-limit-exceeded error code if further
     % transfers are received
-    State1 = State0#state{incoming_unsettled = Unsettled},
-    case book_transfer_received(State1, Link1) of
-        {ok, Link2, State2} ->
+    case book_transfer_received(State0, Link2) of
+        {ok, Link3, State1} ->
             % deliver
             TargetPid ! {amqp10_msg, LinkRef, Msg},
-            State = auto_flow(Link2, State2),
+            State = auto_flow(Link3, State1),
             {keep_state, State};
-        {credit_exhausted, Link2, State} ->
+        {credit_exhausted, Link3, State} ->
             TargetPid ! {amqp10_msg, LinkRef, Msg},
-            notify_credit_exhausted(Link2),
+            notify_credit_exhausted(Link3),
             {keep_state, State};
-        {transfer_limit_exceeded, Link2, State} ->
-            logger:warning("transfer_limit_exceeded for link ~tp", [Link2]),
-            Link = detach_with_error_cond(Link2, State,
+        {transfer_limit_exceeded, Link3, State} ->
+            logger:warning("transfer_limit_exceeded for link ~tp", [Link3]),
+            Link = detach_with_error_cond(Link3, State,
                                            ?V_1_0_LINK_ERROR_TRANSFER_LIMIT_EXCEEDED),
             {keep_state, update_link(Link, State)}
     end;
@@ -501,12 +503,15 @@ mapped({call, From},
     end;
 
 mapped({call, From},
-       {disposition, Role, First, Last, Settled0, DeliveryState},
-       #state{incoming_unsettled = Unsettled0} = State0) ->
+       {disposition, OutputHandle, First, Last, Settled0, DeliveryState},
+       #state{links = Links} = State0) ->
+    #{OutputHandle := Link0 = #link{incoming_unsettled = Unsettled0}} = Links,
     Unsettled = serial_number:foldl(fun maps:remove/2, Unsettled0, First, Last),
-    State = State0#state{incoming_unsettled = Unsettled},
+    Link = Link0#link{incoming_unsettled = Unsettled},
+    State1 = State0#state{links = Links#{OutputHandle := Link}},
+    State = auto_flow(Link, State1),
     Disposition = #'v1_0.disposition'{
-                     role = translate_role(Role),
+                     role = translate_role(receiver),
                      first = {uint, First},
                      last = {uint, Last},
                      settled = Settled0,
@@ -599,7 +604,7 @@ send_transfer(Transfer0, Parts0, MaxMessageSize, #state{socket = Socket,
            {ok, length(Frames)}
     end.
 
-send_flow_link(Send, OutHandle,
+send_flow_link(OutHandle,
                #'v1_0.flow'{link_credit = {uint, Credit}} = Flow0, RenewWhenBelow,
                #state{links = Links,
                       next_incoming_id = NII,
@@ -625,7 +630,7 @@ send_flow_link(Send, OutHandle,
                    %% initial attach frame from the sender this field MUST NOT be set." [2.7.4]
                    delivery_count = maybe_uint(DeliveryCount),
                    available = uint(Available)},
-    ok = Send(Flow, State),
+    ok = send(Flow, State),
     State#state{links = Links#{OutHandle =>
                                Link#link{link_credit = Credit,
                                          auto_flow = AutoFlow}}}.
@@ -777,8 +782,9 @@ send_attach(Send, #{name := Name, role := Role} = Args, {FromPid, _},
                             max_message_size = MaxMessageSize},
     ok = Send(Attach, State),
 
+    LinkRef = make_link_ref(element(1, Role), self(), OutHandle),
     Link = #link{name = Name,
-                 ref = make_link_ref(element(1, Role), self(), OutHandle),
+                 ref = LinkRef,
                  output_handle = OutHandle,
                  state = attach_sent,
                  role = element(1, Role),
@@ -790,7 +796,7 @@ send_attach(Send, #{name := Name, role := Role} = Args, {FromPid, _},
 
     {State#state{links = Links#{OutHandle => Link},
                  next_link_handle = NextLinkHandle,
-                 link_index = LinkIndex#{Name => OutHandle}}, Link#link.ref}.
+                 link_index = LinkIndex#{Name => OutHandle}}, LinkRef}.
 
 -spec handle_session_flow(#'v1_0.flow'{}, #state{}) -> #state{}.
 handle_session_flow(#'v1_0.flow'{next_incoming_id = MaybeNII,
@@ -908,7 +914,6 @@ translate_delivery_state({modified,
 translate_delivery_state(released) -> #'v1_0.released'{};
 translate_delivery_state(received) -> #'v1_0.received'{}.
 
-translate_role(sender) -> false;
 translate_role(receiver) -> true.
 
 maybe_notify_link_credit(#link{role = sender,
@@ -987,9 +992,11 @@ book_transfer_received(#state{next_incoming_id = NID,
 
 auto_flow(#link{link_credit = LC,
                 auto_flow = {auto, RenewWhenBelow, Credit},
-                output_handle = OutHandle}, State)
-  when LC < RenewWhenBelow ->
-    send_flow_link(fun send/2, OutHandle,
+                output_handle = OutHandle,
+                incoming_unsettled = Unsettled},
+          State)
+  when LC + map_size(Unsettled) < RenewWhenBelow ->
+    send_flow_link(OutHandle,
                    #'v1_0.flow'{link_credit = {uint, Credit}},
                    RenewWhenBelow, State);
 auto_flow(_, State) ->
@@ -1045,7 +1052,8 @@ socket_send0({tcp, Socket}, Data) ->
 socket_send0({ssl, Socket}, Data) ->
     ssl:send(Socket, Data).
 
--spec make_link_ref(_, _, _) -> link_ref().
+-spec make_link_ref(link_role(), pid(), output_handle()) ->
+    link_ref().
 make_link_ref(Role, Session, Handle) ->
     #link_ref{role = Role, session = Session, link_handle = Handle}.
 
@@ -1100,7 +1108,6 @@ format_status(Status = #{data := Data0}) ->
            connection_config = ConnectionConfig,
            outgoing_delivery_id = OutgoingDeliveryId,
            outgoing_unsettled = OutgoingUnsettled,
-           incoming_unsettled = IncomingUnsettled,
            notify = Notify
           } = Data0,
     Links = maps:map(
@@ -1119,7 +1126,8 @@ format_status(Status = #{data := Data0}) ->
                         available = Available,
                         drain = Drain,
                         partial_transfers = PartialTransfers0,
-                        auto_flow = AutoFlow
+                        auto_flow = AutoFlow,
+                        incoming_unsettled = IncomingUnsettled
                        }) ->
                       PartialTransfers = case PartialTransfers0 of
                                              undefined ->
@@ -1141,7 +1149,9 @@ format_status(Status = #{data := Data0}) ->
                         available => Available,
                         drain => Drain,
                         partial_transfers => PartialTransfers,
-                        auto_flow => AutoFlow}
+                        auto_flow => AutoFlow,
+                        incoming_unsettled => maps:size(IncomingUnsettled)
+                       }
               end, Links0),
     Data = #{channel => Channel,
              remote_channel => RemoteChannel,
@@ -1160,7 +1170,6 @@ format_status(Status = #{data := Data0}) ->
              connection_config => maps:remove(sasl, ConnectionConfig),
              outgoing_delivery_id => OutgoingDeliveryId,
              outgoing_unsettled => maps:size(OutgoingUnsettled),
-             incoming_unsettled => maps:size(IncomingUnsettled),
              notify => Notify},
     Status#{data := Data}.
 
diff --git a/deps/amqp10_client/test/system_SUITE.erl b/deps/amqp10_client/test/system_SUITE.erl
index 2147b0f156ce..62a7718657ef 100644
--- a/deps/amqp10_client/test/system_SUITE.erl
+++ b/deps/amqp10_client/test/system_SUITE.erl
@@ -514,16 +514,17 @@ subscribe(Config) ->
                                                         <<"sub-receiver">>,
                                                         QueueName, unsettled),
     ok = amqp10_client:flow_link_credit(Receiver, 10, never),
-
-    _ = receive_messages(Receiver, 10),
-    % assert no further messages are delivered
-    timeout = receive_one(Receiver),
-    receive
-        {amqp10_event, {link, Receiver, credit_exhausted}} ->
-            ok
-    after 5000 ->
-              flush(),
-              exit(credit_exhausted_assert)
+    [begin
+         receive {amqp10_msg, Receiver, Msg} ->
+                     ok = amqp10_client:accept_msg(Receiver, Msg)
+         after 2000 -> ct:fail(timeout)
+         end
+     end || _ <- lists:seq(1, 10)],
+    ok = assert_no_message(Receiver),
+
+    receive {amqp10_event, {link, Receiver, credit_exhausted}} -> ok
+    after 5000 -> flush(),
+                  exit(credit_exhausted_assert)
     end,
 
     ok = amqp10_client:end_session(Session),
@@ -539,16 +540,121 @@ subscribe_with_auto_flow(Config) ->
                                                          <<"sub-sender">>,
                                                          QueueName),
     await_link(Sender, credited, link_credit_timeout),
-    _ = publish_messages(Sender, <<"banana">>, 10),
-    {ok, Receiver} = amqp10_client:attach_receiver_link(Session,
-                                                        <<"sub-receiver">>,
-                                                        QueueName, unsettled),
-    ok = amqp10_client:flow_link_credit(Receiver, 5, 2),
-
-    _ = receive_messages(Receiver, 10),
 
-    % assert no further messages are delivered
-    timeout = receive_one(Receiver),
+    _ = publish_messages(Sender, <<"banana">>, 20),
+    %% Use sender settle mode 'settled'.
+    {ok, R1} = amqp10_client:attach_receiver_link(
+                 Session, <<"sub-receiver-1">>, QueueName, settled),
+    await_link(R1, attached, attached_timeout),
+    ok = amqp10_client:flow_link_credit(R1, 5, 2),
+    ?assertEqual(20, count_received_messages(R1)),
+    ok = amqp10_client:detach_link(R1),
+
+    _ = publish_messages(Sender, <<"banana">>, 30),
+    %% Use sender settle mode 'unsettled'.
+    %% This should require us to manually settle message in order to receive more messages.
+    {ok, R2} = amqp10_client:attach_receiver_link(Session, <<"sub-receiver-2">>, QueueName, unsettled),
+    await_link(R2, attached, attached_timeout),
+    ok = amqp10_client:flow_link_credit(R2, 5, 2),
+    %% We should receive exactly 5 messages.
+    [M1, _M2, M3, M4, M5] = receive_messages(R2, 5),
+    ok = assert_no_message(R2),
+
+    %% Even when we accept the first 3 messages, the number of unsettled messages has not yet fallen below 2.
+    %% Therefore, the client should not yet grant more credits to the sender.
+    ok = amqp10_client_session:disposition(
+           R2, amqp10_msg:delivery_id(M1), amqp10_msg:delivery_id(M3), true, accepted),
+    ok = assert_no_message(R2),
+
+    %% When we accept 1 more message (the order in which we accept shouldn't matter, here we accept M5 before M4),
+    %% the number of unsettled messages now falls below 2 (since only M4 is left unsettled).
+    %% Therefore, the client should grant 5 credits to the sender.
+    %% Therefore, we should receive 5 more messages.
+    ok = amqp10_client:accept_msg(R2, M5),
+    [_M6, _M7, _M8, _M9, M10] = receive_messages(R2, 5),
+    ok = assert_no_message(R2),
+
+    %% It shouldn't matter how we settle messages, therefore we use 'rejected' this time.
+    %% Settling all in flight messages should cause us to receive exactly 5 more messages.
+    ok = amqp10_client_session:disposition(
+           R2, amqp10_msg:delivery_id(M4), amqp10_msg:delivery_id(M10), true, rejected),
+    [M11, _M12, _M13, _M14, M15] = receive_messages(R2, 5),
+    ok = assert_no_message(R2),
+
+    %% Dynamically decrease link credit.
+    %% Since we explicitly tell to grant 3 new credits now, we expect to receive 3 more messages.
+    ok = amqp10_client:flow_link_credit(R2, 3, 3),
+    [M16, _M17, M18] = receive_messages(R2, 3),
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client_session:disposition(
+           R2, amqp10_msg:delivery_id(M11), amqp10_msg:delivery_id(M15), true, accepted),
+    %% However, the RenewWhenBelow=3 still refers to all unsettled messages.
+    %% Right now we have 3 messages (M16, M17, M18) unsettled.
+    ok = assert_no_message(R2),
+
+    %% Settling 1 out of these 3 messages causes RenewWhenBelow to fall below 3 resulting
+    %% in 3 new messages to be received.
+    ok = amqp10_client:accept_msg(R2, M18),
+    [_M19, _M20, _M21] = receive_messages(R2, 3),
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client:flow_link_credit(R2, 3, never, true),
+    [_M22, _M23, M24] = receive_messages(R2, 3),
+    ok = assert_no_message(R2),
+
+    %% Since RenewWhenBelow = never, we expect to receive no new messages despite settling.
+    ok = amqp10_client_session:disposition(
+           R2, amqp10_msg:delivery_id(M16), amqp10_msg:delivery_id(M24), true, rejected),
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client:flow_link_credit(R2, 2, never, false),
+    [M25, _M26] = receive_messages(R2, 2),
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client:flow_link_credit(R2, 3, 3),
+    [_M27, _M28, M29] = receive_messages(R2, 3),
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client_session:disposition(
+           R2, amqp10_msg:delivery_id(M25), amqp10_msg:delivery_id(M29), true, accepted),
+    [M30] = receive_messages(R2, 1),
+    ok = assert_no_message(R2),
+    ok = amqp10_client:accept_msg(R2, M30),
+    %% The sender queue is empty now.
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client:flow_link_credit(R2, 3, 1),
+    _ = publish_messages(Sender, <<"banana">>, 1),
+    [M31] = receive_messages(R2, 1),
+    ok = amqp10_client:accept_msg(R2, M31),
+
+    %% Since function flow_link_credit/3 documents
+    %%     "if RenewWhenBelow is an integer, the amqp10_client will automatically grant more
+    %%     Credit to the sender when the sum of the remaining link credit and the number of
+    %%     unsettled messages falls below the value of RenewWhenBelow."
+    %% our expectation is that the amqp10_client has not renewed credit since the sum of
+    %% remaining link credit (2) and unsettled messages (0) is 2.
+    %%
+    %% Therefore, when we publish another 3 messages, we expect to only receive only 2 messages!
+    _ = publish_messages(Sender, <<"banana">>, 5),
+    [M32, M33] = receive_messages(R2, 2),
+    ok = assert_no_message(R2),
+
+    %% When we accept both messages, the sum of the remaining link credit (0) and unsettled messages (0)
+    %% falls below RenewWhenBelow=1 causing the amqp10_client to grant 3 new credits.
+    ok = amqp10_client:accept_msg(R2, M32),
+    ok = assert_no_message(R2),
+    ok = amqp10_client:accept_msg(R2, M33),
+
+    [M35, M36, M37] = receive_messages(R2, 3),
+    ok = amqp10_client:accept_msg(R2, M35),
+    ok = amqp10_client:accept_msg(R2, M36),
+    ok = amqp10_client:accept_msg(R2, M37),
+    %% The sender queue is empty now.
+    ok = assert_no_message(R2),
+
+    ok = amqp10_client:detach_link(R2),
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection).
 
@@ -703,11 +809,19 @@ incoming_heartbeat(Config) ->
 %%% HELPERS
 %%%
 
-receive_messages(Receiver, Num) ->
-    [begin
-         ct:pal("receive_messages ~tp", [T]),
-         ok = receive_one(Receiver)
-     end || T <- lists:seq(1, Num)].
+await_link(Who, What, Err) ->
+    receive
+        {amqp10_event, {link, Who0, What0}}
+          when Who0 =:= Who andalso
+               What0 =:= What ->
+            ok;
+        {amqp10_event, {link, Who0, {detached, Why}}}
+          when Who0 =:= Who ->
+            exit(Why)
+    after 5000 ->
+              flush(),
+              exit(Err)
+    end.
 
 publish_messages(Sender, Data, Num) ->
     [begin
@@ -717,36 +831,42 @@ publish_messages(Sender, Data, Num) ->
         ok = await_disposition(Tag)
      end || T <- lists:seq(1, Num)].
 
-receive_one(Receiver) ->
-    receive
-        {amqp10_msg, Receiver0, Msg}
-          when Receiver0 =:= Receiver ->
-            amqp10_client:accept_msg(Receiver, Msg)
-    after 2000 ->
-          timeout
-    end.
-
 await_disposition(DeliveryTag) ->
     receive
         {amqp10_disposition, {accepted, DeliveryTag0}}
           when DeliveryTag0 =:= DeliveryTag -> ok
     after 3000 ->
               flush(),
-              exit(dispostion_timeout)
+              ct:fail(dispostion_timeout)
     end.
 
-await_link(Who, What, Err) ->
+count_received_messages(Receiver) ->
+    count_received_messages0(Receiver, 0).
+
+count_received_messages0(Receiver, Count) ->
     receive
-        {amqp10_event, {link, Who0, What0}}
-          when Who0 =:= Who andalso
-               What0 =:= What ->
-            ok;
-        {amqp10_event, {link, Who0, {detached, Why}}}
-          when Who0 =:= Who ->
-            exit(Why)
-    after 5000 ->
-              flush(),
-              exit(Err)
+        {amqp10_msg, Receiver, _Msg} ->
+            count_received_messages0(Receiver, Count + 1)
+    after 200 ->
+              Count
+    end.
+
+receive_messages(Receiver, N) ->
+    receive_messages0(Receiver, N, []).
+
+receive_messages0(_Receiver, 0, Acc) ->
+    lists:reverse(Acc);
+receive_messages0(Receiver, N, Acc) ->
+    receive
+        {amqp10_msg, Receiver, Msg} ->
+            receive_messages0(Receiver, N - 1, [Msg | Acc])
+    after 5000  ->
+              ct:fail({timeout, {num_received, length(Acc)}, {num_missing, N}})
+    end.
+
+assert_no_message(Receiver) ->
+    receive {amqp10_msg, Receiver, Msg} -> ct:fail({unexpected_message, Msg})
+    after 50 -> ok
     end.
 
 to_bin(X) when is_list(X) ->
diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl
index 884f54033a9f..c450ca06e346 100644
--- a/deps/rabbit/test/amqp_client_SUITE.erl
+++ b/deps/rabbit/test/amqp_client_SUITE.erl
@@ -492,7 +492,7 @@ receiver_settle_mode_first(Config) ->
     ?assertEqual(DeliveryIdMsg9, serial_number_increment(DeliveryIdMsg8)),
     Last1 = serial_number_increment(serial_number_increment(DeliveryIdMsg9)),
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg8, Last1, true, accepted),
+           Receiver, DeliveryIdMsg8, Last1, true, accepted),
     assert_messages(QName, 8, 7, Config),
 
     %% 2. Ack a range smaller than the number of unacked messages where all delivery IDs
@@ -501,7 +501,7 @@ receiver_settle_mode_first(Config) ->
     DeliveryIdMsg4 = amqp10_msg:delivery_id(Msg4),
     DeliveryIdMsg6 = amqp10_msg:delivery_id(Msg6),
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg4, DeliveryIdMsg6, true, accepted),
+           Receiver, DeliveryIdMsg4, DeliveryIdMsg6, true, accepted),
     assert_messages(QName, 5, 4, Config),
 
     %% 3. Ack a range larger than the number of unacked messages where all delivery IDs
@@ -509,7 +509,7 @@ receiver_settle_mode_first(Config) ->
     DeliveryIdMsg2 = amqp10_msg:delivery_id(Msg2),
     DeliveryIdMsg7 = amqp10_msg:delivery_id(Msg7),
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg2, DeliveryIdMsg7, true, accepted),
+           Receiver, DeliveryIdMsg2, DeliveryIdMsg7, true, accepted),
     assert_messages(QName, 2, 1, Config),
 
     %% Consume the last message.
@@ -523,16 +523,16 @@ receiver_settle_mode_first(Config) ->
     DeliveryIdMsg10 = amqp10_msg:delivery_id(Msg10),
     Last2 = serial_number_increment(DeliveryIdMsg10),
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg1, Last2, true, accepted),
+           Receiver, DeliveryIdMsg1, Last2, true, accepted),
     assert_messages(QName, 0, 0, Config),
 
     %% 5. Ack single delivery ID when there are no unacked messages.
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg1, DeliveryIdMsg1, true, accepted),
+           Receiver, DeliveryIdMsg1, DeliveryIdMsg1, true, accepted),
 
     %% 6. Ack multiple delivery IDs when there are no unacked messages.
     ok = amqp10_client_session:disposition(
-           Session, receiver, DeliveryIdMsg1, DeliveryIdMsg6, true, accepted),
+           Receiver, DeliveryIdMsg1, DeliveryIdMsg6, true, accepted),
     assert_messages(QName, 0, 0, Config),
 
     ok = amqp10_client:detach_link(Receiver),
@@ -684,7 +684,7 @@ amqp_stream_amqpl(Config) ->
          #amqp_msg{props = #'P_basic'{type = <<"amqp-1.0">>}}} ->
             ok
     after 5000 ->
-              exit(basic_deliver_timeout)
+              ct:fail(basic_deliver_timeout)
     end,
     #'queue.delete_ok'{} = amqp_channel:call(Ch, #'queue.delete'{queue = QName}),
     ok = rabbit_ct_client_helpers:close_channel(Ch).
@@ -1736,7 +1736,7 @@ detach_requeues(Config) ->
 
     %% Receiver2 accepts all 4 messages.
     ok = amqp10_client_session:disposition(
-           Session, receiver,
+           Receiver2,
            amqp10_msg:delivery_id(Msg2),
            amqp10_msg:delivery_id(Msg3b),
            true, accepted),
@@ -2328,29 +2328,20 @@ async_notify(SenderSettleMode, QType, Config) ->
     end,
 
     %% Initially, grant 10 credits to the sending queue.
-    %% Whenever credits drops below 5, renew back to 10.
+    %% Whenever the sum of credits and number of unsettled messages drops below 5, renew back to 10.
     ok = amqp10_client:flow_link_credit(Receiver, 10, 5),
 
     %% We should receive all messages.
-    Msgs = receive_messages(Receiver, NumMsgs),
+    Accept = case SenderSettleMode of
+                 settled -> false;
+                 unsettled -> true
+             end,
+    Msgs = receive_all_messages(Receiver, Accept),
     FirstMsg = hd(Msgs),
     LastMsg = lists:last(Msgs),
     ?assertEqual([<<"1">>], amqp10_msg:body(FirstMsg)),
     ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(LastMsg)),
 
-    case SenderSettleMode of
-        settled ->
-            ok;
-        unsettled ->
-            ok = amqp10_client_session:disposition(
-                   Session,
-                   receiver,
-                   amqp10_msg:delivery_id(FirstMsg),
-                   amqp10_msg:delivery_id(LastMsg),
-                   true,
-                   accepted)
-    end,
-
     %% No further messages should be delivered.
     receive Unexpected -> ct:fail({received_unexpected_message, Unexpected})
     after 50 -> ok
@@ -2503,8 +2494,7 @@ queue_and_client_different_nodes(QueueLeaderNode, ClientNode, QueueType, Config)
     ?assertEqual([<<"1">>], amqp10_msg:body(FirstMsg)),
     ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(LastMsg)),
     ok = amqp10_client_session:disposition(
-           Session,
-           receiver,
+           Receiver,
            amqp10_msg:delivery_id(FirstMsg),
            amqp10_msg:delivery_id(LastMsg),
            true,
@@ -2803,7 +2793,7 @@ stream_filtering(Config) ->
                             #{<<"rabbitmq:stream-offset-spec">> => <<"first">>,
                               <<"rabbitmq:stream-filter">> => <<"apple">>}),
     ok = amqp10_client:flow_link_credit(AppleReceiver, 100, 10),
-    AppleMessages = receive_all_messages(AppleReceiver, []),
+    AppleMessages = receive_all_messages(AppleReceiver, true),
     %% we should get less than all the waves combined
     ?assert(length(AppleMessages) < WaveCount * 3),
     %% client-side filtering
@@ -2824,7 +2814,7 @@ stream_filtering(Config) ->
                                   #{<<"rabbitmq:stream-offset-spec">> => <<"first">>,
                                     <<"rabbitmq:stream-filter">> => [<<"apple">>, <<"orange">>]}),
     ok = amqp10_client:flow_link_credit(AppleOrangeReceiver, 100, 10),
-    AppleOrangeMessages = receive_all_messages(AppleOrangeReceiver, []),
+    AppleOrangeMessages = receive_all_messages(AppleOrangeReceiver, true),
     %% we should get less than all the waves combined
     ?assert(length(AppleOrangeMessages) < WaveCount * 3),
     %% client-side filtering
@@ -2848,7 +2838,7 @@ stream_filtering(Config) ->
                                         <<"rabbitmq:stream-match-unfiltered">> => {boolean, true}}),
     ok = amqp10_client:flow_link_credit(AppleUnfilteredReceiver, 100, 10),
 
-    AppleUnfilteredMessages = receive_all_messages(AppleUnfilteredReceiver, []),
+    AppleUnfilteredMessages = receive_all_messages(AppleUnfilteredReceiver, true),
     %% we should get less than all the waves combined
     ?assert(length(AppleUnfilteredMessages) < WaveCount * 3),
     %% client-side filtering
@@ -3351,10 +3341,16 @@ classic_priority_queue(Config) ->
 %% internal
 %%
 
-receive_all_messages(Receiver, Acc) ->
+receive_all_messages(Receiver, Accept) ->
+    receive_all_messages0(Receiver, Accept, []).
+
+receive_all_messages0(Receiver, Accept, Acc) ->
     receive {amqp10_msg, Receiver, Msg} ->
-                ok = amqp10_client:accept_msg(Receiver, Msg),
-                receive_all_messages(Receiver, [Msg | Acc])
+                case Accept of
+                    true -> ok = amqp10_client:accept_msg(Receiver, Msg);
+                    false -> ok
+                end,
+                receive_all_messages0(Receiver, Accept, [Msg | Acc])
     after 500 ->
               lists:reverse(Acc)
     end.
@@ -3501,7 +3497,7 @@ receive_messages0(Receiver, N, Acc) ->
         {amqp10_msg, Receiver, Msg} -> 
             receive_messages0(Receiver, N - 1, [Msg | Acc])
     after 5000  ->
-              exit({timeout, {num_received, length(Acc)}, {num_missing, N}})
+              ct:fail({timeout, {num_received, length(Acc)}, {num_missing, N}})
     end.
 
 count_received_messages(Receiver) ->
diff --git a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl
index b9f053db6ce9..fb46a7def31f 100644
--- a/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl
+++ b/deps/rabbit/test/amqp_credit_api_v2_SUITE.erl
@@ -121,8 +121,8 @@ credit_api_v2(Config) ->
                      filter => #{}},
     {ok, QQReceiver1} = amqp10_client:attach_link(Session, QQAttachArgs),
 
-    ok = consume_and_accept(10, CQReceiver1, Session),
-    ok = consume_and_accept(10, QQReceiver1, Session),
+    ok = consume_and_accept(10, CQReceiver1),
+    ok = consume_and_accept(10, QQReceiver1),
 
     ?assertEqual(ok,
                  rabbit_ct_broker_helpers:enable_feature_flag(Config, ?FUNCTION_NAME)),
@@ -133,12 +133,12 @@ credit_api_v2(Config) ->
                           Session, <<"cq receiver 2">>, CQAddr, unsettled),
     {ok, QQReceiver2} = amqp10_client:attach_receiver_link(
                           Session, <<"qq receiver 2">>, QQAddr, unsettled),
-    ok = consume_and_accept(10, CQReceiver2, Session),
-    ok = consume_and_accept(10, QQReceiver2, Session),
+    ok = consume_and_accept(10, CQReceiver2),
+    ok = consume_and_accept(10, QQReceiver2),
 
     %% Consume via with credit API v1
-    ok = consume_and_accept(10, CQReceiver1, Session),
-    ok = consume_and_accept(10, QQReceiver1, Session),
+    ok = consume_and_accept(10, CQReceiver1),
+    ok = consume_and_accept(10, QQReceiver1),
 
     %% Detach the credit API v1 links and attach with the same output handle.
     ok = detach_sync(CQReceiver1),
@@ -147,8 +147,8 @@ credit_api_v2(Config) ->
     {ok, QQReceiver3} = amqp10_client:attach_link(Session, QQAttachArgs),
 
     %% The new links should use credit API v2
-    ok = consume_and_accept(10, CQReceiver3, Session),
-    ok = consume_and_accept(10, QQReceiver3, Session),
+    ok = consume_and_accept(10, CQReceiver3),
+    ok = consume_and_accept(10, QQReceiver3),
 
     flush(pre_drain),
     %% Draining should also work.
@@ -181,12 +181,11 @@ credit_api_v2(Config) ->
     after 5000 -> ct:fail(missing_closed)
     end.
 
-consume_and_accept(NumMsgs, Receiver, Session) ->
+consume_and_accept(NumMsgs, Receiver) ->
     ok = amqp10_client:flow_link_credit(Receiver, NumMsgs, never),
     Msgs = receive_messages(Receiver, NumMsgs),
     ok = amqp10_client_session:disposition(
-           Session,
-           receiver,
+           Receiver,
            amqp10_msg:delivery_id(hd(Msgs)),
            amqp10_msg:delivery_id(lists:last(Msgs)),
            true,
diff --git a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
index 147626de7f72..492fd535d959 100644
--- a/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
+++ b/deps/rabbitmq_shovel/src/rabbit_amqp10_shovel.erl
@@ -283,30 +283,24 @@ close_dest(#{dest := #{current := #{conn := Conn}}}) ->
 close_dest(_Config) -> ok.
 
 -spec ack(Tag :: tag(), Multi :: boolean(), state()) -> state().
-ack(Tag, true, State = #{source := #{current := #{session := Session},
+ack(Tag, true, State = #{source := #{current := #{link := LinkRef},
                                      last_acked_tag := LastTag} = Src}) ->
     First = LastTag + 1,
-    ok = amqp10_client_session:disposition(Session, receiver, First,
-                                           Tag, true, accepted),
+    ok = amqp10_client_session:disposition(LinkRef, First, Tag, true, accepted),
     State#{source => Src#{last_acked_tag => Tag}};
-ack(Tag, false, State = #{source := #{current :=
-                                      #{session := Session}} = Src}) ->
-    ok = amqp10_client_session:disposition(Session, receiver, Tag,
-                                           Tag, true, accepted),
+ack(Tag, false, State = #{source := #{current := #{link := LinkRef}} = Src}) ->
+    ok = amqp10_client_session:disposition(LinkRef, Tag, Tag, true, accepted),
     State#{source => Src#{last_acked_tag => Tag}}.
 
 -spec nack(Tag :: tag(), Multi :: boolean(), state()) -> state().
-nack(Tag, false, State = #{source :=
-                           #{current := #{session := Session}} = Src}) ->
+nack(Tag, false, State = #{source := #{current := #{link := LinkRef}} = Src}) ->
     % the tag is the same as the deliveryid
-    ok = amqp10_client_session:disposition(Session, receiver, Tag,
-                                           Tag, false, rejected),
+    ok = amqp10_client_session:disposition(LinkRef, Tag, Tag, true, rejected),
     State#{source => Src#{last_nacked_tag => Tag}};
-nack(Tag, true, State = #{source := #{current := #{session := Session},
-                                     last_nacked_tag := LastTag} = Src}) ->
+nack(Tag, true, State = #{source := #{current := #{link := LinkRef},
+                                      last_nacked_tag := LastTag} = Src}) ->
     First = LastTag + 1,
-    ok = amqp10_client_session:disposition(Session, receiver, First,
-                                           Tag, true, accepted),
+    ok = amqp10_client_session:disposition(LinkRef, First, Tag, true, rejected),
     State#{source => Src#{last_nacked_tag => Tag}}.
 
 status(#{dest := #{current := #{link_state := attached}}}) ->
diff --git a/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl b/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl
index 3ae1fbe603a5..6a5dd4151d5a 100644
--- a/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl
+++ b/deps/rabbitmq_stream/test/protocol_interop_SUITE.erl
@@ -178,8 +178,7 @@ amqp_credit_multiple_grants(Config) ->
 
     %% Let's ack all of them.
     ok = amqp10_client_session:disposition(
-           Session,
-           receiver,
+           Receiver,
            amqp10_msg:delivery_id(M1),
            amqp10_msg:delivery_id(M4),
            true,
@@ -226,8 +225,7 @@ amqp_credit_multiple_grants(Config) ->
 
     %% Let's ack them all.
     ok = amqp10_client_session:disposition(
-           Session,
-           receiver,
+           Receiver,
            amqp10_msg:delivery_id(M5),
            amqp10_msg:delivery_id(M11),
            true,
diff --git a/moduleindex.yaml b/moduleindex.yaml
index 19f69504b60d..b4cf2824d75e 100755
--- a/moduleindex.yaml
+++ b/moduleindex.yaml
@@ -3,6 +3,23 @@ accept:
 - accept_header
 - accept_neg
 - accept_parser
+amqp10_client:
+- amqp10_client
+- amqp10_client_app
+- amqp10_client_connection
+- amqp10_client_connection_sup
+- amqp10_client_frame_reader
+- amqp10_client_session
+- amqp10_client_sessions_sup
+- amqp10_client_sup
+- amqp10_client_types
+- amqp10_msg
+amqp10_common:
+- amqp10_binary_generator
+- amqp10_binary_parser
+- amqp10_framing
+- amqp10_framing0
+- serial_number
 amqp_client:
 - amqp_auth_mechanisms
 - amqp_channel
@@ -28,23 +45,6 @@ amqp_client:
 - amqp_util
 - rabbit_routing_util
 - uri_parser
-amqp10_client:
-- amqp10_client
-- amqp10_client_app
-- amqp10_client_connection
-- amqp10_client_connection_sup
-- amqp10_client_frame_reader
-- amqp10_client_session
-- amqp10_client_sessions_sup
-- amqp10_client_sup
-- amqp10_client_types
-- amqp10_msg
-amqp10_common:
-- amqp10_binary_generator
-- amqp10_binary_parser
-- amqp10_framing
-- amqp10_framing0
-- serial_number
 aten:
 - aten
 - aten_app

From 63b3a4f93d0f88ab40bd75f2fd169420a70a0b52 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Wed, 21 Feb 2024 10:51:35 +0100
Subject: [PATCH 08/16] Bump ActiveMQ to 5.18.3

Remove bean `logQuery` as described in
https://github.com/spring-attic/spring-native/issues/1708#issuecomment-1384669898
to avoid ActiveMQ start up failure with reason
```
java.lang.ClassNotFoundException: io.fabric8.insight.log.log4j.Log4jLogQuery
```
---
 deps/amqp10_client/Makefile                   |   2 +-
 deps/amqp10_client/activemq.bzl               |   4 +-
 deps/amqp10_client/test/system_SUITE.erl      | 199 ++++++++++--------
 .../test/system_SUITE_data/conf/activemq.xml  |   6 -
 .../conf/activemq_no_anon.xml                 |   6 -
 moduleindex.yaml                              |  34 +--
 6 files changed, 135 insertions(+), 116 deletions(-)

diff --git a/deps/amqp10_client/Makefile b/deps/amqp10_client/Makefile
index b42ca3017e45..466bde568804 100644
--- a/deps/amqp10_client/Makefile
+++ b/deps/amqp10_client/Makefile
@@ -55,7 +55,7 @@ HEX_TARBALL_FILES += rabbitmq-components.mk \
 # ActiveMQ for the testsuite.
 # --------------------------------------------------------------------
 
-ACTIVEMQ_VERSION := 5.14.4
+ACTIVEMQ_VERSION := 5.18.3
 ACTIVEMQ_URL := 'https://archive.apache.org/dist/activemq/$(ACTIVEMQ_VERSION)/apache-activemq-$(ACTIVEMQ_VERSION)-bin.tar.gz'
 
 ACTIVEMQ := $(abspath test/system_SUITE_data/apache-activemq-$(ACTIVEMQ_VERSION)/bin/activemq)
diff --git a/deps/amqp10_client/activemq.bzl b/deps/amqp10_client/activemq.bzl
index 7ad2f5cb4d88..7cffe4dea891 100644
--- a/deps/amqp10_client/activemq.bzl
+++ b/deps/amqp10_client/activemq.bzl
@@ -1,8 +1,8 @@
 load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive")
 
-ACTIVEMQ_VERSION = "5.14.4"
+ACTIVEMQ_VERSION = "5.18.3"
 ACTIVEMQ_URL = "https://archive.apache.org/dist/activemq/{version}/apache-activemq-{version}-bin.tar.gz".format(version = ACTIVEMQ_VERSION)
-SHA_256 = "16ec52bece0a4759f9d70f4132d7d8da67d662e4af029081c492e65510a695c1"
+SHA_256 = "943381aa6d340707de6c42eadbf7b41b7fdf93df604156d972d50c4da783544f"
 
 def activemq_archive():
     http_archive(
diff --git a/deps/amqp10_client/test/system_SUITE.erl b/deps/amqp10_client/test/system_SUITE.erl
index 62a7718657ef..62ab9c2a461d 100644
--- a/deps/amqp10_client/test/system_SUITE.erl
+++ b/deps/amqp10_client/test/system_SUITE.erl
@@ -17,7 +17,7 @@
 -compile([export_all, nowarn_export_all]).
 
 suite() ->
-    [{timetrap, {seconds, 120}}].
+    [{timetrap, {minutes, 4}}].
 
 all() ->
     [
@@ -64,7 +64,8 @@ shared() ->
      split_transfer,
      transfer_unsettled,
      subscribe,
-     subscribe_with_auto_flow,
+     subscribe_with_auto_flow_settled,
+     subscribe_with_auto_flow_unsettled,
      outgoing_heartbeat,
      roundtrip_large_messages,
      transfer_id_vs_delivery_id
@@ -290,12 +291,15 @@ roundtrip_large_messages(Config) ->
     Hostname = ?config(rmq_hostname, Config),
     Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
     OpenConf = #{address => Hostname, port => Port, sasl => anon},
-    DataKb = crypto:strong_rand_bytes(1024),
-    roundtrip(OpenConf, DataKb),
-    Data1Mb = binary:copy(DataKb, 1024),
-    roundtrip(OpenConf, Data1Mb),
-    roundtrip(OpenConf, binary:copy(Data1Mb, 8)),
-    ok = roundtrip(OpenConf, binary:copy(Data1Mb, 64)).
+
+    DataKb = rand:bytes(1024),
+    DataMb = rand:bytes(1024 * 1024),
+    Data8Mb = rand:bytes(8 * 1024 * 1024),
+    Data64Mb = rand:bytes(64 * 1024 * 1024),
+    ok = roundtrip(OpenConf, DataKb),
+    ok = roundtrip(OpenConf, DataMb),
+    ok = roundtrip(OpenConf, Data8Mb),
+    ok = roundtrip(OpenConf, Data64Mb).
 
 roundtrip(OpenConf) ->
     roundtrip(OpenConf, <<"banana">>).
@@ -321,9 +325,10 @@ roundtrip(OpenConf, Body) ->
     {error, link_not_found} = amqp10_client:detach_link(Sender),
     {ok, Receiver} = amqp10_client:attach_receiver_link(
                        Session, <<"banana-receiver">>, <<"test1">>, settled, unsettled_state),
-    {ok, OutMsg} = amqp10_client:get_msg(Receiver, 60_000 * 4),
+    {ok, OutMsg} = amqp10_client:get_msg(Receiver, 4 * 60_000),
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection),
+
     % ct:pal(?LOW_IMPORTANCE, "roundtrip message Out: ~tp~nIn: ~tp~n", [OutMsg, Msg]),
     #{creation_time := Now} = amqp10_msg:properties(OutMsg),
     #{<<"a_key">> := <<"a_value">>} = amqp10_msg:application_properties(OutMsg),
@@ -502,7 +507,7 @@ transfer_unsettled(Config) ->
 subscribe(Config) ->
     Hostname = ?config(rmq_hostname, Config),
     Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    QueueName = <<"test-sub">>,
+    QueueName = atom_to_binary(?FUNCTION_NAME),
     {ok, Connection} = amqp10_client:open_connection(Hostname, Port),
     {ok, Session} = amqp10_client:begin_session(Connection),
     {ok, Sender} = amqp10_client:attach_sender_link_sync(Session,
@@ -530,10 +535,11 @@ subscribe(Config) ->
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection).
 
-subscribe_with_auto_flow(Config) ->
+subscribe_with_auto_flow_settled(Config) ->
+    SenderSettleMode = settled,
     Hostname = ?config(rmq_hostname, Config),
     Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
-    QueueName = <<"test-sub">>,
+    QueueName = atom_to_binary(?FUNCTION_NAME),
     {ok, Connection} = amqp10_client:open_connection(Hostname, Port),
     {ok, Session} = amqp10_client:begin_session(Connection),
     {ok, Sender} = amqp10_client:attach_sender_link_sync(Session,
@@ -541,93 +547,109 @@ subscribe_with_auto_flow(Config) ->
                                                          QueueName),
     await_link(Sender, credited, link_credit_timeout),
 
-    _ = publish_messages(Sender, <<"banana">>, 20),
-    %% Use sender settle mode 'settled'.
-    {ok, R1} = amqp10_client:attach_receiver_link(
-                 Session, <<"sub-receiver-1">>, QueueName, settled),
-    await_link(R1, attached, attached_timeout),
-    ok = amqp10_client:flow_link_credit(R1, 5, 2),
-    ?assertEqual(20, count_received_messages(R1)),
-    ok = amqp10_client:detach_link(R1),
+    publish_messages(Sender, <<"banana">>, 20),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(
+                       Session, <<"sub-receiver">>, QueueName, SenderSettleMode),
+    await_link(Receiver, attached, attached_timeout),
+
+    ok = amqp10_client:flow_link_credit(Receiver, 5, 2),
+    ?assertEqual(20, count_received_messages(Receiver)),
+
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:detach_link(Sender),
+    ok = amqp10_client:end_session(Session),
+    ok = amqp10_client:close_connection(Connection).
+
+subscribe_with_auto_flow_unsettled(Config) ->
+    SenderSettleMode = unsettled,
+    Hostname = ?config(rmq_hostname, Config),
+    Port = rabbit_ct_broker_helpers:get_node_config(Config, 0, tcp_port_amqp),
+    QueueName = atom_to_binary(?FUNCTION_NAME),
+    {ok, Connection} = amqp10_client:open_connection(Hostname, Port),
+    {ok, Session} = amqp10_client:begin_session(Connection),
+    {ok, Sender} = amqp10_client:attach_sender_link_sync(Session,
+                                                         <<"sub-sender">>,
+                                                         QueueName),
+    await_link(Sender, credited, link_credit_timeout),
 
-    _ = publish_messages(Sender, <<"banana">>, 30),
+    _ = publish_messages(Sender, <<"1-">>, 30),
     %% Use sender settle mode 'unsettled'.
     %% This should require us to manually settle message in order to receive more messages.
-    {ok, R2} = amqp10_client:attach_receiver_link(Session, <<"sub-receiver-2">>, QueueName, unsettled),
-    await_link(R2, attached, attached_timeout),
-    ok = amqp10_client:flow_link_credit(R2, 5, 2),
+    {ok, Receiver} = amqp10_client:attach_receiver_link(Session, <<"sub-receiver-2">>, QueueName, SenderSettleMode),
+    await_link(Receiver, attached, attached_timeout),
+    ok = amqp10_client:flow_link_credit(Receiver, 5, 2),
     %% We should receive exactly 5 messages.
-    [M1, _M2, M3, M4, M5] = receive_messages(R2, 5),
-    ok = assert_no_message(R2),
+    [M1, _M2, M3, M4, M5] = receive_messages(Receiver, 5),
+    ok = assert_no_message(Receiver),
 
     %% Even when we accept the first 3 messages, the number of unsettled messages has not yet fallen below 2.
     %% Therefore, the client should not yet grant more credits to the sender.
     ok = amqp10_client_session:disposition(
-           R2, amqp10_msg:delivery_id(M1), amqp10_msg:delivery_id(M3), true, accepted),
-    ok = assert_no_message(R2),
+           Receiver, amqp10_msg:delivery_id(M1), amqp10_msg:delivery_id(M3), true, accepted),
+    ok = assert_no_message(Receiver),
 
     %% When we accept 1 more message (the order in which we accept shouldn't matter, here we accept M5 before M4),
     %% the number of unsettled messages now falls below 2 (since only M4 is left unsettled).
     %% Therefore, the client should grant 5 credits to the sender.
     %% Therefore, we should receive 5 more messages.
-    ok = amqp10_client:accept_msg(R2, M5),
-    [_M6, _M7, _M8, _M9, M10] = receive_messages(R2, 5),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:accept_msg(Receiver, M5),
+    [_M6, _M7, _M8, _M9, M10] = receive_messages(Receiver, 5),
+    ok = assert_no_message(Receiver),
 
     %% It shouldn't matter how we settle messages, therefore we use 'rejected' this time.
     %% Settling all in flight messages should cause us to receive exactly 5 more messages.
     ok = amqp10_client_session:disposition(
-           R2, amqp10_msg:delivery_id(M4), amqp10_msg:delivery_id(M10), true, rejected),
-    [M11, _M12, _M13, _M14, M15] = receive_messages(R2, 5),
-    ok = assert_no_message(R2),
+           Receiver, amqp10_msg:delivery_id(M4), amqp10_msg:delivery_id(M10), true, rejected),
+    [M11, _M12, _M13, _M14, M15] = receive_messages(Receiver, 5),
+    ok = assert_no_message(Receiver),
 
     %% Dynamically decrease link credit.
     %% Since we explicitly tell to grant 3 new credits now, we expect to receive 3 more messages.
-    ok = amqp10_client:flow_link_credit(R2, 3, 3),
-    [M16, _M17, M18] = receive_messages(R2, 3),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:flow_link_credit(Receiver, 3, 3),
+    [M16, _M17, M18] = receive_messages(Receiver, 3),
+    ok = assert_no_message(Receiver),
 
     ok = amqp10_client_session:disposition(
-           R2, amqp10_msg:delivery_id(M11), amqp10_msg:delivery_id(M15), true, accepted),
+           Receiver, amqp10_msg:delivery_id(M11), amqp10_msg:delivery_id(M15), true, accepted),
     %% However, the RenewWhenBelow=3 still refers to all unsettled messages.
     %% Right now we have 3 messages (M16, M17, M18) unsettled.
-    ok = assert_no_message(R2),
+    ok = assert_no_message(Receiver),
 
     %% Settling 1 out of these 3 messages causes RenewWhenBelow to fall below 3 resulting
     %% in 3 new messages to be received.
-    ok = amqp10_client:accept_msg(R2, M18),
-    [_M19, _M20, _M21] = receive_messages(R2, 3),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:accept_msg(Receiver, M18),
+    [_M19, _M20, _M21] = receive_messages(Receiver, 3),
+    ok = assert_no_message(Receiver),
 
-    ok = amqp10_client:flow_link_credit(R2, 3, never, true),
-    [_M22, _M23, M24] = receive_messages(R2, 3),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:flow_link_credit(Receiver, 3, never, true),
+    [_M22, _M23, M24] = receive_messages(Receiver, 3),
+    ok = assert_no_message(Receiver),
 
     %% Since RenewWhenBelow = never, we expect to receive no new messages despite settling.
     ok = amqp10_client_session:disposition(
-           R2, amqp10_msg:delivery_id(M16), amqp10_msg:delivery_id(M24), true, rejected),
-    ok = assert_no_message(R2),
+           Receiver, amqp10_msg:delivery_id(M16), amqp10_msg:delivery_id(M24), true, rejected),
+    ok = assert_no_message(Receiver),
 
-    ok = amqp10_client:flow_link_credit(R2, 2, never, false),
-    [M25, _M26] = receive_messages(R2, 2),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:flow_link_credit(Receiver, 2, never, false),
+    [M25, _M26] = receive_messages(Receiver, 2),
+    ok = assert_no_message(Receiver),
 
-    ok = amqp10_client:flow_link_credit(R2, 3, 3),
-    [_M27, _M28, M29] = receive_messages(R2, 3),
-    ok = assert_no_message(R2),
+    ok = amqp10_client:flow_link_credit(Receiver, 3, 3),
+    [_M27, _M28, M29] = receive_messages(Receiver, 3),
+    ok = assert_no_message(Receiver),
 
     ok = amqp10_client_session:disposition(
-           R2, amqp10_msg:delivery_id(M25), amqp10_msg:delivery_id(M29), true, accepted),
-    [M30] = receive_messages(R2, 1),
-    ok = assert_no_message(R2),
-    ok = amqp10_client:accept_msg(R2, M30),
+           Receiver, amqp10_msg:delivery_id(M25), amqp10_msg:delivery_id(M29), true, accepted),
+    [M30] = receive_messages(Receiver, 1),
+    ok = assert_no_message(Receiver),
+    ok = amqp10_client:accept_msg(Receiver, M30),
     %% The sender queue is empty now.
-    ok = assert_no_message(R2),
+    ok = assert_no_message(Receiver),
 
-    ok = amqp10_client:flow_link_credit(R2, 3, 1),
-    _ = publish_messages(Sender, <<"banana">>, 1),
-    [M31] = receive_messages(R2, 1),
-    ok = amqp10_client:accept_msg(R2, M31),
+    ok = amqp10_client:flow_link_credit(Receiver, 3, 1),
+    _ = publish_messages(Sender, <<"2-">>, 1),
+    [M31] = receive_messages(Receiver, 1),
+    ok = amqp10_client:accept_msg(Receiver, M31),
 
     %% Since function flow_link_credit/3 documents
     %%     "if RenewWhenBelow is an integer, the amqp10_client will automatically grant more
@@ -637,24 +659,25 @@ subscribe_with_auto_flow(Config) ->
     %% remaining link credit (2) and unsettled messages (0) is 2.
     %%
     %% Therefore, when we publish another 3 messages, we expect to only receive only 2 messages!
-    _ = publish_messages(Sender, <<"banana">>, 5),
-    [M32, M33] = receive_messages(R2, 2),
-    ok = assert_no_message(R2),
+    _ = publish_messages(Sender, <<"3-">>, 5),
+    [M32, M33] = receive_messages(Receiver, 2),
+    ok = assert_no_message(Receiver),
 
     %% When we accept both messages, the sum of the remaining link credit (0) and unsettled messages (0)
     %% falls below RenewWhenBelow=1 causing the amqp10_client to grant 3 new credits.
-    ok = amqp10_client:accept_msg(R2, M32),
-    ok = assert_no_message(R2),
-    ok = amqp10_client:accept_msg(R2, M33),
-
-    [M35, M36, M37] = receive_messages(R2, 3),
-    ok = amqp10_client:accept_msg(R2, M35),
-    ok = amqp10_client:accept_msg(R2, M36),
-    ok = amqp10_client:accept_msg(R2, M37),
+    ok = amqp10_client:accept_msg(Receiver, M32),
+    ok = assert_no_message(Receiver),
+    ok = amqp10_client:accept_msg(Receiver, M33),
+
+    [M35, M36, M37] = receive_messages(Receiver, 3),
+    ok = amqp10_client:accept_msg(Receiver, M35),
+    ok = amqp10_client:accept_msg(Receiver, M36),
+    ok = amqp10_client:accept_msg(Receiver, M37),
     %% The sender queue is empty now.
-    ok = assert_no_message(R2),
+    ok = assert_no_message(Receiver),
 
-    ok = amqp10_client:detach_link(R2),
+    ok = amqp10_client:detach_link(Receiver),
+    ok = amqp10_client:detach_link(Sender),
     ok = amqp10_client:end_session(Session),
     ok = amqp10_client:close_connection(Connection).
 
@@ -817,18 +840,18 @@ await_link(Who, What, Err) ->
             ok;
         {amqp10_event, {link, Who0, {detached, Why}}}
           when Who0 =:= Who ->
-            exit(Why)
+            ct:fail(Why)
     after 5000 ->
               flush(),
-              exit(Err)
+              ct:fail(Err)
     end.
 
-publish_messages(Sender, Data, Num) ->
+publish_messages(Sender, BodyPrefix, Num) ->
     [begin
-        Tag = integer_to_binary(T),
-        Msg = amqp10_msg:new(Tag, Data, false),
-        ok = amqp10_client:send_msg(Sender, Msg),
-        ok = await_disposition(Tag)
+         Tag = integer_to_binary(T),
+         Msg = amqp10_msg:new(Tag, <>, false),
+         ok = amqp10_client:send_msg(Sender, Msg),
+         ok = await_disposition(Tag)
      end || T <- lists:seq(1, Num)].
 
 await_disposition(DeliveryTag) ->
@@ -847,7 +870,7 @@ count_received_messages0(Receiver, Count) ->
     receive
         {amqp10_msg, Receiver, _Msg} ->
             count_received_messages0(Receiver, Count + 1)
-    after 200 ->
+    after 500 ->
               Count
     end.
 
@@ -861,7 +884,15 @@ receive_messages0(Receiver, N, Acc) ->
         {amqp10_msg, Receiver, Msg} ->
             receive_messages0(Receiver, N - 1, [Msg | Acc])
     after 5000  ->
-              ct:fail({timeout, {num_received, length(Acc)}, {num_missing, N}})
+              LastReceivedMsg = case Acc of
+                                    [] -> none;
+                                    [M | _] -> M
+                                end,
+              ct:fail({timeout,
+                       {num_received, length(Acc)},
+                       {num_missing, N},
+                       {last_received_msg, LastReceivedMsg}
+                      })
     end.
 
 assert_no_message(Receiver) ->
diff --git a/deps/amqp10_client/test/system_SUITE_data/conf/activemq.xml b/deps/amqp10_client/test/system_SUITE_data/conf/activemq.xml
index 2b9d37ed1812..9e14c13e84d2 100644
--- a/deps/amqp10_client/test/system_SUITE_data/conf/activemq.xml
+++ b/deps/amqp10_client/test/system_SUITE_data/conf/activemq.xml
@@ -28,12 +28,6 @@
         
     
 
-   
-    
-    
-
     
diff --git a/deps/amqp10_client/test/system_SUITE_data/conf/activemq_no_anon.xml b/deps/amqp10_client/test/system_SUITE_data/conf/activemq_no_anon.xml
index 2e489fa2b9d2..fd2dbfc5bba3 100644
--- a/deps/amqp10_client/test/system_SUITE_data/conf/activemq_no_anon.xml
+++ b/deps/amqp10_client/test/system_SUITE_data/conf/activemq_no_anon.xml
@@ -28,12 +28,6 @@
         
     
 
-   
-    
-    
-
     
diff --git a/moduleindex.yaml b/moduleindex.yaml
index b4cf2824d75e..19f69504b60d 100755
--- a/moduleindex.yaml
+++ b/moduleindex.yaml
@@ -3,23 +3,6 @@ accept:
 - accept_header
 - accept_neg
 - accept_parser
-amqp10_client:
-- amqp10_client
-- amqp10_client_app
-- amqp10_client_connection
-- amqp10_client_connection_sup
-- amqp10_client_frame_reader
-- amqp10_client_session
-- amqp10_client_sessions_sup
-- amqp10_client_sup
-- amqp10_client_types
-- amqp10_msg
-amqp10_common:
-- amqp10_binary_generator
-- amqp10_binary_parser
-- amqp10_framing
-- amqp10_framing0
-- serial_number
 amqp_client:
 - amqp_auth_mechanisms
 - amqp_channel
@@ -45,6 +28,23 @@ amqp_client:
 - amqp_util
 - rabbit_routing_util
 - uri_parser
+amqp10_client:
+- amqp10_client
+- amqp10_client_app
+- amqp10_client_connection
+- amqp10_client_connection_sup
+- amqp10_client_frame_reader
+- amqp10_client_session
+- amqp10_client_sessions_sup
+- amqp10_client_sup
+- amqp10_client_types
+- amqp10_msg
+amqp10_common:
+- amqp10_binary_generator
+- amqp10_binary_parser
+- amqp10_framing
+- amqp10_framing0
+- serial_number
 aten:
 - aten
 - aten_app

From 9440134097ad4dd17e72441e13049e18e0521011 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Fri, 23 Feb 2024 12:47:57 +0100
Subject: [PATCH 09/16] Allow transfer without payload

Fixes test
```
./mvnw test -Dtest=ClientTest#largeMessageWithStreamSender
```
in client rabbitmq-java-model
---
 deps/rabbit/src/rabbit_amqp_session.erl | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 6b937c4c2fee..e72a6c13360a 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -903,6 +903,12 @@ handle_control({Txfr = #'v1_0.transfer'{handle = ?UINT(Handle)}, MsgPart},
                            "Unknown link handle: ~p", [Handle])
     end;
 
+%% Although the AMQP message format [3.2] requires a body, it is valid to send a transfer frame without payload.
+%% For example, when a large multi transfer message is streamed using the ProtonJ2 client, the client could send
+%% a final #'v1_0.transfer'{more=false} frame without a payload.
+handle_control(Txfr = #'v1_0.transfer'{}, State) ->
+    handle_control({Txfr, <<>>}, State);
+
 %% Flow control. These frames come with two pieces of information:
 %% the session window, and optionally, credit for a particular link.
 %% We'll deal with each of them separately.

From b43c26d417338ac817f5f06ef6da3542c8fd1429 Mon Sep 17 00:00:00 2001
From: Karl Nilsson 
Date: Mon, 26 Feb 2024 15:00:57 +0000
Subject: [PATCH 10/16] AMQP: send incoming messages with flow=noflow option.

As this disables the internal credit flow accounting that isn't
used anyway.
---
 deps/rabbit/src/rabbit_amqp_session.erl | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index e72a6c13360a..9ed7daa77764 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -309,8 +309,6 @@ handle_call(Msg, _From, State) ->
 
 handle_info(timeout, State) ->
     noreply(State);
-handle_info({bump_credit, _IgnoreMsg}, State) ->
-    noreply(State);
 handle_info({{'DOWN', QName}, _MRef, process, QPid, Reason},
             #state{queue_states = QStates0,
                    stashed_eol = Eol} = State0) ->
@@ -1613,7 +1611,8 @@ incoming_link_transfer(
                                        "rcv-settle-mode second not supported", []);
                 false -> ok
             end,
-            Opts = #{correlation => {HandleInt, DeliveryId}},
+            Opts = #{correlation => {HandleInt, DeliveryId},
+                     flow => noflow},
             Qs0 = rabbit_amqqueue:lookup_many(QNames),
             Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0),
             case rabbit_queue_type:deliver(Qs, Mc, Opts, QStates0) of

From a3f247b850fbaf1d5ea52b0874a39839bfb60db5 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Mon, 26 Feb 2024 18:54:19 +0100
Subject: [PATCH 11/16] Use default value noflow

---
 deps/rabbit/src/rabbit_amqp_session.erl | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 9ed7daa77764..a97adb27869c 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -1611,8 +1611,7 @@ incoming_link_transfer(
                                        "rcv-settle-mode second not supported", []);
                 false -> ok
             end,
-            Opts = #{correlation => {HandleInt, DeliveryId},
-                     flow => noflow},
+            Opts = #{correlation => {HandleInt, DeliveryId}},
             Qs0 = rabbit_amqqueue:lookup_many(QNames),
             Qs = rabbit_amqqueue:prepend_extra_bcc(Qs0),
             case rabbit_queue_type:deliver(Qs, Mc, Opts, QStates0) of

From 02c29ac1c026472b8fe31f0d8a31c2a74912346f Mon Sep 17 00:00:00 2001
From: Karl Nilsson 
Date: Tue, 27 Feb 2024 13:48:31 +0000
Subject: [PATCH 12/16] AMQP: disable all use of credit flow control for
 classic queues.

This aligns flow control behaviour for AMQP across all queue types.

All flow is controlled by the AMQP credit flow gestures rather than
relying on additional, parallel mechanism.

This will allow us to adjust the flow control approach for all
queue types and expect consistent results.
---
 deps/rabbit/src/rabbit_amqp_session.erl     | 40 ++++-------
 deps/rabbit/src/rabbit_amqqueue_process.erl | 17 ++---
 deps/rabbit/src/rabbit_queue_consumers.erl  | 76 ++++++++++++---------
 3 files changed, 65 insertions(+), 68 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index a97adb27869c..59a35a84db26 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -1128,35 +1128,23 @@ send_pending(#state{remote_incoming_window = Space,
             Flow = session_flow_fields(Flow0, State0),
             rabbit_amqp_writer:send_command(WriterPid, Ch, Flow),
             send_pending(State0#state{outgoing_pending = Buf});
-        {{value, #pending_transfer{
-                    frames = Frames,
-                    queue_pid = QPid,
-                    outgoing_unsettled = #outgoing_unsettled{queue_name = QName}
-                   } = Pending}, Buf1}
+        {{value, #pending_transfer{frames = Frames} = Pending}, Buf1}
           when Space > 0 ->
-            SendFun = case rabbit_queue_type:module(QName, State0#state.queue_states) of
-                          {ok, rabbit_classic_queue} ->
-                              fun(Transfer, Sections) ->
-                                      rabbit_amqp_writer:send_command_and_notify(
-                                        WriterPid, Ch, QPid, self(), Transfer, Sections)
-                              end;
-                          {ok, _QType} ->
-                              fun(Transfer, Sections) ->
-                                      rabbit_amqp_writer:send_command(
-                                        WriterPid, Ch, Transfer, Sections)
-                              end
+            SendFun = fun(Transfer, Sections) ->
+                              rabbit_amqp_writer:send_command(
+                                WriterPid, Ch, Transfer, Sections)
                       end,
             {NumTransfersSent, Buf, State1} =
-            case send_frames(SendFun, Frames, Space) of
-                {all, SpaceLeft} ->
-                    {Space - SpaceLeft,
-                     Buf1,
-                     record_outgoing_unsettled(Pending, State0)};
-                {some, Rest} ->
-                    {Space,
-                     queue:in_r(Pending#pending_transfer{frames = Rest}, Buf1),
-                     State0}
-            end,
+                case send_frames(SendFun, Frames, Space) of
+                    {all, SpaceLeft} ->
+                        {Space - SpaceLeft,
+                         Buf1,
+                         record_outgoing_unsettled(Pending, State0)};
+                    {some, Rest} ->
+                        {Space,
+                         queue:in_r(Pending#pending_transfer{frames = Rest}, Buf1),
+                         State0}
+                end,
             State2 = session_flow_control_sent_transfers(NumTransfersSent, State1),
             State = State2#state{outgoing_pending = Buf},
             send_pending(State);
diff --git a/deps/rabbit/src/rabbit_amqqueue_process.erl b/deps/rabbit/src/rabbit_amqqueue_process.erl
index 3edbce0bd0cf..f645faef88b9 100644
--- a/deps/rabbit/src/rabbit_amqqueue_process.erl
+++ b/deps/rabbit/src/rabbit_amqqueue_process.erl
@@ -1644,14 +1644,15 @@ handle_cast({credit, SessionPid, CTag, DeliveryCountRcv, Credit, Drain, Echo},
                q = Q} = State0) ->
     QName = amqqueue:get_name(Q),
     State = #q{backing_queue_state = PostBQS,
-               backing_queue = BQ} = case rabbit_queue_consumers:process_credit(
-                                            DeliveryCountRcv, Credit, SessionPid, CTag, Consumers0) of
-                                         unchanged ->
-                                             State0;
-                                         {unblocked, Consumers1} ->
-                                             State1 = State0#q{consumers = Consumers1},
-                                             run_message_queue(true, State1)
-                                     end,
+               backing_queue = BQ} =
+        case rabbit_queue_consumers:process_credit(
+               DeliveryCountRcv, Credit, SessionPid, CTag, Consumers0) of
+            unchanged ->
+                State0;
+            {unblocked, Consumers1} ->
+                State1 = State0#q{consumers = Consumers1},
+                run_message_queue(true, State1)
+        end,
     case rabbit_queue_consumers:get_link_state(SessionPid, CTag) of
         {credit_api_v1, PostCred}
           when Drain andalso
diff --git a/deps/rabbit/src/rabbit_queue_consumers.erl b/deps/rabbit/src/rabbit_queue_consumers.erl
index cb83974fe0a1..8cb1d299c16e 100644
--- a/deps/rabbit/src/rabbit_queue_consumers.erl
+++ b/deps/rabbit/src/rabbit_queue_consumers.erl
@@ -232,10 +232,12 @@ deliver(FetchFun, QName, State, SingleActiveConsumerIsOn, ActiveConsumer) ->
 deliver(_FetchFun, _QName, false, State, true, none) ->
     {undelivered, false,
         State#state{use = update_use(State#state.use, inactive)}};
-deliver(FetchFun, QName, false, State = #state{consumers = Consumers}, true, SingleActiveConsumer) ->
+deliver(FetchFun, QName, false, State = #state{consumers = Consumers}, true,
+        SingleActiveConsumer) ->
     {ChPid, Consumer} = SingleActiveConsumer,
-    %% blocked (rate/prefetch limited) consumers are removed from the queue state, but not the exclusive_consumer field,
-    %% so we need to do this check to avoid adding the exclusive consumer to the channel record
+    %% blocked (rate/prefetch limited) consumers are removed from the queue state,
+    %% but not the exclusive_consumer field, so we need to do this check to
+    %% avoid adding the exclusive consumer to the channel record
     %% over and over
     case is_blocked(SingleActiveConsumer) of
         true ->
@@ -271,40 +273,46 @@ deliver(FetchFun, QName, ConsumersChanged,
 
 deliver_to_consumer(FetchFun, E = {ChPid, Consumer}, QName) ->
     C = lookup_ch(ChPid),
-    case is_ch_blocked(C) of
-        true  ->
-            block_consumer(C, E),
-            undelivered;
-        false ->
-            CTag = Consumer#consumer.tag,
-            LinkStates = C#cr.link_states,
-            case maps:find(CTag, LinkStates) of
-                {ok, #link_state{delivery_count = DeliveryCount0,
-                                 credit = Credit} = LinkState0}
-                  when Credit > 0 ->
+    ChBlocked = is_ch_blocked(C),
+    LinkStates = C#cr.link_states,
+    CTag = Consumer#consumer.tag,
+    case LinkStates of
+        #{CTag := #link_state{delivery_count = DeliveryCount0,
+                              credit = Credit} = LinkState0} ->
+            %% bypass credit flow for link credit consumers
+            %% as it is handled separately
+            case Credit > 0 of
+                true ->
                     DeliveryCount = case DeliveryCount0 of
-                                        credit_api_v1 -> DeliveryCount0;
-                                        _ -> serial_number:add(DeliveryCount0, 1)
+                                        credit_api_v1 ->
+                                            DeliveryCount0;
+                                        _ ->
+                                            serial_number:add(DeliveryCount0, 1)
                                     end,
                     LinkState = LinkState0#link_state{delivery_count = DeliveryCount,
                                                       credit = Credit - 1},
                     C1 = C#cr{link_states = maps:update(CTag, LinkState, LinkStates)},
                     {delivered, deliver_to_consumer(FetchFun, Consumer, C1, QName)};
-                {ok, _Exhausted} ->
+                false ->
                     block_consumer(C, E),
+                    undelivered
+            end;
+        _ when ChBlocked ->
+            %% not a link credit consumer, use credit flow
+            block_consumer(C, E),
+            undelivered;
+        _ ->
+            %% not a link credit consumer, use credit flow
+            case rabbit_limiter:can_send(C#cr.limiter,
+                                         Consumer#consumer.ack_required,
+                                         CTag) of
+                {suspend, Limiter} ->
+                    block_consumer(C#cr{limiter = Limiter}, E),
                     undelivered;
-                error ->
-                    case rabbit_limiter:can_send(C#cr.limiter,
-                                                 Consumer#consumer.ack_required,
-                                                 CTag) of
-                        {suspend, Limiter} ->
-                            block_consumer(C#cr{limiter = Limiter}, E),
-                            undelivered;
-                        {continue, Limiter} ->
-                            {delivered, deliver_to_consumer(
-                                          FetchFun, Consumer,
-                                          C#cr{limiter = Limiter}, QName)}
-                    end
+                {continue, Limiter} ->
+                    {delivered, deliver_to_consumer(
+                                  FetchFun, Consumer,
+                                  C#cr{limiter = Limiter}, QName)}
             end
     end.
 
@@ -466,9 +474,10 @@ drained(AdvancedDeliveryCount, ChPid, CTag) ->
     'unchanged' | {'unblocked', state()}.
 process_credit(DeliveryCountRcv, LinkCredit, ChPid, CTag, State) ->
     case lookup_ch(ChPid) of
-        #cr{link_states = LinkStates = #{CTag := LinkState = #link_state{delivery_count = DeliveryCountSnd,
-                                                                         credit = OldLinkCreditSnd}},
-            unsent_message_count = Count} = C0 ->
+        #cr{link_states = LinkStates = #{CTag := LinkState =
+                                             #link_state{delivery_count = DeliveryCountSnd,
+                                                         credit = OldLinkCreditSnd}},
+            unsent_message_count = _Count} = C0 ->
             LinkCreditSnd = case DeliveryCountSnd of
                                 credit_api_v1 ->
                                     %% LinkCredit refers to LinkCreditSnd
@@ -482,8 +491,7 @@ process_credit(DeliveryCountRcv, LinkCredit, ChPid, CTag, State) ->
                                       DeliveryCountSnd)
                             end,
             C = C0#cr{link_states = maps:update(CTag, LinkState#link_state{credit = LinkCreditSnd}, LinkStates)},
-            case Count >= ?UNSENT_MESSAGE_LIMIT orelse
-                 OldLinkCreditSnd > 0 orelse
+            case OldLinkCreditSnd > 0 orelse
                  LinkCreditSnd < 1 of
                 true ->
                     update_ch_record(C),

From c6fb0b9317e87e84302cfb6e6a2561011d1be23e Mon Sep 17 00:00:00 2001
From: Karl Nilsson 
Date: Tue, 27 Feb 2024 14:58:18 +0000
Subject: [PATCH 13/16] Set off_heap for AMQP session and writer processes.

As these are the most likely to potentially run a backlog of
rabbit messages in their mailboxes and we do not want to include
these in gc runs unnecessarily.
---
 deps/rabbit/src/rabbit_amqp_session.erl | 1 +
 deps/rabbit/src/rabbit_amqp_writer.erl  | 1 +
 2 files changed, 2 insertions(+)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 59a35a84db26..c276b2028dba 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -241,6 +241,7 @@ init({ReaderPid, WriterPid, ChannelNum, MaxFrameSize, User, Vhost, ConnName,
                     outgoing_window = ?UINT(RemoteOutgoingWindow),
                     handle_max = HandleMax0}}) ->
     process_flag(trap_exit, true),
+    process_flag(message_queue_data, off_heap),
     ok = pg:join(node(), ?PROCESS_GROUP_NAME, self()),
 
     Alarms0 = rabbit_alarm:register(self(), {?MODULE, conserve_resources, []}),
diff --git a/deps/rabbit/src/rabbit_amqp_writer.erl b/deps/rabbit/src/rabbit_amqp_writer.erl
index f5b1b5adc6d3..9e28b7d833af 100644
--- a/deps/rabbit/src/rabbit_amqp_writer.erl
+++ b/deps/rabbit/src/rabbit_amqp_writer.erl
@@ -98,6 +98,7 @@ init({Sock, MaxFrame, ReaderPid}) ->
                    reader = ReaderPid,
                    pending = [],
                    pending_size = 0},
+    process_flag(message_queue_data, off_heap),
     {ok, State}.
 
 handle_cast({send_command, ChannelNum, MethodRecord}, State0) ->

From cae97046d958ad03c0cdfc32f690f62bd771f768 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Tue, 27 Feb 2024 18:46:22 +0100
Subject: [PATCH 14/16] Delete dead code

---
 deps/rabbit/src/rabbit_amqp_session.erl | 24 ++++++++----------------
 deps/rabbit/src/rabbit_amqp_writer.erl  | 20 +-------------------
 deps/rabbit/test/amqp_client_SUITE.erl  | 14 +++++++-------
 3 files changed, 16 insertions(+), 42 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index c276b2028dba..eb7b0d3c11f5 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -112,8 +112,6 @@
 -record(pending_transfer, {
           frames :: iolist(),
           queue_ack_required :: boolean(),
-          %% queue that sent us this message
-          queue_pid :: pid(),
           delivery_id :: delivery_number(),
           outgoing_unsettled :: #outgoing_unsettled{}
          }).
@@ -1011,9 +1009,8 @@ handle_control(#'v1_0.detach'{handle = Handle = ?UINT(HandleInt),
                          incoming_links = maps:remove(HandleInt, IncomingLinks),
                          outgoing_links = OutgoingLinks,
                          outgoing_unsettled_map = Unsettled},
-    ok = rabbit_amqp_writer:send_command(
-           WriterPid, Ch, #'v1_0.detach'{handle = Handle,
-                                         closed = Closed}),
+    rabbit_amqp_writer:send_command(WriterPid, Ch, #'v1_0.detach'{handle = Handle,
+                                                                  closed = Closed}),
     publisher_or_consumer_deleted(State, State0),
     {noreply, State};
 
@@ -1131,12 +1128,8 @@ send_pending(#state{remote_incoming_window = Space,
             send_pending(State0#state{outgoing_pending = Buf});
         {{value, #pending_transfer{frames = Frames} = Pending}, Buf1}
           when Space > 0 ->
-            SendFun = fun(Transfer, Sections) ->
-                              rabbit_amqp_writer:send_command(
-                                WriterPid, Ch, Transfer, Sections)
-                      end,
             {NumTransfersSent, Buf, State1} =
-                case send_frames(SendFun, Frames, Space) of
+                case send_frames(WriterPid, Ch, Frames, Space) of
                     {all, SpaceLeft} ->
                         {Space - SpaceLeft,
                          Buf1,
@@ -1154,13 +1147,13 @@ send_pending(#state{remote_incoming_window = Space,
             State0
     end.
 
-send_frames(_, [], Left) ->
+send_frames(_, _, [], Left) ->
     {all, Left};
-send_frames(_, Rest, 0) ->
+send_frames(_, _, Rest, 0) ->
     {some, Rest};
-send_frames(SendFun, [[Transfer, Sections] | Rest], Left) ->
-    ok = SendFun(Transfer, Sections),
-    send_frames(SendFun, Rest, Left - 1).
+send_frames(Writer, Ch, [[Transfer, Sections] | Rest], Left) ->
+    rabbit_amqp_writer:send_command(Writer, Ch, Transfer, Sections),
+    send_frames(Writer, Ch, Rest, Left - 1).
 
 record_outgoing_unsettled(#pending_transfer{queue_ack_required = true,
                                             delivery_id = DeliveryId,
@@ -1454,7 +1447,6 @@ handle_deliver(ConsumerTag, AckRequired,
             PendingTransfer = #pending_transfer{
                                  frames = Frames,
                                  queue_ack_required = AckRequired,
-                                 queue_pid = QPid,
                                  delivery_id = DeliveryId,
                                  outgoing_unsettled = Del},
             State#state{outgoing_pending = queue:in(PendingTransfer, Pending),
diff --git a/deps/rabbit/src/rabbit_amqp_writer.erl b/deps/rabbit/src/rabbit_amqp_writer.erl
index 9e28b7d833af..cc47b4ae4c2b 100644
--- a/deps/rabbit/src/rabbit_amqp_writer.erl
+++ b/deps/rabbit/src/rabbit_amqp_writer.erl
@@ -15,7 +15,6 @@
          send_command/3,
          send_command/4,
          send_command_sync/3,
-         send_command_and_notify/6,
          internal_send_command/3]).
 
 %% gen_server callbacks
@@ -71,16 +70,6 @@ send_command_sync(Writer, ChannelNum, MethodRecord) ->
     Request = {send_command, ChannelNum, MethodRecord},
     gen_server:call(Writer, Request, ?CALL_TIMEOUT).
 
--spec send_command_and_notify(pid(),
-                              rabbit_types:channel_number(),
-                              pid(),
-                              pid(),
-                              rabbit_framing:amqp_method_record(),
-                              rabbit_types:content()) -> ok.
-send_command_and_notify(Writer, ChannelNum, QueuePid, SessionPid, MethodRecord, Content) ->
-    Request = {send_command_and_notify, ChannelNum, QueuePid, SessionPid, MethodRecord, Content},
-    gen_server:cast(Writer, Request).
-
 -spec internal_send_command(rabbit_net:socket(),
                             rabbit_framing:amqp_method_record(),
                             amqp10_framing | rabbit_amqp_sasl) -> ok.
@@ -106,10 +95,6 @@ handle_cast({send_command, ChannelNum, MethodRecord}, State0) ->
     no_reply(State);
 handle_cast({send_command, ChannelNum, MethodRecord, Content}, State0) ->
     State = internal_send_command_async(ChannelNum, MethodRecord, Content, State0),
-    no_reply(State);
-handle_cast({send_command_and_notify, ChannelNum, QueuePid, SessionPid, MethodRecord, Content}, State0) ->
-    State = internal_send_command_async(ChannelNum, MethodRecord, Content, State0),
-    rabbit_amqqueue:notify_sent(QueuePid, SessionPid),
     no_reply(State).
 
 handle_call({send_command, ChannelNum, MethodRecord}, _From, State0) ->
@@ -119,10 +104,7 @@ handle_call({send_command, ChannelNum, MethodRecord}, _From, State0) ->
 
 handle_info(timeout, State0) ->
     State = flush(State0),
-    {noreply, State};
-handle_info({'DOWN', _MRef, process, QueuePid, _Reason}, State) ->
-    rabbit_amqqueue:notify_sent_queue_down(QueuePid),
-    no_reply(State).
+    {noreply, State}.
 
 format_status(Status) ->
     maps:update_with(
diff --git a/deps/rabbit/test/amqp_client_SUITE.erl b/deps/rabbit/test/amqp_client_SUITE.erl
index c450ca06e346..2495bf1acd34 100644
--- a/deps/rabbit/test/amqp_client_SUITE.erl
+++ b/deps/rabbit/test/amqp_client_SUITE.erl
@@ -2478,11 +2478,10 @@ queue_and_client_different_nodes(QueueLeaderNode, ClientNode, QueueType, Config)
     end,
     flush(receiver_attached),
 
-    NumMsgs = 10,
-    [begin
-         Bin = integer_to_binary(N),
-         ok = amqp10_client:send_msg(Sender, amqp10_msg:new(Bin, Bin, true))
-     end || N <- lists:seq(1, NumMsgs)],
+    %% Let's test with many messages to make sure we're not
+    %% impacted by RabbitMQ internal credit based flow control.
+    NumMsgs = 1100,
+    ok = send_messages(Sender, NumMsgs, true),
 
     %% Grant credits to the sending queue.
     ok = amqp10_client:flow_link_credit(Receiver, NumMsgs, never),
@@ -2491,8 +2490,9 @@ queue_and_client_different_nodes(QueueLeaderNode, ClientNode, QueueType, Config)
     Msgs = receive_messages(Receiver, NumMsgs),
     FirstMsg = hd(Msgs),
     LastMsg = lists:last(Msgs),
-    ?assertEqual([<<"1">>], amqp10_msg:body(FirstMsg)),
-    ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(LastMsg)),
+    ?assertEqual([integer_to_binary(NumMsgs)], amqp10_msg:body(FirstMsg)),
+    ?assertEqual([<<"1">>], amqp10_msg:body(LastMsg)),
+
     ok = amqp10_client_session:disposition(
            Receiver,
            amqp10_msg:delivery_id(FirstMsg),

From 1e1a6848e1f2da46bc50c3995cdda44f08cb399e Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Tue, 27 Feb 2024 19:14:13 +0100
Subject: [PATCH 15/16] Sort message IDs in rabbit_classic_queue

---
 deps/rabbit/src/rabbit_amqp_session.erl  | 11 +++--------
 deps/rabbit/src/rabbit_classic_queue.erl | 20 ++++++++++----------
 2 files changed, 13 insertions(+), 18 deletions(-)

diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index eb7b0d3c11f5..948d69219251 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -983,7 +983,7 @@ handle_control(#'v1_0.detach'{handle = Handle = ?UINT(HandleInt),
                                   [] ->
                                       {QStates1, Unsettled0, OutgoingLinks1};
                                   _ ->
-                                      case rabbit_queue_type_settle(QName, requeue, Ctag, MsgIds, QStates1) of
+                                      case rabbit_queue_type:settle(QName, requeue, Ctag, MsgIds, QStates1) of
                                           {ok, QStates2, _Actions = []} ->
                                               {QStates2, Unsettled1, OutgoingLinks1};
                                           {protocol_error, _ErrorType, Reason, ReasonArgs} ->
@@ -1084,7 +1084,7 @@ handle_control(#'v1_0.disposition'{role = ?RECV_ROLE,
             {QStates, Actions} =
             maps:fold(
               fun({QName, Ctag}, MsgIds, {QS0, ActionsAcc}) ->
-                      case rabbit_queue_type_settle(QName, SettleOp, Ctag, MsgIds, QS0) of
+                      case rabbit_queue_type:settle(QName, SettleOp, Ctag, MsgIds, QS0) of
                           {ok, QS, Actions0} ->
                               messages_acknowledged(SettleOp, QName, QS, MsgIds),
                               {QS, ActionsAcc ++ Actions0};
@@ -1110,11 +1110,6 @@ handle_control(Frame, _State) ->
                    "Unexpected frame ~tp",
                    [amqp10_framing:pprint(Frame)]).
 
-rabbit_queue_type_settle(QName, SettleOp, Ctag, MsgIds0, QStates) ->
-    %% Classic queues expect message IDs in sorted order.
-    MsgIds = lists:usort(MsgIds0),
-    rabbit_queue_type:settle(QName, SettleOp, Ctag, MsgIds, QStates).
-
 send_pending(#state{remote_incoming_window = Space,
                     outgoing_pending = Buf0,
                     cfg = #cfg{writer_pid = WriterPid,
@@ -1389,7 +1384,7 @@ handle_queue_actions(Actions, State) ->
       end, State, Actions).
 
 handle_deliver(ConsumerTag, AckRequired,
-               Msg = {QName, QPid, MsgId, Redelivered, Mc0},
+               Msg = {QName, _QPid, MsgId, Redelivered, Mc0},
                State = #state{outgoing_pending = Pending,
                               outgoing_delivery_id = DeliveryId,
                               outgoing_links = OutgoingLinks0,
diff --git a/deps/rabbit/src/rabbit_classic_queue.erl b/deps/rabbit/src/rabbit_classic_queue.erl
index 753a7d851c58..42dbcbb096f0 100644
--- a/deps/rabbit/src/rabbit_classic_queue.erl
+++ b/deps/rabbit/src/rabbit_classic_queue.erl
@@ -289,16 +289,16 @@ cancel(Q, ConsumerTag, OkMsg, ActingUser, State) ->
 -spec settle(rabbit_amqqueue:name(), rabbit_queue_type:settle_op(),
              rabbit_types:ctag(), [non_neg_integer()], state()) ->
     {state(), rabbit_queue_type:actions()}.
-settle(_QName, complete, _CTag, MsgIds, State) ->
-    Pid = State#?STATE.pid,
-    delegate:invoke_no_result(Pid,
-                              {gen_server2, cast, [{ack, MsgIds, self()}]}),
-    {State, []};
-settle(_QName, Op, _CTag, MsgIds, State) ->
-    ChPid = self(),
-    ok = delegate:invoke_no_result(State#?STATE.pid,
-                                   {gen_server2, cast,
-                                    [{reject, Op == requeue, MsgIds, ChPid}]}),
+settle(_QName, Op, _CTag, MsgIds0, State = #?STATE{pid = Pid}) ->
+    %% Classic queues expect message IDs in sorted order.
+    MsgIds = lists:usort(MsgIds0),
+    Arg = case Op of
+              complete ->
+                  {ack, MsgIds, self()};
+              _ ->
+                  {reject, Op == requeue, MsgIds, self()}
+          end,
+    delegate:invoke_no_result(Pid, {gen_server2, cast, [Arg]}),
     {State, []}.
 
 credit_v1(_QName, Ctag, LinkCreditSnd, Drain, #?STATE{pid = QPid} = State) ->

From 6d9ed3f4ddfb8f6c39f5e279f5d4b5ffd7c278c6 Mon Sep 17 00:00:00 2001
From: David Ansari 
Date: Wed, 28 Feb 2024 12:48:21 +0100
Subject: [PATCH 16/16] Apply PR feedback

---
 deps/amqp10_common/src/serial_number.erl      | 16 ++++++++
 .../test/serial_number_SUITE.erl              | 27 +++++++++++++
 deps/rabbit/src/rabbit_amqp_session.erl       | 40 +++++++++++--------
 3 files changed, 66 insertions(+), 17 deletions(-)

diff --git a/deps/amqp10_common/src/serial_number.erl b/deps/amqp10_common/src/serial_number.erl
index e3b6e900f875..5dd9ada92018 100644
--- a/deps/amqp10_common/src/serial_number.erl
+++ b/deps/amqp10_common/src/serial_number.erl
@@ -11,6 +11,7 @@
 -export([add/2,
          compare/2,
          ranges/1,
+         in_range/3,
          diff/2,
          foldl/4]).
 
@@ -86,6 +87,21 @@ ranges0([H | Rest], [{First, Last} | AccRest] = Acc0) ->
             ranges0(Rest, Acc)
     end.
 
+-spec in_range(serial_number(), serial_number(), serial_number()) ->
+    boolean().
+in_range(S, First, Last) ->
+    case compare(S, First) of
+        less ->
+            false;
+        _ ->
+            case compare(S, Last) of
+                greater ->
+                    false;
+                _ ->
+                    true
+            end
+    end.
+
 -define(SERIAL_DIFF_BOUND, 16#80000000).
 -spec diff(serial_number(), serial_number()) -> integer().
 diff(A, B) ->
diff --git a/deps/amqp10_common/test/serial_number_SUITE.erl b/deps/amqp10_common/test/serial_number_SUITE.erl
index 637b581b3973..78e8901ede29 100644
--- a/deps/amqp10_common/test/serial_number_SUITE.erl
+++ b/deps/amqp10_common/test/serial_number_SUITE.erl
@@ -15,6 +15,7 @@
                         compare/2,
                         usort/1,
                         ranges/1,
+                        in_range/3,
                         diff/2,
                         foldl/4]).
 
@@ -22,6 +23,7 @@ all() -> [test_add,
           test_compare,
           test_usort,
           test_ranges,
+          test_in_range,
           test_diff,
           test_foldl].
 
@@ -96,6 +98,31 @@ test_ranges(_Config) ->
     ?assertEqual([{4294967294, 1}, {3, 5}, {10, 10}, {18, 19}],
                  ranges([1, 10, 4294967294, 0, 3, 4, 5, 19, 18, 4294967295])).
 
+test_in_range(_Config) ->
+    ?assert(in_range(0, 0, 0)),
+    ?assert(in_range(0, 0, 1)),
+    ?assert(in_range(4294967295, 4294967295, 4294967295)),
+    ?assert(in_range(4294967295, 4294967295, 0)),
+    ?assert(in_range(0, 4294967295, 0)),
+    ?assert(in_range(4294967230, 4294967200, 1000)),
+    ?assert(in_range(88, 4294967200, 1000)),
+
+    ?assertNot(in_range(1, 0, 0)),
+    ?assertNot(in_range(4294967295, 0, 0)),
+    ?assertNot(in_range(0, 1, 1)),
+    ?assertNot(in_range(10, 1, 9)),
+    ?assertNot(in_range(1005, 4294967200, 1000)),
+    ?assertNot(in_range(4294967190, 4294967200, 1000)),
+
+    %% Pass wrong First and Last.
+    ?assertNot(in_range(1, 3, 2)),
+    ?assertNot(in_range(2, 3, 2)),
+    ?assertNot(in_range(3, 3, 2)),
+    ?assertNot(in_range(4, 3, 2)),
+
+    ?assertExit({undefined_serial_comparison, 0, 16#80000000},
+                in_range(0, 16#80000000, 16#80000000)).
+
 test_diff(_Config) ->
     ?assertEqual(0, diff(0, 0)),
     ?assertEqual(0, diff(1, 1)),
diff --git a/deps/rabbit/src/rabbit_amqp_session.erl b/deps/rabbit/src/rabbit_amqp_session.erl
index 948d69219251..24a1e925d6dd 100644
--- a/deps/rabbit/src/rabbit_amqp_session.erl
+++ b/deps/rabbit/src/rabbit_amqp_session.erl
@@ -7,6 +7,8 @@
 
 -module(rabbit_amqp_session).
 
+-compile({inline, [maps_update_with/4]}).
+
 -behaviour(gen_server).
 
 -include_lib("rabbit_common/include/rabbit.hrl").
@@ -1061,20 +1063,16 @@ handle_control(#'v1_0.disposition'{role = ?RECV_ROLE,
                                                consumer_tag = Ctag,
                                                msg_id = MsgId} = Unsettled,
                            {SettledAcc, UnsettledAcc}) ->
-                              DeliveryIdComparedToFirst = compare(DeliveryId, First),
-                              DeliveryIdComparedToLast = compare(DeliveryId, Last),
-                              if DeliveryIdComparedToFirst =:= less orelse
-                                 DeliveryIdComparedToLast =:= greater ->
-                                     %% Delivery ID is outside the DISPOSITION range.
-                                     {SettledAcc, UnsettledAcc#{DeliveryId => Unsettled}};
-                                 true ->
-                                     %% Delivery ID is inside the DISPOSITION range.
-                                     SettledAcc1 = maps:update_with(
-                                                     {QName, Ctag},
-                                                     fun(MsgIds) -> [MsgId | MsgIds] end,
-                                                     [MsgId],
-                                                     SettledAcc),
-                                     {SettledAcc1, UnsettledAcc}
+                              case serial_number:in_range(DeliveryId, First, Last) of
+                                  true ->
+                                      SettledAcc1 = maps_update_with(
+                                                      {QName, Ctag},
+                                                      fun(MsgIds) -> [MsgId | MsgIds] end,
+                                                      [MsgId],
+                                                      SettledAcc),
+                                      {SettledAcc1, UnsettledAcc};
+                                  false ->
+                                      {SettledAcc, UnsettledAcc#{DeliveryId => Unsettled}}
                               end
                       end,
                       {#{}, #{}}, UnsettledMap)
@@ -1209,19 +1207,19 @@ session_flow_control_sent_transfers(
     State#state{remote_incoming_window = RemoteIncomingWindow - NumTransfers,
                 next_outgoing_id = add(NextOutgoingId, NumTransfers)}.
 
-settle_delivery_id(Current, {Settled, Unsettled}) ->
+settle_delivery_id(Current, {Settled, Unsettled} = Acc) ->
     case maps:take(Current, Unsettled) of
         {#outgoing_unsettled{queue_name = QName,
                              consumer_tag = Ctag,
                              msg_id = MsgId}, Unsettled1} ->
-            Settled1 = maps:update_with(
+            Settled1 = maps_update_with(
                          {QName, Ctag},
                          fun(MsgIds) -> [MsgId | MsgIds] end,
                          [MsgId],
                          Settled),
             {Settled1, Unsettled1};
         error ->
-            {Settled, Unsettled}
+            Acc
     end.
 
 settle_op_from_outcome(#'v1_0.accepted'{}) ->
@@ -2276,6 +2274,14 @@ check_user_id(Mc, User) ->
             protocol_error(?V_1_0_AMQP_ERROR_UNAUTHORIZED_ACCESS, Reason, Args)
     end.
 
+maps_update_with(Key, Fun, Init, Map) ->
+    case Map of
+        #{Key := Value} ->
+            Map#{Key := Fun(Value)};
+        _ ->
+            Map#{Key => Init}
+    end.
+
 format_status(
   #{state := #state{cfg = Cfg,
                     outgoing_pending = OutgoingPending,