use case similar to Wait/Notify

2017-07-10 Thread Yuri Krysko
Good morning Everyone,

I am trying to see if I could make my scenario work with Wait/Notify 
processors. Basically, I require the following implementation:

  1.  Pass a single flow file into a processor group
  2.  Keep subsequent all flow flies in a wait queue until the first flow file 
is processed and sent out of the processor group to Notify processor, which 
updates the cache.

Would I be able to accomplish this scenario without writing a custom processor?

Thanks much,
Yuri



LEGAL DISCLAIMER: M.C. Dean, Inc. and its subsidiaries considers this e-mail 
and any files transmitted with it to be protected, proprietary or privileged 
information intended solely for the use of the named recipient(s). Any 
disclosure of this material or the information contained herein, in whole or in 
part, to anyone outside of the intended recipient or affiliates is strictly 
prohibited. M. C. Dean, Inc. accepts no liability for the content of this 
e-mail or for the consequences of any actions taken on the basis of the 
information contained in it, unless that information is subsequently confirmed 
in writing. Employees of M.C. Dean, Inc. are instructed not to infringe on any 
rights of the recipient; any such communication violates company policy. If you 
are not the intended recipient, any disclosure, copying, distribution, or 
action taken or omitted in reliance on this information is strictly prohibited 
by M.C. Dean, Inc.; please notify the sender immediately by return e-mail, 
delete this communication and destroy all copies.


Re: FlowFile position when transferred to Relationship.SELF

2017-07-10 Thread 尹文才
Hi Koji, thanks for the explanation, I checked the NIFI documentation you
provided, do you mean I should use the FIFO prioritizer in my case? Because
as you mentioned the FlowFiles would be put back into their original
positions, so as I  understand using FIFO should make the FlowFiles in
consistent order.

Regards,
Ben

2017-07-10 17:06 GMT+08:00 Koji Kawamura :

> Hi,
>
> I think it puts back a FlowFile to its original position but update
> queued date as implemented here:
> https://github.com/apache/nifi/blob/master/nifi-nar-
> bundles/nifi-framework-bundle/nifi-framework/nifi-framework-
> core/src/main/java/org/apache/nifi/controller/repository/
> StandardProcessSession.java#L1851
>
> In order to pull FlowFiles from a queue in consistent order, you need
> to specify a prioritizer.
> https://nifi.apache.org/docs/nifi-docs/html/user-guide.html#prioritization
>
> I'm just curious about the functionality you added. Wait processor has
> 'Releasable FlowFile Count' and it could be used to make a batch of
> FlowFiles wait and go. Or Notify's 'Signal Counter Delta' could be
> useful, too.
>
> Regards,
> Koji
>
> On Mon, Jul 10, 2017 at 4:43 PM, 尹文才  wrote:
> > Hi guys, I have written a customized processor whose functionality is
> > similar to the NIFI's Wait processor, the difference is my processor
> needs
> > to wait a batch of data and when the batch end flag is found, it will
> > transfer the batch of data to destinations.
> >
> > I checked the source code of Wait processor and also transferred the
> > flowfiles to Relationship.SELF which is the incoming queue when the batch
> > of data is not yet complete. The problem I found was sometimes I could
> see
> > the sequence of the FlowFiles transferred from my processor to
> destinations
> > were not in order.
> > I then added sequence attribute(number starting from 1) to all FlowFiles
> > coming into my processor and I could verify that this problem happen from
> > time to time, but I couldn't find the stable way to reproduce it.
> >
> > My question is how does NIFI handle the FlowFile when it's being
> > transferred to Relationship.SELF, does it put back to its original
> position
> > in the incoming queue? Thanks.
> >
> > Regards,
> > Ben
>


Re: FlowFile position when transferred to Relationship.SELF

2017-07-10 Thread Koji Kawamura
Hi,

I think it puts back a FlowFile to its original position but update
queued date as implemented here:
https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java#L1851

In order to pull FlowFiles from a queue in consistent order, you need
to specify a prioritizer.
https://nifi.apache.org/docs/nifi-docs/html/user-guide.html#prioritization

I'm just curious about the functionality you added. Wait processor has
'Releasable FlowFile Count' and it could be used to make a batch of
FlowFiles wait and go. Or Notify's 'Signal Counter Delta' could be
useful, too.

Regards,
Koji

On Mon, Jul 10, 2017 at 4:43 PM, 尹文才  wrote:
> Hi guys, I have written a customized processor whose functionality is
> similar to the NIFI's Wait processor, the difference is my processor needs
> to wait a batch of data and when the batch end flag is found, it will
> transfer the batch of data to destinations.
>
> I checked the source code of Wait processor and also transferred the
> flowfiles to Relationship.SELF which is the incoming queue when the batch
> of data is not yet complete. The problem I found was sometimes I could see
> the sequence of the FlowFiles transferred from my processor to destinations
> were not in order.
> I then added sequence attribute(number starting from 1) to all FlowFiles
> coming into my processor and I could verify that this problem happen from
> time to time, but I couldn't find the stable way to reproduce it.
>
> My question is how does NIFI handle the FlowFile when it's being
> transferred to Relationship.SELF, does it put back to its original position
> in the incoming queue? Thanks.
>
> Regards,
> Ben


FlowFile position when transferred to Relationship.SELF

2017-07-10 Thread 尹文才
Hi guys, I have written a customized processor whose functionality is
similar to the NIFI's Wait processor, the difference is my processor needs
to wait a batch of data and when the batch end flag is found, it will
transfer the batch of data to destinations.

I checked the source code of Wait processor and also transferred the
flowfiles to Relationship.SELF which is the incoming queue when the batch
of data is not yet complete. The problem I found was sometimes I could see
the sequence of the FlowFiles transferred from my processor to destinations
were not in order.
I then added sequence attribute(number starting from 1) to all FlowFiles
coming into my processor and I could verify that this problem happen from
time to time, but I couldn't find the stable way to reproduce it.

My question is how does NIFI handle the FlowFile when it's being
transferred to Relationship.SELF, does it put back to its original position
in the incoming queue? Thanks.

Regards,
Ben