diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java index 13e6e317d5b8..e6ff53aaf6b5 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java @@ -319,7 +319,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS)); getLogger().info("Transformation Completed {}", original); } catch (final ProcessException e) { - getLogger().error("Transformation Failed", original, e); + getLogger().error("Transformation Failed {}", original, e); session.transfer(original, REL_FAILURE); } } diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java index 4807d63bb8dd..262d784fb2ce 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestTransformXml.java @@ -16,8 +16,11 @@ */ package org.apache.nifi.processors.standard; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.nifi.lookup.SimpleKeyValueLookupService; +import org.apache.nifi.processor.Relationship; import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.util.LogMessage; import org.apache.nifi.util.MockComponentLog; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.PropertyMigrationResult; @@ -25,23 +28,40 @@ import org.apache.nifi.util.TestRunners; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.BufferedReader; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStreamReader; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Stream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestTransformXml { + private static final String XML_FOR_TESTING_PARAMETERS = """ + + + Some data + + """; + private TestRunner runner; + @TempDir + private Path temptDir; + @BeforeEach void setUp() { runner = TestRunners.newTestRunner(TransformXml.class); @@ -353,6 +373,202 @@ public void testMessageNonTerminate() throws IOException { assertTrue(logger.getWarnMessages().isEmpty()); } + @Test + void testParameterDeclaredAndSet() throws IOException { + final String xslt = """ + + + + + + + Value Selected: + + + + + """; + + final Path xsltPath = writeXslt(xslt, "someTransform.xslt"); + runner.setProperty(TransformXml.XSLT_FILE_NAME, xsltPath.toString()); + runner.setProperty("customParam", "From NIFI"); + runner.enqueue(XML_FOR_TESTING_PARAMETERS); + + runner.run(); + runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS); + final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).getFirst(); + final String expectedTransform = """ + + + + Value Selected: From NIFI + + """; + transformed.assertContentEquals(expectedTransform); + } + + @Test + void testParameterSetButNotDeclared() throws IOException { + final String xslt = """ + + + + + + Value Selected: + + + + + """; + + final Path xsltPath = writeXslt(xslt, "someTransform.xslt"); + runner.setProperty(TransformXml.XSLT_FILE_NAME, xsltPath.toString()); + runner.setProperty("customParam", "From NIFI"); + runner.enqueue(XML_FOR_TESTING_PARAMETERS); + + runner.run(); + runner.assertAllFlowFilesTransferred(TransformXml.REL_SUCCESS); + final MockFlowFile transformed = runner.getFlowFilesForRelationship(TransformXml.REL_SUCCESS).getFirst(); + final String expectedTransform = + """ + + + + Value Selected: + + + """; + transformed.assertContentEquals(expectedTransform); + } + + @Test + void testParameterNotDeclaredButUsedInXslt() throws IOException { + final String xslt = """ + + + + + + Value Selected: + + + + + """; + + final Path xsltPath = writeXslt(xslt, "someTransform.xslt"); + runner.setProperty(TransformXml.XSLT_FILE_NAME, xsltPath.toString()); + runner.setProperty("customParam", "From NIFI"); + runner.enqueue(XML_FOR_TESTING_PARAMETERS); + + runner.run(); + + runner.assertAllFlowFilesTransferred(TransformXml.REL_FAILURE); + final MockComponentLog logger = runner.getLogger(); + final List errorMessages = logger.getErrorMessages(); + assertTrue(errorMessages.getFirst().getMsg().contains("Variable $customParam has not been declared")); + } + + @ParameterizedTest + @MethodSource("parameterAsSpecificTypeArgs") + void testParameterAsSpecificType(String paramType, String parameterValue, String defaultValue, Relationship expectedRelationship, String expectedTransform) throws IOException { + final String parameterName = "customParam"; + final String xslt = getXSLTWithParameterDefinedWithType(paramType, defaultValue); + final Path xsltPath = writeXslt(xslt, "someTransform.xslt"); + runner.setProperty(TransformXml.XSLT_FILE_NAME, xsltPath.toString()); + runner.setProperty(parameterName, parameterValue); + runner.enqueue(XML_FOR_TESTING_PARAMETERS); + + runner.run(); + runner.assertAllFlowFilesTransferred(expectedRelationship); + + if (expectedTransform != null) { + final MockFlowFile transformed = runner.getFlowFilesForRelationship(expectedRelationship).getFirst(); + transformed.assertContentEquals(expectedTransform); + } + + if (expectedRelationship == TransformXml.REL_FAILURE) { + assertTrue(runner.getLogger().getErrorMessages().stream() + .map(LogMessage::getThrowable) + .map(ExceptionUtils::getStackTrace) + .anyMatch(stackTrace -> stackTrace.contains("ValidationException"))); + } + } + + private static Stream parameterAsSpecificTypeArgs() { + return Stream.of( + Arguments.argumentSet("Valid number", "xs:integer", "100", "0", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:integer value is 100 + + """), + Arguments.argumentSet("Invalid number", "xs:integer", "NIFI", "0", TransformXml.REL_FAILURE, null), + Arguments.argumentSet("Valid boolean lowercase", "xs:boolean", "true", "false", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:boolean value is true + + """), + Arguments.argumentSet("Invalid boolean uppercase", "xs:boolean", "TRUE", "false", TransformXml.REL_FAILURE, null), + Arguments.argumentSet("Valid ISO 8601 date", "xs:date", "2026-01-01", "xs:date('1970-01-01')", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:date value is 2026-01-01 + + """, null), //29 April 2003 + Arguments.argumentSet("Invalid ISO 8601 date", "xs:date", "1 January 2026", "xs:date('1970-01-01')", TransformXml.REL_FAILURE, null), + Arguments.argumentSet("Valid ISO 8601 date time", "xs:dateTime", "2026-01-01T00:00:00", "xs:dateTime('1970-01-01T00:00:00')", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:dateTime value is 2026-01-01T00:00:00 + + """), + Arguments.argumentSet("Invalid ISO 8601 date time", "xs:dateTime", "1970-01-01 00:00:00", "xs:dateTime('1970-01-01T00:00:00')", TransformXml.REL_FAILURE, null), + Arguments.argumentSet("Valid time without timezone/offset", "xs:time", "12:34:56.789", "current-time()", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:time value is 12:34:56.789 + + """), + Arguments.argumentSet("Valid UTC time", "xs:time", "12:34:56Z", "current-time()", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:time value is 12:34:56Z + + """), + Arguments.argumentSet("Valid Offset from UTC time", "xs:time", "12:34:56-05:00", "current-time()", TransformXml.REL_SUCCESS, + """ + + + Param of type xs:time value is 12:34:56-05:00 + + """) + + ); + } + + private String getXSLTWithParameterDefinedWithType(String type, String defaultValue) { + return """ + + + + + Param of type %1$s value is + + + + """.formatted(type, defaultValue); + } + @Test void testMigrateProperties() { final Map expectedRenamed = Map.ofEntries( @@ -369,4 +585,11 @@ void testMigrateProperties() { final PropertyMigrationResult propertyMigrationResult = runner.migrateProperties(); assertEquals(expectedRenamed, propertyMigrationResult.getPropertiesRenamed()); } + + private Path writeXslt(String xslt, String xsltName) throws IOException { + final Path xsltPath = temptDir.resolve(xsltName); + Files.writeString(xsltPath, xslt); + + return xsltPath; + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java b/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java index c0014612a231..f9730580cbad 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/CapturingLogger.java @@ -88,7 +88,8 @@ public void trace(String format, Object arg1, Object arg2) { @Override public void trace(String format, Object... arguments) { - traceMessages.add(new LogMessage(null, format, null, arguments)); + final Throwable throwable = lastArgIsException(arguments) ? (Throwable) arguments[arguments.length - 1] : null; + traceMessages.add(new LogMessage(null, format, throwable, arguments)); logger.trace(format, arguments); } @@ -161,7 +162,8 @@ public void debug(String format, Object arg1, Object arg2) { @Override public void debug(String format, Object... arguments) { - debugMessages.add(new LogMessage(null, format, null, arguments)); + final Throwable throwable = lastArgIsException(arguments) ? (Throwable) arguments[arguments.length - 1] : null; + debugMessages.add(new LogMessage(null, format, throwable, arguments)); logger.debug(format, arguments); } @@ -232,8 +234,9 @@ public void info(String format, Object arg1, Object arg2) { @Override public void info(String format, Object... arguments) { + final Throwable throwable = lastArgIsException(arguments) ? (Throwable) arguments[arguments.length - 1] : null; String message = MessageFormatter.arrayFormat(format, arguments).getMessage(); - infoMessages.add(new LogMessage(null, message, null, arguments)); + infoMessages.add(new LogMessage(null, message, throwable, arguments)); logger.info(format, arguments); } @@ -303,8 +306,9 @@ public void warn(String format, Object arg1, Object arg2) { @Override public void warn(String format, Object... arguments) { + final Throwable throwable = lastArgIsException(arguments) ? (Throwable) arguments[arguments.length - 1] : null; String message = MessageFormatter.arrayFormat(format, arguments).getMessage(); - warnMessages.add(new LogMessage(null, message, null, arguments)); + warnMessages.add(new LogMessage(null, message, throwable, arguments)); logger.warn(format, arguments); } @@ -384,8 +388,9 @@ public void error(String format, Object arg1, Throwable t) { @Override public void error(String format, Object... arguments) { + final Throwable throwable = lastArgIsException(arguments) ? (Throwable) arguments[arguments.length - 1] : null; final String message = MessageFormatter.arrayFormat(format, arguments).getMessage(); - errorMessages.add(new LogMessage(null, message, null, arguments)); + errorMessages.add(new LogMessage(null, message, throwable, arguments)); logger.error(format, arguments); } @@ -434,4 +439,7 @@ public void error(Marker marker, String msg, Throwable t) { logger.error(marker, msg, t); } + private boolean lastArgIsException(final Object[] os) { + return (os != null && os.length > 0 && (os[os.length - 1] instanceof Throwable)); + } }