mirror of https://github.com/apache/nifi.git
NIFI-4080: Added EL support to fields in ValidateCSV
This commit is contained in:
parent
270892f926
commit
836c324258
|
@ -120,6 +120,7 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
+ "processors to apply. The following cell processors are allowed in the schema definition: "
|
||||
+ allowedOperators.toString() + ". Note: cell processors cannot be nested except with Optional.")
|
||||
.required(true)
|
||||
.expressionLanguageSupported(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
|
@ -139,6 +140,7 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
.description("Character used as 'quote' in the incoming data. Example: \"")
|
||||
.required(true)
|
||||
.defaultValue("\"")
|
||||
.expressionLanguageSupported(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
|
@ -148,6 +150,7 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
.description("Character used as 'delimiter' in the incoming data. Example: ,")
|
||||
.required(true)
|
||||
.defaultValue(",")
|
||||
.expressionLanguageSupported(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
|
@ -157,6 +160,7 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
.description("Symbols used as 'end of line' in the incoming data. Example: \\n")
|
||||
.required(true)
|
||||
.defaultValue("\\n")
|
||||
.expressionLanguageSupported(true)
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.build();
|
||||
|
||||
|
@ -213,9 +217,9 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
|
||||
@Override
|
||||
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
|
||||
String schema = validationContext.getProperty(SCHEMA).getValue();
|
||||
String schema = validationContext.getProperty(SCHEMA).evaluateAttributeExpressions().getValue();
|
||||
try {
|
||||
this.parseSchema(validationContext.getProperty(SCHEMA).getValue());
|
||||
this.parseSchema(schema);
|
||||
} catch (Exception e) {
|
||||
final List<ValidationResult> problems = new ArrayList<>(1);
|
||||
problems.add(new ValidationResult.Builder().subject(SCHEMA.getName())
|
||||
|
@ -234,9 +238,9 @@ public class ValidateCsv extends AbstractProcessor {
|
|||
// input is transferred over to Java as is. So when you type the characters "\"
|
||||
// and "n" into the UI the Java string will end up being those two characters
|
||||
// not the interpreted value "\n".
|
||||
final String msgDemarcator = context.getProperty(END_OF_LINE_CHARACTER).getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||
this.preference.set(new CsvPreference.Builder(context.getProperty(QUOTE_CHARACTER).getValue().charAt(0),
|
||||
context.getProperty(DELIMITER_CHARACTER).getValue().charAt(0), msgDemarcator).build());
|
||||
final String msgDemarcator = context.getProperty(END_OF_LINE_CHARACTER).evaluateAttributeExpressions().getValue().replace("\\n", "\n").replace("\\r", "\r").replace("\\t", "\t");
|
||||
this.preference.set(new CsvPreference.Builder(context.getProperty(QUOTE_CHARACTER).evaluateAttributeExpressions().getValue().charAt(0),
|
||||
context.getProperty(DELIMITER_CHARACTER).evaluateAttributeExpressions().getValue().charAt(0), msgDemarcator).build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -275,6 +275,37 @@ public class TestValidateCsv {
|
|||
runner.assertNotValid();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testValidateWithEL() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ValidateCsv());
|
||||
runner.setProperty(ValidateCsv.DELIMITER_CHARACTER, "${comma}");
|
||||
runner.setProperty(ValidateCsv.END_OF_LINE_CHARACTER, "${crlf}");
|
||||
runner.setProperty(ValidateCsv.QUOTE_CHARACTER, "${quote}");
|
||||
runner.setProperty(ValidateCsv.HEADER, "false");
|
||||
|
||||
runner.setProperty(ValidateCsv.SCHEMA, "RequireSubString(\"test\")");
|
||||
runner.assertNotValid();
|
||||
|
||||
runner.setProperty(ValidateCsv.SCHEMA, "''");
|
||||
runner.assertNotValid();
|
||||
|
||||
runner.setProperty(ValidateCsv.SCHEMA, "\"\"");
|
||||
runner.assertNotValid();
|
||||
|
||||
runner.setProperty(ValidateCsv.SCHEMA, "${schema}");
|
||||
runner.assertValid();
|
||||
|
||||
int hashcode = "test".hashCode();
|
||||
runner.setVariable("schema", "RequireHashCode(" + hashcode + "), RequireSubStr(\"test\")");
|
||||
runner.setVariable("comma", ",");
|
||||
runner.setVariable("quote", "\"");
|
||||
runner.setVariable("crlf", "\r\n");
|
||||
|
||||
runner.enqueue("test,test");
|
||||
runner.run();
|
||||
runner.assertAllFlowFilesTransferred(ValidateCsv.REL_VALID, 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseSchemaCommaBoundary() {
|
||||
final TestRunner runner = TestRunners.newTestRunner(new ValidateCsv());
|
||||
|
|
Loading…
Reference in New Issue