NIFI-15483: Route PublishAMQP to failure on undeliverable messages instead of silent success#11213
Open
rakesh-rsky wants to merge 3 commits intoapache:mainfrom
Open
Conversation
…r cannot deliver message PublishAMQP uses mandatory=true on basicPublish() so the broker returns messages it cannot route to any queue. However, the return arrives asynchronously via ReturnListener.handleReturn() on the AMQP I/O thread while the publishing thread had already moved on to session.transfer(REL_SUCCESS). The UndeliverableMessageLogger only logged a warning — it never signaled failure back to publish() or onTrigger(), so every unroutable message was silently counted as a success despite never reaching any consumer. Fix: - Enabled Publisher Confirms (channel.confirmSelect()) in the constructor. The broker's basic.return frame for an unroutable message is guaranteed to arrive before the corresponding confirm frame, so waitForConfirms() acts as a synchronization barrier that makes return detection reliable. - Added an AtomicReference<String> field (undeliverableReturnReason) that UndeliverableMessageLogger.handleReturn() populates with exchange/routingKey/ replyCode/replyText when a message is returned. - publish() now: resets the field before each call, calls waitForConfirms(5s) to synchronize with the broker, then checks the field and throws AMQPException if the message was returned — causing onTrigger() to route to REL_FAILURE. - Broker NACKs (e.g., resource alarm) are also now surfaced as AMQPException because waitForConfirms() returns false on NACK. Co-authored-by: Rakesh Kumar Singh <rsky.rakesh@gmail.com>
5da2156 to
ace38df
Compare
Contributor
|
@rakesh-rsky Thanks for working on this issue. Please note the unit tests are failing. I tried the new error handling in my local environment but I'm getting the following runtime error (instead of routing the FlowFile to failure): |
…ssing exchange When the broker closes the channel with a 404 NOT_FOUND error (e.g., exchange does not exist), waitForConfirms() throws ShutdownSignalException instead of returning normally. This was propagating as an unhandled processor failure rather than routing the FlowFile to REL_FAILURE. - Added ShutdownSignalException to the catch block in AMQPPublisher.publish() - Converts the channel-close signal into AMQPException so PublishAMQP routes the FlowFile to REL_FAILURE with a descriptive error message - Added ShutdownSignalException import Co-authored-by: Rakesh Kumar Singh <rsky.rakesh@gmail.com>
…dling
Added regression tests to verify that AMQPPublisher and PublishAMQP correctly
route FlowFiles to REL_FAILURE for all broker-side failure modes:
- TestChannel: implemented confirmSelect() (no-op) and waitForConfirms(timeout)
with simulation flags for ShutdownSignalException and NACK
- TestChannel: added simulateSynchronousReturn flag for deterministic
undeliverable-message tests (fires ReturnListeners synchronously)
- TestConnection: exposed getTestChannel() for test configuration
- AMQPPublisherTest: added 3 new unit tests
* failPublishWhenBrokerClosesChannelDuringConfirm
* failPublishWhenBrokerNacksMessage
* failPublishWhenMessageReturnedAsUndeliverable
- PublishAMQPTest: added 2 new integration tests
* validateFlowFileRoutedToFailureWhenBrokerClosesChannel
* validateFlowFileRoutedToFailureOnBrokerNack
All 45 tests pass. Tests were verified to FAIL without the corresponding fix.
Co-authored-by: Rakesh Kumar Singh <rsky.rakesh@gmail.com>
Author
|
@turcsanyip Thank you for validating this. Root cause: When the exchange does not exist, the broker closes the channel with This has been fixed in the latest commits — |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
Summary
PublishAMQPsilently routes FlowFiles toREL_SUCCESSeven when the AMQP broker cannot deliver the message, causing silent data loss.Two failure modes are addressed:
Undeliverable message (
basic.return) — broker returns the message when no queue is bound to the exchange/routing-key. The fix uses AMQP Publisher Confirms +basic.returnto detect and surface delivery failures, routing the FlowFile toREL_FAILURE.Exchange not found (
ShutdownSignalException) — when the exchange does not exist, the broker closes the channel with404 NOT_FOUND, causingwaitForConfirms()to throwShutdownSignalException. This is now caught and converted toAMQPExceptionso the FlowFile routes toREL_FAILUREinstead of causing an unhandled processor failure.Testing
failurewith the broker's return reasonfailureinstead of unhandled processor errorsuccessFixes: https://issues.apache.org/jira/browse/NIFI-15483