It sounds like there is sufficient support for adding error-handling/DLQ
into Beam; that will be a great boost for users.

I'll start to take a look at formalizing some sort of BIP / Design doc, and
once that takes shape, will surface on list [ am out of my normal routine,
traveling for work, so that might take some time ... so anyone sufficiently
interested, do not feel blocked by me ].

I'm wondering about even intermixing (a) and (b).  Ex: short-run we could
look to define an API surface in the Beam Core API that we are happy with,
which *could* rely on asgarde [ in github.com/apache/beam-asgarde ] in the
short run.



On Wed, Sep 13, 2023 at 5:48 PM Alexey Romanenko <aromanenko....@gmail.com>
wrote:

> I agree with Cham on these two options.
>
> In the end, it would be great to have such functionality (error handling /
> DLQ) integrated into Beam core API, but it will require, for sure, some
> technical discussions and reviews before - so it will take more time.
>
> Though, to make it available for users soon as a part of Beam
> distribution, adding this as an extension looks very feasible for me.
>
> —
> Alexey
>
> On 12 Sep 2023, at 19:44, Chamikara Jayalath via dev <dev@beam.apache.org>
> wrote:
>
> Thanks Mazlum, this sounds great. I think there are two ways we can
> proceed if we decide to integrate the Asgarde library into Beam.
>
> (1) Directly import the code into Beam without significant modifications
> and/or a review (though we may add tests).
>
> (2) Go through a design/code review to determine whether this is the best
> approach for implementing error handling / DLQ in Beam transforms or
> whether there are other alternatives/modifications to Asgarde we want to
> consider.
>
> If we do (1) I prefer adding Asgarde as a separate Gradle module in Beam.
> We can later integrate it into the core module after a design/code review.
>
> Thank,
> Cham
>
>
>
> On Tue, Sep 12, 2023 at 10:26 AM Mazlum TOSUN <mazlum.to...@gmail.com>
> wrote:
>
>> Hello Austin and everyone,
>>
>> I am open for discussion.
>>
>> My first intention with Asgarde was to help the Beam community, because
>> Dead Letter Queue is so important in Beam and all the data pipeline
>> frameworks.
>> When I worked with Beam on production with my customers, we needed to
>> catch errors with side outputs and dead letter queue.
>>
>> This library really helped us to keep a less verbose code while applying
>> all the error handling logic, that is error prone and verbose if it is
>> repeated.
>>
>> As Kennet said, my intention was to stay as close as possible to Beam,
>> with a Wrapper and a Failure Monad on top of a PCollection, to handle all
>> the code and complexity for try catch blocks and side output.
>>
>> For the governance, even if I am the creator of this library, the most
>> important isn't me but the community and to help the community.
>> If the best solution to help the community is including the library
>> directly on Beam, we can go in this direction, with of course your reviews
>> and recommendations.
>>
>> Then the library will belong to the community and we will continue to
>> improve it.
>>
>> For the decision about the best place, I will comply with the majority.
>>
>> Best regards,
>>
>> Mazlum
>>
>> On Mon, Sep 11, 2023 at 11:15 PM Austin Bennett <aus...@apache.org>
>> wrote:
>>
>>> @Mazlum TOSUN <mazlum.to...@gmail.com> --  you and I have spoken a few
>>> times about this.  it'd be good for you to comment here on list, on any of
>>> your concerns with governance, and/or other thoughts.  Ex: if you think
>>> contributing asgarde directly is the thing [ or perhaps expressing any
>>> interest helping write/contribute the relevant functionality into beam ...
>>> it is possible that by adding the actual functionality into beam - like
>>> Kenn's mentioned 'other place' we could make asgarde as an separate add-on
>>> obsolete ].
>>>
>>>
>>>
>>> On Fri, Sep 8, 2023 at 8:55 AM Kenneth Knowles <k...@apache.org> wrote:
>>>
>>>> For anyone who hasn't clicked over the Asgarde, my TL;DR description of
>>>> it is that it adds the "failure monad" aka "andThen" style error/result
>>>> handling on top of chaining of PCollections. So it is at a similar level of
>>>> abstraction of our basic transforms and generally useful for chaining
>>>> dead-letter side outputs. It is no more or less appropriate for the core
>>>> SDK than, say, the Project/Filter/Join transforms, or Watch, etc. If we
>>>> actually aspired to have a thin core with the accessories like that in
>>>> another place, then it should go to that other place.
>>>>
>>>> Kenn
>>>>
>>>> On Fri, Sep 8, 2023 at 11:24 AM Daniel Collins via dev <
>>>> dev@beam.apache.org> wrote:
>>>>
>>>>> > until we *require* Asgard on a core transform, it shouldn't be in
>>>>> the main repo
>>>>>
>>>>> I don't think this is necessarily true if it solves end user use
>>>>> cases. If there is a specific transform that solves a specific use case, 
>>>>> we
>>>>> could include it in the transforms folder for end-users, even if it isn't
>>>>> utilized in the I/Os at present. Hence the suggestion to take the most
>>>>> promising transforms and propose adding them with documentation, apis and
>>>>> rationale.
>>>>>
>>>>> -Daniel
>>>>>
>>>>> On Fri, Sep 8, 2023 at 11:20 AM Robert Burke <rob...@frantil.com>
>>>>> wrote:
>>>>>
>>>>>> I would say until we *require* Asgard on a core transform, it
>>>>>> shouldn't be in the main repo.
>>>>>>
>>>>>> Incorporating something before there's a need for it is premature
>>>>>> abstraction. We can't do things because they *might* be useful. Let's see
>>>>>> concrete places where they are useful, or we're already having a similar
>>>>>> need solved a different way.
>>>>>>
>>>>>> Beam is complicated by itself, and we do encourage multiple ways of
>>>>>> solving problems, but that says to me that having an out of repo 
>>>>>> ecosystem
>>>>>> is the right path, rather than incorporation.
>>>>>>
>>>>>> On Fri, Sep 8, 2023, 8:14 AM Daniel Collins via dev <
>>>>>> dev@beam.apache.org> wrote:
>>>>>>
>>>>>>> I think there are a lot of interesting and relatively isolated
>>>>>>> components of the project, it might make sense to write per-transform 
>>>>>>> one
>>>>>>> pagers for isolated things like the most useful pieces (just basically
>>>>>>> copying the documentation and justifying the API) instead of doing a
>>>>>>> one-shot import or having it live forever in an external project.
>>>>>>>
>>>>>>> -Daniel
>>>>>>>
>>>>>>> On Fri, Sep 8, 2023 at 11:10 AM Kenneth Knowles <k...@apache.org>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> I agree with everyone about "not everything has to be in the Beam
>>>>>>>> repo". I really like the idea of having a clearer "ecosystem" section 
>>>>>>>> of
>>>>>>>> the website, which is sort of started at
>>>>>>>> https://beam.apache.org/community/integrations/ but that is not
>>>>>>>> very prominent.
>>>>>>>>
>>>>>>>> Agree with John though. The transforms in Asgarde could potentially
>>>>>>>> be used in Beam. Potentially best accomplished by just adding them as
>>>>>>>> transforms to the core Java SDK?
>>>>>>>>
>>>>>>>> Kenn
>>>>>>>>
>>>>>>>> On Wed, Sep 6, 2023 at 1:46 PM John Casey via dev <
>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>
>>>>>>>>> Agreed on documentation and on keeping it in a separate repo.
>>>>>>>>>
>>>>>>>>> We have a few pretty significant beam extensions (scio and
>>>>>>>>> Dataflow Templates also come to mind) that Beam should highlight, but 
>>>>>>>>> are
>>>>>>>>> separate repos for their own governance, contributions, and release 
>>>>>>>>> reasons.
>>>>>>>>>
>>>>>>>>> The difference with Asgarde is that we might want to use it in
>>>>>>>>> Beam itself, which makes it more reasonable to include in the main 
>>>>>>>>> repo.
>>>>>>>>>
>>>>>>>>> On Tue, Sep 5, 2023 at 8:36 PM Robert Bradshaw via dev <
>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> I think this is a great library. I'm on the fence of whether it
>>>>>>>>>> makes sense to include with Beam proper vs. be a library that builds 
>>>>>>>>>> on top
>>>>>>>>>> of Beam. (Would there be benefits of tighter integration? There is 
>>>>>>>>>> the
>>>>>>>>>> maintenance/loss of governance issue.) I am definitely not on the 
>>>>>>>>>> side that
>>>>>>>>>> the entire Beam ecosystem needs to be distributed/maintained by Beam
>>>>>>>>>> itself.
>>>>>>>>>>
>>>>>>>>>> Regardless of the direction we go, I think it could make a lot of
>>>>>>>>>> sense to put pointers to it in our documentation.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Tue, Sep 5, 2023 at 7:21 AM Danny McCormick via dev <
>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>
>>>>>>>>>>> I think my only concerns here are around the toil we'll be
>>>>>>>>>>> taking on, and will we be leaving the asgarde project in a better 
>>>>>>>>>>> or worse
>>>>>>>>>>> place.
>>>>>>>>>>>
>>>>>>>>>>> From a release standpoint, we would need to release it with the
>>>>>>>>>>> same cadence as Beam. Adding asgarde into our standard release 
>>>>>>>>>>> process
>>>>>>>>>>> seems fairly straightforward, though, so I'm not too worried about 
>>>>>>>>>>> it -
>>>>>>>>>>> looks like it's basically (1) add a commit like this
>>>>>>>>>>> <https://github.com/tosun-si/asgarde/commit/432de527d67dc71f06507328319b466b6d0fb56a>,
>>>>>>>>>>> (2) run this workflow
>>>>>>>>>>> <https://github.com/tosun-si/asgarde/blob/main/.github/workflows/publish-project.yml>,
>>>>>>>>>>> and (3) tag/mark the release as released on GitHub.
>>>>>>>>>>>
>>>>>>>>>>> In terms of bug fixes and improvements, though, I'm a little
>>>>>>>>>>> worried that we might be leaving things in a worse state since 
>>>>>>>>>>> Mazlum has
>>>>>>>>>>> been the only contributor thus far, and he would lose some 
>>>>>>>>>>> governance (and
>>>>>>>>>>> possibly the ability to commit code on his own). An extra motivated
>>>>>>>>>>> community member or two could change the math a bit, but I'm not 
>>>>>>>>>>> sure if
>>>>>>>>>>> there are actually clear advantages to including it in Apache other 
>>>>>>>>>>> than
>>>>>>>>>>> visibility. Would adding links to our docs calling Asgarde out as 
>>>>>>>>>>> an option
>>>>>>>>>>> accomplish the same purpose?
>>>>>>>>>>>
>>>>>>>>>>> > Let's be careful about whether these tests are included in our
>>>>>>>>>>> presubmits. Contrib code with flaky tests has been a major pain 
>>>>>>>>>>> point in
>>>>>>>>>>> the past.
>>>>>>>>>>>
>>>>>>>>>>> +1 - I think if we do this I'd vote that it be in a separate
>>>>>>>>>>> repo (github.com/apache/beam-asgarde made sense to me).
>>>>>>>>>>>
>>>>>>>>>>> ---------------------------------------
>>>>>>>>>>>
>>>>>>>>>>> Overall, I'm probably a slight -1 to adding this to the Apache
>>>>>>>>>>> workspace, but +1 to at least adding links from the Beam docs to 
>>>>>>>>>>> Asgarde.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Danny
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Tue, Sep 5, 2023 at 12:03 AM Reuven Lax via dev <
>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Let's be careful about whether these tests are included in our
>>>>>>>>>>>> presubmits. Contrib code with flaky tests has been a major pain 
>>>>>>>>>>>> point in
>>>>>>>>>>>> the past.
>>>>>>>>>>>>
>>>>>>>>>>>> On Sat, Sep 2, 2023 at 12:02 PM Austin Bennett <
>>>>>>>>>>>> aus...@apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Wanting us to not miss this. @Mazlum TOSUN
>>>>>>>>>>>>> <mazlum.to...@gmail.com> is happy to donate Asgarde to
>>>>>>>>>>>>> our project.
>>>>>>>>>>>>>
>>>>>>>>>>>>> It looks like he'd need a SGA and CCLA [ 1 ] on file; anything
>>>>>>>>>>>>> else?
>>>>>>>>>>>>>
>>>>>>>>>>>>> I recalled the donation of Euphoria [ 2 ] , so I looked at
>>>>>>>>>>>>> those threads [ 3 ]  for insights into the process.  It didn't 
>>>>>>>>>>>>> look like
>>>>>>>>>>>>> there was a needed VOTE, so mostly a matter of ensuring necessary
>>>>>>>>>>>>> signatures, and ideally some sort of consensus [ or 
>>>>>>>>>>>>> non-opposition ] to the
>>>>>>>>>>>>> donation.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> [ 1 ]
>>>>>>>>>>>>> https://www.apache.org/licenses/contributor-agreements.html
>>>>>>>>>>>>> [ 2 ]
>>>>>>>>>>>>> https://beam.apache.org/documentation/sdks/java/euphoria/
>>>>>>>>>>>>> [ 3 ]
>>>>>>>>>>>>> https://lists.apache.org/thread/xzlx4rm2tvc36mmwvhyvtdvsw7bnjscp
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Jun 15, 2023 at 7:05 AM Kerry Donny-Clark via dev <
>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> This looks like an excellent contribution. I can easily
>>>>>>>>>>>>>> understand the motivation, and I think Beam would benefit from a 
>>>>>>>>>>>>>> higher
>>>>>>>>>>>>>> level abstraction for error handling.
>>>>>>>>>>>>>> Kerry
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Jun 14, 2023, 6:31 PM Austin Bennett <
>>>>>>>>>>>>>> aus...@apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hi Beam Devs,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> @Mazlum <https://www.linkedin.com/in/mazlum-tosun-900b1812/>
>>>>>>>>>>>>>>> was suggested to consider donating Asgarde
>>>>>>>>>>>>>>> <https://github.com/tosun-si/asgarde> to Beam for
>>>>>>>>>>>>>>> Java/Kotlin error handling to Beam [ see:
>>>>>>>>>>>>>>> https://2022.beamsummit.org/sessions/error-handling-asgarde/
>>>>>>>>>>>>>>> for last year's Beam Summit talk ], he is also the author of
>>>>>>>>>>>>>>> Pasgard <https://github.com/tosun-si/pasgarde>e [ for
>>>>>>>>>>>>>>> Python ] and Milgard [ for a simplified Kotlin API ].
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Would Asgarde be a good contribution, something the Beam
>>>>>>>>>>>>>>> community would be willing to accept?  I imagine we might want 
>>>>>>>>>>>>>>> it to live
>>>>>>>>>>>>>>> at github.com/apache/beam-asgarde ?  Or perhaps there is a
>>>>>>>>>>>>>>> good place in github.com/apache/beam ??
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Especially once/if officially part of Beam, I imagine we'd
>>>>>>>>>>>>>>> add follow-up items like getting onto the website/docs, and 
>>>>>>>>>>>>>>> related.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Cheers,
>>>>>>>>>>>>>>> Austin
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> P.S.  This might warrant separate/additional conversations
>>>>>>>>>>>>>>> for his other libraries, but let's focus any discussion on 
>>>>>>>>>>>>>>> Asgarde for
>>>>>>>>>>>>>>> now?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>

Reply via email to