Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Collection;
Expand Down Expand Up @@ -74,6 +73,7 @@
import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.util.StandardValidators;
import org.apache.nifi.stream.io.StreamUtils;
import org.apache.nifi.util.StringUtils;

@SupportsBatching
@Tags({"email", "put", "notify", "smtp"})
Expand Down Expand Up @@ -226,6 +226,18 @@ public class PutEmail extends AbstractProcessor {
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.defaultValue("false")
.build();
public static final PropertyDescriptor INPUT_CHARACTER_SET = new PropertyDescriptor.Builder()
.name("input-character-set")
.displayName("Input Character Set")
.description("Specifies the character set of the FlowFile contents "
+ "for reading input FlowFile contents to generate the message body "
+ "or as an attachment to the message. "
+ "If not set, UTF-8 will be the default value.")
.required(false)
.expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
.defaultValue("UTF-8")
.build();
public static final PropertyDescriptor INCLUDE_ALL_ATTRIBUTES = new PropertyDescriptor.Builder()
.name("Include All Attributes In Message")
.description("Specifies whether or not all FlowFile attributes should be recorded in the body of the email message")
Expand All @@ -243,8 +255,6 @@ public class PutEmail extends AbstractProcessor {
.description("FlowFiles that fail to send will be routed to this relationship")
.build();

private static final Charset CONTENT_CHARSET = StandardCharsets.UTF_8;

private List<PropertyDescriptor> properties;

private Set<Relationship> relationships;
Expand Down Expand Up @@ -285,6 +295,7 @@ protected void init(final ProcessorInitializationContext context) {
properties.add(SUBJECT);
properties.add(MESSAGE);
properties.add(CONTENT_AS_MESSAGE);
properties.add(INPUT_CHARACTER_SET);
properties.add(ATTACH_FILE);
properties.add(INCLUDE_ALL_ATTRIBUTES);
this.properties = Collections.unmodifiableList(properties);
Expand Down Expand Up @@ -324,7 +335,7 @@ protected Collection<ValidationResult> customValidate(final ValidationContext co
@OnScheduled
public void onScheduled(final ProcessContext context) {
final String attributeNameRegex = context.getProperty(ATTRIBUTE_NAME_REGEX).getValue();
this.attributeNamePattern = attributeNameRegex == null ? null : Pattern.compile(attributeNameRegex);
this.attributeNamePattern = StringUtils.isBlank(attributeNameRegex) ? null : Pattern.compile(attributeNameRegex);
}

private void setMessageHeader(final String header, final String value, final Message message) throws MessagingException {
Expand Down Expand Up @@ -362,17 +373,16 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
}
this.setMessageHeader("X-Mailer", context.getProperty(HEADER_XMAILER).evaluateAttributeExpressions(flowFile).getValue(), message);
message.setSubject(context.getProperty(SUBJECT).evaluateAttributeExpressions(flowFile).getValue());

final String messageText = getMessage(flowFile, context, session);

final String contentType = context.getProperty(CONTENT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
final String contentType = context.getProperty(CONTENT_TYPE).evaluateAttributeExpressions(flowFile).getValue().split(";")[0] + "; charset=\"utf-8\"";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The split() seems unnecessary, can it be removed? Appending charset="utf-8" implies that the Content Type will always be UTF-8, is that correct?

message.setContent(messageText, contentType);
message.setSentDate(new Date());

if (context.getProperty(ATTACH_FILE).asBoolean()) {
final MimeBodyPart mimeText = new PreencodedMimeBodyPart("base64");
mimeText.setDataHandler(new DataHandler(new ByteArrayDataSource(
Base64.encodeBase64(messageText.getBytes(CONTENT_CHARSET)), contentType + "; charset=\"utf-8\"")));
Base64.encodeBase64(messageText.getBytes(StandardCharsets.UTF_8)), contentType)));
Comment on lines -375 to +385
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should StandardCharsets.UTF_8 be replaced with the configured value of Input Character Set?

final MimeBodyPart mimeFile = new MimeBodyPart();
session.read(flowFile, stream -> {
try {
Expand Down Expand Up @@ -401,15 +411,17 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
}
}

private String getMessage(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) {
private String getMessage(final FlowFile flowFile, final ProcessContext context, final ProcessSession session) throws UnsupportedEncodingException {
String messageText = "";
String inputCharacterSet = context.getProperty(INPUT_CHARACTER_SET).evaluateAttributeExpressions(flowFile).getValue();
inputCharacterSet = StringUtils.isBlank(inputCharacterSet) ? "UTF-8" : inputCharacterSet;

if(context.getProperty(CONTENT_AS_MESSAGE).evaluateAttributeExpressions(flowFile).asBoolean()) {
// reading all the content of the input flow file
final byte[] byteBuffer = new byte[(int) flowFile.getSize()];
session.read(flowFile, in -> StreamUtils.fillBuffer(in, byteBuffer, false));

messageText = new String(byteBuffer, 0, byteBuffer.length, CONTENT_CHARSET);
messageText = new String(byteBuffer, 0, byteBuffer.length, inputCharacterSet);
} else if (context.getProperty(MESSAGE).isSet()) {
messageText = context.getProperty(MESSAGE).evaluateAttributeExpressions(flowFile).getValue();
}
Expand Down