mirror of
https://github.com/apache/nifi.git
synced 2025-02-10 03:55:22 +00:00
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 <mattyb149@apache.org> This closes #2767
This commit is contained in:
parent
568959baf5
commit
d4d9259664
@ -294,8 +294,18 @@ public class ReplaceText extends AbstractProcessor {
|
|||||||
|
|
||||||
final StopWatch stopWatch = new StopWatch(true);
|
final StopWatch stopWatch = new StopWatch(true);
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
flowFile = replacementStrategyExecutor.replace(flowFile, session, context, evaluateMode, charset, maxBufferSize);
|
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});
|
logger.info("Transferred {} to 'success'", new Object[] {flowFile});
|
||||||
session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
session.getProvenanceReporter().modifyContent(flowFile, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||||
session.transfer(flowFile, REL_SUCCESS);
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
|
@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets;
|
|||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.nio.file.Paths;
|
import java.nio.file.Paths;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
@ -1230,6 +1231,30 @@ public class TestReplaceText {
|
|||||||
out.assertContentEquals("WO$1R$2D");
|
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 {
|
private String translateNewLines(final File file) throws IOException {
|
||||||
return translateNewLines(file.toPath());
|
return translateNewLines(file.toPath());
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user