From d4d925966416a449ccb1aac0296e2eed7e8b43e1 Mon Sep 17 00:00:00 2001 From: Mike Moser Date: Wed, 6 Jun 2018 18:25:32 +0000 Subject: [PATCH] NIFI-5274 avoid rollback on uncaught errors in ReplaceText forgot to penalize flowfiles sent to failure NIFI-5274 avoid rollback on StackOverflowError in ReplaceText add comment removed penalize on failure, doesn't really apply here Signed-off-by: Matthew Burgess This closes #2767 --- .../nifi/processors/standard/ReplaceText.java | 12 ++++++++- .../processors/standard/TestReplaceText.java | 25 +++++++++++++++++++ 2 files changed, 36 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java index f303796e45..e424108ddc 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java @@ -294,7 +294,17 @@ public class ReplaceText extends AbstractProcessor { final StopWatch stopWatch = new StopWatch(true); - flowFile = replacementStrategyExecutor.replace(flowFile, session, context, evaluateMode, charset, maxBufferSize); + try { + + flowFile = replacementStrategyExecutor.replace(flowFile, session, context, evaluateMode, charset, maxBufferSize); + + } catch (StackOverflowError e) { + // Some regular expressions can produce many matches on large input data size using recursive code + // do not log the StackOverflowError stack trace + logger.info("Transferred {} to 'failure' due to {}", new Object[] {flowFile, e.toString()}); + session.transfer(flowFile, REL_FAILURE); + return; + } logger.info("Transferred {} to 'success'", new Object[] {flowFile}); session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS)); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java index 7505233447..3c432e5b12 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestReplaceText.java @@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.regex.Pattern; @@ -1230,6 +1231,30 @@ public class TestReplaceText { out.assertContentEquals("WO$1R$2D"); } + /* + * A repeated alternation regex such as (A|B)* can lead to StackOverflowError + * on large input strings. + */ + @Test + public void testForStackOverflow() throws Exception { + final TestRunner runner = getRunner(); + runner.setProperty(ReplaceText.REPLACEMENT_VALUE, "New text"); + runner.setProperty(ReplaceText.REPLACEMENT_STRATEGY, ReplaceText.REGEX_REPLACE); + runner.setProperty(ReplaceText.EVALUATION_MODE, ReplaceText.ENTIRE_TEXT); + runner.setProperty(ReplaceText.MAX_BUFFER_SIZE, "10 MB"); + runner.setProperty(ReplaceText.SEARCH_VALUE, "(?s)(^(A|B)*$)"); + runner.assertValid(); + + char[] data = new char[1_000_000]; + Arrays.fill(data, 'A'); + runner.enqueue(new String(data)); + + runner.run(); + + // we want the large file to fail, rather than rollback and yield + runner.assertAllFlowFilesTransferred(ReplaceText.REL_FAILURE, 1); + } + private String translateNewLines(final File file) throws IOException { return translateNewLines(file.toPath()); }