Jira issues for adding exception handling in Java and Python SDKs:

https://issues.apache.org/jira/browse/BEAM-5638
https://issues.apache.org/jira/browse/BEAM-5639

I'll plan to have a complete PR for the Java SDK put together in the next
few days.

On Wed, Oct 3, 2018 at 1:29 PM Jeff Klukas <[email protected]> wrote:

> I don't personally have experience with the Python SDK, so am not
> immediately in a position to comment on how feasible it would be to
> introduce a similar change there. I'll plan to write up two separate issues
> for adding exception handling in the Java and Python SDKs.
>
> On Wed, Oct 3, 2018 at 12:17 PM Thomas Weise <[email protected]> wrote:
>
>> +1 for the proposal as well as the suggestion to offer it in other SDKs,
>> where applicable
>>
>> On Wed, Oct 3, 2018 at 8:58 AM Chamikara Jayalath <[email protected]>
>> wrote:
>>
>>> Sounds like a very good addition. I'd say this can be a single PR since
>>> changes are related. Please open a JIRA for tracking.
>>>
>>> Have you though about introducing a similar change to Python SDK ?
>>> (doesn't have to be the same PR).
>>>
>>> - Cham
>>>
>>> On Wed, Oct 3, 2018 at 8:31 AM Jeff Klukas <[email protected]> wrote:
>>>
>>>> If this looks good for MapElements, I agree that it makes sense to
>>>> extend to FlatMapElements and Filter and to keep the API consistent between
>>>> them.
>>>>
>>>> Do you have suggestions on how to submit changes with that wider scope?
>>>> Would one PR altering MapElements, FlatMapElements, Filter, ParseJsons, and
>>>> AsJsons be too large to reasonably review? Should I open an overall JIRA
>>>> ticket to track and break this into smaller  PRs?
>>>>
>>>> On Wed, Oct 3, 2018 at 10:31 AM Reuven Lax <[email protected]> wrote:
>>>>
>>>>> Sounds cool. Why not support this on other transforms as well?
>>>>> (FlatMapElements, Filter, etc.)
>>>>>
>>>>> Reuven
>>>>>
>>>>> On Tue, Oct 2, 2018 at 4:49 PM Jeff Klukas <[email protected]>
>>>>> wrote:
>>>>>
>>>>>> I've seen a few Beam users mention the need to handle errors in their
>>>>>> transforms by using a try/catch and routing to different outputs based on
>>>>>> whether an exception was thrown. This was particularly nicely written up 
>>>>>> in
>>>>>> a post by Vallery Lancey:
>>>>>>
>>>>>>
>>>>>> https://medium.com/@vallerylancey/error-handling-elements-in-apache-beam-pipelines-fffdea91af2a
>>>>>>
>>>>>> I'd love to see this pattern better supported directly in the Beam
>>>>>> API, because it currently requires the user to implement a full DoFn even
>>>>>> for the simplest cases.
>>>>>>
>>>>>> I propose we support for a MapElements-like transform that allows the
>>>>>> user to specify a set of exceptions to catch and route to a failure 
>>>>>> output.
>>>>>> Something like:
>>>>>>
>>>>>> MapElements
>>>>>> .via(myFunctionThatThrows)
>>>>>> .withSuccessTag(successTag)
>>>>>> .withFailureTag(failureTag, JsonParsingException.class)
>>>>>>
>>>>>> which would output a PCollectionTuple with both the successful
>>>>>> outcomes of the map operation and also a collection of the inputs that
>>>>>> threw JsonParsingException.
>>>>>>
>>>>>> To make this more concrete, I put together a proof of concept PR:
>>>>>> https://github.com/apache/beam/pull/6518  I'd appreciate feedback
>>>>>> about whether this seems like a worthwhile addition and a feasible 
>>>>>> approach.
>>>>>>
>>>>>

Reply via email to