mirror of https://github.com/apache/nifi.git
NIFI-12121 Added Transform Result Filter to JSLTTransformJSON
This closes #7922 Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
61bb01560d
commit
832d4455c1
|
@ -62,6 +62,7 @@ import java.io.BufferedReader;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.StringReader;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -87,6 +88,8 @@ import static org.apache.nifi.processors.jslt.JSLTTransformJSON.TransformationSt
|
|||
+ "fails, the original FlowFile is routed to the 'failure' relationship.")
|
||||
public class JSLTTransformJSON extends AbstractProcessor {
|
||||
|
||||
public static String JSLT_FILTER_DEFAULT = ". != null and . != {} and . != []";
|
||||
|
||||
public static final PropertyDescriptor JSLT_TRANSFORM = new PropertyDescriptor.Builder()
|
||||
.name("jslt-transform-transformation")
|
||||
.displayName("JSLT Transformation")
|
||||
|
@ -127,6 +130,18 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
.required(true)
|
||||
.build();
|
||||
|
||||
public static final PropertyDescriptor RESULT_FILTER = new PropertyDescriptor.Builder()
|
||||
.name("jslt-transform-result-filter")
|
||||
.displayName("Transform Result Filter")
|
||||
.description("A filter for output JSON results using a JSLT expression. This property supports changing the default filter,"
|
||||
+ " which removes JSON objects with null values, empty objects and empty arrays from the output JSON."
|
||||
+ " This JSLT must return true for each JSON object to be included and false for each object to be removed."
|
||||
+ " Using a filter value of \"true\" to disables filtering.")
|
||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||
.required(true)
|
||||
.defaultValue(JSLT_FILTER_DEFAULT)
|
||||
.build();
|
||||
|
||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||
.name("success")
|
||||
.description("The FlowFile with transformed content will be routed to this relationship")
|
||||
|
@ -146,7 +161,8 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
JSLT_TRANSFORM,
|
||||
TRANSFORMATION_STRATEGY,
|
||||
PRETTY_PRINT,
|
||||
TRANSFORM_CACHE_SIZE
|
||||
TRANSFORM_CACHE_SIZE,
|
||||
RESULT_FILTER
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -174,26 +190,33 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
|
||||
final List<ValidationResult> results = new ArrayList<>(super.customValidate(validationContext));
|
||||
|
||||
final ValidationResult.Builder transformBuilder = new ValidationResult.Builder().subject(JSLT_TRANSFORM.getDisplayName());
|
||||
|
||||
final PropertyValue transformProperty = validationContext.getProperty(JSLT_TRANSFORM);
|
||||
if (transformProperty.isExpressionLanguagePresent()) {
|
||||
transformBuilder.valid(true);
|
||||
final ValidationResult.Builder transformBuilder = new ValidationResult.Builder().subject(JSLT_TRANSFORM.getDisplayName());
|
||||
results.add(transformBuilder.valid(true).build());
|
||||
} else {
|
||||
try {
|
||||
final String transform = readTransform(transformProperty);
|
||||
Parser.compileString(transform);
|
||||
transformBuilder.valid(true);
|
||||
} catch (final RuntimeException e) {
|
||||
final String explanation = String.format("JSLT Transform not valid: %s", e.getMessage());
|
||||
transformBuilder.valid(false).explanation(explanation);
|
||||
}
|
||||
results.add(validateJSLT(JSLT_TRANSFORM, transformProperty));
|
||||
}
|
||||
|
||||
results.add(transformBuilder.build());
|
||||
final PropertyValue filterProperty = validationContext.getProperty(RESULT_FILTER);
|
||||
results.add(validateJSLT(RESULT_FILTER, filterProperty));
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
private ValidationResult validateJSLT(PropertyDescriptor property, PropertyValue value) {
|
||||
final ValidationResult.Builder builder = new ValidationResult.Builder().subject(property.getDisplayName());
|
||||
try {
|
||||
final String transform = readTransform(value);
|
||||
getJstlExpression(transform, null);
|
||||
builder.valid(true);
|
||||
} catch (final RuntimeException e) {
|
||||
final String explanation = String.format("%s not valid: %s", property.getDisplayName(), e.getMessage());
|
||||
builder.valid(false).explanation(explanation);
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
@OnScheduled
|
||||
public void onScheduled(final ProcessContext context) {
|
||||
int maxTransformsToCache = context.getProperty(TRANSFORM_CACHE_SIZE).asInteger();
|
||||
|
@ -202,10 +225,11 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
.build();
|
||||
// Precompile the transform if it hasn't been done already (and if there is no Expression Language present)
|
||||
final PropertyValue transformProperty = context.getProperty(JSLT_TRANSFORM);
|
||||
final PropertyValue filterProperty = context.getProperty(RESULT_FILTER);
|
||||
if (!transformProperty.isExpressionLanguagePresent()) {
|
||||
try {
|
||||
final String transform = readTransform(transformProperty);
|
||||
transformCache.put(transform, Parser.compileString(transform));
|
||||
transformCache.put(transform, getJstlExpression(transform, filterProperty.getValue()));
|
||||
} catch (final RuntimeException e) {
|
||||
throw new ProcessException("JSLT Transform compilation failed", e);
|
||||
}
|
||||
|
@ -223,12 +247,13 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
final StopWatch stopWatch = new StopWatch(true);
|
||||
|
||||
final PropertyValue transformProperty = context.getProperty(JSLT_TRANSFORM);
|
||||
final PropertyValue filterProperty = context.getProperty(RESULT_FILTER);
|
||||
FlowFile transformed;
|
||||
final JsonFactory jsonFactory = new JsonFactory();
|
||||
|
||||
try {
|
||||
final String transform = readTransform(transformProperty, original);
|
||||
final Expression jsltExpression = transformCache.get(transform, currString -> Parser.compileString(transform));
|
||||
final Expression jsltExpression = transformCache.get(transform, currString -> getJstlExpression(transform, filterProperty.getValue()));
|
||||
final boolean prettyPrint = context.getProperty(PRETTY_PRINT).asBoolean();
|
||||
|
||||
transformed = session.write(original, (inputStream, outputStream) -> {
|
||||
|
@ -296,7 +321,18 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
@OnStopped
|
||||
@OnShutdown
|
||||
public void onStopped() {
|
||||
transformCache.cleanUp();
|
||||
if (transformCache != null) {
|
||||
transformCache.cleanUp();
|
||||
}
|
||||
}
|
||||
|
||||
private Expression getJstlExpression(String transform, String jsltFilter) {
|
||||
Parser parser = new Parser(new StringReader(transform))
|
||||
.withSource("<inline>");
|
||||
if (jsltFilter != null && !jsltFilter.isEmpty() && !jsltFilter.equals(JSLT_FILTER_DEFAULT)) {
|
||||
parser = parser.withObjectFilter(jsltFilter);
|
||||
}
|
||||
return parser.compile();
|
||||
}
|
||||
|
||||
private JsonNode readJson(final InputStream in) throws IOException {
|
||||
|
@ -321,6 +357,9 @@ public class JSLTTransformJSON extends AbstractProcessor {
|
|||
|
||||
private String readTransform(final PropertyValue propertyValue) {
|
||||
final ResourceReference resourceReference = propertyValue.asResource();
|
||||
if (resourceReference == null) {
|
||||
return propertyValue.getValue();
|
||||
}
|
||||
try (final BufferedReader reader = new BufferedReader(new InputStreamReader(resourceReference.read()))) {
|
||||
return reader.lines().collect(Collectors.joining());
|
||||
} catch (final IOException e) {
|
||||
|
|
|
@ -151,6 +151,30 @@ public class TestJSLTTransformJSON {
|
|||
flowFile.assertContentEquals(new byte[0]);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithNullValueRemoved() {
|
||||
runner.setProperty(JSLTTransformJSON.RESULT_FILTER, ". != null and . != {} and . != []");
|
||||
runTransform("inputWithNull.json", "simpleTransform.json", "simpleOutputWithoutNull.json");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithNullValueIncluded() {
|
||||
runner.setProperty(JSLTTransformJSON.RESULT_FILTER, ". != {} and . != []");
|
||||
runTransform("inputWithNull.json", "simpleTransform.json", "simpleOutputWithNull.json");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTransformWithNoFilter() {
|
||||
runner.setProperty(JSLTTransformJSON.RESULT_FILTER, "true");
|
||||
runTransform("inputWithNull.json", "simpleTransform.json", "simpleOutputWithNull.json");
|
||||
}
|
||||
|
||||
// This test verifies transformCache cleanup does not throw an exception
|
||||
@Test
|
||||
public void testShutdown() {
|
||||
runner.stop();
|
||||
}
|
||||
|
||||
private void runTransform(final String inputFileName, final String transformFileName, final String outputFileName) {
|
||||
setTransformEnqueueJson(transformFileName, inputFileName);
|
||||
|
||||
|
|
|
@ -0,0 +1,13 @@
|
|||
{
|
||||
"rating": {
|
||||
"primary": {
|
||||
"value": 3
|
||||
},
|
||||
"series": {
|
||||
"value": [5,4]
|
||||
},
|
||||
"quality": {
|
||||
"value": null
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,8 @@
|
|||
{
|
||||
"SecondaryRatings" : {
|
||||
"quality" : {
|
||||
"Value" : 3,
|
||||
"RatingRange" : null
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,7 @@
|
|||
{
|
||||
"SecondaryRatings" : {
|
||||
"quality" : {
|
||||
"Value" : 3
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue