mirror of https://github.com/apache/nifi.git
NIFI-8042: Fixed bug that was escaping Expression Language references for use in a Regular Expression (i.e., Pattern.quote) even though it wasn't being used in a Regular Expression
This closes #4685 Signed-off-by: Mike Thomsen <mthomsen@apache.org>
This commit is contained in:
parent
d84583690f
commit
29ea872f2c
|
@ -602,8 +602,7 @@ public class ReplaceText extends AbstractProcessor {
|
|||
@Override
|
||||
public FlowFile replace(FlowFile flowFile, final ProcessSession session, final ProcessContext context, final String evaluateMode, final Charset charset, final int maxBufferSize) {
|
||||
final String replacementValue = context.getProperty(REPLACEMENT_VALUE).evaluateAttributeExpressions(flowFile).getValue();
|
||||
final AttributeValueDecorator quotedAttributeDecorator = Pattern::quote;
|
||||
final String searchValue = context.getProperty(SEARCH_VALUE).evaluateAttributeExpressions(flowFile, quotedAttributeDecorator).getValue();
|
||||
final String searchValue = context.getProperty(SEARCH_VALUE).evaluateAttributeExpressions(flowFile).getValue();
|
||||
|
||||
if (evaluateMode.equalsIgnoreCase(ENTIRE_TEXT)) {
|
||||
final int flowFileSize = (int) flowFile.getSize();
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.nio.file.Files;
|
|||
import java.nio.file.Path;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -53,6 +54,66 @@ public class TestReplaceText {
|
|||
return runner;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLiteralReplaceWithExpressionLanguageInSearchEntireText() {
|
||||
final TestRunner runner = getRunner();
|
||||
runner.enqueue("Me, you, and the other", Collections.singletonMap("search.value", "you, and"));
|
||||
runner.setProperty(ReplaceText.SEARCH_VALUE, "Me, ${search.value}");
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\"Replacement\"");
|
||||
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE.getValue());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
|
||||
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
|
||||
out.assertContentEquals("\"Replacement\" the other");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLiteralReplaceWithExpressionLanguageInReplacementEntireText() {
|
||||
final TestRunner runner = getRunner();
|
||||
runner.enqueue("Me, you, and the other", Collections.singletonMap("replacement.value", "us"));
|
||||
runner.setProperty(ReplaceText.SEARCH_VALUE, "Me, you,");
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "With ${replacement.value}");
|
||||
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE.getValue());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
|
||||
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
|
||||
out.assertContentEquals("With us and the other");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLiteralReplaceWithExpressionLanguageInSearchLineByLine() {
|
||||
final TestRunner runner = getRunner();
|
||||
runner.enqueue("Me, you, and the other", Collections.singletonMap("search.value", "you, and"));
|
||||
runner.setProperty(ReplaceText.SEARCH_VALUE, "Me, ${search.value}");
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "\"Replacement\"");
|
||||
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE.getValue());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
|
||||
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
|
||||
out.assertContentEquals("\"Replacement\" the other");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLiteralReplaceWithExpressionLanguageInReplacementLineByLine() {
|
||||
final TestRunner runner = getRunner();
|
||||
runner.enqueue("Me, you, and the other", Collections.singletonMap("replacement.value", "us"));
|
||||
runner.setProperty(ReplaceText.SEARCH_VALUE, "Me, you,");
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "With ${replacement.value}");
|
||||
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.LINE_BY_LINE);
|
||||
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.LITERAL_REPLACE.getValue());
|
||||
runner.run();
|
||||
|
||||
runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
|
||||
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
|
||||
out.assertContentEquals("With us and the other");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConfigurationCornerCase() throws IOException {
|
||||
final TestRunner runner = getRunner();
|
||||
|
|
Loading…
Reference in New Issue