nifi-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Uwe@Moosheimer.com" <...@Moosheimer.com>
Subject Re: [EXT] Re: New Standard Pattern - Put Exception that caused failure in an attribute
Date Sat, 27 Oct 2018 16:46:10 GMT
Do you really want to mix provenance and data lineage with logging/error information?

Writing exception information/logging information within an attribute is not a bad idea in
my opinion.
If a user wants to use this for routing, why not ... or whatever the user wants to do.

I could imagine that this can be switched on and off by a property via config. E.g. in development
on and on production off.

Regards,
Uwe

> Am 26.10.2018 um 09:26 schrieb Pierre Villard <pierre.villard.fr@gmail.com>:
> 
> Adding another option to the list.
> 
> Peter - if I understand correctly and based on my own experience, the idea
> is not to have an 'exception' attribute to perform custom routing after the
> failure relationship but rather have a more user friendly way to see what
> happened without going through all the logs for a given flow file.
> 
> If that's correct, then could we add this information somehow to the
> provenance event generated by the processor? Ideally adding a new field to
> a provenance event or using the existing 'details' field?
> 
> Pierre
> 
> 
> Le ven. 26 oct. 2018 à 08:40, Koji Kawamura <ijokarumawak@gmail.com> a
> écrit :
> 
>> Hi all,
>> 
>> I'd like to add another option to Matt's list of solutions:
>> 
>> 4) Add a processor property, 'Enable detailed error handling'
>> (defaults to false), then toggle available list of relationships. This
>> way, existing flows such as Peter's don't have to change, while he can
>> opt-in new relationships. RouteOnAttribute can be a reference
>> implementation.
>> 
>> I like the idea of thinking relationships as potential exceptions. It
>> can be better if relationships have hierarchy.
>> Some users need more granular relationships while others don't.
>> For NiFi 2.0 or later, supporting relationship hierarchy at framework
>> can mitigate having additional property at each processor.
>> 
>> Thanks,
>> Koji
>> On Fri, Oct 26, 2018 at 11:49 AM Matt Burgess <mattyb149@apache.org>
>> wrote:
>>> 
>>> Peter,
>>> 
>>> Totally agree, RDBMS/JDBC is in a weird class as always, there is a
>>> teaspoon of exception types for an ocean of causes. For NiFi 1.x, it
>>> seems like we need to pick from a set of less-than-ideal solutions:
>>> 
>>> 1) Add new relationships, but then your (possibly hundreds of)
>>> processors are invalid
>>> 2) Add new auto-terminated relationships, but then your
>>> previously-handled errors are "lost"
>>> 3) Add an attribute, but then each NiFi instance/release/flow is
>>> responsible for parsing the error and handling it as desired.
>>> 
>>> We could mitigate 1-2 with a tool that updates your flow/template by
>>> sending all new failure relationships to the same target as the
>>> existing one, but then the tool itself suffers from maintainability
>>> issues (as does option #3). If we could recognize that the new
>>> relationships are self-terminated and then send the errors out to the
>>> original failure relationship, that could be quite confusing to the
>>> user, especially as time goes on (how to suppress the "new" errors,
>>> e.g.).
>>> 
>>> IMHO I think we're between a rock and a hard place here, I guess with
>>> great entropy comes great responsibility :P
>>> 
>>> P.S. For your use case, is the workaround to just keep retrying? Or
>>> are there other constraints at play?
>>> 
>>> Regards,
>>> Matt
>>> 
>>> On Thu, Oct 25, 2018 at 10:27 PM Peter Wicks (pwicks) <pwicks@micron.com>
>> wrote:
>>>> 
>>>> Matt,
>>>> 
>>>> If I were to split an existing failure relationship into several
>> relationships, I do not think I would want to auto-terminate in most cases.
>> Specifically, I'm interested in a failure relationship for a database
>> disconnect during SQL execution (database was online when the connection
>> was verified in the DBCP pool, but went down during execution). If I were
>> to find a way to separate this into its own relationship, I do not think
>> most users would appreciate it being a condition silently not handled by
>> the normal failure path.
>>>> 
>>>> Thanks,
>>>>  Peter
>>>> 
>>>> -----Original Message-----
>>>> From: Matt Burgess [mailto:mattyb149@apache.org]
>>>> Sent: Friday, October 26, 2018 10:18 AM
>>>> To: dev@nifi.apache.org
>>>> Subject: Re: [EXT] Re: New Standard Pattern - Put Exception that
>> caused failure in an attribute
>>>> 
>>>> NiFi (as of the last couple releases I think) has the ability to set
>> auto-terminating relationships; this IMO is one of those use cases (for
>> NiFi 1.x). If new relationships are added, they could default to
>> auto-terminate; then the existing processors should remain valid.
>>>> However we might want an "omnibus Jira" to capture those relationships
>> we'd like to remove the auto-termination from in NiFi 2.0.
>>>> 
>>>> Regards,
>>>> Matt
>>>> On Thu, Oct 25, 2018 at 10:12 PM Peter Wicks (pwicks) <
>> pwicks@micron.com> wrote:
>>>>> 
>>>>> Mark,
>>>>> 
>>>>> I agree with you that this is the best option in general terms.
>> After thinking about it some more I think the biggest use case is for
>> troubleshooting. If a file routes to failure, you need to be watching the
>> UI to see what the exception was. An admin may have access to the NiFi log
>> files and could grep the error out, but a normal user who checks in on the
>> flow and sees a FlowFile in the error queue will not know what the cause
>> was; this is especially frustrating if retrying the file works without
>> failure the second time... Capturing the error message in an attribute
>> makes this easy to find.
>>>>> 
>>>>> One thing I worry about too is adding new relationships to core
>> processors. After an upgrade, won't users need to go to each instance of
>> that processor and handle the new relationship? Right now I'd swagger we
>> have at least five thousand ExecuteSQL processors in our environment; and
>> while we have strong scripting skills in my NiFi team, I would not want to
>> encounter this without that.
>>>>> 
>>>>> Thanks,
>>>>>  Peter
>>>>> 
>>>>> -----Original Message-----
>>>>> From: Mark Payne [mailto:markap14@hotmail.com]
>>>>> Sent: Thursday, October 25, 2018 10:38 PM
>>>>> To: dev@nifi.apache.org
>>>>> Subject: [EXT] Re: New Standard Pattern - Put Exception that caused
>>>>> failure in an attribute
>>>>> 
>>>>> I agree - the notion of adding a "failure.reason" attribute is, in
>> my opinion, an anti-pattern that should be avoided. Relationships are not a
>> workaround but rather the preferred approach in this scenario - an
>> attribute I would consider a workaround. This is due to the fact that not
>> only is it brittle and complex to add processors that route on such things,
>> but there's no reason at all to assume that from release to release (even
>> bug fix/increment releases) that the Exception type or message will be the
>> same, so the flow could stop working at any time after upgrading nifi.
>>>>> Relationships offer a well-defined way to explicitly indicate "these
>> are the possible outcomes,"
>>>>> similar IMO to Java Exception classes vs. throwing Strings in C.
>>>>> 
>>>>> 
>>>>>> On Oct 25, 2018, at 9:47 AM, Bryan Bende <bbende@gmail.com>
wrote:
>>>>>> 
>>>>>> I think processors should really have well defined relationships
>> for
>>>>>> the error scenarios that need to be handled. Having the exception
>>>>>> message is ok for a human who wants to see it, but in order to do
>>>>>> anything with it in the flow you will have to have a bunch of
>>>>>> parsing/interpreting of the message with a bunch of routing
>>>>>> processors, which seems more brittle than just having the
>>>>>> appropriate relationships.
>>>>>> On Thu, Oct 25, 2018 at 1:36 AM Peter Wicks (pwicks) <
>> pwicks@micron.com> wrote:
>>>>>>> 
>>>>>>> When a FlowFile is routed to failure, frequently there is no
>> clear reason without looking into the actual error message.
>>>>>>> Some processors work around this by creating many different
>> relationships, but even then frequently the generic Failure relationship
>> also provides little guidance.
>>>>>>> 
>>>>>>> I've seen a few cases recently where processors are including
the
>> exception message as an attribute on the FlowFile when routing to failure
>> (ExecuteStreamCommand, new PR for ExecuteSQL). Should this be a standard
>> pattern so that it's easier for users to route failures?
>>>>>>> 
>>>>>>> --Peter
>>>>> 
>> 


Mime
View raw message