[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16313332#comment-16313332
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user markap14 commented on the issue:

https://github.com/apache/nifi/pull/2362
  
ok all looks good after testing! +1 merged to master. Thanks for updating 
this @ijokarumawak!


> 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
> Fix For: 1.5.0
>
>
> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1631#comment-1631
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user asfgit closed the pull request at:

https://github.com/apache/nifi/pull/2362


> 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
> Fix For: 1.5.0
>
>
> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-05 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16313331#comment-16313331
 ] 

ASF subversion and git services commented on NIFI-4724:
---

Commit e5ed62a98fe0c15fcba865edfe00eac673a6ac51 in nifi's branch 
refs/heads/master from [~ijokarumawak]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=e5ed62a ]

NIFI-4724: Support 0 byte message with PublishKafka
Before this fix, PublishKafka (0.9) and PublishKafka_0_10 fail with empty 
incoming FlowFiles due to 'transfer relationship not specified' error.
Because the internal 'publish' method is not called as StreamDemarcator does 
not emit any token regardless whether demarcator is set or not.

As for PublishKafka_0_11 and PublishKafka_1_0, empty FlowFiles are transferred 
to 'success' relationship, however no Kafka message is sent to Kafka.

Since Kafka allows 0 byte body empty messages, NiFi should be able to send it, 
too.

This commit changes above current situation to the followings, with all 
PublishKafka_* processors:

- If demarcator is not set, then publish incoming FlowFile content as it is. 
This enables sending an empty Kafka message.
- If demarcator is set, send each token as a separate message.
  Even if no token is found (empty incoming FlowFile), transfer the FlowFile to 
'success'.

This closes #2362.

Signed-off-by: Mark Payne 


> 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
> Fix For: 1.5.0
>
>
> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16313276#comment-16313276
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user markap14 commented on the issue:

https://github.com/apache/nifi/pull/2362
  
@ijokarumawak ah, it didn't occur to me that ProducerRecord will not allow 
us to take `size` and `offset` into account when creating the ProducerRecord. 
In that case, scrap that idea :) All else looks good code-wise. Will give it a 
once-over testing and assuming all is good will merge to master. Thanks for 
updating!


> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-04 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16312282#comment-16312282
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user ijokarumawak commented on the issue:

https://github.com/apache/nifi/pull/2362
  
@markap14 Thanks for your feedback, I've added another commit to 
incorporate your suggestions. Also rebased with the latest master just in case.

I considered to implement the use of BlockingQueue for further 
optimization, too. However, I think it may be dangerous to reuse the same 
byte[] object because different FlowFiles may have different size of content. 
For example, the 1st FlowFile content is 'foo bar', then the 2nd FlowFile's is 
'baz', then the 2nd message would be 'baz bar'. In addition to that, the 
`ProducerRecord` provided by Kafka client library does not take `size` and 
`offset` those are usually used to specify a part of a larger byte array. So, I 
think we need to create new byte[] object for each message. I hope my 
understanding is correct. But if there's more optimization can be done, please 
file a separate JIRA as I think it's out of scope for this ticket.


> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-04 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16312279#comment-16312279
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user ijokarumawak commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2362#discussion_r159791308
  
--- 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) {
+// Send FlowFile content as it is, to support sending 0 
byte message.
+final ByteArrayOutputStream bos = new 
ByteArrayOutputStream();
--- End diff --

@markap14 Thanks for the advice. I switched to use StreamUtils as you 
suggested.


> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-04 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16312270#comment-16312270
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

Github user ijokarumawak commented on a diff in the pull request:

https://github.com/apache/nifi/pull/2362#discussion_r159790221
  
--- 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 --

@markap14 I agree with that, added a code to transfer FlowFiles to 
'failure' if content size exceeds the maxMessageSize. Thanks!


> 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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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--69761c4e] 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)


[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2018-01-02 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16308670#comment-16308670
 ] 

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_r159309499
  
