[ https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16308669#comment-16308669 ]
ASF GitHub Bot commented on NIFI-4724: -------------------------------------- Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/2362#discussion_r159308383 --- Diff: nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java --- @@ -71,9 +72,18 @@ void publish(final FlowFile flowFile, final InputStream flowFileContent, final b tracker = new InFlightMessageTracker(); } - try (final StreamDemarcator demarcator = new StreamDemarcator(flowFileContent, demarcatorBytes, maxMessageSize)) { + try { byte[] messageContent; - try { + if (demarcatorBytes == null || demarcatorBytes.length == 0) { --- End diff -- @ijokarumawak in this case, we end up blindly buffering the entire contents of the FlowFile into memory. If a 1 GB FlowFile is sent in, for example, we will buffer a full 1 GB of data in heap. Currently, in this case, the StreamDemarcator would have buffered only 1 MB (by default) and then thrown a TokenTooLargeException, which would avoid exhausting the JVM heap. I think we need to also do the same here, checking flowFile.getSize() and if it's larger than the maxMessageSize throw an Exception instead of copying the content to a ByteArrayOutputStream. > Publish kafka processors fails with FlowFileHandlingException if the flow > file is empty > --------------------------------------------------------------------------------------- > > Key: NIFI-4724 > URL: https://issues.apache.org/jira/browse/NIFI-4724 > Project: Apache NiFi > Issue Type: Bug > Components: Extensions > Affects Versions: 1.1.0 > Reporter: Mahesh Nayak > Assignee: Koji Kawamura > > 1. Construct the flow GenerateFlowFile --> PublishKafka --> PutFile > 2. In GenerateFlowFile set the "File Size" to 0B. > 3. Start the flow. > Result : Kafka processor throws below exception > {code:None} > 2017-12-27 02:49:21,933 WARN [Timer-Driven Process Thread-9] > o.a.n.c.t.ContinuallyRunProcessorTask Administratively Yielding > PublishKafka_0_10[id=95dbc77a-0160-1000-0000-000069761c4e] due to uncaught > Exception: org.apache.nifi.processor.exception.FlowFileHandlingException: > StandardFlowFileRecord[uuid=4d6cb989-b6a7-4129-9dfc-1598ee2b3937,claim=,offset=0,name=7061091998478433,size=0] > transfer relationship not specified > 2017-12-27 02:49:21,933 WARN [Timer-Driven Process Thread-9] > o.a.n.c.t.ContinuallyRunProcessorTask > org.apache.nifi.processor.exception.FlowFileHandlingException: > StandardFlowFileRecord[uuid=4d6cb989-b6a7-4129-9dfc-1598ee2b3937,claim=,offset=0,name=7061091998478433,size=0] > transfer relationship not specified > at > org.apache.nifi.controller.repository.StandardProcessSession.checkpoint(StandardProcessSession.java:251) > at > org.apache.nifi.controller.repository.StandardProcessSession.commit(StandardProcessSession.java:321) > at > org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:28) > at > org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1120) > at > org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:147) > at > org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47) > at > org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:128) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)