diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/NetworkUtils.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/NetworkUtils.java
new file mode 100644
index 0000000000..b2deeccbfe
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/NetworkUtils.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+package org.apache.nifi.remote.io.socket;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+
+/**
+ *
+ */
+public class NetworkUtils {
+
+ /**
+ * Will determine the available port
+ */
+ public final static int availablePort() {
+ ServerSocket s = null;
+ try {
+ s = new ServerSocket(0);
+ s.setReuseAddress(true);
+ return s.getLocalPort();
+ } catch (Exception e) {
+ throw new IllegalStateException("Failed to discover available port.", e);
+ } finally {
+ try {
+ s.close();
+ } catch (IOException e) {
+ // ignore
+ }
+ }
+ }
+}
diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
index a657030e1a..b1ebf2fbcb 100644
--- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
+++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/stream/io/LimitingInputStream.java
@@ -16,6 +16,7 @@
*/
package org.apache.nifi.stream.io;
+import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
@@ -24,16 +25,50 @@ public class LimitingInputStream extends InputStream {
private final InputStream in;
private final long limit;
private long bytesRead = 0;
+ private final boolean exceptionOnLimit;
+ /**
+ * Constructs a limited input stream whereby if the limit is reached all
+ * subsequent calls to read will return a -1.
+ *
+ * @param in
+ * the underlying input stream
+ * @param limit
+ * maximum length of bytes to read from underlying input stream
+ */
public LimitingInputStream(final InputStream in, final long limit) {
+ this(in, limit, false);
+ }
+
+ /**
+ * Constructs a limited input stream whereby if the limit is reached all
+ * subsequent calls to read will return a -1 or EOFexception as configured
+ *
+ * @param in
+ * the underlying input stream
+ * @param limit
+ * maximum length of bytes to read from underlying input stream
+ * @param eofOnLimit
+ * true if EOF should occur on all read calls once limit reached;
+ * false if -1 should be returned instead
+ */
+ public LimitingInputStream(final InputStream in, final long limit, final boolean eofOnLimit) {
this.in = in;
this.limit = limit;
+ exceptionOnLimit = eofOnLimit;
+ }
+
+ private int limitReached() throws IOException {
+ if (exceptionOnLimit) {
+ throw new EOFException("Limit of allowed bytes read from input stream reached");
+ }
+ return -1;
}
@Override
public int read() throws IOException {
if (bytesRead >= limit) {
- return -1;
+ return limitReached();
}
final int val = in.read();
@@ -46,7 +81,7 @@ public class LimitingInputStream extends InputStream {
@Override
public int read(final byte[] b) throws IOException {
if (bytesRead >= limit) {
- return -1;
+ return limitReached();
}
final int maxToRead = (int) Math.min(b.length, limit - bytesRead);
@@ -61,7 +96,7 @@ public class LimitingInputStream extends InputStream {
@Override
public int read(byte[] b, int off, int len) throws IOException {
if (bytesRead >= limit) {
- return -1;
+ return limitReached();
}
final int maxToRead = (int) Math.min(len, limit - bytesRead);
diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/LimitingInputStreamTest.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/LimitingInputStreamTest.java
new file mode 100644
index 0000000000..5bb4362e1a
--- /dev/null
+++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/stream/io/LimitingInputStreamTest.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.stream.io;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+public class LimitingInputStreamTest extends TestCase {
+ private final static byte[] TEST_BUFFER = new byte[] { 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 };
+
+ public void testReadLimitNotReached() throws IOException {
+ LimitingInputStream is = new LimitingInputStream(new ByteArrayInputStream(TEST_BUFFER), 50, false);
+ long bytesRead = StreamUtils.copy(is, new ByteArrayOutputStream());
+ assertEquals(bytesRead, TEST_BUFFER.length);
+
+ is = new LimitingInputStream(new ByteArrayInputStream(TEST_BUFFER), 50, true);
+ bytesRead = StreamUtils.copy(is, new ByteArrayOutputStream());
+ assertEquals(bytesRead, TEST_BUFFER.length);
+ }
+
+ public void testReadLimitExceeded() throws IOException {
+ final LimitingInputStream is = new LimitingInputStream(new ByteArrayInputStream(TEST_BUFFER), 9);
+ final long bytesRead = StreamUtils.copy(is, new ByteArrayOutputStream());
+ assertEquals(bytesRead, 9);
+ }
+
+ public void testReadLimitExceededEof() throws IOException {
+ final LimitingInputStream is = new LimitingInputStream(new ByteArrayInputStream(TEST_BUFFER), 9, true);
+ try {
+ StreamUtils.copy(is, new ByteArrayOutputStream());
+ fail("Should not get here");
+ } catch (final EOFException eof) {
+ }
+ }
+}
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/LICENSE
new file mode 100644
index 0000000000..d645695673
--- /dev/null
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/LICENSE
@@ -0,0 +1,202 @@
+
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
+
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+ 1. Definitions.
+
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
+
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
+
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
+
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
+
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
+
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
+
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
+
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
+
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
+
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
+
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
+
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
+
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
+
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
+
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
+
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
+
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
+
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
+
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
+
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
+
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
+
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
+
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
+
+ END OF TERMS AND CONDITIONS
+
+ APPENDIX: How to apply the Apache License to your work.
+
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright [yyyy] [name of copyright owner]
+
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/NOTICE
new file mode 100644
index 0000000000..dc462ceb12
--- /dev/null
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-nar/src/main/resources/META-INF/NOTICE
@@ -0,0 +1,37 @@
+nifi-email-nar
+Copyright 2014-2016 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+******************
+Apache Software License v2
+******************
+
+The following binary components are provided under the Apache Software License v2
+
+ (ASLv2) Spring Framework
+ The following NOTICE information applies:
+ Spring Framework
+ Copyright 2002-2016
+
+ (ASLv2) SubEthaSMTP - A SMTP mail server
+ The following NOTICE information applies:
+ Spring Framework
+ Copyright 2006-2007
+
+************************
+Common Development and Distribution License 1.1
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.1. See project link for details.
+
+ (CDDL 1.1) (GPL2 w/ CPE) JavaMail API (compat) (javax.mail:mail:jar:1.4.7 - http://kenai.com/projects/javamail/mail)
+
+************************
+Common Development and Distribution License 1.0
+************************
+
+The following binary components are provided under the Common Development and Distribution License 1.0. See project link for details.
+
+ (CDDL 1.0) JavaBeans Activation Framework (JAF) (javax.activation:activation:jar:1.1 - http://java.sun.com/products/javabeans/jaf/index.jsp)
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
index 75cd3d2119..34125f4263 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/pom.xml
@@ -53,6 +53,12 @@
org.springframework.integration
spring-integration-mail
4.3.0.RELEASE
+
+
+ org.springframework.retry
+ spring-retry
+
+
commons-logging
diff --git a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
index 0f17838837..52d448168b 100644
--- a/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
+++ b/nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
@@ -13,16 +13,15 @@
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
-*/
+ */
package org.apache.nifi.processors.email;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.SSLSocket;
-import javax.net.ssl.SSLSocketFactory;
import java.io.IOException;
-import java.io.InputStream;
+import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.net.Socket;
+import java.security.cert.Certificate;
+import java.security.cert.X509Certificate;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -31,71 +30,57 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.StringUtils;
-
-import org.subethamail.smtp.server.SMTPServer;
-
-
-import org.apache.nifi.annotation.lifecycle.OnStopped;
-import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-import org.apache.nifi.processor.DataUnit;
-
-import org.apache.nifi.annotation.lifecycle.OnScheduled;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.processor.AbstractProcessor;
-import org.apache.nifi.processor.ProcessorInitializationContext;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.processor.util.StandardValidators;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLSocket;
+import javax.net.ssl.SSLSocketFactory;
+import org.apache.commons.io.IOUtils;
import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.behavior.WritesAttributes;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSession;
-import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.io.OutputStreamCallback;
+import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.ssl.SSLContextService;
-
-import org.apache.nifi.processors.email.smtp.event.SmtpEvent;
-import org.apache.nifi.processors.email.smtp.handler.SMTPResultCode;
-import org.apache.nifi.processors.email.smtp.handler.SMTPMessageHandlerFactory;
+import org.apache.nifi.stream.io.LimitingInputStream;
+import org.springframework.util.StringUtils;
+import org.subethamail.smtp.server.SMTPServer;
@Tags({"listen", "email", "smtp"})
@InputRequirement(InputRequirement.Requirement.INPUT_FORBIDDEN)
-@CapabilityDescription("This processor implements a lightweight SMTP server to an arbitrary port, " +
- "allowing nifi to listen for incoming email. " +
- "" +
- "Note this server does not perform any email validation. If direct exposure to the internet is sought," +
- "it may be a better idea to use the combination of NiFi and an industrial scale MTA (e.g. Postfix)")
+@CapabilityDescription("This processor implements a lightweight SMTP server to an arbitrary port, "
+ + "allowing nifi to listen for incoming email. Note this server does not perform any email "
+ + "validation. If direct exposure to the internet is sought, it may be a better idea to use "
+ + "the combination of NiFi and an industrial scale MTA (e.g. Postfix)")
@WritesAttributes({
- @WritesAttribute(attribute = "mime.type", description = "The value used during HELO"),
- @WritesAttribute(attribute = "smtp.helo", description = "The value used during HELO"),
- @WritesAttribute(attribute = "smtp.certificates.*.serial", description = "The serial numbers for each of the " +
- "certificates used by an TLS peer"),
- @WritesAttribute(attribute = "smtp.certificates.*.principal", description = "The principal for each of the " +
- "certificates used by an TLS peer"),
- @WritesAttribute(attribute = "smtp.from", description = "The value used during MAIL FROM (i.e. envelope)"),
- @WritesAttribute(attribute = "smtp.to", description = "The value used during RCPT TO (i.e. envelope)"),
- @WritesAttribute(attribute = "smtp.src", description = "The source IP of the SMTP connection")})
-
-public class ListenSMTP extends AbstractProcessor {
- public static final String SMTP_HELO = "smtp.helo";
- public static final String SMTP_FROM = "smtp.from";
- public static final String SMTP_TO = "smtp.to";
- public static final String MIME_TYPE = "message/rfc822";
- public static final String SMTP_SRC_IP = "smtp.src";
-
-
- protected static final PropertyDescriptor SMTP_PORT = new PropertyDescriptor.Builder()
+ @WritesAttribute(attribute = "smtp.helo", description = "The value used during HELO"),
+ @WritesAttribute(attribute = "smtp.certificates.*.serial", description = "The serial numbers for each of the " +
+ "certificates used by an TLS peer"),
+ @WritesAttribute(attribute = "smtp.certificates.*.principal", description = "The principal for each of the " +
+ "certificates used by an TLS peer"),
+ @WritesAttribute(attribute = "smtp.src", description = "The source IP of the SMTP connection"),
+ @WritesAttribute(attribute = "smtp.from", description = "The value used during MAIL FROM (i.e. envelope)"),
+ @WritesAttribute(attribute = "smtp.recipient", description = "The value used during RCPT TO (i.e. envelope)"),
+ @WritesAttribute(attribute = "mime.type", description = "Mime type of the message")})
+public class ListenSMTP extends AbstractSessionFactoryProcessor {
+ static final PropertyDescriptor SMTP_PORT = new PropertyDescriptor.Builder()
.name("SMTP_PORT")
.displayName("Listening Port")
.description("The TCP port the ListenSMTP processor will bind to." +
@@ -106,26 +91,17 @@ public class ListenSMTP extends AbstractProcessor {
.addValidator(StandardValidators.PORT_VALIDATOR)
.build();
- protected static final PropertyDescriptor SMTP_HOSTNAME = new PropertyDescriptor.Builder()
- .name("SMTP_HOSTNAME")
- .displayName("SMTP hostname")
- .description("The hostname to be embedded into the banner displayed when an " +
- "SMTP client connects to the processor TCP port .")
- .required(true)
- .expressionLanguageSupported(false)
- .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
- .build();
-
- protected static final PropertyDescriptor SMTP_MAXIMUM_CONNECTIONS = new PropertyDescriptor.Builder()
+ static final PropertyDescriptor SMTP_MAXIMUM_CONNECTIONS = new PropertyDescriptor.Builder()
.name("SMTP_MAXIMUM_CONNECTIONS")
.displayName("Maximum number of SMTP connection")
.description("The maximum number of simultaneous SMTP connections.")
.required(true)
+ .defaultValue("1")
.expressionLanguageSupported(false)
.addValidator(StandardValidators.INTEGER_VALIDATOR)
.build();
- protected static final PropertyDescriptor SMTP_TIMEOUT = new PropertyDescriptor.Builder()
+ static final PropertyDescriptor SMTP_TIMEOUT = new PropertyDescriptor.Builder()
.name("SMTP_TIMEOUT")
.displayName("SMTP connection timeout")
.description("The maximum time to wait for an action of SMTP client.")
@@ -135,29 +111,17 @@ public class ListenSMTP extends AbstractProcessor {
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
- protected static final PropertyDescriptor SMTP_MAXIMUM_MSG_SIZE = new PropertyDescriptor.Builder()
+ static final PropertyDescriptor SMTP_MAXIMUM_MSG_SIZE = new PropertyDescriptor.Builder()
.name("SMTP_MAXIMUM_MSG_SIZE")
.displayName("SMTP Maximum Message Size")
.description("The maximum number of bytes the server will accept.")
.required(true)
- .defaultValue("20MB")
+ .defaultValue("20 MB")
.expressionLanguageSupported(false)
- .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+ .addValidator(StandardValidators.createDataSizeBoundsValidator(1, Integer.MAX_VALUE))
.build();
- protected static final PropertyDescriptor SMTP_MAXIMUM_INCOMING_MESSAGE_QUEUE = new PropertyDescriptor.Builder()
- .name("SMTP_MAXIMUM_INCOMING_MESSAGE_QUEUE")
- .displayName("SMTP message buffer length")
- .description("This property control the size of the Queue utilised by the processor to hold messages as they are processed. " +
- "Setting a very small value will decrease the number of emails the processor simultaneously, while setting an very large" +
- "queue will result in higher memory and CPU utilisation. The default setting of 1024 is generally a fair number.")
- .required(true)
- .expressionLanguageSupported(false)
- .addValidator(StandardValidators.INTEGER_VALIDATOR)
- .defaultValue("1024")
- .build();
-
- public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
+ static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
.name("SSL_CONTEXT_SERVICE")
.displayName("SSL Context Service")
.description("The Controller Service to use in order to obtain an SSL Context. If this property is set, " +
@@ -166,7 +130,7 @@ public class ListenSMTP extends AbstractProcessor {
.identifiesControllerService(SSLContextService.class)
.build();
- public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
+ static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
.name("CLIENT_AUTH")
.displayName("Client Auth")
.description("The client authentication policy to use for the SSL Context. Only used if an SSL Context Service is provided.")
@@ -174,309 +138,243 @@ public class ListenSMTP extends AbstractProcessor {
.allowableValues(SSLContextService.ClientAuth.NONE.toString(), SSLContextService.ClientAuth.REQUIRED.toString())
.build();
- @Override
- protected Collection customValidate(final ValidationContext validationContext) {
- final List results = new ArrayList<>();
-
- final String clientAuth = validationContext.getProperty(CLIENT_AUTH).getValue();
- final SSLContextService sslContextService = validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
-
- if (sslContextService != null && StringUtils.isBlank(clientAuth)) {
- results.add(new ValidationResult.Builder()
- .explanation("Client Auth must be provided when using TLS/SSL")
- .valid(false).subject("Client Auth").build());
- }
-
- return results;
-
- }
-
-
- public static final Relationship REL_SUCCESS = new Relationship.Builder()
- .name("success")
- .description("Extraction was successful")
+ protected static final PropertyDescriptor SMTP_HOSTNAME = new PropertyDescriptor.Builder()
+ .name("SMTP_HOSTNAME")
+ .displayName("SMTP hostname")
+ .description("The hostname to be embedded into the banner displayed when an " +
+ "SMTP client connects to the processor TCP port .")
+ .expressionLanguageSupported(false)
+ .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
.build();
- private Set relationships;
- private List propertyDescriptors;
- private volatile LinkedBlockingQueue incomingMessages;
+ static final Relationship REL_SUCCESS = new Relationship.Builder()
+ .name("success")
+ .description("All new messages will be routed as FlowFiles to this relationship")
+ .build();
- private volatile SMTPServer server;
- private AtomicBoolean initialized = new AtomicBoolean(false);
- private AtomicBoolean stopping = new AtomicBoolean(false);
+ private final static List propertyDescriptors;
+ private final static Set relationships;
+
+ static {
+ List _propertyDescriptors = new ArrayList<>();
+ _propertyDescriptors.add(SMTP_PORT);
+ _propertyDescriptors.add(SMTP_MAXIMUM_CONNECTIONS);
+ _propertyDescriptors.add(SMTP_TIMEOUT);
+ _propertyDescriptors.add(SMTP_MAXIMUM_MSG_SIZE);
+ _propertyDescriptors.add(SSL_CONTEXT_SERVICE);
+ _propertyDescriptors.add(CLIENT_AUTH);
+ _propertyDescriptors.add(SMTP_HOSTNAME);
+ propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
+
+ Set _relationships = new HashSet<>();
+ _relationships.add(REL_SUCCESS);
+ relationships = Collections.unmodifiableSet(_relationships);
+ }
+
+ private volatile SMTPServer smtp;
+
+ private volatile SmtpConsumer smtpConsumer;
+
+ private volatile int maxMessageSize;
+
+ /**
+ *
+ */
+ @Override
+ public void onTrigger(ProcessContext context, ProcessSessionFactory sessionFactory) throws ProcessException {
+ ProcessSession processSession = sessionFactory.createSession();
+ if (this.smtp == null) {
+ this.setupSmtpIfNecessary(context, processSession);
+ }
+
+ /*
+ * Will consume incoming message directly from the wire and into
+ * FlowFile/Content repository before exiting. This essentially limits
+ * any potential data loss by allowing SMTPServer thread to actually
+ * commit NiFi session if all good. However in the event of exception,
+ * such exception will be propagated back to the email sender via
+ * "undeliverable message" allowing such user to re-send the message
+ */
+ this.smtpConsumer.consumeUsing((inputDataStream) -> {
+ FlowFile flowFile = processSession.create();
+ AtomicInteger size = new AtomicInteger();
+ try {
+ flowFile = processSession.write(flowFile, new OutputStreamCallback() {
+ @Override
+ public void process(OutputStream out) throws IOException {
+ size.set(IOUtils.copy(new LimitingInputStream(inputDataStream, ListenSMTP.this.maxMessageSize, true), out));
+ }
+ });
+ flowFile = updateFlowFileWithAttributes(flowFile, processSession);
+
+ processSession.getProvenanceReporter().receive(flowFile,
+ "smtp://" + ListenSMTP.this.smtp.getHostName() + ":" + ListenSMTP.this.smtp.getPort() + "/");
+ processSession.transfer(flowFile, REL_SUCCESS);
+ processSession.commit();
+ return size.get();
+ } catch (Exception e) {
+ context.yield();
+ this.getLogger().error("Failed while processing incoming mail. " + e.getMessage(), e);
+ throw new IllegalStateException("Failed while processing incoming mail. " + e.getMessage(), e);
+ }
+ });
+ }
+
+ /**
+ *
+ */
+ @OnStopped
+ public void stop() {
+ this.getLogger().info("Stopping SMTPServer");
+ this.smtp.stop();
+ this.smtp = null;
+ this.getLogger().info("SMTPServer stopped");
+ }
+
+ /**
+ *
+ */
@Override
public Set getRelationships() {
return relationships;
}
+ /**
+ *
+ */
+ @Override
+ protected Collection customValidate(ValidationContext validationContext) {
+ List results = new ArrayList<>();
+
+ String clientAuth = validationContext.getProperty(CLIENT_AUTH).getValue();
+ SSLContextService sslContextService = validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+
+ if (sslContextService != null && !StringUtils.hasText(clientAuth)) {
+ results.add(new ValidationResult.Builder()
+ .subject(CLIENT_AUTH.getDisplayName())
+ .explanation(CLIENT_AUTH.getDisplayName() + " must be provided when using " + SSL_CONTEXT_SERVICE.getDisplayName())
+ .valid(false)
+ .build());
+ } else if (sslContextService == null && StringUtils.hasText(clientAuth)) {
+ results.add(new ValidationResult.Builder()
+ .subject(SSL_CONTEXT_SERVICE.getDisplayName())
+ .explanation(SSL_CONTEXT_SERVICE.getDisplayName() + " must be provided when selecting " + CLIENT_AUTH.getDisplayName())
+ .valid(false)
+ .build());
+ }
+ return results;
+ }
+
+ /**
+ *
+ */
@Override
protected List getSupportedPropertyDescriptors() {
return propertyDescriptors;
}
- @Override
- protected void init(final ProcessorInitializationContext context) {
- final Set relationships = new HashSet<>();
- relationships.add(REL_SUCCESS);
- this.relationships = Collections.unmodifiableSet(relationships);
-
- final List props = new ArrayList<>();
- props.add(SMTP_PORT);
- props.add(SMTP_HOSTNAME);
- props.add(SMTP_MAXIMUM_CONNECTIONS);
- props.add(SMTP_TIMEOUT);
- props.add(SMTP_MAXIMUM_MSG_SIZE);
- props.add(SMTP_MAXIMUM_INCOMING_MESSAGE_QUEUE);
- props.add(SSL_CONTEXT_SERVICE);
- props.add(CLIENT_AUTH);
- this.propertyDescriptors = Collections.unmodifiableList(props);
-
- }
-
- // Upon Schedule, reset the initialized state to false
- @OnScheduled
- public void onScheduled(ProcessContext context) {
- initialized.set(false);
- stopping.set(false);
- }
-
- protected synchronized void initializeSMTPServer(final ProcessContext context) throws Exception {
-
- // check if we are already running or if it is stopping
- if (initialized.get() && server.isRunning() || stopping.get() ) {
- return;
+ /**
+ *
+ */
+ private FlowFile updateFlowFileWithAttributes(FlowFile flowFile, ProcessSession processSession) {
+ Map attributes = new HashMap<>();
+ Certificate[] tlsPeerCertificates = this.smtpConsumer.getMessageContext().getTlsPeerCertificates();
+ if (tlsPeerCertificates != null) {
+ for (int i = 0; i < tlsPeerCertificates.length; i++) {
+ if (tlsPeerCertificates[i] instanceof X509Certificate) {
+ X509Certificate x509Cert = (X509Certificate) tlsPeerCertificates[i];
+ attributes.put("smtp.certificate." + i + ".serial", x509Cert.getSerialNumber().toString());
+ attributes.put("smtp.certificate." + i + ".subjectName", x509Cert.getSubjectDN().getName());
+ }
+ }
}
- incomingMessages = new LinkedBlockingQueue<>(context.getProperty(SMTP_MAXIMUM_INCOMING_MESSAGE_QUEUE).asInteger());
+ attributes.put("smtp.helo", this.smtpConsumer.getMessageContext().getHelo());
+ attributes.put("smtp.remote.addr", this.smtpConsumer.getMessageContext().getRemoteAddress().toString());
+ attributes.put("smtp.from", this.smtpConsumer.getFrom());
+ attributes.put("smtp.recepient", this.smtpConsumer.getRecipient());
+ attributes.put(CoreAttributes.MIME_TYPE.key(), "message/rfc822");
+ return processSession.putAllAttributes(flowFile, attributes);
+ }
- String clientAuth = null;
+ /**
+ *
+ */
+ private synchronized void setupSmtpIfNecessary(ProcessContext context, ProcessSession processSession) {
+ if (this.smtp == null) {
+ SmtpConsumer consumer = new SmtpConsumer();
+ SMTPServer smtpServer = this.createServerInstance(context, consumer);
+ smtpServer.setSoftwareName("Apache NiFi");
+ smtpServer.setPort(context.getProperty(SMTP_PORT).asInteger());
+ smtpServer.setMaxConnections(context.getProperty(SMTP_MAXIMUM_CONNECTIONS).asInteger());
+ this.maxMessageSize = context.getProperty(SMTP_MAXIMUM_MSG_SIZE).asDataSize(DataUnit.B).intValue();
+ smtpServer.setMaxMessageSize(this.maxMessageSize);
+ smtpServer.setConnectionTimeout(context.getProperty(SMTP_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue());
+ if (context.getProperty(SMTP_HOSTNAME).isSet()) {
+ smtpServer.setHostName(context.getProperty(SMTP_HOSTNAME).getValue());
+ }
- // If an SSLContextService was provided then create an SSLContext to pass down to the server
- SSLContext sslContext = null;
- final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
- if (sslContextService != null) {
- clientAuth = context.getProperty(CLIENT_AUTH).getValue();
- sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.valueOf(clientAuth));
+ this.smtpConsumer = consumer;
+ this.smtp = smtpServer;
+ this.smtp.start();
}
+ }
- final SSLContext finalSslContext = sslContext;
-
- SMTPMessageHandlerFactory smtpMessageHandlerFactory = new SMTPMessageHandlerFactory(incomingMessages, getLogger());
- final SMTPServer server = new SMTPServer(smtpMessageHandlerFactory) {
-
+ /**
+ *
+ */
+ private SMTPServer createServerInstance(ProcessContext context, SmtpConsumer consumer) {
+ SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+ SMTPServer smtpServer = sslContextService == null ? new ConsumerAwareSmtpServer(consumer) : new ConsumerAwareSmtpServer(consumer) {
@Override
public SSLSocket createSSLSocket(Socket socket) throws IOException {
InetSocketAddress remoteAddress = (InetSocketAddress) socket.getRemoteSocketAddress();
+ String clientAuth = context.getProperty(CLIENT_AUTH).getValue();
+ SSLContext sslContext = sslContextService.createSSLContext(SSLContextService.ClientAuth.valueOf(clientAuth));
+ SSLSocketFactory socketFactory = sslContext.getSocketFactory();
+ SSLSocket sslSocket = (SSLSocket) (socketFactory.createSocket(socket, remoteAddress.getHostName(),socket.getPort(), true));
+ sslSocket.setUseClientMode(false);
- SSLSocketFactory socketFactory = finalSslContext.getSocketFactory();
-
- SSLSocket s = (SSLSocket) (socketFactory.createSocket(socket, remoteAddress.getHostName(), socket.getPort(), true));
-
- s.setUseClientMode(false);
-
-
- // For some reason the createSSLContext above is not enough to enforce
- // client side auth
- // If client auth is required...
- if (SSLContextService.ClientAuth.REQUIRED.toString().equals(context.getProperty(CLIENT_AUTH).getValue())) {
- s.setNeedClientAuth(true);
+ if (SSLContextService.ClientAuth.REQUIRED.toString().equals(clientAuth)) {
+ this.setRequireTLS(true);
+ sslSocket.setNeedClientAuth(true);
}
-
-
- return s;
+ return sslSocket;
}
};
-
- // Set some parameters to our server
- server.setSoftwareName("Apache NiFi");
-
-
- // Set the Server options based on properties
- server.setPort(context.getProperty(SMTP_PORT).asInteger());
- server.setHostName(context.getProperty(SMTP_HOSTNAME).getValue());
- server.setMaxMessageSize(context.getProperty(SMTP_MAXIMUM_MSG_SIZE).asDataSize(DataUnit.B).intValue());
- server.setMaxConnections(context.getProperty(SMTP_MAXIMUM_CONNECTIONS).asInteger());
- server.setConnectionTimeout(context.getProperty(SMTP_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS).intValue());
-
-
- // Check if TLS should be enabled
if (sslContextService != null) {
- server.setEnableTLS(true);
+ smtpServer.setEnableTLS(true);
} else {
- server.setHideTLS(true);
+ smtpServer.setHideTLS(true);
}
-
- // Set TLS to required in case CLIENT_AUTH = required
- if (SSLContextService.ClientAuth.REQUIRED.toString().equals(context.getProperty(CLIENT_AUTH).getValue())) {
- server.setRequireTLS(true);
- }
-
- this.server = server;
- server.start();
-
- getLogger().info("Server started and listening on port " + server.getPort());
-
- initialized.set(true);
- stopping.set(false);
+ return smtpServer;
}
- @OnUnscheduled
- public void startShutdown() throws Exception {
- if (server != null) {
- stopping.set(true);
- getLogger().info("Shutting down processor P{}", new Object[]{server});
- server.stop();
- getLogger().info("Shut down {}", new Object[]{server});
- }
- }
+ /**
+ * Wrapper over {@link SMTPServer} that is aware of the {@link SmtpConsumer}
+ * to ensure that its stop() operation is called during server stoppage.
+ */
+ private static class ConsumerAwareSmtpServer extends SMTPServer {
- @OnStopped
- public void completeShutdown() throws Exception {
- if (server != null) {
- if (!server.isRunning() && stopping.get() ) {
- stopping.set(false);
- }
- getLogger().info("Completed shut down {}", new Object[]{server});
- }
- }
-
- @Override
- public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
-
- try {
- initializeSMTPServer(context);
- } catch (Exception e) {
- context.yield();
- throw new ProcessException("Failed to initialize the SMTP server", e);
+ /**
+ *
+ */
+ public ConsumerAwareSmtpServer(SmtpConsumer consumer) {
+ super(consumer);
}
- while (!incomingMessages.isEmpty()) {
- SmtpEvent message = incomingMessages.poll();
-
- if (message == null) {
- return;
- }
-
- synchronized (message) {
- if (resultCodeSetAndIsError(message)) {
- SMTPResultCode resultCode = SMTPResultCode.fromCode(message.getReturnCode());
- getLogger().warn("Message failed before onTrigger processing message was: " + resultCode.getLogMessage());
- continue;
- }
-
- try {
- FlowFile flowfile = session.create();
-
- if (message.getMessageData() != null) {
- flowfile = session.write(flowfile, out -> {
- InputStream inputStream = message.getMessageData();
- byte [] buffer = new byte[1024];
-
- int rd;
- long totalBytesRead =0;
-
- while ((rd = inputStream.read(buffer, 0, buffer.length)) != -1 ) {
- totalBytesRead += rd;
- if (totalBytesRead > server.getMaxMessageSize() ) {
- message.setReturnCode(500);
- message.setProcessed();
- break;
- }
- out.write(buffer, 0, rd);
- }
- out.flush();
- });
- } else {
- getLogger().debug("Message body was null");
- message.setReturnCode(SMTPResultCode.UNKNOWN_ERROR_CODE.getCode());
- message.setProcessed();
- }
-
- if (!message.getProcessed()) {
- HashMap attributes = new HashMap<>();
- // Gather message attributes
- attributes.put(SMTP_HELO, message.getHelo());
- attributes.put(SMTP_SRC_IP, message.getHelo());
- attributes.put(SMTP_FROM, message.getFrom());
- attributes.put(SMTP_TO, message.getTo());
-
- List