--- 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) {
+// Send FlowFile content as it is, to support sending 0 
byte message.
+final ByteArrayOutputStream bos = new 
ByteArrayOutputStream();
--- End diff --

This is expensive, as it will create a new ByteArrayOutputStream for each 
FlowFile, then constantly re-copy that byte array each time that it needs to 
expand the internal buffer. Instead, I would suggest we just do something like:
```
byte[] messageContent = new byte[(int) flowFile.getSize()];
StreamUtils.fillBuffer(flowFileContent, messageContent);
```
Even that, though, is going to create a good bit of garbage that we can 
avoid. A better approach might actually be to use a BlockingQueue and 
call poll() on that. If we get a byte[] back, then use it. If not, then create 
a new byte[maxMessageSize] and then use that. At the end, add the byte[] back 
to the queue. Then in the close() method clear the queue in case the processor 
is re-scheduled with fewer threads. This is nice because it means that we can 
avoid constantly creating these byte[] objects, which can cause stress on the 
GC.

That being said, if you think it's out of scope for this ticket, I would 
recommend just creating the byte[] inline as described above and then we can 
create a new JIRA to optimize this.


> 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--69761c4e] 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 
> 

[jira] [Commented] (NIFI-4724) Publish kafka processors fails with FlowFileHandlingException if the flow file is empty

2017-12-27 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-4724?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16304366#comment-16304366
 ] 

ASF GitHub Bot commented on NIFI-4724:
--

GitHub user ijokarumawak opened a pull request:

https://github.com/apache/nifi/pull/2362

NIFI-4724: Support 0 byte message with PublishKafka

Before this fix, PublishKafka (0.9) and PublishKafka_0_10 fail with empty 
incoming FlowFiles due to 'transfer relationship not specified' error.
Because the internal 'publish' method is not called as StreamDemarcator 
does not emit any token regardless whether demarcator is set or not.

As for PublishKafka_0_11 and PublishKafka_1_0, empty FlowFiles are 
transferred to 'success' relationship, however no Kafka message is sent to 
Kafka.

Since Kafka allows 0 byte body empty messages, NiFi should be able to send 
it, too.

This commit changes above current situation to the followings, with all 
PublishKafka_* processors:

- If demarcator is not set, then publish incoming FlowFile content as it 
is. This enables sending an empty Kafka message.
- If demarcator is set, send each token as a separate message.
  Even if no token is found (empty incoming FlowFile), transfer the 
FlowFile to 'success'.

Thank you for submitting a contribution to Apache NiFi.

In order to streamline the review of the contribution we ask you
to ensure the following steps have been taken:

### For all changes:
- [x] Is there a JIRA ticket associated with this PR? Is it referenced 
 in the commit message?

- [x] Does your PR title start with NIFI- where  is the JIRA number 
you are trying to resolve? Pay particular attention to the hyphen "-" character.

- [x] Has your PR been rebased against the latest commit within the target 
branch (typically master)?

- [x] Is your initial contribution a single, squashed commit?

### For code changes:
- [x] Have you ensured that the full suite of tests is executed via mvn 
-Pcontrib-check clean install at the root nifi folder?
- [x] Have you written or updated unit tests to verify your changes?
- [ ] If adding new dependencies to the code, are these dependencies 
licensed in a way that is compatible for inclusion under [ASF 
2.0](http://www.apache.org/legal/resolved.html#category-a)? 
- [ ] If applicable, have you updated the LICENSE file, including the main 
LICENSE file under nifi-assembly?
- [ ] If applicable, have you updated the NOTICE file, including the main 
NOTICE file found under nifi-assembly?
- [ ] If adding new Properties, have you added .displayName in addition to 
.name (programmatic access) for each of the new properties?

### For documentation related changes:
- [ ] Have you ensured that format looks appropriate for the output in 
which it is rendered?

### Note:
Please ensure that once the PR is submitted, you check travis-ci for build 
issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/ijokarumawak/nifi nifi-4724

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/nifi/pull/2362.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #2362


commit 513722c3382e5c7a9757b4ad71490483c151fa1f
Author: Koji Kawamura 
Date:   2017-12-27T08:38:57Z

NIFI-4724: Support 0 byte message with PublishKafka

Before this fix, PublishKafka (0.9) and PublishKafka_0_10 fail with empty 
incoming FlowFiles due to 'transfer relationship not specified' error.
Because the internal 'publish' method is not called as StreamDemarcator 
does not emit any token regardless whether demarcator is set or not.

As for PublishKafka_0_11 and PublishKafka_1_0, empty FlowFiles are 
transferred to 'success' relationship, however no Kafka message is sent to 
Kafka.

Since Kafka allows 0 byte body empty messages, NiFi should be able to send 
it, too.

This commit changes above current situation to the followings, with all 
PublishKafka_* processors:

- If demarcator is not set, then publish incoming FlowFile content as it 
is. This enables sending an empty Kafka message.
- If demarcator is set, send each token as a separate message.
  Even if no token is found (empty incoming FlowFile), transfer the 
FlowFile to 'success'.




> 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: