mirror of https://github.com/apache/nifi.git
NIFI-7183 - This closes #4073. Improve ReplaceText when removing FF's content
Signed-off-by: Joe Witt <joewitt@apache.org>
This commit is contained in:
parent
722b99432c
commit
9a8a551e03
|
@ -26,6 +26,7 @@ import org.apache.nifi.annotation.behavior.SystemResource;
|
|||
import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException;
|
||||
import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
|
@ -199,6 +200,7 @@ public class ReplaceText extends AbstractProcessor {
|
|||
|
||||
private List<PropertyDescriptor> properties;
|
||||
private Set<Relationship> relationships;
|
||||
private ReplacementStrategyExecutor replacementStrategyExecutor;
|
||||
|
||||
@Override
|
||||
protected void init(final ProcessorInitializationContext context) {
|
||||
|
@ -254,21 +256,12 @@ public class ReplaceText extends AbstractProcessor {
|
|||
return errors;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final ComponentLog logger = getLogger();
|
||||
|
||||
@OnScheduled
|
||||
public void setup(ProcessContext context) {
|
||||
final String replacementStrategy = context.getProperty(REPLACEMENT_STRATEGY).getValue();
|
||||
|
||||
final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
|
||||
final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
|
||||
final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
|
||||
final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
|
||||
final byte[] buffer;
|
||||
if (replacementStrategy.equalsIgnoreCase(regexReplaceValue) || replacementStrategy.equalsIgnoreCase(literalReplaceValue)) {
|
||||
buffer = new byte[maxBufferSize];
|
||||
|
@ -276,7 +269,6 @@ public class ReplaceText extends AbstractProcessor {
|
|||
buffer = null;
|
||||
}
|
||||
|
||||
ReplacementStrategyExecutor replacementStrategyExecutor;
|
||||
switch (replacementStrategy) {
|
||||
case prependValue:
|
||||
replacementStrategyExecutor = new PrependReplace();
|
||||
|
@ -288,6 +280,10 @@ public class ReplaceText extends AbstractProcessor {
|
|||
// for backward compatibility - if replacement regex is ".*" then we will simply always replace the content.
|
||||
if (context.getProperty(SEARCH_VALUE).getValue().equals(".*")) {
|
||||
replacementStrategyExecutor = new AlwaysReplace();
|
||||
} else if (context.getProperty(SEARCH_VALUE).getValue().equals(DEFAULT_REGEX)
|
||||
&& evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)
|
||||
&& context.getProperty(REPLACEMENT_VALUE).getValue().isEmpty()) {
|
||||
replacementStrategyExecutor = new AlwaysReplace();
|
||||
} else {
|
||||
replacementStrategyExecutor = new RegexReplace(buffer, context);
|
||||
}
|
||||
|
@ -302,6 +298,20 @@ public class ReplaceText extends AbstractProcessor {
|
|||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
final ComponentLog logger = getLogger();
|
||||
|
||||
final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
|
||||
final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();
|
||||
final String evaluateMode = context.getProperty(EVALUATION_MODE).getValue();
|
||||
|
||||
if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
|
||||
if (flowFile.getSize() > maxBufferSize && replacementStrategyExecutor.isAllDataBufferedForEntireText()) {
|
||||
|
@ -325,6 +335,7 @@ public class ReplaceText extends AbstractProcessor {
|
|||
session.transfer(flowFile, REL_FAILURE);
|
||||
return;
|
||||
}
|
||||
|
||||
logger.info("Transferred {} to 'success'", new Object[] {flowFile});
|
||||
session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||
session.transfer(flowFile, REL_SUCCESS);
|
||||
|
|
Loading…
Reference in New Issue