From bd1ad8f9f4c94ff9c93aa14b20ed60943662dc9a Mon Sep 17 00:00:00 2001 From: Jim Steinebrey Date: Tue, 11 Jun 2024 14:28:13 -0400 Subject: [PATCH] NIFI-13389 Streamlined use of putAllAttributes() in ConsumeJMS This closes #8955 Signed-off-by: David Handermann --- .../nifi/jms/processors/ConsumeJMS.java | 27 +++---------------- 1 file changed, 4 insertions(+), 23 deletions(-) diff --git a/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java b/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java index dfb18216c0..87444869bc 100644 --- a/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java +++ b/nifi-extension-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/ConsumeJMS.java @@ -61,7 +61,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Stream; @@ -364,12 +363,10 @@ public class ConsumeJMS extends AbstractJMSProcessor { final Map jmsHeaders = response.getMessageHeaders(); final Map jmsProperties = response.getMessageProperties(); - - flowFile = updateFlowFileAttributesWithJMSAttributes(mergeJmsAttributes(jmsHeaders, jmsProperties), flowFile, processSession); - flowFile = processSession.putAttribute(flowFile, JMS_SOURCE_DESTINATION_NAME, destinationName); - flowFile = processSession.putAttribute(flowFile, JMS_MESSAGETYPE, response.getMessageType()); - - return flowFile; + Map attributes = mergeJmsAttributes(jmsHeaders, jmsProperties); + attributes.put(JMS_SOURCE_DESTINATION_NAME, destinationName); + attributes.put(JMS_MESSAGETYPE, response.getMessageType()); + return processSession.putAllAttributes(flowFile, attributes); } private void processMessageSet(ProcessContext context, ProcessSession session, JMSConsumer consumer, String destinationName, String errorQueueName, @@ -496,22 +493,6 @@ public class ConsumeJMS extends AbstractJMSProcessor { } } - /** - * Copies JMS attributes (i.e., headers and properties) as FF attributes. - * Given that FF attributes mandate that values are of type String, the - * copied values of JMS attributes will be "stringified" via - * String.valueOf(attribute). - */ - private FlowFile updateFlowFileAttributesWithJMSAttributes(Map jmsAttributes, FlowFile flowFile, ProcessSession processSession) { - Map attributes = new HashMap<>(); - for (Entry entry : jmsAttributes.entrySet()) { - attributes.put(entry.getKey(), entry.getValue()); - } - - flowFile = processSession.putAllAttributes(flowFile, attributes); - return flowFile; - } - private Map mergeJmsAttributes(Map headers, Map properties) { final Map jmsAttributes = new HashMap<>(headers); properties.forEach((key, value) -> {