NIFI-2142 Added Guava cache for XSLT stylesheets in TransformXml

This closes #609

Signed-off-by: jpercivall <joepercivall@yahoo.com>
This commit is contained in:
Joey Frazee 2016-06-28 15:29:43 -07:00 committed by jpercivall
parent 557d6365bf
commit bb24312709
6 changed files with 153 additions and 35 deletions

View File

@ -189,6 +189,10 @@ language governing permissions and limitations under the License. -->
<artifactId>jBcrypt</artifactId>
<version>0.4.1</version>
</dependency>
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
<dependency>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-mock</artifactId>

View File

@ -28,7 +28,10 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import javax.xml.transform.OutputKeys;
import javax.xml.transform.Templates;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerConfigurationException;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
@ -41,6 +44,7 @@ import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.annotation.documentation.Tags;
import org.apache.nifi.annotation.lifecycle.OnScheduled;
import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.components.ValidationContext;
import org.apache.nifi.components.ValidationResult;
@ -60,6 +64,10 @@ import org.apache.nifi.stream.io.BufferedInputStream;
import org.apache.nifi.util.StopWatch;
import org.apache.nifi.util.Tuple;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
@EventDriven
@SideEffectFree
@SupportsBatching
@ -76,13 +84,43 @@ public class TransformXml extends AbstractProcessor {
.name("XSLT file name")
.description("Provides the name (including full path) of the XSLT file to apply to the flowfile XML content.")
.required(true)
.expressionLanguageSupported(true)
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
.build();
public static final PropertyDescriptor INDENT_OUTPUT = new PropertyDescriptor.Builder()
.name("indent-output")
.displayName("Indent")
.description("Whether or not to indent the output.")
.required(true)
.defaultValue("true")
.allowableValues("true", "false")
.addValidator(StandardValidators.BOOLEAN_VALIDATOR)
.build();
public static final PropertyDescriptor CACHE_SIZE = new PropertyDescriptor.Builder()
.name("cache-size")
.displayName("Cache size")
.description("Maximum number of stylesheets to cache. Zero disables the cache.")
.required(true)
.defaultValue("10")
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
.build();
public static final PropertyDescriptor CACHE_TTL_AFTER_LAST_ACCESS = new PropertyDescriptor.Builder()
.name("cache-ttl-after-last-access")
.displayName("Cache TTL after last access")
.description("The cache TTL (time-to-live) or how long to keep stylesheets in the cache after last access.")
.required(true)
.defaultValue("60 secs")
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
.build();
public static final Relationship REL_SUCCESS = new Relationship.Builder()
.name("success")
.description("The FlowFile with transformed content will be routed to this relationship")
.build();
public static final Relationship REL_FAILURE = new Relationship.Builder()
.name("failure")
.description("If a FlowFile fails processing for any reason (for example, the FlowFile is not valid XML), it will be routed to this relationship")
@ -90,11 +128,15 @@ public class TransformXml extends AbstractProcessor {
private List<PropertyDescriptor> properties;
private Set<Relationship> relationships;
private LoadingCache<String, Templates> cache;
@Override
protected void init(final ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>();
properties.add(XSLT_FILE_NAME);
properties.add(INDENT_OUTPUT);
properties.add(CACHE_SIZE);
properties.add(CACHE_TTL_AFTER_LAST_ACCESS);
this.properties = Collections.unmodifiableList(properties);
final Set<Relationship> relationships = new HashSet<>();
@ -124,6 +166,35 @@ public class TransformXml extends AbstractProcessor {
.build();
}
private Templates newTemplates(String path) throws TransformerConfigurationException {
TransformerFactory factory = TransformerFactory.newInstance();
return factory.newTemplates(new StreamSource(path));
}
@OnScheduled
public void onScheduled(final ProcessContext context) {
final ComponentLog logger = getLogger();
final Integer cacheSize = context.getProperty(CACHE_SIZE).asInteger();
final Long cacheTTL = context.getProperty(CACHE_TTL_AFTER_LAST_ACCESS).asTimePeriod(TimeUnit.SECONDS);
if (cacheSize > 0) {
CacheBuilder cacheBuilder = CacheBuilder.newBuilder().maximumSize(cacheSize);
if (cacheTTL > 0) {
cacheBuilder = cacheBuilder.expireAfterAccess(cacheTTL, TimeUnit.SECONDS);
}
cache = cacheBuilder.build(
new CacheLoader<String, Templates>() {
public Templates load(String path) throws TransformerConfigurationException {
return newTemplates(path);
}
});
} else {
cache = null;
logger.warn("Stylesheet cache disabled because cache size is set to 0");
}
}
@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final FlowFile original = session.get();
@ -133,17 +204,25 @@ public class TransformXml extends AbstractProcessor {
final ComponentLog logger = getLogger();
final StopWatch stopWatch = new StopWatch(true);
final String xsltFileName = context.getProperty(XSLT_FILE_NAME)
.evaluateAttributeExpressions(original)
.getValue();
final Boolean indentOutput = context.getProperty(INDENT_OUTPUT).asBoolean();
try {
FlowFile transformed = session.write(original, new StreamCallback() {
@Override
public void process(final InputStream rawIn, final OutputStream out) throws IOException {
try (final InputStream in = new BufferedInputStream(rawIn)) {
final Templates templates;
if (cache != null) {
templates = cache.get(xsltFileName);
} else {
templates = newTemplates(xsltFileName);
}
File stylesheet = new File(context.getProperty(XSLT_FILE_NAME).getValue());
StreamSource styleSource = new StreamSource(stylesheet);
TransformerFactory tfactory = new net.sf.saxon.TransformerFactoryImpl();
Transformer transformer = tfactory.newTransformer(styleSource);
final Transformer transformer = templates.newTransformer();
transformer.setOutputProperty(OutputKeys.INDENT, (indentOutput ? "yes" : "no"));
// pass all dynamic properties to the transformer
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {

View File

@ -16,12 +16,15 @@
*/
package org.apache.nifi.processors.standard;
import java.io.IOException;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.HashMap;
import java.util.Map;
@ -29,7 +32,6 @@ import java.util.Map;
import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners;
import org.junit.Ignore;
import org.junit.Test;
@ -59,7 +61,6 @@ public class TestTransformXml {
original.assertContentEquals("not xml");
}
@Ignore("this test fails")
@Test
public void testTransformMath() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
@ -72,12 +73,11 @@ public class TestTransformXml {
runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
final String transformedContent = new String(transformed.toByteArray(), StandardCharsets.UTF_8);
final String expectedContent = new String(Files.readAllBytes(Paths.get("src/test/resources/TestTransformXml/math.html"))).trim();
transformed.assertContentEquals(Paths.get("src/test/resources/TestTransformXml/math.html"));
transformed.assertContentEquals(expectedContent);
}
@Ignore("this test fails")
@Test
public void testTransformCsv() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
@ -108,9 +108,44 @@ public class TestTransformXml {
runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
final String transformedContent = new String(transformed.toByteArray(), StandardCharsets.ISO_8859_1);
final String expectedContent = new String(Files.readAllBytes(Paths.get("src/test/resources/TestTransformXml/tokens.xml")));
transformed.assertContentEquals(Paths.get("src/test/resources/TestTransformXml/tokens.xml"));
transformed.assertContentEquals(expectedContent);
}
}
@Test
public void testTransformExpressionLanguage() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
runner.setProperty("header", "Test for mod");
runner.setProperty(TransformXml.XSLT_FILE_NAME, "${xslt.path}");
final Map<String, String> attributes = new HashMap<>();
attributes.put("xslt.path", "src/test/resources/TestTransformXml/math.xsl");
runner.enqueue(Paths.get("src/test/resources/TestTransformXml/math.xml"), attributes);
runner.run();
runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
final String expectedContent = new String(Files.readAllBytes(Paths.get("src/test/resources/TestTransformXml/math.html"))).trim();
transformed.assertContentEquals(expectedContent);
}
@Test
public void testTransformNoCache() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
runner.setProperty("header", "Test for mod");
runner.setProperty(TransformXml.CACHE_SIZE, "0");
runner.setProperty(TransformXml.XSLT_FILE_NAME, "src/test/resources/TestTransformXml/math.xsl");
runner.enqueue(Paths.get("src/test/resources/TestTransformXml/math.xml"));
runner.run();
runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS);
final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).get(0);
final String expectedContent = new String(Files.readAllBytes(Paths.get("src/test/resources/TestTransformXml/math.html"))).trim();
transformed.assertContentEquals(expectedContent);
}
}