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

Aldrin Piri commented on NIFI-730:
----------------------------------

Code looks good and contrib all passes.

For general cases, the flow has been working fine.  I have yet to test against 
a clustered instance, but have run into some issues on my standalone.  My flow 
for testing was simply a GenerateFlowFile sending to an UpdateAttribute.  This 
seemed to work quite well overall.

I then configured the connection between the two components to have an 
expiration of flowfiles after 30 seconds.  I let this run for a while (queue 
~2GB of 1 kB files) and then initiated a purge.  At this point, I noticed 
significant swapping in the logs to the point where it seemed that was all that 
was occurring.  
{code}
2015-10-27 12:35:31,581 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:32,007 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:32,433 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:32,829 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:33,252 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:33,678 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:34,146 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:34,587 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:35,013 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:35,431 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
2015-10-27 12:35:35,853 INFO [Drop FlowFiles for Connection 
239dc171-e1d9-4782-8ead-e4f31bee3496] o.a.n.c.r.WriteAheadFlowFileRepository 
Repository updated to reflect that 10000 FlowFiles were swapped in to 
FlowFileQueue[id=239dc171-e1d9-4782-8ead-e4f31bee3496]
{code}

I had accidentally refreshed the page when trying to refresh stats of the flow, 
and then lost the ability to view the flow, only seeing the glowing drop.  I 
tried stopping my instance via nifi.sh, but this seemed to fail and the 
background processes continued to run.  Forcibly quitting, I was able to start 
NiFi again and let it run for a while.  At startup, there were several errors 
concerning provenance, with several of the below exception:

{code}
015-10-27 12:33:36,905 ERROR [main] o.a.n.p.PersistentProvenanceRepository 
Failed to read Provenance Event File ./provenance_repository/43401826.prov.gz 
due to {}
java.io.EOFException: Unexpected end of ZLIB input stream
        at java.util.zip.InflaterInputStream.fill(InflaterInputStream.java:240) 
~[na:1.8.0_60]
        at java.util.zip.InflaterInputStream.read(InflaterInputStream.java:158) 
~[na:1.8.0_60]
        at java.util.zip.GZIPInputStream.read(GZIPInputStream.java:117) 
~[na:1.8.0_60]
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) 
~[na:1.8.0_60]
        at java.io.BufferedInputStream.read(BufferedInputStream.java:265) 
~[na:1.8.0_60]
        at 
org.apache.nifi.stream.io.ByteCountingInputStream.read(ByteCountingInputStream.java:41)
 ~[nifi-utils-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at java.io.DataInputStream.readUnsignedShort(DataInputStream.java:337) 
~[na:1.8.0_60]
        at java.io.DataInputStream.readUTF(DataInputStream.java:589) 
~[na:1.8.0_60]
        at java.io.DataInputStream.readUTF(DataInputStream.java:564) 
~[na:1.8.0_60]
        at 
org.apache.nifi.provenance.StandardRecordReader.<init>(StandardRecordReader.java:91)
 ~[nifi-persistent-provenance-repository-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.provenance.serialization.RecordReaders.newRecordReader(RecordReaders.java:106)
 ~[nifi-persistent-provenance-repository-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.provenance.PersistentProvenanceRepository.recover(PersistentProvenanceRepository.java:548)
 [nifi-persistent-provenance-repository-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.provenance.PersistentProvenanceRepository.initialize(PersistentProvenanceRepository.java:219)
 [nifi-persistent-provenance-repository-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.controller.FlowController.<init>(FlowController.java:406) 
[nifi-framework-core-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.controller.FlowController.createStandaloneInstance(FlowController.java:349)
 [nifi-framework-core-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.apache.nifi.spring.FlowControllerFactoryBean.getObject(FlowControllerFactoryBean.java:63)
 [nifi-framework-core-0.3.1-SNAPSHOT.jar:0.3.1-SNAPSHOT]
        at 
org.springframework.beans.factory.support.FactoryBeanRegistrySupport.doGetObjectFromFactoryBean(FactoryBeanRegistrySupport.java:168)
 [spring-beans-4.1.6.RELEASE.jar:4.1.6.RELEASE]
{code}

I tried a few times to recreate this, but could not get the same result as 
above.

Not sure if there is a way around it, but ultimately, the blocking on the 
operation and being able to access the UI, which could be confusing for other 
people trying to access a given instance.  Is it feasible to heartbeat from the 
user issuing the request to the background processing and causing the 
termination to occur if I should navigate away?  This wouldn't solve the issue 
of the blocking, but would at least prevent losing control of the process 
should I navigate away.

Thoughts on any of the above?


> Purge Queue from UI
> -------------------
>
>                 Key: NIFI-730
>                 URL: https://issues.apache.org/jira/browse/NIFI-730
>             Project: Apache NiFi
>          Issue Type: Improvement
>          Components: Core Framework, Core UI
>            Reporter: Brian Ghigiarelli
>            Priority: Minor
>             Fix For: 0.4.0
>
>
> Making changes to connections between NiFi processors is difficult when data 
> is queued up between those processors. A workaround to this data build up is 
> to stop both processors, reduce/set the FlowFile Expiration of the connection 
> to a low number (e.g., 2 sec), then start the receiving processor to age off 
> the data.
> A more user-friendly solution is to provide a "Purge" or "Delete" context 
> menu option on the queues to remove the data and/or age it off immediately.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to