[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-14 Thread olegz
Github user olegz closed the pull request at:

https://github.com/apache/nifi/pull/827


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-12 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74527137
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +90,17 @@
 .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()
--- End diff --

It is propagated to the target API


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74527077
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +90,17 @@
 .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()
--- End diff --

@olegz is SMTP_MAXIMUM_CONNECTIONS being enforced at all? 
```

$ date && telnet 0 2525
Fri Aug 12 09:02:31 AEST 2016
Trying 0.0.0.0...
Connected to 0.
Escape character is '^]'.
220 localhost ESMTP Apache NiFi
```

$ date && telnet 0 2525
Fri Aug 12 09:02:30 AEST 2016
Trying 0.0.0.0...
Connected to 0.
Escape character is '^]'.
220 localhost ESMTP Apache NiFi

I did this test and it doesn't seem to be:

```
$ date  && netstat -an| grep 2525 | pcregrep "2525\s+ES"
Fri Aug 12 09:02:33 AEST 2016
tcp0  0 0.0.0.0:25250.0.0.0:*   
LISTEN
tcp0  0 127.0.0.1:38840 127.0.0.1:2525  
ESTABLISHED
tcp0  0 127.0.0.1:38842 127.0.0.1:2525  
ESTABLISHED
```




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74423248
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

@olegz I see where you come from but put file is controlled by the DFM, 
ListenSMTP is an exposed service

True, we don't do it everywhere but it is present on GetFile as well as 
other processors crossing administrative domains (DFM <> FS) and so it goes



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74427203
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

Agree with that. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74425136
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

If you are intending to speak SMTP tgan you need to follow the relevant 
RFC, this means a connecting MTA should be actively informed of a time out 
otherwise it will indefinitely hang connected as you do not return a 4XX code 
nor an ok,  instead we just leave the  connection for dead. This means all 
clients will have to wait for their own timeouts before disconnecting 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74425869
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

We can definitely look at this as an improvement in the future


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74425691
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

I have no intention to speak SMTP. I am relying on the underlying API to do 
that and if it's inadequate, then let's look for a better API.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74422525
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

I am not sure hat you're saying. If Subethasmtp is inadequate API, then we 
have to scrap it and use something different. But once again, let's keep in 
mind that the job of this processor is to simply open up a socket with attached 
application layer capable of speaking SMTP. That is all. For anything else use 
commercial server and attache to it via available mail protocols.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74421827
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

Yes it goes straight to disc via the Function that is passed to the 
SmtpConsumer. 
As far as enforcing data size we can address it in the future if that is a 
problem. For example we don't have the same enforcement in PutFile and other 
processors unless memory is involved which in this case it isn't. 
Also, in all honesty I can't see someone using ListenSMTP or email in 
general to send large data or to move data. There are better more suitable 
protocols for it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74421721
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

It will cause clients to hang around indefinitely 

Subethasmtp is a weird API. I don't truly understand the reason why it was 
coded like that but it transfers good chunks of the smtp conversation to the 
developer making use of it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74420800
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

I imagine you write straight to disk? If so we should enforce this somehow. 
It would be sort of concerning to have a client filling up your disk with a 
single message? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74418537
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

Same as above. The property is simply propagated to the underlying API


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74418332
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .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()
--- End diff --

Is SMTP_TIMEOUT being enforced at the DATA stage?

The server seems to hang around the data stage with the connection open no 
matter how long the client idles for.

Typing . and quit do not work

```
$ time telnet 0 2525
Trying 0.0.0.0...
Connected to 0.
Escape character is '^]'.
220 localhost ESMTP Apache NiFi
helo .
250 localhost
mail from: x
250 Ok
rcpt to: a
250 Ok
data
354 End data with .
H

.
^]
telnet> quit
Connection closed.

real1m24.348s
user0m0.000s
sys 0m0.002s
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74417054
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

I was just propagating property value to it's corresponding property in 
SMTPServer, so IMO if server doesn't care why should we? Note that with the 
refactoring we never write a message into memory so even if you try to send TB 
it will be successful.
If on the other hand we are to provide some type of enforcement on the NiFi 
side , then we should completely disassociate this property from SMTPServer or 
have some other property to eliminate the confusion. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74416389
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -135,29 +89,17 @@
 .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()
--- End diff --

Is SMTP_MAXIMUM_SIZE being enforced? I've set it to 512 B and ended up with 
a message of 1KB?

Note that setMaxMessageSize has no effective role in dropping messages, 
instead all it does is to configure the EHLO response:

```
$ telnet 0 2525
Trying 0.0.0.0...
Connected to 0.
Escape character is '^]'.
220 localhost ESMTP Apache NiFi
EHLO .
250-localhost
250-8BITMIME
250-SIZE 512
250 Ok

```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74414544
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
-protected Collection customValidate(final 
ValidationContext validationContext) {
-final List results = new ArrayList<>();
+static final Relationship REL_SUCCESS = new Relationship.Builder()
+.name("success")
+.description("All new messages will be routed as FlowFiles to 
this relationship")
+.build();
 
-final String clientAuth = 
validationContext.getProperty(CLIENT_AUTH).getValue();
-final SSLContextService sslContextService = 
validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+private final static List propertyDescriptors;
 
-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());
-}
+private final static Set relationships;
 
-return results;
+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 = 
Collections.unmodifiableList(_propertyDescriptors);
 
+Set _relationships = new HashSet<>();
+_relationships.add(REL_SUCCESS);
+relationships = Collections.unmodifiableSet(_relationships);
 }
 
+private volatile SMTPServer smtp;
 
-public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
-.name("success")
-.description("Extraction was successful")
-.build();
+private volatile SmtpConsumer smtpConsumer;
 
-private Set relationships;
-private List propertyDescriptors;
-private volatile LinkedBlockingQueue incomingMessages;
+/**
+ *
+ */
+@Override
+public void onTrigger(ProcessContext context, ProcessSessionFactory 
sessionFactory) throws ProcessException {
+ProcessSession processSession = sessionFactory.createSession();
+if (this.smtp == null) {
+this.setupSmtpIfNecessary(context, processSession);
+}
+
+if (this.smtpConsumer.hasMessage()) {
+try {
+/*
+ * 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();
+flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
+@Override
+public void process(final OutputStream out) throws 
IOException {
+size.set(IOUtils.copy(inputDataStream, out));
+}
+});
+

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-11 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74414087
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-   

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74364045
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/test/java/org/apache/nifi/processors/email/TestListenSMTP.java
 ---
@@ -13,307 +13,174 @@
  *  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 static org.junit.Assert.assertTrue;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.commons.mail.Email;
 import org.apache.commons.mail.EmailException;
 import org.apache.commons.mail.SimpleEmail;
-import org.apache.nifi.processor.ProcessContext;
-import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.remote.io.socket.NetworkUtils;
+import org.apache.nifi.ssl.SSLContextService;
 import org.apache.nifi.ssl.StandardSSLContextService;
-import org.apache.nifi.util.MockFlowFile;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
-
-import org.apache.nifi.ssl.SSLContextService;
-
-import org.junit.Assert;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
 public class TestListenSMTP {
 
-@Test(timeout=15000)
-public void ValidEmailTls() throws Exception {
-boolean[] failed = {false};
-ListenSMTP listenSmtp = new ListenSMTP();
-final TestRunner runner = TestRunners.newTestRunner(listenSmtp);
-
-runner.setProperty(ListenSMTP.SMTP_PORT, "0");
-runner.setProperty(ListenSMTP.SMTP_HOSTNAME, "bermudatriangle");
-runner.setProperty(ListenSMTP.SMTP_MAXIMUM_CONNECTIONS, "3");
-runner.setProperty(ListenSMTP.SMTP_TIMEOUT, "10 seconds");
-
-// Setup the SSL Context
-final SSLContextService sslContextService = new 
StandardSSLContextService();
-runner.addControllerService("ssl-context", sslContextService);
-runner.setProperty(sslContextService, 
StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
-runner.setProperty(sslContextService, 
StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
-runner.setProperty(sslContextService, 
StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
-runner.setProperty(sslContextService, 
StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
-runner.setProperty(sslContextService, 
StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
-runner.setProperty(sslContextService, 
StandardSSLContextService.KEYSTORE_TYPE, "JKS");
-runner.enableControllerService(sslContextService);
-
-// and add the SSL context to the runner
-runner.setProperty(ListenSMTP.SSL_CONTEXT_SERVICE, "ssl-context");
-runner.setProperty(ListenSMTP.CLIENT_AUTH, 
SSLContextService.ClientAuth.NONE.name());
-
+private ScheduledExecutorService executor;
 
+/**
+ *
+ */
+@Before
+public void before() {
+this.executor = Executors.newScheduledThreadPool(2);
+}
 
-final ProcessSessionFactory processSessionFactory = 
runner.getProcessSessionFactory();
-final ProcessContext context = runner.getProcessContext();
-
-// NOTE: This test routine uses  the same strategy used by 
TestListenAndPutSyslog
-// where listenSmtp method calls are used to allow the processor 
to be started using
-// port "0" without triggering a violation of PORT_VALIDATOR
-
-listenSmtp.onScheduled(context);
-listenSmtp.initializeSMTPServer(context);
-
-final int port = listenSmtp.getPort();
-
-try {
-final Thread clientThread = new Thread(new Runnable() {
-@Override
-public void run() {
-try {
-
-
-System.setProperty("mail.smtp.ssl.trust", "*");
-System.setProperty("javax.net.ssl.keyStore", 
"src/test/resources/localhost-ks.jks");
-
System.setProperty("javax.net.ssl.keyStorePassword", "localtest");
-
-Email email = new SimpleEmail();
-
-email.setHostName("127.0.0.1");
-email.setSmtpPort(port);
  

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74363458
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
-protected Collection customValidate(final 
ValidationContext validationContext) {
-final List results = new ArrayList<>();
+static final Relationship REL_SUCCESS = new Relationship.Builder()
+.name("success")
+.description("All new messages will be routed as FlowFiles to 
this relationship")
+.build();
 
-final String clientAuth = 
validationContext.getProperty(CLIENT_AUTH).getValue();
-final SSLContextService sslContextService = 
validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+private final static List propertyDescriptors;
 
-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());
-}
+private final static Set relationships;
 
-return results;
+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 = 
Collections.unmodifiableList(_propertyDescriptors);
 
+Set _relationships = new HashSet<>();
+_relationships.add(REL_SUCCESS);
+relationships = Collections.unmodifiableSet(_relationships);
 }
 
+private volatile SMTPServer smtp;
 
-public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
-.name("success")
-.description("Extraction was successful")
-.build();
+private volatile SmtpConsumer smtpConsumer;
 
-private Set relationships;
-private List propertyDescriptors;
-private volatile LinkedBlockingQueue incomingMessages;
+/**
+ *
+ */
+@Override
+public void onTrigger(ProcessContext context, ProcessSessionFactory 
sessionFactory) throws ProcessException {
+ProcessSession processSession = sessionFactory.createSession();
+if (this.smtp == null) {
+this.setupSmtpIfNecessary(context, processSession);
+}
+
+if (this.smtpConsumer.hasMessage()) {
+try {
+/*
+ * 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();
+flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
+@Override
+public void process(final OutputStream out) throws 
IOException {
+size.set(IOUtils.copy(inputDataStream, out));
+}
+});
+

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362947
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
--- End diff --

Will put it back


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362949
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-   

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362934
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .addValidator(StandardValidators.PORT_VALIDATOR)
 .build();
 
-protected static final PropertyDescriptor SMTP_HOSTNAME = new 
PropertyDescriptor.Builder()
-.name("SMTP_HOSTNAME")
--- End diff --

Perhaps I am not fully in line with that due to not fully understanding it, 
but if it's only used for "display" I am ok with putting it back in. Please 
confirm


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362870
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362680
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .addValidator(StandardValidators.PORT_VALIDATOR)
 .build();
 
-protected static final PropertyDescriptor SMTP_HOSTNAME = new 
PropertyDescriptor.Builder()
-.name("SMTP_HOSTNAME")
--- End diff --

Because I may have a system using AWS's weird naming structures but look to 
display a fancy name like mx.trixpanmailservices.com




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362612
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-   

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74362408
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
-protected Collection customValidate(final 
ValidationContext validationContext) {
-final List results = new ArrayList<>();
+static final Relationship REL_SUCCESS = new Relationship.Builder()
+.name("success")
+.description("All new messages will be routed as FlowFiles to 
this relationship")
+.build();
 
-final String clientAuth = 
validationContext.getProperty(CLIENT_AUTH).getValue();
-final SSLContextService sslContextService = 
validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+private final static List propertyDescriptors;
 
-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());
-}
+private final static Set relationships;
 
-return results;
+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 = 
Collections.unmodifiableList(_propertyDescriptors);
 
+Set _relationships = new HashSet<>();
+_relationships.add(REL_SUCCESS);
+relationships = Collections.unmodifiableSet(_relationships);
 }
 
+private volatile SMTPServer smtp;
 
-public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
-.name("success")
-.description("Extraction was successful")
-.build();
+private volatile SmtpConsumer smtpConsumer;
 
-private Set relationships;
-private List propertyDescriptors;
-private volatile LinkedBlockingQueue incomingMessages;
+/**
+ *
+ */
+@Override
+public void onTrigger(ProcessContext context, ProcessSessionFactory 
sessionFactory) throws ProcessException {
+ProcessSession processSession = sessionFactory.createSession();
+if (this.smtp == null) {
+this.setupSmtpIfNecessary(context, processSession);
+}
+
+if (this.smtpConsumer.hasMessage()) {
+try {
+/*
+ * 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();
+flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
+@Override
+public void process(final OutputStream out) throws 
IOException {
+size.set(IOUtils.copy(inputDataStream, out));
+}
+});
+

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361967
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
-protected Collection customValidate(final 
ValidationContext validationContext) {
-final List results = new ArrayList<>();
+static final Relationship REL_SUCCESS = new Relationship.Builder()
+.name("success")
+.description("All new messages will be routed as FlowFiles to 
this relationship")
+.build();
 
-final String clientAuth = 
validationContext.getProperty(CLIENT_AUTH).getValue();
-final SSLContextService sslContextService = 
validationContext.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
+private final static List propertyDescriptors;
 
-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());
-}
+private final static Set relationships;
 
-return results;
+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 = 
Collections.unmodifiableList(_propertyDescriptors);
 
+Set _relationships = new HashSet<>();
+_relationships.add(REL_SUCCESS);
+relationships = Collections.unmodifiableSet(_relationships);
 }
 
+private volatile SMTPServer smtp;
 
-public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
-.name("success")
-.description("Extraction was successful")
-.build();
+private volatile SmtpConsumer smtpConsumer;
 
-private Set relationships;
-private List propertyDescriptors;
-private volatile LinkedBlockingQueue incomingMessages;
+/**
+ *
+ */
+@Override
+public void onTrigger(ProcessContext context, ProcessSessionFactory 
sessionFactory) throws ProcessException {
+ProcessSession processSession = sessionFactory.createSession();
+if (this.smtp == null) {
+this.setupSmtpIfNecessary(context, processSession);
+}
+
+if (this.smtpConsumer.hasMessage()) {
+try {
+/*
+ * 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();
+flowFile = processSession.write(flowFile, new 
OutputStreamCallback() {
+@Override
+public void process(final OutputStream out) throws 
IOException {
+size.set(IOUtils.copy(inputDataStream, out));
+}
+});
+

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361819
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
--- End diff --

I believe you are talking about customValidate() you had there before. I 
think it was an oversight on my end. Will put it back 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361725
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-

[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361570
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -166,317 +108,158 @@
 .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.")
 .required(false)
 .allowableValues(SSLContextService.ClientAuth.NONE.toString(), 
SSLContextService.ClientAuth.REQUIRED.toString())
 .build();
 
-@Override
--- End diff --

Although harmless, this should not be a valid configuration:


![image](https://cloud.githubusercontent.com/assets/3108527/17576962/e5e0414a-5fbb-11e6-9050-9234dc1fe7d1.png)



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread olegz
Github user olegz commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361483
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .addValidator(StandardValidators.PORT_VALIDATOR)
 .build();
 
-protected static final PropertyDescriptor SMTP_HOSTNAME = new 
PropertyDescriptor.Builder()
-.name("SMTP_HOSTNAME")
--- End diff --

True, so my question is whatever it defaults to is based on the network 
settings of your machine. Exposing it means I can set it to any name, so I am 
wondering why would this be important?
Don't get me wrong, putting it back is trivial, but I am looking for 
justification.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74361191
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -106,26 +69,17 @@
 .addValidator(StandardValidators.PORT_VALIDATOR)
 .build();
 
-protected static final PropertyDescriptor SMTP_HOSTNAME = new 
PropertyDescriptor.Builder()
-.name("SMTP_HOSTNAME")
--- End diff --

Without this SubEtha seems to default to localhost

```
$ telnet 10.0.2.15 2525
Trying 10.0.2.15...
Connected to 10.0.2.15.
Escape character is '^]'.
220 localhost ESMTP Apache NiFi

```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] nifi pull request #827: NIFI-2519 Fixed and refactored ListenSMTP processor

2016-08-10 Thread trixpan
Github user trixpan commented on a diff in the pull request:

https://github.com/apache/nifi/pull/827#discussion_r74359964
  
--- Diff: 
nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/ListenSMTP.java
 ---
@@ -13,89 +13,52 @@
  *  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.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 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 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 java.util.concurrent.atomic.AtomicInteger;
 
-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.annotation.lifecycle.OnStopped;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+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.components.ValidationContext;
-import org.apache.nifi.components.ValidationResult;
-import org.apache.nifi.flowfile.FlowFile;
+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.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)")
-@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)"),
-