mirror of https://github.com/apache/nifi.git
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
This commit is contained in:
commit
93b361e69b
|
@ -287,6 +287,28 @@ public class StandardValidators {
|
||||||
return createAttributeExpressionLanguageValidator(expectedResultType, true);
|
return createAttributeExpressionLanguageValidator(expectedResultType, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Validator createDataSizeBoundsValidator(final long minBytesInclusive, final long maxBytesInclusive) {
|
||||||
|
return new Validator() {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
|
||||||
|
final ValidationResult vr = DATA_SIZE_VALIDATOR.validate(subject, input, context);
|
||||||
|
if(!vr.isValid()){
|
||||||
|
return vr;
|
||||||
|
}
|
||||||
|
final long dataSizeBytes = DataUnit.parseDataSize(input, DataUnit.B).longValue();
|
||||||
|
if(dataSizeBytes < minBytesInclusive){
|
||||||
|
return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Cannot be smaller than " + minBytesInclusive + " bytes").build();
|
||||||
|
}
|
||||||
|
if(dataSizeBytes > maxBytesInclusive){
|
||||||
|
return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Cannot be larger than " + maxBytesInclusive + " bytes").build();
|
||||||
|
}
|
||||||
|
return new ValidationResult.Builder().subject(subject).input(input).valid(true).build();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
public static Validator createRegexMatchingValidator(final Pattern pattern) {
|
public static Validator createRegexMatchingValidator(final Pattern pattern) {
|
||||||
return new Validator() {
|
return new Validator() {
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -51,4 +51,35 @@ public class TestStandardValidators {
|
||||||
vr = val.validate("TimePeriodTest", "1 sec", null);
|
vr = val.validate("TimePeriodTest", "1 sec", null);
|
||||||
assertTrue(vr.isValid());
|
assertTrue(vr.isValid());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDataSizeBoundsValidator() {
|
||||||
|
Validator val = StandardValidators.createDataSizeBoundsValidator(100, 1000);
|
||||||
|
ValidationResult vr;
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "5 GB", null);
|
||||||
|
assertFalse(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "0 B", null);
|
||||||
|
assertFalse(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "99 B", null);
|
||||||
|
assertFalse(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "100 B", null);
|
||||||
|
assertTrue(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "999 B", null);
|
||||||
|
assertTrue(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "1000 B", null);
|
||||||
|
assertTrue(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "1001 B", null);
|
||||||
|
assertFalse(vr.isValid());
|
||||||
|
|
||||||
|
vr = val.validate("DataSizeBounds", "water", null);
|
||||||
|
assertFalse(vr.isValid());
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,9 +52,10 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
@EventDriven
|
@EventDriven
|
||||||
@SideEffectFree
|
@SideEffectFree
|
||||||
@SupportsBatching
|
@SupportsBatching
|
||||||
@Tags({"evaluate", "Text", "Regular Expression", "regex", "experimental"})
|
@Tags({"deprecated"})
|
||||||
@CapabilityDescription(
|
@CapabilityDescription(
|
||||||
"Evaluates one or more Regular Expressions against the content of a FlowFile. "
|
"WARNING: This has been deprecated and will be removed in 0.2.0. Use ExtractText instead.\n"
|
||||||
|
+ "Evaluates one or more Regular Expressions against the content of a FlowFile. "
|
||||||
+ "The results of those Regular Expressions are assigned to FlowFile Attributes. "
|
+ "The results of those Regular Expressions are assigned to FlowFile Attributes. "
|
||||||
+ "Regular Expressions are entered by adding user-defined properties; "
|
+ "Regular Expressions are entered by adding user-defined properties; "
|
||||||
+ "the name of the property maps to the Attribute Name into which the result will be placed. "
|
+ "the name of the property maps to the Attribute Name into which the result will be placed. "
|
||||||
|
@ -62,7 +63,7 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
+ "If the Regular Expression matches more than once, only the first match will be used. "
|
+ "If the Regular Expression matches more than once, only the first match will be used. "
|
||||||
+ "If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. "
|
+ "If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. "
|
||||||
+ "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' and no attributes will be applied to the FlowFile.")
|
+ "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' and no attributes will be applied to the FlowFile.")
|
||||||
|
@Deprecated
|
||||||
public class EvaluateRegularExpression extends AbstractProcessor {
|
public class EvaluateRegularExpression extends AbstractProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
|
||||||
|
|
|
@ -0,0 +1,313 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.nifi.processors.standard;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.nio.charset.Charset;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.regex.Matcher;
|
||||||
|
import java.util.regex.Pattern;
|
||||||
|
|
||||||
|
import org.apache.nifi.components.PropertyDescriptor;
|
||||||
|
import org.apache.nifi.flowfile.FlowFile;
|
||||||
|
import org.apache.nifi.stream.io.StreamUtils;
|
||||||
|
import org.apache.nifi.logging.ProcessorLog;
|
||||||
|
import org.apache.nifi.processor.AbstractProcessor;
|
||||||
|
import org.apache.nifi.processor.DataUnit;
|
||||||
|
import org.apache.nifi.processor.ProcessContext;
|
||||||
|
import org.apache.nifi.processor.ProcessSession;
|
||||||
|
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||||
|
import org.apache.nifi.processor.Relationship;
|
||||||
|
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||||
|
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||||
|
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||||
|
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||||
|
import org.apache.nifi.annotation.documentation.Tags;
|
||||||
|
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||||
|
import org.apache.nifi.processor.util.StandardValidators;
|
||||||
|
|
||||||
|
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||||
|
|
||||||
|
@EventDriven
|
||||||
|
@SideEffectFree
|
||||||
|
@SupportsBatching
|
||||||
|
@Tags({"evaluate", "extract", "Text", "Regular Expression", "regex"})
|
||||||
|
@CapabilityDescription(
|
||||||
|
"Evaluates one or more Regular Expressions against the content of a FlowFile. "
|
||||||
|
+ "The results of those Regular Expressions are assigned to FlowFile Attributes. "
|
||||||
|
+ "Regular Expressions are entered by adding user-defined properties; "
|
||||||
|
+ "the name of the property maps to the Attribute Name into which the result will be placed. "
|
||||||
|
+ "The first capture group, if any found, will be placed into that attribute name."
|
||||||
|
+ "But all catpure groups, including the matching string sequence itself will also be "
|
||||||
|
+ "provided at that attribute name with an index value provided."
|
||||||
|
+ "The value of the property must be a valid Regular Expressions with one or more capturing groups. "
|
||||||
|
+ "If the Regular Expression matches more than once, only the first match will be used. "
|
||||||
|
+ "If any provided Regular Expression matches, the FlowFile(s) will be routed to 'matched'. "
|
||||||
|
+ "If no provided Regular Expression matches, the FlowFile will be routed to 'unmatched' "
|
||||||
|
+ "and no attributes will be applied to the FlowFile.")
|
||||||
|
|
||||||
|
public class ExtractText extends AbstractProcessor {
|
||||||
|
|
||||||
|
public static final PropertyDescriptor CHARACTER_SET = new PropertyDescriptor.Builder()
|
||||||
|
.name("Character Set")
|
||||||
|
.description("The Character Set in which the file is encoded")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
|
||||||
|
.defaultValue("UTF-8")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Maximum Buffer Size")
|
||||||
|
.description("Specifies the maximum amount of data to buffer (per file) in order to apply the regular expressions. Files larger than the specified maximum will not be fully evaluated.")
|
||||||
|
.required(true)
|
||||||
|
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||||
|
.addValidator(StandardValidators.createDataSizeBoundsValidator(0, Integer.MAX_VALUE))
|
||||||
|
.defaultValue("1 MB")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor MAX_CAPTURE_GROUP_LENGTH = new PropertyDescriptor.Builder()
|
||||||
|
.name("Maximum Capture Group Length")
|
||||||
|
.description("Specifies the maximum number of characters a given capture group value can have. Any characters beyond the max will be truncated.")
|
||||||
|
.required(false)
|
||||||
|
.defaultValue("1024")
|
||||||
|
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor CANON_EQ = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Canonical Equivalence")
|
||||||
|
.description("Indicates that two characters match only when their full canonical decompositions match.")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor CASE_INSENSITIVE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Case-insensitive Matching")
|
||||||
|
.description("Indicates that two characters match even if they are in a different case. Can also be specified via the embeded flag (?i).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor COMMENTS = new PropertyDescriptor.Builder()
|
||||||
|
.name("Permit Whitespace and Comments in Pattern")
|
||||||
|
.description("In this mode, whitespace is ignored, and embedded comments starting with # are ignored until the end of a line. Can also be specified via the embeded flag (?x).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor DOTALL = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable DOTALL Mode")
|
||||||
|
.description("Indicates that the expression '.' should match any character, including a line terminator. Can also be specified via the embeded flag (?s).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor LITERAL = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Literal Parsing of the Pattern")
|
||||||
|
.description("Indicates that Metacharacters and escape characters should be given no special meaning.")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor MULTILINE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Multiline Mode")
|
||||||
|
.description("Indicates that '^' and '$' should match just after and just before a line terminator or end of sequence, instead of only the begining or end of the entire input. Can also be specified via the embeded flag (?m).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor UNICODE_CASE = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Unicode-aware Case Folding")
|
||||||
|
.description("When used with 'Enable Case-insensitive Matching', matches in a manner consistent with the Unicode Standard. Can also be specified via the embeded flag (?u).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor UNICODE_CHARACTER_CLASS = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Unicode Predefined Character Classes")
|
||||||
|
.description("Specifies conformance with the Unicode Technical Standard #18: Unicode Regular Expression Annex C: Compatibility Properties. Can also be specified via the embeded flag (?U).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final PropertyDescriptor UNIX_LINES = new PropertyDescriptor.Builder()
|
||||||
|
.name("Enable Unix Lines Mode")
|
||||||
|
.description("Indicates that only the '\n' line terminator is recognized int the behavior of '.', '^', and '$'. Can also be specified via the embeded flag (?d).")
|
||||||
|
.required(true)
|
||||||
|
.allowableValues("true", "false")
|
||||||
|
.defaultValue("false")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final Relationship REL_MATCH = new Relationship.Builder()
|
||||||
|
.name("matched")
|
||||||
|
.description(
|
||||||
|
"FlowFiles are routed to this relationship when the Regular Expression is successfully evaluated and the FlowFile "
|
||||||
|
+ "is modified as a result")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
public static final Relationship REL_NO_MATCH = new Relationship.Builder()
|
||||||
|
.name("unmatched")
|
||||||
|
.description(
|
||||||
|
"FlowFiles are routed to this relationship when no provided Regular Expression matches the content of the FlowFile")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
private Set<Relationship> relationships;
|
||||||
|
private List<PropertyDescriptor> properties;
|
||||||
|
private final AtomicReference<Map<String, Pattern>> compiledPattersMapRef = new AtomicReference<>();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void init(final ProcessorInitializationContext context) {
|
||||||
|
final Set<Relationship> rels = new HashSet<>();
|
||||||
|
rels.add(REL_MATCH);
|
||||||
|
rels.add(REL_NO_MATCH);
|
||||||
|
this.relationships = Collections.unmodifiableSet(rels);
|
||||||
|
|
||||||
|
final List<PropertyDescriptor> props = new ArrayList<>();
|
||||||
|
props.add(CHARACTER_SET);
|
||||||
|
props.add(MAX_BUFFER_SIZE);
|
||||||
|
props.add(MAX_CAPTURE_GROUP_LENGTH);
|
||||||
|
props.add(CANON_EQ);
|
||||||
|
props.add(CASE_INSENSITIVE);
|
||||||
|
props.add(COMMENTS);
|
||||||
|
props.add(DOTALL);
|
||||||
|
props.add(LITERAL);
|
||||||
|
props.add(MULTILINE);
|
||||||
|
props.add(UNICODE_CASE);
|
||||||
|
props.add(UNICODE_CHARACTER_CLASS);
|
||||||
|
props.add(UNIX_LINES);
|
||||||
|
this.properties = Collections.unmodifiableList(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Set<Relationship> getRelationships() {
|
||||||
|
return relationships;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
|
return properties;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
||||||
|
return new PropertyDescriptor.Builder()
|
||||||
|
.name(propertyDescriptorName)
|
||||||
|
.expressionLanguageSupported(false)
|
||||||
|
.addValidator(StandardValidators.createRegexValidator(1, 40, true))
|
||||||
|
.required(false)
|
||||||
|
.dynamic(true)
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@OnScheduled
|
||||||
|
public final void onScheduled(final ProcessContext context) throws IOException {
|
||||||
|
final Map<String, Pattern> compiledPatternsMap = new HashMap<>();
|
||||||
|
|
||||||
|
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
|
||||||
|
if (!entry.getKey().isDynamic()) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
final int flags = getCompileFlags(context);
|
||||||
|
final Pattern pattern = Pattern.compile(entry.getValue(), flags);
|
||||||
|
compiledPatternsMap.put(entry.getKey().getName(), pattern);
|
||||||
|
}
|
||||||
|
compiledPattersMapRef.set(compiledPatternsMap);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
||||||
|
FlowFile flowFile = session.get();
|
||||||
|
if (flowFile == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
final ProcessorLog logger = getLogger();
|
||||||
|
final Charset charset = Charset.forName(context.getProperty(CHARACTER_SET).getValue());
|
||||||
|
final int maxCaptureGroupLength = context.getProperty(MAX_CAPTURE_GROUP_LENGTH).asInteger();
|
||||||
|
final long maxBufferSizeL = Math.min(flowFile.getSize(), context.getProperty(MAX_BUFFER_SIZE).asDataSize(DataUnit.B).longValue());
|
||||||
|
final byte[] buffer = new byte[(int) maxBufferSizeL];
|
||||||
|
|
||||||
|
session.read(flowFile, new InputStreamCallback() {
|
||||||
|
@Override
|
||||||
|
public void process(InputStream in) throws IOException {
|
||||||
|
StreamUtils.fillBuffer(in, buffer, false);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
final String contentString = new String(buffer, 0, (int) maxBufferSizeL, charset);
|
||||||
|
final Map<String, String> regexResults = new HashMap<>();
|
||||||
|
|
||||||
|
final Map<String, Pattern> patternMap = compiledPattersMapRef.get();
|
||||||
|
for (final Map.Entry<String, Pattern> entry : patternMap.entrySet()) {
|
||||||
|
|
||||||
|
final Matcher matcher = entry.getValue().matcher(contentString);
|
||||||
|
|
||||||
|
if (matcher.find()) {
|
||||||
|
final String baseKey = entry.getKey();
|
||||||
|
for (int i = 0; i <= matcher.groupCount(); i++) {
|
||||||
|
final String key = new StringBuilder(baseKey).append(".").append(i).toString();
|
||||||
|
String value = matcher.group(i);
|
||||||
|
if (value.length() > maxCaptureGroupLength) {
|
||||||
|
value = value.substring(0, maxCaptureGroupLength);
|
||||||
|
}
|
||||||
|
regexResults.put(key, value);
|
||||||
|
if (i == 1) {
|
||||||
|
regexResults.put(baseKey, value);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!regexResults.isEmpty()) {
|
||||||
|
flowFile = session.putAllAttributes(flowFile, regexResults);
|
||||||
|
session.getProvenanceReporter().modifyAttributes(flowFile);
|
||||||
|
session.transfer(flowFile, REL_MATCH);
|
||||||
|
logger.info("Matched {} Regular Expressions and added attributes to FlowFile {}", new Object[]{regexResults.size(), flowFile});
|
||||||
|
} else {
|
||||||
|
session.transfer(flowFile, REL_NO_MATCH);
|
||||||
|
logger.info("Did not match any Regular Expressions for FlowFile {}", new Object[]{flowFile});
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
int getCompileFlags(ProcessContext context) {
|
||||||
|
int flags = (context.getProperty(UNIX_LINES).asBoolean() ? Pattern.UNIX_LINES : 0)
|
||||||
|
| (context.getProperty(CASE_INSENSITIVE).asBoolean() ? Pattern.CASE_INSENSITIVE : 0)
|
||||||
|
| (context.getProperty(COMMENTS).asBoolean() ? Pattern.COMMENTS : 0)
|
||||||
|
| (context.getProperty(MULTILINE).asBoolean() ? Pattern.MULTILINE : 0)
|
||||||
|
| (context.getProperty(LITERAL).asBoolean() ? Pattern.LITERAL : 0)
|
||||||
|
| (context.getProperty(DOTALL).asBoolean() ? Pattern.DOTALL : 0)
|
||||||
|
| (context.getProperty(UNICODE_CASE).asBoolean() ? Pattern.UNICODE_CASE : 0)
|
||||||
|
| (context.getProperty(CANON_EQ).asBoolean() ? Pattern.CANON_EQ : 0)
|
||||||
|
| (context.getProperty(UNICODE_CHARACTER_CLASS).asBoolean() ? Pattern.UNICODE_CHARACTER_CLASS : 0);
|
||||||
|
return flags;
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,6 +25,7 @@ org.apache.nifi.processors.standard.EvaluateXPath
|
||||||
org.apache.nifi.processors.standard.EvaluateXQuery
|
org.apache.nifi.processors.standard.EvaluateXQuery
|
||||||
org.apache.nifi.processors.standard.ExecuteStreamCommand
|
org.apache.nifi.processors.standard.ExecuteStreamCommand
|
||||||
org.apache.nifi.processors.standard.ExecuteProcess
|
org.apache.nifi.processors.standard.ExecuteProcess
|
||||||
|
org.apache.nifi.processors.standard.ExtractText
|
||||||
org.apache.nifi.processors.standard.GenerateFlowFile
|
org.apache.nifi.processors.standard.GenerateFlowFile
|
||||||
org.apache.nifi.processors.standard.GetFile
|
org.apache.nifi.processors.standard.GetFile
|
||||||
org.apache.nifi.processors.standard.GetFTP
|
org.apache.nifi.processors.standard.GetFTP
|
||||||
|
|
|
@ -16,7 +16,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.processors.standard;
|
package org.apache.nifi.processors.standard;
|
||||||
|
|
||||||
import org.apache.nifi.processors.standard.EvaluateRegularExpression;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
@ -31,19 +30,19 @@ import org.apache.nifi.util.TestRunners;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class TestEvaluateRegularExpression {
|
public class TestExtractText {
|
||||||
|
|
||||||
final String SAMPLE_STRING = "foo\r\nbar1\r\nbar2\r\nbar3\r\nhello\r\nworld\r\n";
|
final String SAMPLE_STRING = "foo\r\nbar1\r\nbar2\r\nbar3\r\nhello\r\nworld\r\n";
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testProcessor() throws Exception {
|
public void testProcessor() throws Exception {
|
||||||
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty("regex.result1", "(?s)(.*)");
|
testRunner.setProperty("regex.result1", "(?s)(.*)");
|
||||||
testRunner.setProperty("regex.result2", "(?s).*(bar1).*");
|
testRunner.setProperty("regex.result2", "(?s).*(bar1).*");
|
||||||
testRunner.setProperty("regex.result3", "(?s).*?(bar\\d).*"); // reluctant gets first
|
testRunner.setProperty("regex.result3", "(?s).*?(bar\\d).*"); // reluctant gets first
|
||||||
testRunner.setProperty("regex.result4", "(?s).*?(?:bar\\d).*?(bar\\d).*"); // reluctant w/ repeated pattern gets second
|
testRunner.setProperty("regex.result4", "(?s).*?(?:bar\\d).*?(bar\\d).*?(bar3).*"); // reluctant w/ repeated pattern gets second
|
||||||
testRunner.setProperty("regex.result5", "(?s).*(bar\\d).*"); // greedy gets last
|
testRunner.setProperty("regex.result5", "(?s).*(bar\\d).*"); // greedy gets last
|
||||||
testRunner.setProperty("regex.result6", "(?s)^(.*)$");
|
testRunner.setProperty("regex.result6", "(?s)^(.*)$");
|
||||||
testRunner.setProperty("regex.result7", "(?s)(XXX)");
|
testRunner.setProperty("regex.result7", "(?s)(XXX)");
|
||||||
|
@ -51,12 +50,16 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
|
||||||
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
||||||
out.assertAttributeEquals("regex.result2", "bar1");
|
out.assertAttributeEquals("regex.result2", "bar1");
|
||||||
out.assertAttributeEquals("regex.result3", "bar1");
|
out.assertAttributeEquals("regex.result3", "bar1");
|
||||||
out.assertAttributeEquals("regex.result4", "bar2");
|
out.assertAttributeEquals("regex.result4", "bar2");
|
||||||
|
out.assertAttributeEquals("regex.result4.0", SAMPLE_STRING);
|
||||||
|
out.assertAttributeEquals("regex.result4.1", "bar2");
|
||||||
|
out.assertAttributeEquals("regex.result4.2", "bar3");
|
||||||
|
out.assertAttributeNotExists("regex.result4.3");
|
||||||
out.assertAttributeEquals("regex.result5", "bar3");
|
out.assertAttributeEquals("regex.result5", "bar3");
|
||||||
out.assertAttributeEquals("regex.result6", SAMPLE_STRING);
|
out.assertAttributeEquals("regex.result6", SAMPLE_STRING);
|
||||||
out.assertAttributeEquals("regex.result7", null);
|
out.assertAttributeEquals("regex.result7", null);
|
||||||
|
@ -65,9 +68,9 @@ public class TestEvaluateRegularExpression {
|
||||||
@Test
|
@Test
|
||||||
public void testProcessorWithDotall() throws Exception {
|
public void testProcessorWithDotall() throws Exception {
|
||||||
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true");
|
testRunner.setProperty(ExtractText.DOTALL, "true");
|
||||||
|
|
||||||
testRunner.setProperty("regex.result1", "(.*)");
|
testRunner.setProperty("regex.result1", "(.*)");
|
||||||
testRunner.setProperty("regex.result2", ".*(bar1).*");
|
testRunner.setProperty("regex.result2", ".*(bar1).*");
|
||||||
|
@ -80,8 +83,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
|
||||||
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
||||||
out.assertAttributeEquals("regex.result2", "bar1");
|
out.assertAttributeEquals("regex.result2", "bar1");
|
||||||
out.assertAttributeEquals("regex.result3", "bar1");
|
out.assertAttributeEquals("regex.result3", "bar1");
|
||||||
|
@ -95,9 +98,9 @@ public class TestEvaluateRegularExpression {
|
||||||
@Test
|
@Test
|
||||||
public void testProcessorWithMultiline() throws Exception {
|
public void testProcessorWithMultiline() throws Exception {
|
||||||
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true");
|
testRunner.setProperty(ExtractText.MULTILINE, "true");
|
||||||
|
|
||||||
testRunner.setProperty("regex.result1", "(.*)");
|
testRunner.setProperty("regex.result1", "(.*)");
|
||||||
testRunner.setProperty("regex.result2", "(bar1)");
|
testRunner.setProperty("regex.result2", "(bar1)");
|
||||||
|
@ -112,8 +115,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
|
||||||
out.assertAttributeEquals("regex.result1", "foo"); // matches everything on the first line
|
out.assertAttributeEquals("regex.result1", "foo"); // matches everything on the first line
|
||||||
out.assertAttributeEquals("regex.result2", "bar1");
|
out.assertAttributeEquals("regex.result2", "bar1");
|
||||||
out.assertAttributeEquals("regex.result3", "bar1");
|
out.assertAttributeEquals("regex.result3", "bar1");
|
||||||
|
@ -128,10 +131,10 @@ public class TestEvaluateRegularExpression {
|
||||||
@Test
|
@Test
|
||||||
public void testProcessorWithMultilineAndDotall() throws Exception {
|
public void testProcessorWithMultilineAndDotall() throws Exception {
|
||||||
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true");
|
testRunner.setProperty(ExtractText.MULTILINE, "true");
|
||||||
testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true");
|
testRunner.setProperty(ExtractText.DOTALL, "true");
|
||||||
|
|
||||||
testRunner.setProperty("regex.result1", "(.*)");
|
testRunner.setProperty("regex.result1", "(.*)");
|
||||||
testRunner.setProperty("regex.result2", "(bar1)");
|
testRunner.setProperty("regex.result2", "(bar1)");
|
||||||
|
@ -146,8 +149,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
|
||||||
|
|
||||||
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
out.assertAttributeEquals("regex.result1", SAMPLE_STRING);
|
||||||
out.assertAttributeEquals("regex.result2", "bar1");
|
out.assertAttributeEquals("regex.result2", "bar1");
|
||||||
|
@ -163,10 +166,10 @@ public class TestEvaluateRegularExpression {
|
||||||
@Test
|
@Test
|
||||||
public void testProcessorWithNoMatches() throws Exception {
|
public void testProcessorWithNoMatches() throws Exception {
|
||||||
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true");
|
testRunner.setProperty(ExtractText.MULTILINE, "true");
|
||||||
testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true");
|
testRunner.setProperty(ExtractText.DOTALL, "true");
|
||||||
|
|
||||||
testRunner.setProperty("regex.result2", "(bar1)");
|
testRunner.setProperty("regex.result2", "(bar1)");
|
||||||
testRunner.setProperty("regex.result3", ".*?(bar\\d).*");
|
testRunner.setProperty("regex.result3", ".*?(bar\\d).*");
|
||||||
|
@ -179,8 +182,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue("YYY".getBytes("UTF-8"));
|
testRunner.enqueue("YYY".getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_NO_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_NO_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_NO_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_NO_MATCH).get(0);
|
||||||
|
|
||||||
out.assertAttributeEquals("regex.result1", null);
|
out.assertAttributeEquals("regex.result1", null);
|
||||||
out.assertAttributeEquals("regex.result2", null);
|
out.assertAttributeEquals("regex.result2", null);
|
||||||
|
@ -195,7 +198,7 @@ public class TestEvaluateRegularExpression {
|
||||||
|
|
||||||
@Test(expected = java.lang.AssertionError.class)
|
@Test(expected = java.lang.AssertionError.class)
|
||||||
public void testNoCaptureGroups() throws UnsupportedEncodingException {
|
public void testNoCaptureGroups() throws UnsupportedEncodingException {
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
testRunner.setProperty("regex.result1", ".*");
|
testRunner.setProperty("regex.result1", ".*");
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
@ -203,25 +206,17 @@ public class TestEvaluateRegularExpression {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNoFlowFile() throws UnsupportedEncodingException {
|
public void testNoFlowFile() throws UnsupportedEncodingException {
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 0);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 0);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = java.lang.AssertionError.class)
|
|
||||||
public void testTooManyCaptureGroups() throws UnsupportedEncodingException {
|
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
|
||||||
testRunner.setProperty("regex.result1", "(.)(.)");
|
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
|
||||||
testRunner.run();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMatchOutsideBuffer() throws Exception {
|
public void testMatchOutsideBuffer() throws Exception {
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(new EvaluateRegularExpression());
|
final TestRunner testRunner = TestRunners.newTestRunner(new ExtractText());
|
||||||
|
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MAX_BUFFER_SIZE, "3 B");//only read the first 3 chars ("foo")
|
testRunner.setProperty(ExtractText.MAX_BUFFER_SIZE, "3 B");//only read the first 3 chars ("foo")
|
||||||
|
|
||||||
testRunner.setProperty("regex.result1", "(foo)");
|
testRunner.setProperty("regex.result1", "(foo)");
|
||||||
testRunner.setProperty("regex.result2", "(world)");
|
testRunner.setProperty("regex.result2", "(world)");
|
||||||
|
@ -229,8 +224,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
testRunner.enqueue(SAMPLE_STRING.getBytes("UTF-8"));
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
testRunner.assertAllFlowFilesTransferred(EvaluateRegularExpression.REL_MATCH, 1);
|
testRunner.assertAllFlowFilesTransferred(ExtractText.REL_MATCH, 1);
|
||||||
final MockFlowFile out = testRunner.getFlowFilesForRelationship(EvaluateRegularExpression.REL_MATCH).get(0);
|
final MockFlowFile out = testRunner.getFlowFilesForRelationship(ExtractText.REL_MATCH).get(0);
|
||||||
|
|
||||||
out.assertAttributeEquals("regex.result1", "foo");
|
out.assertAttributeEquals("regex.result1", "foo");
|
||||||
out.assertAttributeEquals("regex.result2", null); // null because outsk
|
out.assertAttributeEquals("regex.result2", null); // null because outsk
|
||||||
|
@ -239,7 +234,7 @@ public class TestEvaluateRegularExpression {
|
||||||
@Test
|
@Test
|
||||||
public void testGetCompileFlags() {
|
public void testGetCompileFlags() {
|
||||||
|
|
||||||
final EvaluateRegularExpression processor = new EvaluateRegularExpression();
|
final ExtractText processor = new ExtractText();
|
||||||
TestRunner testRunner;
|
TestRunner testRunner;
|
||||||
int flags;
|
int flags;
|
||||||
|
|
||||||
|
@ -250,60 +245,60 @@ public class TestEvaluateRegularExpression {
|
||||||
|
|
||||||
// UNIX_LINES
|
// UNIX_LINES
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.UNIX_LINES, "true");
|
testRunner.setProperty(ExtractText.UNIX_LINES, "true");
|
||||||
assertEquals(Pattern.UNIX_LINES, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.UNIX_LINES, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// CASE_INSENSITIVE
|
// CASE_INSENSITIVE
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.CASE_INSENSITIVE, "true");
|
testRunner.setProperty(ExtractText.CASE_INSENSITIVE, "true");
|
||||||
assertEquals(Pattern.CASE_INSENSITIVE, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.CASE_INSENSITIVE, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// COMMENTS
|
// COMMENTS
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.COMMENTS, "true");
|
testRunner.setProperty(ExtractText.COMMENTS, "true");
|
||||||
assertEquals(Pattern.COMMENTS, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.COMMENTS, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// MULTILINE
|
// MULTILINE
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true");
|
testRunner.setProperty(ExtractText.MULTILINE, "true");
|
||||||
assertEquals(Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// LITERAL
|
// LITERAL
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.LITERAL, "true");
|
testRunner.setProperty(ExtractText.LITERAL, "true");
|
||||||
assertEquals(Pattern.LITERAL, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.LITERAL, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// DOTALL
|
// DOTALL
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true");
|
testRunner.setProperty(ExtractText.DOTALL, "true");
|
||||||
assertEquals(Pattern.DOTALL, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.DOTALL, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// UNICODE_CASE
|
// UNICODE_CASE
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.UNICODE_CASE, "true");
|
testRunner.setProperty(ExtractText.UNICODE_CASE, "true");
|
||||||
assertEquals(Pattern.UNICODE_CASE, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.UNICODE_CASE, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// CANON_EQ
|
// CANON_EQ
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.CANON_EQ, "true");
|
testRunner.setProperty(ExtractText.CANON_EQ, "true");
|
||||||
assertEquals(Pattern.CANON_EQ, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.CANON_EQ, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// UNICODE_CHARACTER_CLASS
|
// UNICODE_CHARACTER_CLASS
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.UNICODE_CHARACTER_CLASS, "true");
|
testRunner.setProperty(ExtractText.UNICODE_CHARACTER_CLASS, "true");
|
||||||
assertEquals(Pattern.UNICODE_CHARACTER_CLASS, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.UNICODE_CHARACTER_CLASS, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
|
|
||||||
// DOTALL and MULTILINE
|
// DOTALL and MULTILINE
|
||||||
testRunner = TestRunners.newTestRunner(processor);
|
testRunner = TestRunners.newTestRunner(processor);
|
||||||
testRunner.setProperty(EvaluateRegularExpression.DOTALL, "true");
|
testRunner.setProperty(ExtractText.DOTALL, "true");
|
||||||
testRunner.setProperty(EvaluateRegularExpression.MULTILINE, "true");
|
testRunner.setProperty(ExtractText.MULTILINE, "true");
|
||||||
assertEquals(Pattern.DOTALL | Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext()));
|
assertEquals(Pattern.DOTALL | Pattern.MULTILINE, processor.getCompileFlags(testRunner.getProcessContext()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGetRelationShips() throws Exception {
|
public void testGetRelationShips() throws Exception {
|
||||||
|
|
||||||
final EvaluateRegularExpression processor = new EvaluateRegularExpression();
|
final ExtractText processor = new ExtractText();
|
||||||
final TestRunner testRunner = TestRunners.newTestRunner(processor);
|
final TestRunner testRunner = TestRunners.newTestRunner(processor);
|
||||||
|
|
||||||
// testRunner.setProperty("regex.result1", "(.*)");
|
// testRunner.setProperty("regex.result1", "(.*)");
|
||||||
|
@ -311,8 +306,8 @@ public class TestEvaluateRegularExpression {
|
||||||
testRunner.run();
|
testRunner.run();
|
||||||
|
|
||||||
Set<Relationship> relationships = processor.getRelationships();
|
Set<Relationship> relationships = processor.getRelationships();
|
||||||
assertTrue(relationships.contains(EvaluateRegularExpression.REL_MATCH));
|
assertTrue(relationships.contains(ExtractText.REL_MATCH));
|
||||||
assertTrue(relationships.contains(EvaluateRegularExpression.REL_NO_MATCH));
|
assertTrue(relationships.contains(ExtractText.REL_NO_MATCH));
|
||||||
assertEquals(2, relationships.size());
|
assertEquals(2, relationships.size());
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue