That's a good point, Mark. I also agree that it's better to give the user
control whenever possible. I imagine the RouteOnAttribute pattern to
eventually "give up" on a FlowFile will be a common pattern, and so so we
should account for that, rather than forcing the user into knowing this
pattern.

On Thu, Jan 28, 2016 at 2:11 PM, Mark Payne <[email protected]> wrote:

>
> The retry idea concerns me a bit. If we were to have a method like:
>
> penalizeOrTransfer(FlowFile flowFile, int numberOfTries, Relationship
> relationship)
>
> I think that leaves out some info - even if a FlowFile is
> penalized, it must be penalized and sent somewhere. So there would have to
> be
> a relationship to send it to if penalized and another to send it to if not
> penalizing.
> This also I think puts more onus on the developer to understand how it
> would be
> used - I believe the user should be making decisions about how many times
> to
> penalize, not the developer.
>
> > On Jan 28, 2016, at 2:03 PM, Bryan Bende <[email protected]> wrote:
> >
> > Regarding throwing an exception... I believe if you are extending
> > AbstractProcessor and an exception is thrown out of onTrigger() then the
> > session is rolled back and any flow files that were accessed are
> penalized,
> > which results in leaving them in the incoming connection to the processor
> > and not being retried until the penalty duration passes. This seems
> similar
> > to what Michael described, although it is not stopping the processor from
> > processing other incoming  flow files.
> >
> > Ricky's retry idea sounds interesting... I think a lot of people handle
> > this today by creating a retry loop using UpdateAttribute and
> > RouteOnAttribute [1].
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/download/attachments/57904847/Retry_Count_Loop.xml?version=1&modificationDate=1433271239000&api=v2
> >
> >
> > On Thu, Jan 28, 2016 at 1:24 PM, Ricky Saltzer <[email protected]>
> wrote:
> >
> >> Is there currently a way to know how many times a FlowFile has been
> >> penalized? Do we have use cases where we want to penalize a FlowFile *n
> >> *number
> >> of times before sending it down an alternate relationship? I could
> imagine
> >> an API like penalizeOrTransfer(FlowFile flowFile, int numberOfTries,
> >> Relationship relationship). For example, someone might want to process a
> >> FlowFile three times before giving up on it.
> >>
> >> On Thu, Jan 28, 2016 at 12:47 PM, Michael de Courci <
> >> [email protected]> wrote:
> >>
> >>> Matt thanks for your reply
> >>>
> >>> I guess what I am saying in that case - if there is an error in a
> >>> FlowFile, then the processor that detects this cannot proceed so
> instead
> >> of
> >>> calling an action to penalize the FlowFile it raises an exception
> >>> OutOFServiceException or ProcessorException.
> >>> You could have an exception cause PeanilisedFlowFileException for this
> >>> case.
> >>>
> >>> But within the processor other error causes may arise for an
> >>> OutOFServiceException
> >>>
> >>> The point is that if the processor threw this exception then there can
> be
> >>> a duration configuration - a time limit to keep this processor out of
> >>> service and the connection to it and possibly any processors leading
> upto
> >>> it - Naturally this will need to be indicated on the DFM - this will
> free
> >>> resources and make the flow well behaved.
> >>>
> >>> Environmental failures will simply be a different category/cause of
> error
> >>> that can be wrapped/captured also with a more general one
> >>>
> >>> With Kind Regards
> >>> Michael de Courci
> >>> [email protected]
> >>>
> >>>
> >>>
> >>>
> >>>> On 28 Jan 2016, at 17:16, Matt Gilman <[email protected]>
> wrote:
> >>>>
> >>>> Just to recap/level set...
> >>>>
> >>>> The distinct between yielding and penalization is important.
> >> Penalization
> >>>> is an action taken on a FlowFile because the FlowFile cannot be
> >> processed
> >>>> right now (like a naming conflict for instance). The Processor is
> >>>> indicating that it cannot process that specific FlowFile at the moment
> >>> but
> >>>> may be able to process the next. Yielding is an indication that the
> >>>> Processor is unable to work at all at the moment likely due to an
> >>>> environmental issue (like the out of service comment).
> >>>>
> >>>> If the concept of penalization were moved to a connection, does it
> >>>> automatically penalize all FlowFile transferred to it? We would lose
> >> some
> >>>> granularity if a Processor wanted to penalize some FlowFile routed to
> a
> >>>> given Relationship but not others. I'm not sure if this is done in
> >>> practice
> >>>> or not, just wanted to mention it.
> >>>>
> >>>> Outside of this minor concern, I like the idea. I especially like that
> >> it
> >>>> would help with the consistency of Processor behavior and transparency
> >>>> about what the data flow is actually doing.
> >>>>
> >>>> Matt
> >>>>
> >>>>
> >>>> On Thu, Jan 28, 2016 at 12:00 PM, Michael de Courci <
> >>>> [email protected]> wrote:
> >>>>
> >>>>> Hi
> >>>>> I think it would be better/simpler to have one “out of service”
> >> concept
> >>>>> to replace penalizing and yielding and when a plugin throws an
> >> exception
> >>>>> then the plugin is deemed out of service, for a duration and so the
> >>>>> connection to that plugin is disabled for the out of service
> duration.
> >>>>>
> >>>>> When a plugin is out of service and the connection disabled - then
> >>>>> resources that it uses will be freed(yielded).
> >>>>>
> >>>>> The question then is what the behaviour of the plugin before the
> >>> disabled
> >>>>> connection - should be.  My thought is to tend towards stability and
> >>> make
> >>>>> sure resources are freed, so there may need to be a “domino
> >>> effect”/cascade
> >>>>> affect where all plugins before are gradually put out of service.
> >>>>>
> >>>>>
> >>>>> With Kind Regards
> >>>>> Michael de Courci
> >>>>> [email protected]
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>>> On 28 Jan 2016, at 16:34, Mark Payne <[email protected]> wrote:
> >>>>>>
> >>>>>> All,
> >>>>>>
> >>>>>> I've been thinking about how we handle the concept of penalizing
> >>>>> FlowFiles. We've had a lot of questions
> >>>>>> lately about how penalization works & the concept in general. Seems
> >> the
> >>>>> following problems exist:
> >>>>>>
> >>>>>> - Confusion about difference between penalization & yielding
> >>>>>> - DFM sees option to configure penalization period on all
> processors,
> >>>>> even if they don't penalize FlowFiles.
> >>>>>> - DFM cannot set penalty duration in 1 case and set a different
> value
> >>>>> for a different case (different relationship, for example).
> >>>>>> - Developers often forget to call penalize()
> >>>>>> - Developer has to determine whether or not to penalize when
> >> building a
> >>>>> processor. It is based on what the developer will
> >>>>>> think may make sense, but in reality DFM's sometimes want to
> penalize
> >>>>> things when the processor doesn't behave that way.
> >>>>>>
> >>>>>> I'm wondering if it doesn't make sense to remove the concept of
> >>>>> penalization all together from Processors and instead
> >>>>>> move the Penalty Duration so that it's a setting on the Connection.
> I
> >>>>> think this would clear up the confusion and give the DFM
> >>>>>> more control over when/how long to penalize. Could set to the
> default
> >>> to
> >>>>> 30 seconds for self-looping connections and no penalization
> >>>>>> for other connections.
> >>>>>>
> >>>>>> Any thoughts?
> >>>>>>
> >>>>>> Thanks
> >>>>>> -Mark
> >>>>>
> >>>>>
> >>>
> >>>
> >>
> >>
> >> --
> >> Ricky Saltzer
> >> http://www.cloudera.com
> >>
>
>


-- 
Ricky Saltzer
http://www.cloudera.com

Reply via email to