Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,10 @@
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;
Expand All @@ -41,6 +44,7 @@
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;
Expand All @@ -60,6 +64,10 @@
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
Expand All @@ -76,25 +84,59 @@ 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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this property a max character length or data size limit?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also, this is a integer validator but the property is interpreted as a long on line 171

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually my new guess is that this is the maximum number of stylesheets stored (the description could be added to, lol)

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@JPercivall Yes, that's correct. So "Maximum number of stylesheets to store in the cache"?

.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")
.build();

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<>();
Expand Down Expand Up @@ -124,6 +166,35 @@ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String
.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");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The cache will still be created and on line 213 it will try to use it to look up an xslt if the cache size is 0. I'd suggest not even creating it in that case.

}
}

@Override
public void onTrigger(final ProcessContext context, final ProcessSession session) {
final FlowFile original = session.get();
Expand All @@ -133,17 +204,25 @@ public void onTrigger(final ProcessContext context, final ProcessSession session

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()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,20 +16,22 @@
*/
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;

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;

Expand Down Expand Up @@ -59,7 +61,6 @@ public void testNonXmlContent() throws IOException {
original.assertContentEquals("not xml");
}

@Ignore("this test fails")
@Test
public void testTransformMath() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new TransformXml());
Expand All @@ -72,12 +73,11 @@ public void testTransformMath() throws IOException {

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());
Expand Down Expand Up @@ -108,9 +108,44 @@ public void testTransformCsv() throws IOException {
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);
}

}
Original file line number Diff line number Diff line change
@@ -1,8 +1,8 @@
<HTML xmlns:xs="http://www.w3.org/2001/XMLSchema">
<H1>Test for mod</H1>
<HR>
<P>Should say "1": 1</P>
<P>Should say "1": 1</P>
<P>Should say "-1": -1</P>
<P>true</P>
</HTML>
<H1>Test for mod</H1>
<HR>
<P>Should say "1": 1</P>
<P>Should say "1": 1</P>
<P>Should say "-1": -1</P>
<P>true</P>
</HTML>
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,10 @@
See the License for the specific language governing permissions and
limitations under the License.
-->
<xsl:stylesheet version="2.0"
xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
<xsl:stylesheet version="2.0"
xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
xmlns:xs="http://www.w3.org/2001/XMLSchema">

<xsl:param name="header" />

<xsl:template match="doc">
Expand All @@ -33,4 +33,4 @@
</P>
</HTML>
</xsl:template>
</xsl:stylesheet>
</xsl:stylesheet>
Original file line number Diff line number Diff line change
@@ -1,17 +1,17 @@
<?xml version="1.0" encoding="ISO-8859-1"?>
<test release="0.0" id="uuid_0">
<event id="uuid_1">
<token>1</token>
<token>2</token>
<token>3</token>
<token>4</token>
<token>C:\dir$abc</token>
<token>6</token>
<token>7</token>
<token>A,B</token>
<token>"don't"</token>
<token>2014-05-01T30:23:00Z</token>
<token>11</token>
<token>12</token>
</event>
<event id="uuid_1">
<token>1</token>
<token>2</token>
<token>3</token>
<token>4</token>
<token>C:\dir$abc</token>
<token>6</token>
<token>7</token>
<token>A,B</token>
<token>"don't"</token>
<token>2014-05-01T30:23:00Z</token>
<token>11</token>
<token>12</token>
</event>
</test>