From afec1a35d9c36a4fc35f3a23d7c781c3853ba9a7 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Fri, 25 Aug 2017 16:37:50 -0400 Subject: [PATCH 1/9] Update ExtractEmailHeaders.java Resolve a null pointer exception when there is no recipient available in a TO, CC, or BCC header field. --- .../processors/email/ExtractEmailHeaders.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index 2018349d5905..dfe79c0db14e 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -168,15 +168,17 @@ public void process(final InputStream rawIn) throws IOException { } } } - if (Array.getLength(originalMessage.getAllRecipients()) > 0) { - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.TO)); toCount++) { - attributes.put(EMAIL_HEADER_TO + "." + toCount, originalMessage.getRecipients(Message.RecipientType.TO)[toCount].toString()); - } - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.BCC)); toCount++) { - attributes.put(EMAIL_HEADER_BCC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.BCC)[toCount].toString()); - } - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.CC)); toCount++) { - attributes.put(EMAIL_HEADER_CC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.CC)[toCount].toString()); + if (originalMessage.getAllRecipients() != null) { + if (Array.getLength(originalMessage.getAllRecipients()) > 0) { + for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.TO)); toCount++) { + attributes.put(EMAIL_HEADER_TO + "." + toCount, originalMessage.getRecipients(Message.RecipientType.TO)[toCount].toString()); + } + for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.BCC)); toCount++) { + attributes.put(EMAIL_HEADER_BCC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.BCC)[toCount].toString()); + } + for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.CC)); toCount++) { + attributes.put(EMAIL_HEADER_CC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.CC)[toCount].toString()); + } } } // Incredibly enough RFC-2822 specified From as a "mailbox-list" so an array I returned by getFrom From 47f0fd8cf2fc005b4d71ba4764fde4fa2ec10d4f Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Thu, 31 Aug 2017 15:10:13 -0400 Subject: [PATCH 2/9] NIFI-4326 Fix strict address parsing in ExtractEmailAttachments.java This uses getHeader() instead of getFrom() in order to avoid strict addressing. This matches changes to ExtractEmailHeaders.java Postfix and other mail servers are much more linient in accepting addresses. Using strict = false from getHeaders() we can accept more addresses that otherwise would be rejected. --- .../processors/email/ExtractEmailAttachments.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java index 18c74e960ea2..44bfc932e86d 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java @@ -31,6 +31,7 @@ import javax.activation.DataSource; import javax.mail.Address; +import javax.mail.internet.InternetAddress; import javax.mail.MessagingException; import javax.mail.Session; import javax.mail.internet.MimeMessage; @@ -131,11 +132,15 @@ public void process(final InputStream rawIn) throws IOException { MimeMessageParser parser = new MimeMessageParser(originalMessage).parse(); // RFC-2822 determines that a message must have a "From:" header // if a message lacks the field, it is flagged as invalid - Address[] from = originalMessage.getFrom(); + if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false) == null) { + if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false) == null) { + throw new MessagingException("Message failed RFC2822 validation: No Sender"); + } + } Date sentDate = originalMessage.getSentDate(); - if (from == null || sentDate == null) { + if (sentDate == null) { // Throws MessageException due to lack of minimum required headers - throw new MessagingException("Message failed RFC2822 validation"); + throw new MessagingException("Message failed RFC2822 validation: No Sent Date"); } originalFlowFilesList.add(originalFlowFile); if (parser.hasAttachments()) { @@ -209,4 +214,3 @@ public final List getSupportedPropertyDescriptors() { } - From 437e9b627f4976ca7587d21b7c7da97a0dc5f371 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Thu, 31 Aug 2017 15:11:16 -0400 Subject: [PATCH 3/9] NIFI-4326 Fix NullPointerException and strict addressing This uses parseHeader() instead of getFrom() and getRecipients() in order to avoid strict addressing. It also checks for null to solve a null pointer exception. By contract, this processor should grab information "if available". Which means it should not fail if the info is unavailable. --- .../processors/email/ExtractEmailHeaders.java | 57 +++++++++++++------ 1 file changed, 39 insertions(+), 18 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index dfe79c0db14e..e4d50e3e2112 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -42,6 +42,7 @@ import org.apache.nifi.stream.io.BufferedInputStream; import javax.mail.Address; +import javax.mail.internet.InternetAddress; import javax.mail.Header; import javax.mail.Message; import javax.mail.MessagingException; @@ -153,11 +154,15 @@ public void process(final InputStream rawIn) throws IOException { MimeMessageParser parser = new MimeMessageParser(originalMessage).parse(); // RFC-2822 determines that a message must have a "From:" header // if a message lacks the field, it is flagged as invalid - Address[] from = originalMessage.getFrom(); + if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false) == null) { + if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false) == null) { + throw new MessagingException("Message failed RFC2822 validation: No Sender"); + } + } Date sentDate = originalMessage.getSentDate(); - if (from == null || sentDate == null ) { + if (sentDate == null ) { // Throws MessageException due to lack of minimum required headers - throw new MessagingException("Message failed RFC2822 validation"); + throw new MessagingException("Message failed RFC2822 validation: No Sent Date"); } else if (capturedHeadersList.size() > 0){ Enumeration headers = originalMessage.getAllHeaders(); while (headers.hasMoreElements()) { @@ -168,23 +173,40 @@ public void process(final InputStream rawIn) throws IOException { } } } - if (originalMessage.getAllRecipients() != null) { - if (Array.getLength(originalMessage.getAllRecipients()) > 0) { - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.TO)); toCount++) { - attributes.put(EMAIL_HEADER_TO + "." + toCount, originalMessage.getRecipients(Message.RecipientType.TO)[toCount].toString()); - } - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.BCC)); toCount++) { - attributes.put(EMAIL_HEADER_BCC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.BCC)[toCount].toString()); - } - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getRecipients(Message.RecipientType.CC)); toCount++) { - attributes.put(EMAIL_HEADER_CC + "." + toCount, originalMessage.getRecipients(Message.RecipientType.CC)[toCount].toString()); - } + + // Get Non-Strict Recipient Addresses + InternetAddress[] recipients; + if (originalMessage.getHeader(Message.RecipientType.TO.toString(), ",") != null) { + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.TO.toString(), ","), false); + for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { + attributes.put(EMAIL_HEADER_TO + "." + toCount, recipients[toCount].toString()); } } - // Incredibly enough RFC-2822 specified From as a "mailbox-list" so an array I returned by getFrom - for (int toCount = 0; toCount < ArrayUtils.getLength(originalMessage.getFrom()); toCount++) { - attributes.put(EMAIL_HEADER_FROM + "." + toCount, originalMessage.getFrom()[toCount].toString()); + if (originalMessage.getHeader(Message.RecipientType.BCC.toString(), ",") != null) { + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.BCC.toString(), ","), false); + for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { + attributes.put(EMAIL_HEADER_BCC + "." + toCount, recipients[toCount].toString()); + } } + if (originalMessage.getHeader(Message.RecipientType.CC.toString(), ",") != null) { + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.CC.toString(), ","), false); + for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { + attributes.put(EMAIL_HEADER_CC + "." + toCount, recipients[toCount].toString()); + } + } + + // Get Non-Strict Sender Addresses + InternetAddress[] sender = null; + if (originalMessage.getHeader("From",",") != null) { + sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false)); + } + if (originalMessage.getHeader("Sender",",") != null) { + sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false)); + } + for (int toCount = 0; toCount < ArrayUtils.getLength(sender); toCount++) { + attributes.put(EMAIL_HEADER_FROM + "." + toCount, sender[toCount].toString()); + } + if (StringUtils.isNotEmpty(originalMessage.getMessageID())) { attributes.put(EMAIL_HEADER_MESSAGE_ID, originalMessage.getMessageID()); } @@ -234,4 +256,3 @@ public final List getSupportedPropertyDescriptors() { return descriptors; } } - From 4380b4deacbf00a324085a55085e33e858318073 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Wed, 6 Sep 2017 16:24:18 -0400 Subject: [PATCH 4/9] NIFI-4326 Added property for strict addresses Added a property and corresponding variable for setting strict addressing. The default is ON so as not to impact current users relying on the processor to reject weakly addressed email. --- .../processors/email/ExtractEmailHeaders.java | 29 ++++++++++++++----- 1 file changed, 22 insertions(+), 7 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index e4d50e3e2112..4c4a2cb78cbc 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -104,6 +104,18 @@ public class ExtractEmailHeaders extends AbstractProcessor { .defaultValue("x-mailer") .build(); + public static final PropertyDescriptor STRICT_ADDRESSING = new PropertyDescriptor.Builder() + .name("STRICT_ADDRESSING") + .displayName("Use Strict Email Addresses") + .description("If true, strict adderss rules will be applied. Some mail messages may fail if they have" + + " poorly constructed emails. Setting this to false will allow more mail messages through the " + + "processor, and behave more like sendmail. Try setting this to false if you want poorly constructed" + + " addresses to be accepted.") + .required(true) + .defaultValue("true") + .allowableValues("true", "false") + .build(); + public static final Relationship REL_SUCCESS = new Relationship.Builder() .name("success") .description("Extraction was successful") @@ -126,6 +138,7 @@ protected void init(final ProcessorInitializationContext context) { final List descriptors = new ArrayList<>(); descriptors.add(CAPTURED_HEADERS); + descriptors.add(STRICT_ADDRESSING); this.descriptors = Collections.unmodifiableList(descriptors); } @@ -141,6 +154,8 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } + final boolean strict = context.getProperty(STRICT_ADDRESSING).asBoolean(); + final List capturedHeadersList = Arrays.asList(context.getProperty(CAPTURED_HEADERS).getValue().toLowerCase().split(":")); final Map attributes = new HashMap<>(); @@ -154,8 +169,8 @@ public void process(final InputStream rawIn) throws IOException { MimeMessageParser parser = new MimeMessageParser(originalMessage).parse(); // RFC-2822 determines that a message must have a "From:" header // if a message lacks the field, it is flagged as invalid - if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false) == null) { - if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false) == null) { + if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), strict) == null) { + if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), strict) == null) { throw new MessagingException("Message failed RFC2822 validation: No Sender"); } } @@ -177,19 +192,19 @@ public void process(final InputStream rawIn) throws IOException { // Get Non-Strict Recipient Addresses InternetAddress[] recipients; if (originalMessage.getHeader(Message.RecipientType.TO.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.TO.toString(), ","), false); + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.TO.toString(), ","), strict); for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { attributes.put(EMAIL_HEADER_TO + "." + toCount, recipients[toCount].toString()); } } if (originalMessage.getHeader(Message.RecipientType.BCC.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.BCC.toString(), ","), false); + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.BCC.toString(), ","), strict); for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { attributes.put(EMAIL_HEADER_BCC + "." + toCount, recipients[toCount].toString()); } } if (originalMessage.getHeader(Message.RecipientType.CC.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.CC.toString(), ","), false); + recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.CC.toString(), ","), strict); for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { attributes.put(EMAIL_HEADER_CC + "." + toCount, recipients[toCount].toString()); } @@ -198,10 +213,10 @@ public void process(final InputStream rawIn) throws IOException { // Get Non-Strict Sender Addresses InternetAddress[] sender = null; if (originalMessage.getHeader("From",",") != null) { - sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false)); + sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("From", ","), strict)); } if (originalMessage.getHeader("Sender",",") != null) { - sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false)); + sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), strict)); } for (int toCount = 0; toCount < ArrayUtils.getLength(sender); toCount++) { attributes.put(EMAIL_HEADER_FROM + "." + toCount, sender[toCount].toString()); From ccfb2ee3c2725801978d9de0402a2e4656686920 Mon Sep 17 00:00:00 2001 From: Kevin Doran Date: Mon, 11 Sep 2017 13:24:47 -0400 Subject: [PATCH 5/9] NIFI-4326: Add unit test cases --- .../email/ExtractEmailAttachments.java | 1 - .../processors/email/ExtractEmailHeaders.java | 85 ++++++-------- .../processors/email/GenerateAttachment.java | 27 +++-- .../email/TestExtractEmailHeaders.java | 109 ++++++++++++++++++ 4 files changed, 159 insertions(+), 63 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java index 44bfc932e86d..09a198109713 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java @@ -30,7 +30,6 @@ import java.util.Date; import javax.activation.DataSource; -import javax.mail.Address; import javax.mail.internet.InternetAddress; import javax.mail.MessagingException; import javax.mail.Session; diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index 4c4a2cb78cbc..ef674bcdf12d 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -42,7 +42,6 @@ import org.apache.nifi.stream.io.BufferedInputStream; import javax.mail.Address; -import javax.mail.internet.InternetAddress; import javax.mail.Header; import javax.mail.Message; import javax.mail.MessagingException; @@ -50,7 +49,6 @@ import javax.mail.internet.MimeMessage; import java.io.IOException; import java.io.InputStream; -import java.lang.reflect.Array; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -68,7 +66,7 @@ @SideEffectFree @Tags({"split", "email"}) @InputRequirement(Requirement.INPUT_REQUIRED) -@CapabilityDescription("Using the flowfile content as source of data, extract header from an RFC compliant email file adding the relevant attributes to the flowfile. " + +@CapabilityDescription("Using the flowfile content as source of data, extract header from an RFC compliant email file adding the relevant attributes to the flowfile. " + "This processor does not perform extensive RFC validation but still requires a bare minimum compliance with RFC 2822") @WritesAttributes({ @WritesAttribute(attribute = "email.headers.bcc.*", description = "Each individual BCC recipient (if available)"), @@ -104,15 +102,18 @@ public class ExtractEmailHeaders extends AbstractProcessor { .defaultValue("x-mailer") .build(); + private static final String STRICT_ADDRESSING_DEFAULT_VALUE = "true"; public static final PropertyDescriptor STRICT_ADDRESSING = new PropertyDescriptor.Builder() - .name("STRICT_ADDRESSING") - .displayName("Use Strict Email Addresses") - .description("If true, strict adderss rules will be applied. Some mail messages may fail if they have" + - " poorly constructed emails. Setting this to false will allow more mail messages through the " + - "processor, and behave more like sendmail. Try setting this to false if you want poorly constructed" + - " addresses to be accepted.") - .required(true) - .defaultValue("true") + .name("STRICT_ADDRESS_PARSING") + .displayName("Use Strict Address Parsing") + .description("If true, strict address format parsing rules are applied to mailbox and mailbox list fields, " + + "such as \"to\" and \"from\" headers, and FlowFiles with poorly formed addresses will be routed " + + "to the failure relationship, similar to messages that fail RFC compliant format validation. " + + "If false, the processor will extract the contents of mailbox list headers as comma-separated " + + "values without attempting to parse each value as well-formed Internet mailbox addresses. " + + "This is optional and defaults to " + STRICT_ADDRESSING_DEFAULT_VALUE) + .required(false) + .defaultValue(STRICT_ADDRESSING_DEFAULT_VALUE) .allowableValues("true", "false") .build(); @@ -154,8 +155,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final boolean strict = context.getProperty(STRICT_ADDRESSING).asBoolean(); - + final String requireStrictAddresses = context.getProperty(STRICT_ADDRESSING).getValue(); final List capturedHeadersList = Arrays.asList(context.getProperty(CAPTURED_HEADERS).getValue().toLowerCase().split(":")); final Map attributes = new HashMap<>(); @@ -164,20 +164,20 @@ public void onTrigger(final ProcessContext context, final ProcessSession session public void process(final InputStream rawIn) throws IOException { try (final InputStream in = new BufferedInputStream(rawIn)) { Properties props = new Properties(); - Session mailSession = Session.getDefaultInstance(props, null); + props.put("mail.mime.address.strict", requireStrictAddresses); + Session mailSession = Session.getInstance(props); MimeMessage originalMessage = new MimeMessage(mailSession, in); MimeMessageParser parser = new MimeMessageParser(originalMessage).parse(); // RFC-2822 determines that a message must have a "From:" header // if a message lacks the field, it is flagged as invalid - if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), strict) == null) { - if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), strict) == null) { - throw new MessagingException("Message failed RFC2822 validation: No Sender"); - } + Address[] from = originalMessage.getFrom(); + if (from == null) { + throw new MessagingException("Message failed RFC-2822 validation: No Sender"); } Date sentDate = originalMessage.getSentDate(); if (sentDate == null ) { // Throws MessageException due to lack of minimum required headers - throw new MessagingException("Message failed RFC2822 validation: No Sent Date"); + throw new MessagingException("Message failed RFC-2822 validation: No Sent Date"); } else if (capturedHeadersList.size() > 0){ Enumeration headers = originalMessage.getAllHeaders(); while (headers.hasMoreElements()) { @@ -189,38 +189,10 @@ public void process(final InputStream rawIn) throws IOException { } } - // Get Non-Strict Recipient Addresses - InternetAddress[] recipients; - if (originalMessage.getHeader(Message.RecipientType.TO.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.TO.toString(), ","), strict); - for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { - attributes.put(EMAIL_HEADER_TO + "." + toCount, recipients[toCount].toString()); - } - } - if (originalMessage.getHeader(Message.RecipientType.BCC.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.BCC.toString(), ","), strict); - for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { - attributes.put(EMAIL_HEADER_BCC + "." + toCount, recipients[toCount].toString()); - } - } - if (originalMessage.getHeader(Message.RecipientType.CC.toString(), ",") != null) { - recipients = InternetAddress.parseHeader(originalMessage.getHeader(Message.RecipientType.CC.toString(), ","), strict); - for (int toCount = 0; toCount < ArrayUtils.getLength(recipients); toCount++) { - attributes.put(EMAIL_HEADER_CC + "." + toCount, recipients[toCount].toString()); - } - } - - // Get Non-Strict Sender Addresses - InternetAddress[] sender = null; - if (originalMessage.getHeader("From",",") != null) { - sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("From", ","), strict)); - } - if (originalMessage.getHeader("Sender",",") != null) { - sender = (InternetAddress[])ArrayUtils.addAll(sender, InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), strict)); - } - for (int toCount = 0; toCount < ArrayUtils.getLength(sender); toCount++) { - attributes.put(EMAIL_HEADER_FROM + "." + toCount, sender[toCount].toString()); - } + putAddressListInAttributes(attributes, EMAIL_HEADER_TO, originalMessage.getRecipients(Message.RecipientType.TO)); + putAddressListInAttributes(attributes, EMAIL_HEADER_CC, originalMessage.getRecipients(Message.RecipientType.CC)); + putAddressListInAttributes(attributes, EMAIL_HEADER_BCC, originalMessage.getRecipients(Message.RecipientType.BCC)); + putAddressListInAttributes(attributes, EMAIL_HEADER_FROM, originalMessage.getFrom()); // RFC-2822 specifies "From" as mailbox-list if (StringUtils.isNotEmpty(originalMessage.getMessageID())) { attributes.put(EMAIL_HEADER_MESSAGE_ID, originalMessage.getMessageID()); @@ -270,4 +242,15 @@ public Set getRelationships() { public final List getSupportedPropertyDescriptors() { return descriptors; } + + private static void putAddressListInAttributes( + Map attributes, + final String attributePrefix, + Address[] addresses) { + if (addresses != null) { + for (int count = 0; count < ArrayUtils.getLength(addresses); count++) { + attributes.put(attributePrefix + "." + count, addresses[count].toString()); + } + } + } } diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/GenerateAttachment.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/GenerateAttachment.java index ef100b2cc3ea..621597fa96c4 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/GenerateAttachment.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/GenerateAttachment.java @@ -44,6 +44,20 @@ public GenerateAttachment(String from, String to, String subject, String message } public byte[] SimpleEmail() { + MimeMessage mimeMessage = SimpleEmailMimeMessage(); + ByteArrayOutputStream output = new ByteArrayOutputStream(); + try { + mimeMessage.writeTo(output); + } catch (IOException e) { + e.printStackTrace(); + } catch (MessagingException e) { + e.printStackTrace(); + } + + return output.toByteArray(); + } + + public MimeMessage SimpleEmailMimeMessage() { Email email = new SimpleEmail(); try { email.setFrom(from); @@ -56,19 +70,10 @@ public byte[] SimpleEmail() { e.printStackTrace(); } - ByteArrayOutputStream output = new ByteArrayOutputStream(); - MimeMessage mimeMessage = email.getMimeMessage(); - try { - mimeMessage.writeTo(output); - } catch (IOException e) { - e.printStackTrace(); - } catch (MessagingException e) { - e.printStackTrace(); - } - - return output.toByteArray(); + return email.getMimeMessage(); } + public byte[] WithAttachments(int amount) { MultiPartEmail email = new MultiPartEmail(); try { diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java index aed2292c25be..d9fcffa0630c 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java @@ -17,11 +17,15 @@ package org.apache.nifi.processors.email; +import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; import org.junit.Test; +import javax.mail.MessagingException; +import javax.mail.internet.MimeMessage; +import java.io.IOException; import java.util.List; public class TestExtractEmailHeaders { @@ -79,6 +83,111 @@ public void testValidEmailWithoutAttachments() throws Exception { splits.get(0).assertAttributeExists("email.headers.mime-version"); } + /** + * Test case added for NIFI-4326 for a potential NPE bug + * if the email message contains no recipient header fields, ie, + * TO, CC, BCC. + */ + @Test + public void testValidEmailWithNoRecipients() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExtractEmailHeaders()); + runner.setProperty(ExtractEmailHeaders.CAPTURED_HEADERS, "MIME-Version"); + + MimeMessage simpleEmailMimeMessage = attachmentGenerator.SimpleEmailMimeMessage(); + + simpleEmailMimeMessage.removeHeader("To"); + simpleEmailMimeMessage.removeHeader("Cc"); + simpleEmailMimeMessage.removeHeader("Bcc"); + + ByteArrayOutputStream messageBytes = new ByteArrayOutputStream(); + try { + simpleEmailMimeMessage.writeTo(messageBytes); + } catch (IOException | MessagingException e) { + e.printStackTrace(); + } + + runner.enqueue(messageBytes.toByteArray()); + runner.run(); + + runner.assertTransferCount(ExtractEmailHeaders.REL_SUCCESS, 1); + runner.assertTransferCount(ExtractEmailHeaders.REL_FAILURE, 0); + + runner.assertQueueEmpty(); + final List splits = runner.getFlowFilesForRelationship(ExtractEmailHeaders.REL_SUCCESS); + splits.get(0).assertAttributeEquals("email.headers.from.0", from); + splits.get(0).assertAttributeExists("email.headers.mime-version"); + splits.get(0).assertAttributeNotExists("email.headers.to"); + splits.get(0).assertAttributeNotExists("email.headers.cc"); + splits.get(0).assertAttributeNotExists("email.headers.bcc"); + } + + /** + * NIFI-4326 adds a new feature to disable strict address parsing for + * mailbox list header fields. This is a test case that asserts that + * lax address parsing passes (when set to "strict=false") for malformed + * addresses. + */ + @Test + public void testNonStrictParsingPassesForInvalidAddresses() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExtractEmailHeaders()); + runner.setProperty(ExtractEmailHeaders.STRICT_ADDRESSING, "false"); + + MimeMessage simpleEmailMimeMessage = attachmentGenerator.SimpleEmailMimeMessage(); + + simpleEmailMimeMessage.setHeader("From", ""); + simpleEmailMimeMessage.setHeader("To", "<>, Joe, \"\" <>"); + + ByteArrayOutputStream messageBytes = new ByteArrayOutputStream(); + try { + simpleEmailMimeMessage.writeTo(messageBytes); + } catch (IOException | MessagingException e) { + e.printStackTrace(); + } + + runner.enqueue(messageBytes.toByteArray()); + runner.run(); + + runner.assertTransferCount(ExtractEmailHeaders.REL_SUCCESS, 1); + runner.assertTransferCount(ExtractEmailHeaders.REL_FAILURE, 0); + + + runner.assertQueueEmpty(); + final List splits = runner.getFlowFilesForRelationship(ExtractEmailHeaders.REL_SUCCESS); + splits.get(0).assertAttributeEquals("email.headers.from.0", "bad_email"); + splits.get(0).assertAttributeEquals("email.headers.to.0", ""); + splits.get(0).assertAttributeEquals("email.headers.to.1", "Joe"); + splits.get(0).assertAttributeEquals("email.headers.to.2", ""); + } + + /** + * NIFI-4326 adds a new feature to disable strict address parsing for + * mailbox list header fields. This is a test case that asserts that + * strict address parsing fails (when set to "strict=true") for malformed + * addresses. + */ + @Test + public void testStrictParsingFailsForInvalidAddresses() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(new ExtractEmailHeaders()); + runner.setProperty(ExtractEmailHeaders.STRICT_ADDRESSING, "true"); + + MimeMessage simpleEmailMimeMessage = attachmentGenerator.SimpleEmailMimeMessage(); + + simpleEmailMimeMessage.setHeader("From", ""); + simpleEmailMimeMessage.setHeader("To", "<>, Joe, "); + + ByteArrayOutputStream messageBytes = new ByteArrayOutputStream(); + try { + simpleEmailMimeMessage.writeTo(messageBytes); + } catch (IOException | MessagingException e) { + e.printStackTrace(); + } + + runner.enqueue(messageBytes.toByteArray()); + runner.run(); + + runner.assertTransferCount(ExtractEmailHeaders.REL_SUCCESS, 0); + runner.assertTransferCount(ExtractEmailHeaders.REL_FAILURE, 1); + } @Test public void testInvalidEmail() throws Exception { From 8308cabb9cb5a2a9984d03817d3882e85a046fe2 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Wed, 13 Sep 2017 17:53:40 -0400 Subject: [PATCH 6/9] NIFI-4326 Add non-strict email address parsing Added mailSession properties for non-strict address parsing to mirror what was done in the ExtractEmailHeaders.java This relaxes the requirement for valid email addresses. Email addresses are not used in this processor and has no impact on extracting attachments. In order to process more emails and extract more attachments, the mail.mime.address.strict property was set to false. This was done with a variable in order to make it easier to add a property to the processor later if desired. --- .../processors/email/ExtractEmailAttachments.java | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java index 09a198109713..e92889cfc69a 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailAttachments.java @@ -30,7 +30,7 @@ import java.util.Date; import javax.activation.DataSource; -import javax.mail.internet.InternetAddress; +import javax.mail.Address; import javax.mail.MessagingException; import javax.mail.Session; import javax.mail.internet.MimeMessage; @@ -121,20 +121,22 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final List invalidFlowFilesList = new ArrayList<>(); final List originalFlowFilesList = new ArrayList<>(); + final String requireStrictAddresses = "false"; + session.read(originalFlowFile, new InputStreamCallback() { @Override public void process(final InputStream rawIn) throws IOException { try (final InputStream in = new BufferedInputStream(rawIn)) { Properties props = new Properties(); - Session mailSession = Session.getDefaultInstance(props, null); + props.put("mail.mime.address.strict", requireStrictAddresses); + Session mailSession = Session.getInstance(props); MimeMessage originalMessage = new MimeMessage(mailSession, in); MimeMessageParser parser = new MimeMessageParser(originalMessage).parse(); // RFC-2822 determines that a message must have a "From:" header // if a message lacks the field, it is flagged as invalid - if (InternetAddress.parseHeader(originalMessage.getHeader("From", ","), false) == null) { - if (InternetAddress.parseHeader(originalMessage.getHeader("Sender", ","), false) == null) { - throw new MessagingException("Message failed RFC2822 validation: No Sender"); - } + Address[] from = originalMessage.getFrom(); + if (from == null) { + throw new MessagingException("Message failed RFC-2822 validation: No Sender"); } Date sentDate = originalMessage.getSentDate(); if (sentDate == null) { From 60aa7fe221abc4c9f83d57b85190b3ed38dded88 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Tue, 19 Sep 2017 14:37:52 -0400 Subject: [PATCH 7/9] Add AllowableValue to ExtractEmailHeaders.java Added AllowableValues, and tweaked variable names and documentation to be less ambiguous. --- .../processors/email/ExtractEmailHeaders.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index ef674bcdf12d..378659cb5ce3 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -30,6 +30,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.AbstractProcessor; @@ -102,19 +103,20 @@ public class ExtractEmailHeaders extends AbstractProcessor { .defaultValue("x-mailer") .build(); - private static final String STRICT_ADDRESSING_DEFAULT_VALUE = "true"; - public static final PropertyDescriptor STRICT_ADDRESSING = new PropertyDescriptor.Builder() + private static final AllowableValue STRICT_ADDRESSING = new AllowableValue("true", "Strict Address Parsing", "Strict email address format will be enforced. FlowFiles will be transfered to the failure relationship if the email address is invalid."); + private static final AllowableValue NONSTRICT_ADDRESSING = new AllowableValue("false", "Non-Strict Address Parsing", "Accept emails, even if the address is poorly formed and doesn't strictly comply with RFC Validation."); + public static final PropertyDescriptor STRICT_PARSING = new PropertyDescriptor.Builder() .name("STRICT_ADDRESS_PARSING") - .displayName("Use Strict Address Parsing") - .description("If true, strict address format parsing rules are applied to mailbox and mailbox list fields, " + + .displayName("Email Address Parsing") + .description("If \"strict\", strict address format parsing rules are applied to mailbox and mailbox list fields, " + "such as \"to\" and \"from\" headers, and FlowFiles with poorly formed addresses will be routed " + "to the failure relationship, similar to messages that fail RFC compliant format validation. " + - "If false, the processor will extract the contents of mailbox list headers as comma-separated " + + "If \"non-strict\", the processor will extract the contents of mailbox list headers as comma-separated " + "values without attempting to parse each value as well-formed Internet mailbox addresses. " + - "This is optional and defaults to " + STRICT_ADDRESSING_DEFAULT_VALUE) + "This is optional and defaults to " + STRICT_ADDRESSING.getDisplayName()) .required(false) - .defaultValue(STRICT_ADDRESSING_DEFAULT_VALUE) - .allowableValues("true", "false") + .defaultValue(STRICT_ADDRESSING.getValue()) + .allowableValues(STRICT_ADDRESSING, NONSTRICT_ADDRESSING) .build(); public static final Relationship REL_SUCCESS = new Relationship.Builder() @@ -139,7 +141,7 @@ protected void init(final ProcessorInitializationContext context) { final List descriptors = new ArrayList<>(); descriptors.add(CAPTURED_HEADERS); - descriptors.add(STRICT_ADDRESSING); + descriptors.add(STRICT_PARSING); this.descriptors = Collections.unmodifiableList(descriptors); } @@ -155,7 +157,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final String requireStrictAddresses = context.getProperty(STRICT_ADDRESSING).getValue(); + final String requireStrictAddresses = context.getProperty(STRICT_PARSING).getValue(); final List capturedHeadersList = Arrays.asList(context.getProperty(CAPTURED_HEADERS).getValue().toLowerCase().split(":")); final Map attributes = new HashMap<>(); From 05612444cad27b09355209248232dd5cf3c90d57 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Tue, 19 Sep 2017 14:42:12 -0400 Subject: [PATCH 8/9] Rename variable STRICT_ADDRESSING to STRICT_PARSING Had to rename the strict addressing variable due to the rename of the property in the ExtractEmailHeaders.java processor STRICT_ADDRESSING is now STRICT_PARSING. --- .../nifi/processors/email/TestExtractEmailHeaders.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java index d9fcffa0630c..4cb0009feb65 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestExtractEmailHeaders.java @@ -130,7 +130,7 @@ public void testValidEmailWithNoRecipients() throws Exception { @Test public void testNonStrictParsingPassesForInvalidAddresses() throws Exception { final TestRunner runner = TestRunners.newTestRunner(new ExtractEmailHeaders()); - runner.setProperty(ExtractEmailHeaders.STRICT_ADDRESSING, "false"); + runner.setProperty(ExtractEmailHeaders.STRICT_PARSING, "false"); MimeMessage simpleEmailMimeMessage = attachmentGenerator.SimpleEmailMimeMessage(); @@ -168,7 +168,7 @@ public void testNonStrictParsingPassesForInvalidAddresses() throws Exception { @Test public void testStrictParsingFailsForInvalidAddresses() throws Exception { final TestRunner runner = TestRunners.newTestRunner(new ExtractEmailHeaders()); - runner.setProperty(ExtractEmailHeaders.STRICT_ADDRESSING, "true"); + runner.setProperty(ExtractEmailHeaders.STRICT_PARSING, "true"); MimeMessage simpleEmailMimeMessage = attachmentGenerator.SimpleEmailMimeMessage(); @@ -199,4 +199,4 @@ public void testInvalidEmail() throws Exception { runner.assertTransferCount(ExtractEmailHeaders.REL_FAILURE, 1); } -} \ No newline at end of file +} From f8c12a6f0249f4e43a084f72ab8441662a12a172 Mon Sep 17 00:00:00 2001 From: btwood <4839861+btwood@users.noreply.github.com> Date: Wed, 20 Sep 2017 14:41:52 -0400 Subject: [PATCH 9/9] Styling Guide and Aesthetic Fixes Updating the code to add line breaks for line length to stay under 200, per the style guide. This will pass with "-Pcontrib-check" in the maven build. --- .../apache/nifi/processors/email/ExtractEmailHeaders.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java index 378659cb5ce3..22936fdcd3c9 100644 --- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java +++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ExtractEmailHeaders.java @@ -103,8 +103,10 @@ public class ExtractEmailHeaders extends AbstractProcessor { .defaultValue("x-mailer") .build(); - private static final AllowableValue STRICT_ADDRESSING = new AllowableValue("true", "Strict Address Parsing", "Strict email address format will be enforced. FlowFiles will be transfered to the failure relationship if the email address is invalid."); - private static final AllowableValue NONSTRICT_ADDRESSING = new AllowableValue("false", "Non-Strict Address Parsing", "Accept emails, even if the address is poorly formed and doesn't strictly comply with RFC Validation."); + private static final AllowableValue STRICT_ADDRESSING = new AllowableValue("true", "Strict Address Parsing", + "Strict email address format will be enforced. FlowFiles will be transfered to the failure relationship if the email address is invalid."); + private static final AllowableValue NONSTRICT_ADDRESSING = new AllowableValue("false", "Non-Strict Address Parsing", + "Accept emails, even if the address is poorly formed and doesn't strictly comply with RFC Validation."); public static final PropertyDescriptor STRICT_PARSING = new PropertyDescriptor.Builder() .name("STRICT_ADDRESS_PARSING") .displayName("Email Address Parsing")