diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java index d2859f6381..a251ca9760 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-components/src/main/java/org/apache/nifi/logging/repository/StandardLogRepository.java @@ -68,8 +68,8 @@ public class StandardLogRepository implements LogRepository { @Override public void addLogMessage(final LogLevel level, final String format, final Object[] params) { - replaceThrowablesWithMessage(params); final Optional flowFileUuid = getFirstFlowFileUuidFromObjects(params); + simplifyArgs(params); final String formattedMessage = MessageFormatter.arrayFormat(format, params).getMessage(); final LogMessage logMessage = new LogMessage.Builder(System.currentTimeMillis(), level) .message(formattedMessage) @@ -80,8 +80,8 @@ public class StandardLogRepository implements LogRepository { @Override public void addLogMessage(final LogLevel level, final String format, final Object[] params, final Throwable t) { - replaceThrowablesWithMessage(params); final Optional flowFileUuid = getFirstFlowFileUuidFromObjects(params); + simplifyArgs(params); final String formattedMessage = MessageFormatter.arrayFormat(format, params, t).getMessage(); final LogMessage logMessage = new LogMessage.Builder(System.currentTimeMillis(), level) .message(formattedMessage) @@ -105,14 +105,23 @@ public class StandardLogRepository implements LogRepository { return Optional.ofNullable(flowFileFound).map(ff -> ff.getAttribute(CoreAttributes.UUID.key())); } - private void replaceThrowablesWithMessage(final Object[] params) { + private void simplifyArgs(final Object[] params) { for (int i = 0; i < params.length; i++) { - if (params[i] instanceof Throwable) { - params[i] = ((Throwable) params[i]).getLocalizedMessage(); - } + params[i] = simplifyArg(params[i]); } } + private Object simplifyArg(final Object param) { + if (param instanceof Throwable) { + return ((Throwable) param).getLocalizedMessage(); + } else if (param instanceof FlowFile) { + final FlowFile flowFile = (FlowFile) param; + return "FlowFile[filename=" + flowFile.getAttribute(CoreAttributes.FILENAME.key()) + "]"; + } + + return param; + } + @Override public void setObservationLevel(String observerIdentifier, LogLevel level) { writeLock.lock();