[ https://issues.apache.org/jira/browse/NIFI-1899?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15375925#comment-15375925 ]
ASF GitHub Bot commented on NIFI-1899: -------------------------------------- Github user trixpan commented on a diff in the pull request: https://github.com/apache/nifi/pull/483#discussion_r70721418 --- Diff: nifi-nar-bundles/nifi-email-bundle/nifi-email-processors/src/main/java/org/apache/nifi/processors/email/smtp/handler/SMTPMessageHandlerFactory.java --- @@ -0,0 +1,147 @@ +/* + * 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.processors.email.smtp.handler; + +import java.io.IOException; +import java.io.InputStream; +import java.security.cert.X509Certificate; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.stream.io.ByteArrayOutputStream; +import org.apache.nifi.util.StopWatch; +import org.subethamail.smtp.DropConnectionException; +import org.subethamail.smtp.MessageContext; +import org.subethamail.smtp.MessageHandler; +import org.subethamail.smtp.MessageHandlerFactory; +import org.subethamail.smtp.RejectException; +import org.subethamail.smtp.TooMuchDataException; +import org.subethamail.smtp.server.SMTPServer; + +import org.apache.nifi.processors.email.smtp.event.SmtpEvent; + + +public class SMTPMessageHandlerFactory implements MessageHandlerFactory { + final LinkedBlockingQueue<SmtpEvent> incomingMessages; + final ComponentLog logger; + + public SMTPMessageHandlerFactory(LinkedBlockingQueue<SmtpEvent> incomingMessages, ComponentLog logger) { + this.incomingMessages = incomingMessages; + this.logger = logger; + } + + @Override + public MessageHandler create(MessageContext messageContext) { + return new Handler(messageContext, incomingMessages, logger); + } + + class Handler implements MessageHandler { + final MessageContext messageContext; + String from; + String recipient; + byte [] messageBody; + + + public Handler(MessageContext messageContext, LinkedBlockingQueue<SmtpEvent> incomingMessages, ComponentLog logger){ + this.messageContext = messageContext; + } + + @Override + public void from(String from) throws RejectException { + // TODO: possibly whitelist senders? + this.from = from; + } + + @Override + public void recipient(String recipient) throws RejectException { + // TODO: possibly whitelist receivers? + this.recipient = recipient; + } + + @Override + public void data(InputStream inputStream) throws RejectException, TooMuchDataException, IOException { + // Start counting the timer... + + StopWatch watch = new StopWatch(false); + + SMTPServer server = messageContext.getSMTPServer(); + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + byte [] buffer = new byte[1024]; + int rd; + + while ((rd = inputStream.read(buffer, 0, buffer.length)) != -1) { + baos.write(buffer, 0, rd); + } + if (baos.getBufferLength() > server.getMaxMessageSize()) { + throw new TooMuchDataException("Data exceeds the amount allowed."); + } + + baos.flush(); + this.messageBody = baos.toByteArray(); + + + X509Certificate[] certificates = new X509Certificate[]{}; + + String remoteIP = messageContext.getRemoteAddress().toString(); + String helo = messageContext.getHelo(); + if (messageContext.getTlsPeerCertificates() != null ){ + certificates = (X509Certificate[]) messageContext.getTlsPeerCertificates().clone(); + } + + SmtpEvent message = new SmtpEvent(remoteIP, helo, from, recipient, certificates, messageBody); + try { + // Try to queue the message back to the NiFi session + incomingMessages.put(message); + } catch (InterruptedException e) { + // Throws an error to the incoming server alerting about issues queuing the message in NiFi + logger.error("Hit an error sending message back to NiFi main thread {}. Sending and error back to SMTP client and discarding message", new Object [] {e}); + throw new DropConnectionException(451, "NiFi something went wrong while processing you message, please retry again later"); + } + + // Once message has been sent to the queue, it should be processed by NiFi onTrigger, + // a flowfile created and its processed status updated before an acknowledgment is + // given back to the SMTP client + try { + synchronized(message) { + while(! message.getProcessed()) { + // Check to see if it is too late... + final long serverTimeout = TimeUnit.NANOSECONDS.convert(messageContext.getSMTPServer().getConnectionTimeout(), TimeUnit.MILLISECONDS); + if ( watch.getElapsed(TimeUnit.MILLISECONDS) <= serverTimeout) { + // Will should allow for a couple of retries; + message.wait(messageContext.getSMTPServer().getConnectionTimeout() / 20); + } else { + logger.error("Did not receive the onTrigger reponse within the acceptable timeframes"); + throw new DropConnectionException(451, "The processing of your message timed-out, we may have received it but you better off sending it again"); --- End diff -- @JPercivall Regarding the assumptions about timing. I got that. Brilliant approach that will keep in mind in the future (I believe they should all have been removed by now?) Regarding the third race condition, isn't this condition similar to what happens in ConsumeMqtt ? I ask because when I read [this](https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-mqtt-bundle/nifi-mqtt-processors/src/main/java/org/apache/nifi/processors/mqtt/ConsumeMQTT.java#L294) I interpret(perhaps incorrectly?) that final test tries to remove the message from the queue (after doing its work over a peek'ed variable). If the remove fails it logs a warning. Given Line 129 is the only remove / take / poll present I assume (also perhaps incorrectly?) that for that remove to fail, something would have gone totally belly up, but more likely than not, another thread would have won the race and consumed the object from processor queue faster than the thread logging the warning? It this is the case, wouldn't that mean that mqttMessage also exists on both threads at the same time, having been removed from one, while the other still processing it? Having said that I am fine to have a look at the synchronized again if you still find it the best approach (I take your judgement on that). > Create ListenSMTP & ExtractEmailAttachment processors > ----------------------------------------------------- > > Key: NIFI-1899 > URL: https://issues.apache.org/jira/browse/NIFI-1899 > Project: Apache NiFi > Issue Type: New Feature > Reporter: Andre > -- This message was sent by Atlassian JIRA (v6.3.4#6332)