Skip to content

Commit

Permalink
NIFI-2752 Correct ReplaceText default pattern and unit tests
Browse files Browse the repository at this point in the history
 * Corrected the DEFAULT_REGEX pattern.
 * Added tests to isolate regex capture group problem and verify corrected functionality.
 * Removed short circuit logic that masked configuration errors and created inconsistent processor behavior.

This closes apache#1007.
  • Loading branch information
jskora authored and pvillard31 committed Sep 13, 2016
1 parent 02ca2a0 commit c72a9aa
Show file tree
Hide file tree
Showing 2 changed files with 60 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ public class ReplaceText extends AbstractProcessor {
public static final String literalReplaceValue = "Literal Replace";
public static final String alwaysReplace = "Always Replace";
private static final Pattern backReferencePattern = Pattern.compile("\\$(\\d+)");
private static final String DEFAULT_REGEX = "(?s:^.*$)";
private static final String DEFAULT_REGEX = "(?s)(^.*$)";
private static final String DEFAULT_REPLACEMENT_VALUE = "$1";

// Prepend and Append will just insert the replacement value at the beginning or end
Expand Down Expand Up @@ -214,13 +214,6 @@ public void onTrigger(final ProcessContext context, final ProcessSession session
String unsubstitutedReplacement = context.getProperty(REPLACEMENT_VALUE).getValue();
final String replacementStrategy = context.getProperty(REPLACEMENT_STRATEGY).getValue();

if (replacementStrategy.equalsIgnoreCase(regexReplaceValue) && unsubstitutedRegex.equals(DEFAULT_REGEX) && unsubstitutedReplacement.equals(DEFAULT_REPLACEMENT_VALUE)) {
// This pattern says replace content with itself. We can highly optimize this process by simply transferring
// all FlowFiles to the 'success' relationship
session.transfer(flowFiles, REL_SUCCESS);
return;
}

final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
final int maxBufferSize = context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).intValue();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,10 +30,15 @@
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;

public class TestReplaceText {

@Rule
public ExpectedException exception = ExpectedException.none();

@Test
public void testConfigurationCornerCase() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
Expand Down Expand Up @@ -64,7 +69,7 @@ public void testSimple() throws IOException {
}

@Test
public void testWithEscaped$InReplacemenmt() throws IOException {
public void testWithEscaped$InReplacement() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
runner.setValidateExpressionUsage(false);
runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
Expand Down Expand Up @@ -1106,7 +1111,60 @@ public void testAlwaysReplaceLineByLine() {
out.assertContentEquals("abc.txt\nabc.txt\r\nabc.txt\n");
}

@Test
public void testRegexWithBadCaptureGroup() throws IOException {
// Test the old Default Regex and with a custom Replacement Value that should fail because the
// Perl regex "(?s:^.*$)" must be written "(?s)(^.*$)" in Java for there to be a capture group.
// private static final String DEFAULT_REGEX = "(?s:^.*$)";
final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
runner.setValidateExpressionUsage(false);
runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${'$1':toUpper()}"); // should uppercase group but there is none
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);

runner.enqueue("testing\n123".getBytes());
runner.run();

runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
out.assertContentEquals("");
}

@Test
public void testRegexWithGoodCaptureGroup() throws IOException {
// Test the new Default Regex and with a custom Replacement Values that should succeed.
final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
runner.setValidateExpressionUsage(false);
runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^.*$)");
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "${'$1':toUpper()}"); // will uppercase group with good Java regex
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);

runner.enqueue("testing\n123".getBytes());
runner.run();

runner.assertAllFlowFilesTransferred(ReplaceText.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ReplaceText.REL_SUCCESS).get(0);
out.assertContentEquals("TESTING\n123");
}

@Test
public void testRegexNoCaptureDefaultReplacement() throws IOException {
// Test the old Default Regex and new Default Regex with the default replacement. This should fail
// because the regex does not create a capture group.
final TestRunner runner = TestRunners.newTestRunner(new ReplaceText());
runner.setValidateExpressionUsage(false);
runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s:^.*$)");
runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "$1");
runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE);
runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT);

exception.expect(AssertionError.class);
exception.expectMessage("java.lang.IndexOutOfBoundsException: No group 1");
runner.enqueue("testing\n123".getBytes());
runner.run();
}

private String translateNewLines(final File file) throws IOException {
return translateNewLines(file.toPath());
Expand Down

0 comments on commit c72a9aa

Please sign in to comment.