You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2020/01/19 16:05:02 UTC
[nifi] branch master updated: NIFI-7039 - This closes #3995. Fix:
PublishJMS outgoing flowfile attribute cleanup could lead to
ConcurrentModificationException
This is an automated email from the ASF dual-hosted git repository.
joewitt pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git
The following commit(s) were added to refs/heads/master by this push:
new b35ad7c NIFI-7039 - This closes #3995. Fix: PublishJMS outgoing flowfile attribute cleanup could lead to ConcurrentModificationException
b35ad7c is described below
commit b35ad7cd2089c20f34692fff51f39488df0feaf9
Author: Tamas Palfy <tp...@cloudera.com>
AuthorDate: Fri Jan 17 17:17:25 2020 +0100
NIFI-7039 - This closes #3995. Fix: PublishJMS outgoing flowfile attribute cleanup could lead to ConcurrentModificationException
Signed-off-by: Joe Witt <jo...@apache.org>
---
.../java/org/apache/nifi/jms/processors/PublishJMS.java | 13 ++-----------
.../java/org/apache/nifi/jms/processors/PublishJMSIT.java | 8 +++++++-
2 files changed, 9 insertions(+), 12 deletions(-)
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
index c95ec9d..9cb2451 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/PublishJMS.java
@@ -133,17 +133,8 @@ public class PublishJMS extends AbstractJMSProcessor<JMSPublisher> {
for (final Map.Entry<String, String> entry : flowFile.getAttributes().entrySet()) {
final String key = entry.getKey();
if (pattern.matcher(key).matches()) {
- attributesToSend.put(key, flowFile.getAttribute(key));
- }
- }
-
- // Optionally remove illegal headers names apart from .type attributes for JMS variable types
- if (!allowIllegalChars) {
- for (final Map.Entry<String,String> entry : attributesToSend.entrySet()) {
- if (!entry.getKey().endsWith(".type")){
- if (entry.getKey().contains("-") || entry.getKey().contains(".")) {
- attributesToSend.remove(entry.getKey());
- }
+ if (allowIllegalChars || key.endsWith(".type") || (!key.contains("-") && !key.contains("."))) {
+ attributesToSend.put(key, flowFile.getAttribute(key));
}
}
}
diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
index ad3febd..1f489d5 100644
--- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
+++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/processors/PublishJMSIT.java
@@ -63,7 +63,10 @@ public class PublishJMSIT {
Map<String, String> attributes = new HashMap<>();
attributes.put("foo", "foo");
attributes.put(JmsHeaders.REPLY_TO, "cooQueue");
- attributes.put("test-attribute", "value");
+ attributes.put("test-attribute.type", "allowed1");
+ attributes.put("test.attribute.type", "allowed2");
+ attributes.put("test-attribute", "notAllowed1");
+ attributes.put("jms.source.destination", "notAllowed2");
runner.enqueue("Hey dude!".getBytes(), attributes);
runner.run(1, false); // Run once but don't shut down because we want the Connection Factory left in tact so that we can use it.
@@ -77,7 +80,10 @@ public class PublishJMSIT {
assertEquals("Hey dude!", new String(messageBytes));
assertEquals("cooQueue", ((Queue) message.getJMSReplyTo()).getQueueName());
assertEquals("foo", message.getStringProperty("foo"));
+ assertEquals("allowed1", message.getStringProperty("test-attribute.type"));
+ assertEquals("allowed2", message.getStringProperty("test.attribute.type"));
assertNull(message.getStringProperty("test-attribute"));
+ assertNull(message.getStringProperty("jms.source.destination"));
runner.run(1, true, false); // Run once just so that we can trigger the shutdown of the Connection Factory
}