You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@beam.apache.org by Chamikara Jayalath via user <us...@beam.apache.org> on 2023/08/17 23:28:19 UTC

Re: [Request for Feedback] Swift SDK Prototype

Thanks Byron. This sounds great. I wonder if there is interest in Swift SDK
from folks currently subscribed to the +user <us...@beam.apache.org> list.

On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
wrote:

> Hello everyone,
>
> A couple of months ago I decided that I wanted to really understand how
> the Beam FnApi works and how it interacts with the Portable Runner. For me
> at least that usually means I need to write some code so I can see things
> happening in a debugger and to really prove to myself I understood what was
> going on I decided I couldn't use an existing SDK language to do it since
> there would be the temptation to read some code and convince myself that I
> actually understood what was going on.
>
> One thing led to another and it turns out that to get a minimal FnApi
> integration going you end up writing a fair bit of an SDK. So I decided to
> take things to a point where I had an SDK that could execute a word count
> example via a portable runner backend. I've now reached that point and
> would like to submit my prototype SDK to the list for feedback.
>
> It's currently living in a branch on my fork here:
>
> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>
> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
> Intel Macs, but should also work using beta builds of 5.9 for Linux running
> on Intel hardware. I haven't had a chance to try it on ARM hardware and
> make sure all of the endian checks are complete. The
> "IntegrationTests.swift" file contains a word count example that reads some
> local files (as well as a missing file to exercise DLQ functionality) and
> output counts through two separate group by operations to get it past the
> "map reduce" size of pipeline. I've tested it against the Python Portable
> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
> time.
>
> I've shown it to a couple of folks already and incorporated some of that
> feedback already (for example pardo was originally called dofn when
> defining pipelines). In general I've tried to make the API as "Swift-y" as
> possible, hence the heavy reliance on closures and while there aren't yet
> composite PTransforms there's the beginnings of what would be needed for a
> SwiftUI-like declarative API for creating them.
>
> There are of course a ton of missing bits still to be implemented, like
> counters, metrics, windowing, state, timers, etc.
>

This should be fine and we can get the code documented without these
features. I think support for composites and adding an external transform
(see, Java
<https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
Python
<https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
Go
<https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
TypeScript
<https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
to add support for multi-lang will bring in a lot of features (for example,
I/O connectors) for free.


>
> Any and all feedback welcome and happy to submit a PR if folks are
> interested, though the "Swift Way" would be to have it in its own repo so
> that it can easily be used from the Swift Package Manager.
>

+1 for creating a PR (may be as a draft initially). Also it'll be easier to
comment on a PR :)

- Cham

[1]
[2]
[3]


>
> Best,
> B
>
>
>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Thanks Austin, glad to hear the commentary isn’t just irritating. 🙂

On Sat, Aug 26, 2023 at 2:29 PM Austin Bennett <au...@apache.org> wrote:

> This is great that is coming together, and am glad for the messages along
> the way to understand process, choices, ...!
>
>
>
> On Fri, Aug 25, 2023, 2:04 PM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> Okay, after a brief detour through "get this working in the Flink
>> Portable Runner" I think I have something pretty workable.
>>
>> PInput and POutput can actually be structs rather than protocols, which
>> simplifies things quite a bit. It also allows us to use them with property
>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>> (which is what I was originally intending to use them for). That also means
>> the function signature you use for closures would match full-fledged DoFn
>> definitions for the most part which is satisfying.
>>
>>
>>
>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Okay, I tried a couple of different things.
>>>
>>> Implicitly passing the timestamp and window during iteration did not go
>>> well. While physically possible it introduces an invisible side effect into
>>> loop iteration which confused me when I tried to use it and I implemented
>>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>>> nightmare continuing down that path.
>>>
>>> What I decided to do instead was the following:
>>>
>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>> they always emit a window and timestamp along with their value. This
>>> eliminates the side effect but lets us keep iteration in a bundle where
>>> that might be convenient. For example, in my cheesy GCS implementation it
>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>> as a local variable, which is convenient. It's a bit more typing for users
>>> of pstream, but the expectation here is that if you're using pstream
>>> functions You Know What You Are Doing and most people won't be using it
>>> directly.
>>>
>>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>>> but enough to test the functionality and decide I liked it) which take a
>>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>> struct with friendlier names. Not strictly necessary, but makes the code
>>> nicer to read I think. POutput introduces emit functions that optionally
>>> allow you to specify a timestamp and a window. If you don't for either one
>>> it will take the timestamp and/or window of the input.
>>>
>>> Trying to use that was pretty pleasant to use so I think we should
>>> continue down that path. If you'd like to see it in use, I reimplemented
>>> map() and flatMap() in terms of this new pardo functionality.
>>>
>>> Code has been pushed to the branch/PR if you're interested in taking a
>>> look.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Gotcha, I think there's a fairly easy solution to link input and output
>>>> streams.... Let me try it out... might even be possible to have both
>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>> use @DoFn as a macro)
>>>>
>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>> work, as I think it's more natural overall.
>>>>>>>
>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>
>>>>>>
>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>> would be a valid output of that bundle.
>>>>>>
>>>>>>>
>>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>>> The question is how to do the input_element:output_element correlation for
>>>>> automatically propagating metadata.
>>>>>
>>>>>
>>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>> the easiest/most natural.)
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> Ah, I see.
>>>>>>>>>
>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>> something more explicit).
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>
>>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>>
>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>
>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>   ...
>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>
>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>> it somehow.
>>>>>>>>>>>
>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>> it yet.
>>>>>>>>>>>
>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>
>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>> work with GCS.
>>>>>>>>>>>
>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>>> there.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Neat.
>>>>>>>>>>>>
>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>
>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Austin Bennett <au...@apache.org>.
This is great that is coming together, and am glad for the messages along
the way to understand process, choices, ...!



On Fri, Aug 25, 2023, 2:04 PM Byron Ellis via user <us...@beam.apache.org>
wrote:

> Okay, after a brief detour through "get this working in the Flink Portable
> Runner" I think I have something pretty workable.
>
> PInput and POutput can actually be structs rather than protocols, which
> simplifies things quite a bit. It also allows us to use them with property
> wrappers for a SwiftUI-like experience if we want when defining DoFns
> (which is what I was originally intending to use them for). That also means
> the function signature you use for closures would match full-fledged DoFn
> definitions for the most part which is satisfying.
>
>
>
> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com> wrote:
>
>> Okay, I tried a couple of different things.
>>
>> Implicitly passing the timestamp and window during iteration did not go
>> well. While physically possible it introduces an invisible side effect into
>> loop iteration which confused me when I tried to use it and I implemented
>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>> nightmare continuing down that path.
>>
>> What I decided to do instead was the following:
>>
>> 1. Rename the existing "pardo" functions to "pstream" and require that
>> they always emit a window and timestamp along with their value. This
>> eliminates the side effect but lets us keep iteration in a bundle where
>> that might be convenient. For example, in my cheesy GCS implementation it
>> means that I can keep an OAuth token around for the lifetime of the bundle
>> as a local variable, which is convenient. It's a bit more typing for users
>> of pstream, but the expectation here is that if you're using pstream
>> functions You Know What You Are Doing and most people won't be using it
>> directly.
>>
>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>> but enough to test the functionality and decide I liked it) which take a
>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>> struct with friendlier names. Not strictly necessary, but makes the code
>> nicer to read I think. POutput introduces emit functions that optionally
>> allow you to specify a timestamp and a window. If you don't for either one
>> it will take the timestamp and/or window of the input.
>>
>> Trying to use that was pretty pleasant to use so I think we should
>> continue down that path. If you'd like to see it in use, I reimplemented
>> map() and flatMap() in terms of this new pardo functionality.
>>
>> Code has been pushed to the branch/PR if you're interested in taking a
>> look.
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Gotcha, I think there's a fairly easy solution to link input and output
>>> streams.... Let me try it out... might even be possible to have both
>>> element and stream-wise closure pardos. Definitely possible to have that at
>>> the DoFn level (called SerializableFn in the SDK because I want to
>>> use @DoFn as a macro)
>>>
>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>> work, as I think it's more natural overall.
>>>>>>
>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>> one to emit timestamp-windowed-values.
>>>>>>
>>>>>
>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>> to the current worker) so any output from the current bundle invocation
>>>>> would be a valid output of that bundle.
>>>>>
>>>>>>
>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>> The question is how to do the input_element:output_element correlation for
>>>> automatically propagating metadata.
>>>>
>>>>
>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>> the easiest/most natural.)
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>>
>>>>>>>> Ah, I see.
>>>>>>>>
>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>> something more explicit).
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>
>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>
>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>
>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>   ...
>>>>>>>>>>   output.emit(data)
>>>>>>>>>>
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>>>> October timeframe if history is any guide)
>>>>>>>>>>
>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>> it somehow.
>>>>>>>>>>
>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>> it yet.
>>>>>>>>>>
>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>
>>>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>> work with GCS.
>>>>>>>>>>
>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>> there.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Neat.
>>>>>>>>>>>
>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>> other SDKs.
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>
>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>
>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Chamikara Jayalath via user <us...@beam.apache.org>.
On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <da...@google.com>
wrote:

> > I think the process should be similar to other code/design reviews for
> large contributions. I don't think you need a PMC involvement here.
>
> I think it does require PMC involvement to create the actual repo once we
> have public consensus. I tried the flow at
> https://infra.apache.org/version-control.html#create but it seems like
> its PMC only. It's unclear to me if consensus has been achieved, maybe a
> dedicated voting thread with implied lazy consensus would help here.
>

Yeah, it seems like a PMC member needs to create the repo.


>
> > Sure, we could definitely include things as a submodule for stuff like
> testing multi-language, though I think there's actually a cleaner way just
> using the Swift package manager's test facilities to access the swift sdk
> repo.
>
> +1 on avoiding submodules. If needed we could also use multi-repo checkout
> with GitHub Actions. I think my biggest question is what we'd actually be
> enforcing though. In general, I'd expect the normal update flow to be
>
> 1) Update Beam protos and/or multi-lang components (though the set of
> things that needs updated for multi-lang is unclear to me)
>

Regarding multi-lang, the protocol does not require consistent versioning
but we may need testing to make sure things work consistently/correctly
when used from a released Swift SDK. For example, Python multi-lang
wrappers look for a Java version with the same version number as the Python
SDK being used.


> 2) Mirror those changes to the Swift SDK.
>
> The thing that is most likely to be forgotten is the 2nd step, and that is
> hard to enforce with automation since the automation would either be on the
> first step which doesn't have anything to enforce or on some sort of
> schedule in the swift repo, which is less likely to be visible. I'm a
> little worried we wouldn't notice breakages until release time.
>
> I wonder how much stuff happens outside of the proto directory that needs
> to be mirrored. Could we just create scheduled automation to exactly copy
> changes in the proto directory and version changes for multi-lang stuff to
> the swift SDK repo?
>
> ---------------------------------------------------------------------
>
> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
> organizational weirdness than push that pain to users.
>
> Thanks,
> Danny
>
> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> Sure, we could definitely include things as a submodule for stuff like
>> testing multi-language, though I think there's actually a cleaner way just
>> using the Swift package manager's test facilities to access the swift sdk
>> repo.
>>
>>  That would also be consistent with the user-side experience and let us
>> test things like build-time integrations with multi-language as well (which
>> is possible in Swift through compiler plugins) in the same way as a
>> pipeline author would. You also maybe get backwards compatibility testing
>> as a side effect in that case as well.
>>
>>
>>
>>
>>
>>
>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>>
>>>
>>>
>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I've chatted with a couple of people offline about this and my
>>>> impression is that folks are generally amenable to a separate repo to match
>>>> the target community? I have no idea what the next steps would be though
>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>> Should I write something up somewhere?
>>>>
>>>
>>> I think the process should be similar to other code/design reviews for
>>> large contributions. I don't think you need a PMC involvement here.
>>>
>>>
>>>>
>>>> Best,
>>>> B
>>>>
>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I've been on vacation, but mostly working on getting External
>>>>> Transform support going (which in turn basically requires Schema support as
>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>> able to use those to do some compile-time automation. In particular, this
>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>> the feature).
>>>>>
>>>>> I do have one question for the group though: should the Swift SDK
>>>>> distribution take on Beam community properties or Swift community
>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>> that there's overhead in setting up the various github actions and other
>>>>> CI/CD bits and bobs.
>>>>>
>>>>>
>>>
>>>> The alternative would be to keep it in the beam repo itself like it is
>>>>> now, but we'd probably want to move Package.swift to the root since for
>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>> happens a lot.
>>>>>
>>>>> There's a third option which is basically "do both" but honestly that
>>>>> just seems like the worst of both worlds as it would require constant
>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>> unreleased SDKs for development and testing.
>>>>>
>>>>> Personally, I would lean towards the former option (and would
>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>> for the actual users of the SDK and more consistent with the community
>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>> also chose this route---most notably GRPC which started with the latter
>>>>> approach and has moved to the former.
>>>>>
>>>>
>>> I believe existing SDKs benefit from living in the same repo. For
>>> example, it's easier to keep them consistent with any model/proto changes
>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>> we'll probably have to add tooling/scripts to keep things consistent.
>>> Is it possible to create a separate repo, but also add a reference (and
>>> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
>>> sure that things stay consistent ?
>>>
>>> Thanks,
>>> Cham
>>>
>>>
>>>>
>>>>> Interested to hear any feedback on the subject since I'm guessing it
>>>>> probably came up with the Go SDK back in the day?
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>> more forgiving.
>>>>>>
>>>>>> Also added support for bounded vs unbounded pcollections through the
>>>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>>>> believe?
>>>>>>
>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>>
>>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>>> means the function signature you use for closures would match full-fledged
>>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>
>>>>>>>> Implicitly passing the timestamp and window during iteration did
>>>>>>>> not go well. While physically possible it introduces an invisible side
>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>
>>>>>>>> What I decided to do instead was the following:
>>>>>>>>
>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>>>> directly.
>>>>>>>>
>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>>>> it will take the timestamp and/or window of the input.
>>>>>>>>
>>>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>
>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>> taking a look.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I would like to figure out a way to get the stream-y interface
>>>>>>>>>>>> to work, as I think it's more natural overall.
>>>>>>>>>>>>
>>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> filenames is the input stream and then output and errors
>>>>>>>>>>>>>>>> are both output streams. In theory you can have as many output streams as
>>>>>>>>>>>>>>>> you like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how
>>>>>>>>>>>>>>>>> the FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get
>>>>>>>>>>>>>>>>>>>>> a minimal FnApi integration going you end up writing a fair bit of an SDK.
>>>>>>>>>>>>>>>>>>>>> So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR
>>>>>>>>>>>>>>>>>>>>> if folks are interested, though the "Swift Way" would be to have it in its
>>>>>>>>>>>>>>>>>>>>> own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
I would say the first actual release would still be a ways out, though this
would make it easier to contribute and accelerate that process :-) (I am
implicitly +1 of course if that matters). FWIW the release process for
Swift libraries is "create a github tag" + any relevant testing.

On Mon, Sep 25, 2023 at 10:07 AM Valentyn Tymofieiev via user <
user@beam.apache.org> wrote:

> Do we anticipate any short-term changes to the release process to start
> releasing switft SDK artifacts or we can hold that off for a certain time
> while SDK is in active development?
>
> On Mon, Sep 25, 2023 at 9:56 AM Robert Burke <ro...@frantil.com> wrote:
>
>> I lost this thread for a bit. I'm glad Prism showed some use while it's
>> doing unfused stages!
>>
>> I have no objections to a separate repo, and in a "Beam Go SDK V3" world
>> that's what I'd want as well, because it works better for the Go usage
>> patterns and is more natural for the tooling. And it would be a cleaner way
>> to do a full overhaul of the user API given the way Go has evolved since
>> it's initial design, and our own experience with it. But that's a very
>> different topic for when I have a real proposal around it.
>>
>> I do see the clean thread Kenn started, but since i have no objections,
>> I'll leave it to silent consensus.
>>
>> I agree that copying/building the protos isn't a burden, since that's
>> entirely what protos are for. We're already treating them as properly
>> stable and not making breaking proto, so compatibility is maintained by
>> normal proto behavior.
>>
>> Robert Burke
>> Beam Go Busybody
>>
>> On Thu, Sep 21, 2023, 9:52 AM Byron Ellis via user <us...@beam.apache.org>
>> wrote:
>>
>>> Also, seems like we're getting something like a consensus? One the repo
>>> exists I'm happy to do the slog work of moving everything around (though
>>> I'm not a committer so somebody else actually has to do the pushes). We can
>>> do that in chunks to make life easier on people and I'm not super concerned
>>> with losing the commit history on my current branch
>>>
>>> On Wed, Sep 20, 2023 at 11:10 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> I actually don't think we'll need any of the multi-repo github actions,
>>>> Swift packages are basically 1:1 with repos so the build process will
>>>> actually do all the checkouts. What we'd do is put a test package in the
>>>> sdks/swift, which works fine since it doesn't ever get used as a dependency
>>>> that depends on the swift SDKs with the appropriate dependencies we want to
>>>> make sure we're testing. This should also catch breaking changes to the
>>>> protos (which in theory proto is helping us avoid).
>>>>
>>>> Syncing the protos hasn't been a huge deal and it's already scripted so
>>>> definitely easy to automate. I  also don't think we would want to do that
>>>> all the time anyway as that would require pipeline authors to install
>>>> protoc for something that doesn't happen all that often. We can take care
>>>> of that for users.
>>>>
>>>>
>>>> On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <
>>>> dannymccormick@google.com> wrote:
>>>>
>>>>> > I think the process should be similar to other code/design reviews
>>>>> for large contributions. I don't think you need a PMC involvement here.
>>>>>
>>>>> I think it does require PMC involvement to create the actual repo once
>>>>> we have public consensus. I tried the flow at
>>>>> https://infra.apache.org/version-control.html#create but it seems
>>>>> like its PMC only. It's unclear to me if consensus has been achieved, maybe
>>>>> a dedicated voting thread with implied lazy consensus would help here.
>>>>>
>>>>> > Sure, we could definitely include things as a submodule for stuff
>>>>> like testing multi-language, though I think there's actually a cleaner way
>>>>> just using the Swift package manager's test facilities to access the swift
>>>>> sdk repo.
>>>>>
>>>>> +1 on avoiding submodules. If needed we could also use multi-repo
>>>>> checkout with GitHub Actions. I think my biggest question is what we'd
>>>>> actually be enforcing though. In general, I'd expect the normal update flow
>>>>> to be
>>>>>
>>>>> 1) Update Beam protos and/or multi-lang components (though the set of
>>>>> things that needs updated for multi-lang is unclear to me)
>>>>> 2) Mirror those changes to the Swift SDK.
>>>>>
>>>>> The thing that is most likely to be forgotten is the 2nd step, and
>>>>> that is hard to enforce with automation since the automation would either
>>>>> be on the first step which doesn't have anything to enforce or on some sort
>>>>> of schedule in the swift repo, which is less likely to be visible. I'm a
>>>>> little worried we wouldn't notice breakages until release time.
>>>>>
>>>>> I wonder how much stuff happens outside of the proto directory that
>>>>> needs to be mirrored. Could we just create scheduled automation to exactly
>>>>> copy changes in the proto directory and version changes for multi-lang
>>>>> stuff to the swift SDK repo?
>>>>>
>>>>> ---------------------------------------------------------------------
>>>>>
>>>>> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
>>>>> organizational weirdness than push that pain to users.
>>>>>
>>>>> Thanks,
>>>>> Danny
>>>>>
>>>>> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <
>>>>> user@beam.apache.org> wrote:
>>>>>
>>>>>> Sure, we could definitely include things as a submodule for stuff
>>>>>> like testing multi-language, though I think there's actually a cleaner way
>>>>>> just using the Swift package manager's test facilities to access the swift
>>>>>> sdk repo.
>>>>>>
>>>>>>  That would also be consistent with the user-side experience and let
>>>>>> us test things like build-time integrations with multi-language as well
>>>>>> (which is possible in Swift through compiler plugins) in the same way as a
>>>>>> pipeline author would. You also maybe get backwards compatibility testing
>>>>>> as a side effect in that case as well.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <
>>>>>> chamikara@google.com> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi all,
>>>>>>>>
>>>>>>>> I've chatted with a couple of people offline about this and my
>>>>>>>> impression is that folks are generally amenable to a separate repo to match
>>>>>>>> the target community? I have no idea what the next steps would be though
>>>>>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>>>>>> Should I write something up somewhere?
>>>>>>>>
>>>>>>>
>>>>>>> I think the process should be similar to other code/design reviews
>>>>>>> for large contributions. I don't think you need a PMC involvement here.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi all,
>>>>>>>>>
>>>>>>>>> I've been on vacation, but mostly working on getting External
>>>>>>>>> Transform support going (which in turn basically requires Schema support as
>>>>>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>>>>>> able to use those to do some compile-time automation. In particular, this
>>>>>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>>>>>> the feature).
>>>>>>>>>
>>>>>>>>> I do have one question for the group though: should the Swift SDK
>>>>>>>>> distribution take on Beam community properties or Swift community
>>>>>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>>>>>> that there's overhead in setting up the various github actions and other
>>>>>>>>> CI/CD bits and bobs.
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>>> The alternative would be to keep it in the beam repo itself like it
>>>>>>>>> is now, but we'd probably want to move Package.swift to the root since for
>>>>>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>>>>>> happens a lot.
>>>>>>>>>
>>>>>>>>> There's a third option which is basically "do both" but honestly
>>>>>>>>> that just seems like the worst of both worlds as it would require constant
>>>>>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>>>>>> unreleased SDKs for development and testing.
>>>>>>>>>
>>>>>>>>> Personally, I would lean towards the former option (and would
>>>>>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>>>>>> for the actual users of the SDK and more consistent with the community
>>>>>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>>>>>> also chose this route---most notably GRPC which started with the latter
>>>>>>>>> approach and has moved to the former.
>>>>>>>>>
>>>>>>>>
>>>>>>> I believe existing SDKs benefit from living in the same repo. For
>>>>>>> example, it's easier to keep them consistent with any model/proto changes
>>>>>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>>>>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>>>>>> we'll probably have to add tooling/scripts to keep things consistent.
>>>>>>> Is it possible to create a separate repo, but also add a reference
>>>>>>> (and Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to
>>>>>>> make sure that things stay consistent ?
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Cham
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>>> Interested to hear any feedback on the subject since I'm guessing
>>>>>>>>> it probably came up with the Go SDK back in the day?
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>>>>>> more forgiving.
>>>>>>>>>>
>>>>>>>>>> Also added support for bounded vs unbounded pcollections through
>>>>>>>>>> the "type" parameter when adding a pardo. Impulse is a bounded pcollection
>>>>>>>>>> I believe?
>>>>>>>>>>
>>>>>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Okay, after a brief detour through "get this working in the
>>>>>>>>>>> Flink Portable Runner" I think I have something pretty workable.
>>>>>>>>>>>
>>>>>>>>>>> PInput and POutput can actually be structs rather than
>>>>>>>>>>> protocols, which simplifies things quite a bit. It also allows us to use
>>>>>>>>>>> them with property wrappers for a SwiftUI-like experience if we want when
>>>>>>>>>>> defining DoFns (which is what I was originally intending to use them for).
>>>>>>>>>>> That also means the function signature you use for closures would match
>>>>>>>>>>> full-fledged DoFn definitions for the most part which is satisfying.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>>>>>
>>>>>>>>>>>> Implicitly passing the timestamp and window during iteration
>>>>>>>>>>>> did not go well. While physically possible it introduces an invisible side
>>>>>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>>>>>
>>>>>>>>>>>> What I decided to do instead was the following:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and
>>>>>>>>>>>> require that they always emit a window and timestamp along with their
>>>>>>>>>>>> value. This eliminates the side effect but lets us keep iteration in a
>>>>>>>>>>>> bundle where that might be convenient. For example, in my cheesy GCS
>>>>>>>>>>>> implementation it means that I can keep an OAuth token around for the
>>>>>>>>>>>> lifetime of the bundle as a local variable, which is convenient. It's a bit
>>>>>>>>>>>> more typing for users of pstream, but the expectation here is that if
>>>>>>>>>>>> you're using pstream functions You Know What You Are Doing and most people
>>>>>>>>>>>> won't be using it directly.
>>>>>>>>>>>>
>>>>>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of
>>>>>>>>>>>> them yet, but enough to test the functionality and decide I liked it) which
>>>>>>>>>>>> take a function signature of (any PInput<InputType>,any
>>>>>>>>>>>> POutput<OutputType>). PInput takes the (InputType,Date,Window) tuple and
>>>>>>>>>>>> converts it into a struct with friendlier names. Not strictly necessary,
>>>>>>>>>>>> but makes the code nicer to read I think. POutput introduces emit functions
>>>>>>>>>>>> that optionally allow you to specify a timestamp and a window. If you don't
>>>>>>>>>>>> for either one it will take the timestamp and/or window of the input.
>>>>>>>>>>>>
>>>>>>>>>>>> Trying to use that was pretty pleasant to use so I think we
>>>>>>>>>>>> should continue down that path. If you'd like to see it in use, I
>>>>>>>>>>>> reimplemented map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>>>>>
>>>>>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>>>>>> taking a look.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input
>>>>>>>>>>>>> and output streams.... Let me try it out... might even be possible to have
>>>>>>>>>>>>> both element and stream-wise closure pardos. Definitely possible to have
>>>>>>>>>>>>> that at the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I would like to figure out a way to get the stream-y
>>>>>>>>>>>>>>>> interface to work, as I think it's more natural overall.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One hypothesis is that if any elements are carried over
>>>>>>>>>>>>>>>> loop iterations, there will likely be some that are carried over beyond the
>>>>>>>>>>>>>>>> loop (after all the callee doesn't know when the loop is supposed to end).
>>>>>>>>>>>>>>>> We could reject "plain" elements that are emitted after this point,
>>>>>>>>>>>>>>>> requiring one to emit timestamp-windowed-values.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets
>>>>>>>>>>>>>>> of data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included
>>>>>>>>>>>>>>>>>>> element-wise collection operations like "map" that eliminate the need for
>>>>>>>>>>>>>>>>>>> pardo in many cases. the groupBy command is actually a map + groupByKey
>>>>>>>>>>>>>>>>>>> under the hood. That was to be more consistent with Swift's collection
>>>>>>>>>>>>>>>>>>> protocol (and is also why PCollection and PCollectionStream are different
>>>>>>>>>>>>>>>>>>> types... PCollection implements map and friends as pipeline construction
>>>>>>>>>>>>>>>>>>> operations whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses
>>>>>>>>>>>>>>>>>>> map rather than pardo in a couple of places to do a true wordcount using
>>>>>>>>>>>>>>>>>>> good ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo
>>>>>>>>>>>>>>>>>>>> syntax quite a bit before settling on where I ended up. Ultimately I
>>>>>>>>>>>>>>>>>>>> decided to go with something that felt more Swift-y than anything else
>>>>>>>>>>>>>>>>>>>> which means that rather than dealing with a single element like you do in
>>>>>>>>>>>>>>>>>>>> the other SDKs you're dealing with a stream of elements (which of course
>>>>>>>>>>>>>>>>>>>> will often be of size 1). That's a really natural paradigm in the Swift
>>>>>>>>>>>>>>>>>>>> world especially with the async / await structures. So when you see
>>>>>>>>>>>>>>>>>>>> something like:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> filenames is the input stream and then output and
>>>>>>>>>>>>>>>>>>>> errors are both output streams. In theory you can have as many output
>>>>>>>>>>>>>>>>>>>> streams as you like though at the moment there's a compiler bug in the new
>>>>>>>>>>>>>>>>>>>> type pack feature that limits it to "as many as I felt like supporting".
>>>>>>>>>>>>>>>>>>>> Presumably this will get fixed before the official 5.9 release which will
>>>>>>>>>>>>>>>>>>>> probably be in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that
>>>>>>>>>>>>>>>>>>>> would look like pardo("Parameter") { param,filenames,output,error in ... }
>>>>>>>>>>>>>>>>>>>> where "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like
>>>>>>>>>>>>>>>>>>>> you have in ES6 and other things where "_" is Swift for "ignore." In this
>>>>>>>>>>>>>>>>>>>> case PCollectionStreams have an element signature of (Of,Date,Window) so
>>>>>>>>>>>>>>>>>>>> you can optionally extract the timestamp and the window if you want to
>>>>>>>>>>>>>>>>>>>> manipulate it somehow.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> That said it would also be natural to provide
>>>>>>>>>>>>>>>>>>>> elementwise pardos--- that would probably mean having explicit type
>>>>>>>>>>>>>>>>>>>> signatures in the closure. I had that at one point, but it felt less
>>>>>>>>>>>>>>>>>>>> natural the more I used it. I'm also slowly working towards adding a more
>>>>>>>>>>>>>>>>>>>> "traditional" DoFn implementation approach where you implement the DoFn as
>>>>>>>>>>>>>>>>>>>> an object type. In that case it would be very very easy to support both by
>>>>>>>>>>>>>>>>>>>> having a default stream implementation call the equivalent of
>>>>>>>>>>>>>>>>>>>> processElement. To make that performant I need to implement an @DoFn macro
>>>>>>>>>>>>>>>>>>>> and I just haven't gotten to it yet.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities
>>>>>>>>>>>>>>>>>>>> that would be Swift native. In particularly, I've been looking at the
>>>>>>>>>>>>>>>>>>>> native Swift binding for DuckDB (which is C++ based). DuckDB is SQL based
>>>>>>>>>>>>>>>>>>>> but not distributed in the same was as, say, Beam SQL... but it would allow
>>>>>>>>>>>>>>>>>>>> for SQL statements on individual files with projection pushdown supported
>>>>>>>>>>>>>>>>>>>> for things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand
>>>>>>>>>>>>>>>>>>>>> how the FnAPI works :). I like the use of groupBy. I have to admit I'm a
>>>>>>>>>>>>>>>>>>>>> bit mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine
>>>>>>>>>>>>>>>>>>>>>> yet though (there's a good chance there are a few places that need to
>>>>>>>>>>>>>>>>>>>>>> properly address endianness. Specifically timestamps in windowed values and
>>>>>>>>>>>>>>>>>>>>>> length in iterable coders as those both use specifically
>>>>>>>>>>>>>>>>>>>>>> bigendian representations)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there
>>>>>>>>>>>>>>>>>>>>>>>> is interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev
>>>>>>>>>>>>>>>>>>>>>>>> <de...@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to
>>>>>>>>>>>>>>>>>>>>>>>>> get a minimal FnApi integration going you end up writing a fair bit of an
>>>>>>>>>>>>>>>>>>>>>>>>> SDK. So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode
>>>>>>>>>>>>>>>>>>>>>>>>> Beta using Swift 5.9 on Intel Macs, but should also work using beta builds
>>>>>>>>>>>>>>>>>>>>>>>>> of 5.9 for Linux running on Intel hardware. I haven't had a chance to try
>>>>>>>>>>>>>>>>>>>>>>>>> it on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to
>>>>>>>>>>>>>>>>>>>>>>>>> be implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code
>>>>>>>>>>>>>>>>>>>>>>>> documented without these features. I think support for composites and
>>>>>>>>>>>>>>>>>>>>>>>> adding an external transform (see, Java
>>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a
>>>>>>>>>>>>>>>>>>>>>>>>> PR if folks are interested, though the "Swift Way" would be to have it in
>>>>>>>>>>>>>>>>>>>>>>>>> its own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Valentyn Tymofieiev via user <us...@beam.apache.org>.
Do we anticipate any short-term changes to the release process to start
releasing switft SDK artifacts or we can hold that off for a certain time
while SDK is in active development?

On Mon, Sep 25, 2023 at 9:56 AM Robert Burke <ro...@frantil.com> wrote:

> I lost this thread for a bit. I'm glad Prism showed some use while it's
> doing unfused stages!
>
> I have no objections to a separate repo, and in a "Beam Go SDK V3" world
> that's what I'd want as well, because it works better for the Go usage
> patterns and is more natural for the tooling. And it would be a cleaner way
> to do a full overhaul of the user API given the way Go has evolved since
> it's initial design, and our own experience with it. But that's a very
> different topic for when I have a real proposal around it.
>
> I do see the clean thread Kenn started, but since i have no objections,
> I'll leave it to silent consensus.
>
> I agree that copying/building the protos isn't a burden, since that's
> entirely what protos are for. We're already treating them as properly
> stable and not making breaking proto, so compatibility is maintained by
> normal proto behavior.
>
> Robert Burke
> Beam Go Busybody
>
> On Thu, Sep 21, 2023, 9:52 AM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> Also, seems like we're getting something like a consensus? One the repo
>> exists I'm happy to do the slog work of moving everything around (though
>> I'm not a committer so somebody else actually has to do the pushes). We can
>> do that in chunks to make life easier on people and I'm not super concerned
>> with losing the commit history on my current branch
>>
>> On Wed, Sep 20, 2023 at 11:10 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> I actually don't think we'll need any of the multi-repo github actions,
>>> Swift packages are basically 1:1 with repos so the build process will
>>> actually do all the checkouts. What we'd do is put a test package in the
>>> sdks/swift, which works fine since it doesn't ever get used as a dependency
>>> that depends on the swift SDKs with the appropriate dependencies we want to
>>> make sure we're testing. This should also catch breaking changes to the
>>> protos (which in theory proto is helping us avoid).
>>>
>>> Syncing the protos hasn't been a huge deal and it's already scripted so
>>> definitely easy to automate. I  also don't think we would want to do that
>>> all the time anyway as that would require pipeline authors to install
>>> protoc for something that doesn't happen all that often. We can take care
>>> of that for users.
>>>
>>>
>>> On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <
>>> dannymccormick@google.com> wrote:
>>>
>>>> > I think the process should be similar to other code/design reviews
>>>> for large contributions. I don't think you need a PMC involvement here.
>>>>
>>>> I think it does require PMC involvement to create the actual repo once
>>>> we have public consensus. I tried the flow at
>>>> https://infra.apache.org/version-control.html#create but it seems like
>>>> its PMC only. It's unclear to me if consensus has been achieved, maybe a
>>>> dedicated voting thread with implied lazy consensus would help here.
>>>>
>>>> > Sure, we could definitely include things as a submodule for stuff
>>>> like testing multi-language, though I think there's actually a cleaner way
>>>> just using the Swift package manager's test facilities to access the swift
>>>> sdk repo.
>>>>
>>>> +1 on avoiding submodules. If needed we could also use multi-repo
>>>> checkout with GitHub Actions. I think my biggest question is what we'd
>>>> actually be enforcing though. In general, I'd expect the normal update flow
>>>> to be
>>>>
>>>> 1) Update Beam protos and/or multi-lang components (though the set of
>>>> things that needs updated for multi-lang is unclear to me)
>>>> 2) Mirror those changes to the Swift SDK.
>>>>
>>>> The thing that is most likely to be forgotten is the 2nd step, and that
>>>> is hard to enforce with automation since the automation would either be on
>>>> the first step which doesn't have anything to enforce or on some sort of
>>>> schedule in the swift repo, which is less likely to be visible. I'm a
>>>> little worried we wouldn't notice breakages until release time.
>>>>
>>>> I wonder how much stuff happens outside of the proto directory that
>>>> needs to be mirrored. Could we just create scheduled automation to exactly
>>>> copy changes in the proto directory and version changes for multi-lang
>>>> stuff to the swift SDK repo?
>>>>
>>>> ---------------------------------------------------------------------
>>>>
>>>> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
>>>> organizational weirdness than push that pain to users.
>>>>
>>>> Thanks,
>>>> Danny
>>>>
>>>> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <
>>>> user@beam.apache.org> wrote:
>>>>
>>>>> Sure, we could definitely include things as a submodule for stuff like
>>>>> testing multi-language, though I think there's actually a cleaner way just
>>>>> using the Swift package manager's test facilities to access the swift sdk
>>>>> repo.
>>>>>
>>>>>  That would also be consistent with the user-side experience and let
>>>>> us test things like build-time integrations with multi-language as well
>>>>> (which is possible in Swift through compiler plugins) in the same way as a
>>>>> pipeline author would. You also maybe get backwards compatibility testing
>>>>> as a side effect in that case as well.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi all,
>>>>>>>
>>>>>>> I've chatted with a couple of people offline about this and my
>>>>>>> impression is that folks are generally amenable to a separate repo to match
>>>>>>> the target community? I have no idea what the next steps would be though
>>>>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>>>>> Should I write something up somewhere?
>>>>>>>
>>>>>>
>>>>>> I think the process should be similar to other code/design reviews
>>>>>> for large contributions. I don't think you need a PMC involvement here.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi all,
>>>>>>>>
>>>>>>>> I've been on vacation, but mostly working on getting External
>>>>>>>> Transform support going (which in turn basically requires Schema support as
>>>>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>>>>> able to use those to do some compile-time automation. In particular, this
>>>>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>>>>> the feature).
>>>>>>>>
>>>>>>>> I do have one question for the group though: should the Swift SDK
>>>>>>>> distribution take on Beam community properties or Swift community
>>>>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>>>>> that there's overhead in setting up the various github actions and other
>>>>>>>> CI/CD bits and bobs.
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>>> The alternative would be to keep it in the beam repo itself like it
>>>>>>>> is now, but we'd probably want to move Package.swift to the root since for
>>>>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>>>>> happens a lot.
>>>>>>>>
>>>>>>>> There's a third option which is basically "do both" but honestly
>>>>>>>> that just seems like the worst of both worlds as it would require constant
>>>>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>>>>> unreleased SDKs for development and testing.
>>>>>>>>
>>>>>>>> Personally, I would lean towards the former option (and would
>>>>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>>>>> for the actual users of the SDK and more consistent with the community
>>>>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>>>>> also chose this route---most notably GRPC which started with the latter
>>>>>>>> approach and has moved to the former.
>>>>>>>>
>>>>>>>
>>>>>> I believe existing SDKs benefit from living in the same repo. For
>>>>>> example, it's easier to keep them consistent with any model/proto changes
>>>>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>>>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>>>>> we'll probably have to add tooling/scripts to keep things consistent.
>>>>>> Is it possible to create a separate repo, but also add a reference
>>>>>> (and Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to
>>>>>> make sure that things stay consistent ?
>>>>>>
>>>>>> Thanks,
>>>>>> Cham
>>>>>>
>>>>>>
>>>>>>>
>>>>>>>> Interested to hear any feedback on the subject since I'm guessing
>>>>>>>> it probably came up with the Go SDK back in the day?
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>>>>> more forgiving.
>>>>>>>>>
>>>>>>>>> Also added support for bounded vs unbounded pcollections through
>>>>>>>>> the "type" parameter when adding a pardo. Impulse is a bounded pcollection
>>>>>>>>> I believe?
>>>>>>>>>
>>>>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>>>>>
>>>>>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>>>>>> means the function signature you use for closures would match full-fledged
>>>>>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>>>>
>>>>>>>>>>> Implicitly passing the timestamp and window during iteration did
>>>>>>>>>>> not go well. While physically possible it introduces an invisible side
>>>>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>>>>
>>>>>>>>>>> What I decided to do instead was the following:
>>>>>>>>>>>
>>>>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and
>>>>>>>>>>> require that they always emit a window and timestamp along with their
>>>>>>>>>>> value. This eliminates the side effect but lets us keep iteration in a
>>>>>>>>>>> bundle where that might be convenient. For example, in my cheesy GCS
>>>>>>>>>>> implementation it means that I can keep an OAuth token around for the
>>>>>>>>>>> lifetime of the bundle as a local variable, which is convenient. It's a bit
>>>>>>>>>>> more typing for users of pstream, but the expectation here is that if
>>>>>>>>>>> you're using pstream functions You Know What You Are Doing and most people
>>>>>>>>>>> won't be using it directly.
>>>>>>>>>>>
>>>>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of
>>>>>>>>>>> them yet, but enough to test the functionality and decide I liked it) which
>>>>>>>>>>> take a function signature of (any PInput<InputType>,any
>>>>>>>>>>> POutput<OutputType>). PInput takes the (InputType,Date,Window) tuple and
>>>>>>>>>>> converts it into a struct with friendlier names. Not strictly necessary,
>>>>>>>>>>> but makes the code nicer to read I think. POutput introduces emit functions
>>>>>>>>>>> that optionally allow you to specify a timestamp and a window. If you don't
>>>>>>>>>>> for either one it will take the timestamp and/or window of the input.
>>>>>>>>>>>
>>>>>>>>>>> Trying to use that was pretty pleasant to use so I think we
>>>>>>>>>>> should continue down that path. If you'd like to see it in use, I
>>>>>>>>>>> reimplemented map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>>>>
>>>>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>>>>> taking a look.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input
>>>>>>>>>>>> and output streams.... Let me try it out... might even be possible to have
>>>>>>>>>>>> both element and stream-wise closure pardos. Definitely possible to have
>>>>>>>>>>>> that at the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I would like to figure out a way to get the stream-y
>>>>>>>>>>>>>>> interface to work, as I think it's more natural overall.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses
>>>>>>>>>>>>>>>>>> map rather than pardo in a couple of places to do a true wordcount using
>>>>>>>>>>>>>>>>>> good ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo
>>>>>>>>>>>>>>>>>>> syntax quite a bit before settling on where I ended up. Ultimately I
>>>>>>>>>>>>>>>>>>> decided to go with something that felt more Swift-y than anything else
>>>>>>>>>>>>>>>>>>> which means that rather than dealing with a single element like you do in
>>>>>>>>>>>>>>>>>>> the other SDKs you're dealing with a stream of elements (which of course
>>>>>>>>>>>>>>>>>>> will often be of size 1). That's a really natural paradigm in the Swift
>>>>>>>>>>>>>>>>>>> world especially with the async / await structures. So when you see
>>>>>>>>>>>>>>>>>>> something like:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> filenames is the input stream and then output and errors
>>>>>>>>>>>>>>>>>>> are both output streams. In theory you can have as many output streams as
>>>>>>>>>>>>>>>>>>> you like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that
>>>>>>>>>>>>>>>>>>> would look like pardo("Parameter") { param,filenames,output,error in ... }
>>>>>>>>>>>>>>>>>>> where "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like
>>>>>>>>>>>>>>>>>>> you have in ES6 and other things where "_" is Swift for "ignore." In this
>>>>>>>>>>>>>>>>>>> case PCollectionStreams have an element signature of (Of,Date,Window) so
>>>>>>>>>>>>>>>>>>> you can optionally extract the timestamp and the window if you want to
>>>>>>>>>>>>>>>>>>> manipulate it somehow.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> That said it would also be natural to provide
>>>>>>>>>>>>>>>>>>> elementwise pardos--- that would probably mean having explicit type
>>>>>>>>>>>>>>>>>>> signatures in the closure. I had that at one point, but it felt less
>>>>>>>>>>>>>>>>>>> natural the more I used it. I'm also slowly working towards adding a more
>>>>>>>>>>>>>>>>>>> "traditional" DoFn implementation approach where you implement the DoFn as
>>>>>>>>>>>>>>>>>>> an object type. In that case it would be very very easy to support both by
>>>>>>>>>>>>>>>>>>> having a default stream implementation call the equivalent of
>>>>>>>>>>>>>>>>>>> processElement. To make that performant I need to implement an @DoFn macro
>>>>>>>>>>>>>>>>>>> and I just haven't gotten to it yet.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities
>>>>>>>>>>>>>>>>>>> that would be Swift native. In particularly, I've been looking at the
>>>>>>>>>>>>>>>>>>> native Swift binding for DuckDB (which is C++ based). DuckDB is SQL based
>>>>>>>>>>>>>>>>>>> but not distributed in the same was as, say, Beam SQL... but it would allow
>>>>>>>>>>>>>>>>>>> for SQL statements on individual files with projection pushdown supported
>>>>>>>>>>>>>>>>>>> for things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how
>>>>>>>>>>>>>>>>>>>> the FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there
>>>>>>>>>>>>>>>>>>>>>>> is interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to
>>>>>>>>>>>>>>>>>>>>>>>> get a minimal FnApi integration going you end up writing a fair bit of an
>>>>>>>>>>>>>>>>>>>>>>>> SDK. So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode
>>>>>>>>>>>>>>>>>>>>>>>> Beta using Swift 5.9 on Intel Macs, but should also work using beta builds
>>>>>>>>>>>>>>>>>>>>>>>> of 5.9 for Linux running on Intel hardware. I haven't had a chance to try
>>>>>>>>>>>>>>>>>>>>>>>> it on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to
>>>>>>>>>>>>>>>>>>>>>>>> be implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code
>>>>>>>>>>>>>>>>>>>>>>> documented without these features. I think support for composites and
>>>>>>>>>>>>>>>>>>>>>>> adding an external transform (see, Java
>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a
>>>>>>>>>>>>>>>>>>>>>>>> PR if folks are interested, though the "Swift Way" would be to have it in
>>>>>>>>>>>>>>>>>>>>>>>> its own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Burke <ro...@frantil.com>.
I lost this thread for a bit. I'm glad Prism showed some use while it's
doing unfused stages!

I have no objections to a separate repo, and in a "Beam Go SDK V3" world
that's what I'd want as well, because it works better for the Go usage
patterns and is more natural for the tooling. And it would be a cleaner way
to do a full overhaul of the user API given the way Go has evolved since
it's initial design, and our own experience with it. But that's a very
different topic for when I have a real proposal around it.

I do see the clean thread Kenn started, but since i have no objections,
I'll leave it to silent consensus.

I agree that copying/building the protos isn't a burden, since that's
entirely what protos are for. We're already treating them as properly
stable and not making breaking proto, so compatibility is maintained by
normal proto behavior.

Robert Burke
Beam Go Busybody

On Thu, Sep 21, 2023, 9:52 AM Byron Ellis via user <us...@beam.apache.org>
wrote:

> Also, seems like we're getting something like a consensus? One the repo
> exists I'm happy to do the slog work of moving everything around (though
> I'm not a committer so somebody else actually has to do the pushes). We can
> do that in chunks to make life easier on people and I'm not super concerned
> with losing the commit history on my current branch
>
> On Wed, Sep 20, 2023 at 11:10 AM Byron Ellis <by...@google.com>
> wrote:
>
>> I actually don't think we'll need any of the multi-repo github actions,
>> Swift packages are basically 1:1 with repos so the build process will
>> actually do all the checkouts. What we'd do is put a test package in the
>> sdks/swift, which works fine since it doesn't ever get used as a dependency
>> that depends on the swift SDKs with the appropriate dependencies we want to
>> make sure we're testing. This should also catch breaking changes to the
>> protos (which in theory proto is helping us avoid).
>>
>> Syncing the protos hasn't been a huge deal and it's already scripted so
>> definitely easy to automate. I  also don't think we would want to do that
>> all the time anyway as that would require pipeline authors to install
>> protoc for something that doesn't happen all that often. We can take care
>> of that for users.
>>
>>
>> On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <
>> dannymccormick@google.com> wrote:
>>
>>> > I think the process should be similar to other code/design reviews for
>>> large contributions. I don't think you need a PMC involvement here.
>>>
>>> I think it does require PMC involvement to create the actual repo once
>>> we have public consensus. I tried the flow at
>>> https://infra.apache.org/version-control.html#create but it seems like
>>> its PMC only. It's unclear to me if consensus has been achieved, maybe a
>>> dedicated voting thread with implied lazy consensus would help here.
>>>
>>> > Sure, we could definitely include things as a submodule for stuff like
>>> testing multi-language, though I think there's actually a cleaner way just
>>> using the Swift package manager's test facilities to access the swift sdk
>>> repo.
>>>
>>> +1 on avoiding submodules. If needed we could also use multi-repo
>>> checkout with GitHub Actions. I think my biggest question is what we'd
>>> actually be enforcing though. In general, I'd expect the normal update flow
>>> to be
>>>
>>> 1) Update Beam protos and/or multi-lang components (though the set of
>>> things that needs updated for multi-lang is unclear to me)
>>> 2) Mirror those changes to the Swift SDK.
>>>
>>> The thing that is most likely to be forgotten is the 2nd step, and that
>>> is hard to enforce with automation since the automation would either be on
>>> the first step which doesn't have anything to enforce or on some sort of
>>> schedule in the swift repo, which is less likely to be visible. I'm a
>>> little worried we wouldn't notice breakages until release time.
>>>
>>> I wonder how much stuff happens outside of the proto directory that
>>> needs to be mirrored. Could we just create scheduled automation to exactly
>>> copy changes in the proto directory and version changes for multi-lang
>>> stuff to the swift SDK repo?
>>>
>>> ---------------------------------------------------------------------
>>>
>>> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
>>> organizational weirdness than push that pain to users.
>>>
>>> Thanks,
>>> Danny
>>>
>>> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <
>>> user@beam.apache.org> wrote:
>>>
>>>> Sure, we could definitely include things as a submodule for stuff like
>>>> testing multi-language, though I think there's actually a cleaner way just
>>>> using the Swift package manager's test facilities to access the swift sdk
>>>> repo.
>>>>
>>>>  That would also be consistent with the user-side experience and let us
>>>> test things like build-time integrations with multi-language as well (which
>>>> is possible in Swift through compiler plugins) in the same way as a
>>>> pipeline author would. You also maybe get backwards compatibility testing
>>>> as a side effect in that case as well.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> I've chatted with a couple of people offline about this and my
>>>>>> impression is that folks are generally amenable to a separate repo to match
>>>>>> the target community? I have no idea what the next steps would be though
>>>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>>>> Should I write something up somewhere?
>>>>>>
>>>>>
>>>>> I think the process should be similar to other code/design reviews for
>>>>> large contributions. I don't think you need a PMC involvement here.
>>>>>
>>>>>
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi all,
>>>>>>>
>>>>>>> I've been on vacation, but mostly working on getting External
>>>>>>> Transform support going (which in turn basically requires Schema support as
>>>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>>>> able to use those to do some compile-time automation. In particular, this
>>>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>>>> the feature).
>>>>>>>
>>>>>>> I do have one question for the group though: should the Swift SDK
>>>>>>> distribution take on Beam community properties or Swift community
>>>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>>>> that there's overhead in setting up the various github actions and other
>>>>>>> CI/CD bits and bobs.
>>>>>>>
>>>>>>>
>>>>>
>>>>>> The alternative would be to keep it in the beam repo itself like it
>>>>>>> is now, but we'd probably want to move Package.swift to the root since for
>>>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>>>> happens a lot.
>>>>>>>
>>>>>>> There's a third option which is basically "do both" but honestly
>>>>>>> that just seems like the worst of both worlds as it would require constant
>>>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>>>> unreleased SDKs for development and testing.
>>>>>>>
>>>>>>> Personally, I would lean towards the former option (and would
>>>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>>>> for the actual users of the SDK and more consistent with the community
>>>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>>>> also chose this route---most notably GRPC which started with the latter
>>>>>>> approach and has moved to the former.
>>>>>>>
>>>>>>
>>>>> I believe existing SDKs benefit from living in the same repo. For
>>>>> example, it's easier to keep them consistent with any model/proto changes
>>>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>>>> we'll probably have to add tooling/scripts to keep things consistent.
>>>>> Is it possible to create a separate repo, but also add a reference
>>>>> (and Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to
>>>>> make sure that things stay consistent ?
>>>>>
>>>>> Thanks,
>>>>> Cham
>>>>>
>>>>>
>>>>>>
>>>>>>> Interested to hear any feedback on the subject since I'm guessing it
>>>>>>> probably came up with the Go SDK back in the day?
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>>>> more forgiving.
>>>>>>>>
>>>>>>>> Also added support for bounded vs unbounded pcollections through
>>>>>>>> the "type" parameter when adding a pardo. Impulse is a bounded pcollection
>>>>>>>> I believe?
>>>>>>>>
>>>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>>>>
>>>>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>>>>> means the function signature you use for closures would match full-fledged
>>>>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>>>
>>>>>>>>>> Implicitly passing the timestamp and window during iteration did
>>>>>>>>>> not go well. While physically possible it introduces an invisible side
>>>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>>>
>>>>>>>>>> What I decided to do instead was the following:
>>>>>>>>>>
>>>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>>>>>> directly.
>>>>>>>>>>
>>>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of
>>>>>>>>>> them yet, but enough to test the functionality and decide I liked it) which
>>>>>>>>>> take a function signature of (any PInput<InputType>,any
>>>>>>>>>> POutput<OutputType>). PInput takes the (InputType,Date,Window) tuple and
>>>>>>>>>> converts it into a struct with friendlier names. Not strictly necessary,
>>>>>>>>>> but makes the code nicer to read I think. POutput introduces emit functions
>>>>>>>>>> that optionally allow you to specify a timestamp and a window. If you don't
>>>>>>>>>> for either one it will take the timestamp and/or window of the input.
>>>>>>>>>>
>>>>>>>>>> Trying to use that was pretty pleasant to use so I think we
>>>>>>>>>> should continue down that path. If you'd like to see it in use, I
>>>>>>>>>> reimplemented map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>>>
>>>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>>>> taking a look.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I would like to figure out a way to get the stream-y
>>>>>>>>>>>>>> interface to work, as I think it's more natural overall.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo
>>>>>>>>>>>>>>>>>> syntax quite a bit before settling on where I ended up. Ultimately I
>>>>>>>>>>>>>>>>>> decided to go with something that felt more Swift-y than anything else
>>>>>>>>>>>>>>>>>> which means that rather than dealing with a single element like you do in
>>>>>>>>>>>>>>>>>> the other SDKs you're dealing with a stream of elements (which of course
>>>>>>>>>>>>>>>>>> will often be of size 1). That's a really natural paradigm in the Swift
>>>>>>>>>>>>>>>>>> world especially with the async / await structures. So when you see
>>>>>>>>>>>>>>>>>> something like:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> filenames is the input stream and then output and errors
>>>>>>>>>>>>>>>>>> are both output streams. In theory you can have as many output streams as
>>>>>>>>>>>>>>>>>> you like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like
>>>>>>>>>>>>>>>>>> you have in ES6 and other things where "_" is Swift for "ignore." In this
>>>>>>>>>>>>>>>>>> case PCollectionStreams have an element signature of (Of,Date,Window) so
>>>>>>>>>>>>>>>>>> you can optionally extract the timestamp and the window if you want to
>>>>>>>>>>>>>>>>>> manipulate it somehow.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities
>>>>>>>>>>>>>>>>>> that would be Swift native. In particularly, I've been looking at the
>>>>>>>>>>>>>>>>>> native Swift binding for DuckDB (which is C++ based). DuckDB is SQL based
>>>>>>>>>>>>>>>>>> but not distributed in the same was as, say, Beam SQL... but it would allow
>>>>>>>>>>>>>>>>>> for SQL statements on individual files with projection pushdown supported
>>>>>>>>>>>>>>>>>> for things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how
>>>>>>>>>>>>>>>>>>> the FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to
>>>>>>>>>>>>>>>>>>>>>>> get a minimal FnApi integration going you end up writing a fair bit of an
>>>>>>>>>>>>>>>>>>>>>>> SDK. So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to
>>>>>>>>>>>>>>>>>>>>>>> be implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code
>>>>>>>>>>>>>>>>>>>>>> documented without these features. I think support for composites and
>>>>>>>>>>>>>>>>>>>>>> adding an external transform (see, Java
>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a
>>>>>>>>>>>>>>>>>>>>>>> PR if folks are interested, though the "Swift Way" would be to have it in
>>>>>>>>>>>>>>>>>>>>>>> its own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Also, seems like we're getting something like a consensus? One the repo
exists I'm happy to do the slog work of moving everything around (though
I'm not a committer so somebody else actually has to do the pushes). We can
do that in chunks to make life easier on people and I'm not super concerned
with losing the commit history on my current branch

On Wed, Sep 20, 2023 at 11:10 AM Byron Ellis <by...@google.com> wrote:

> I actually don't think we'll need any of the multi-repo github actions,
> Swift packages are basically 1:1 with repos so the build process will
> actually do all the checkouts. What we'd do is put a test package in the
> sdks/swift, which works fine since it doesn't ever get used as a dependency
> that depends on the swift SDKs with the appropriate dependencies we want to
> make sure we're testing. This should also catch breaking changes to the
> protos (which in theory proto is helping us avoid).
>
> Syncing the protos hasn't been a huge deal and it's already scripted so
> definitely easy to automate. I  also don't think we would want to do that
> all the time anyway as that would require pipeline authors to install
> protoc for something that doesn't happen all that often. We can take care
> of that for users.
>
>
> On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <
> dannymccormick@google.com> wrote:
>
>> > I think the process should be similar to other code/design reviews for
>> large contributions. I don't think you need a PMC involvement here.
>>
>> I think it does require PMC involvement to create the actual repo once we
>> have public consensus. I tried the flow at
>> https://infra.apache.org/version-control.html#create but it seems like
>> its PMC only. It's unclear to me if consensus has been achieved, maybe a
>> dedicated voting thread with implied lazy consensus would help here.
>>
>> > Sure, we could definitely include things as a submodule for stuff like
>> testing multi-language, though I think there's actually a cleaner way just
>> using the Swift package manager's test facilities to access the swift sdk
>> repo.
>>
>> +1 on avoiding submodules. If needed we could also use multi-repo
>> checkout with GitHub Actions. I think my biggest question is what we'd
>> actually be enforcing though. In general, I'd expect the normal update flow
>> to be
>>
>> 1) Update Beam protos and/or multi-lang components (though the set of
>> things that needs updated for multi-lang is unclear to me)
>> 2) Mirror those changes to the Swift SDK.
>>
>> The thing that is most likely to be forgotten is the 2nd step, and that
>> is hard to enforce with automation since the automation would either be on
>> the first step which doesn't have anything to enforce or on some sort of
>> schedule in the swift repo, which is less likely to be visible. I'm a
>> little worried we wouldn't notice breakages until release time.
>>
>> I wonder how much stuff happens outside of the proto directory that needs
>> to be mirrored. Could we just create scheduled automation to exactly copy
>> changes in the proto directory and version changes for multi-lang stuff to
>> the swift SDK repo?
>>
>> ---------------------------------------------------------------------
>>
>> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
>> organizational weirdness than push that pain to users.
>>
>> Thanks,
>> Danny
>>
>> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <
>> user@beam.apache.org> wrote:
>>
>>> Sure, we could definitely include things as a submodule for stuff like
>>> testing multi-language, though I think there's actually a cleaner way just
>>> using the Swift package manager's test facilities to access the swift sdk
>>> repo.
>>>
>>>  That would also be consistent with the user-side experience and let us
>>> test things like build-time integrations with multi-language as well (which
>>> is possible in Swift through compiler plugins) in the same way as a
>>> pipeline author would. You also maybe get backwards compatibility testing
>>> as a side effect in that case as well.
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <
>>> chamikara@google.com> wrote:
>>>
>>>>
>>>>
>>>>
>>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I've chatted with a couple of people offline about this and my
>>>>> impression is that folks are generally amenable to a separate repo to match
>>>>> the target community? I have no idea what the next steps would be though
>>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>>> Should I write something up somewhere?
>>>>>
>>>>
>>>> I think the process should be similar to other code/design reviews for
>>>> large contributions. I don't think you need a PMC involvement here.
>>>>
>>>>
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> I've been on vacation, but mostly working on getting External
>>>>>> Transform support going (which in turn basically requires Schema support as
>>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>>> able to use those to do some compile-time automation. In particular, this
>>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>>> the feature).
>>>>>>
>>>>>> I do have one question for the group though: should the Swift SDK
>>>>>> distribution take on Beam community properties or Swift community
>>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>>> that there's overhead in setting up the various github actions and other
>>>>>> CI/CD bits and bobs.
>>>>>>
>>>>>>
>>>>
>>>>> The alternative would be to keep it in the beam repo itself like it is
>>>>>> now, but we'd probably want to move Package.swift to the root since for
>>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>>> happens a lot.
>>>>>>
>>>>>> There's a third option which is basically "do both" but honestly that
>>>>>> just seems like the worst of both worlds as it would require constant
>>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>>> unreleased SDKs for development and testing.
>>>>>>
>>>>>> Personally, I would lean towards the former option (and would
>>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>>> for the actual users of the SDK and more consistent with the community
>>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>>> also chose this route---most notably GRPC which started with the latter
>>>>>> approach and has moved to the former.
>>>>>>
>>>>>
>>>> I believe existing SDKs benefit from living in the same repo. For
>>>> example, it's easier to keep them consistent with any model/proto changes
>>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>>> we'll probably have to add tooling/scripts to keep things consistent.
>>>> Is it possible to create a separate repo, but also add a reference (and
>>>> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
>>>> sure that things stay consistent ?
>>>>
>>>> Thanks,
>>>> Cham
>>>>
>>>>
>>>>>
>>>>>> Interested to hear any feedback on the subject since I'm guessing it
>>>>>> probably came up with the Go SDK back in the day?
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>>> more forgiving.
>>>>>>>
>>>>>>> Also added support for bounded vs unbounded pcollections through the
>>>>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>>>>> believe?
>>>>>>>
>>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>>>
>>>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>>>> means the function signature you use for closures would match full-fledged
>>>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>>
>>>>>>>>> Implicitly passing the timestamp and window during iteration did
>>>>>>>>> not go well. While physically possible it introduces an invisible side
>>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>>
>>>>>>>>> What I decided to do instead was the following:
>>>>>>>>>
>>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>>>>> directly.
>>>>>>>>>
>>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>>>>> it will take the timestamp and/or window of the input.
>>>>>>>>>
>>>>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>>
>>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>>> taking a look.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I would like to figure out a way to get the stream-y interface
>>>>>>>>>>>>> to work, as I think it's more natural overall.
>>>>>>>>>>>>>
>>>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> filenames is the input stream and then output and errors
>>>>>>>>>>>>>>>>> are both output streams. In theory you can have as many output streams as
>>>>>>>>>>>>>>>>> you like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities
>>>>>>>>>>>>>>>>> that would be Swift native. In particularly, I've been looking at the
>>>>>>>>>>>>>>>>> native Swift binding for DuckDB (which is C++ based). DuckDB is SQL based
>>>>>>>>>>>>>>>>> but not distributed in the same was as, say, Beam SQL... but it would allow
>>>>>>>>>>>>>>>>> for SQL statements on individual files with projection pushdown supported
>>>>>>>>>>>>>>>>> for things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how
>>>>>>>>>>>>>>>>>> the FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get
>>>>>>>>>>>>>>>>>>>>>> a minimal FnApi integration going you end up writing a fair bit of an SDK.
>>>>>>>>>>>>>>>>>>>>>> So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR
>>>>>>>>>>>>>>>>>>>>>> if folks are interested, though the "Swift Way" would be to have it in its
>>>>>>>>>>>>>>>>>>>>>> own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
I actually don't think we'll need any of the multi-repo github actions,
Swift packages are basically 1:1 with repos so the build process will
actually do all the checkouts. What we'd do is put a test package in the
sdks/swift, which works fine since it doesn't ever get used as a dependency
that depends on the swift SDKs with the appropriate dependencies we want to
make sure we're testing. This should also catch breaking changes to the
protos (which in theory proto is helping us avoid).

Syncing the protos hasn't been a huge deal and it's already scripted so
definitely easy to automate. I  also don't think we would want to do that
all the time anyway as that would require pipeline authors to install
protoc for something that doesn't happen all that often. We can take care
of that for users.


On Wed, Sep 20, 2023 at 10:48 AM Danny McCormick <da...@google.com>
wrote:

> > I think the process should be similar to other code/design reviews for
> large contributions. I don't think you need a PMC involvement here.
>
> I think it does require PMC involvement to create the actual repo once we
> have public consensus. I tried the flow at
> https://infra.apache.org/version-control.html#create but it seems like
> its PMC only. It's unclear to me if consensus has been achieved, maybe a
> dedicated voting thread with implied lazy consensus would help here.
>
> > Sure, we could definitely include things as a submodule for stuff like
> testing multi-language, though I think there's actually a cleaner way just
> using the Swift package manager's test facilities to access the swift sdk
> repo.
>
> +1 on avoiding submodules. If needed we could also use multi-repo checkout
> with GitHub Actions. I think my biggest question is what we'd actually be
> enforcing though. In general, I'd expect the normal update flow to be
>
> 1) Update Beam protos and/or multi-lang components (though the set of
> things that needs updated for multi-lang is unclear to me)
> 2) Mirror those changes to the Swift SDK.
>
> The thing that is most likely to be forgotten is the 2nd step, and that is
> hard to enforce with automation since the automation would either be on the
> first step which doesn't have anything to enforce or on some sort of
> schedule in the swift repo, which is less likely to be visible. I'm a
> little worried we wouldn't notice breakages until release time.
>
> I wonder how much stuff happens outside of the proto directory that needs
> to be mirrored. Could we just create scheduled automation to exactly copy
> changes in the proto directory and version changes for multi-lang stuff to
> the swift SDK repo?
>
> ---------------------------------------------------------------------
>
> Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
> organizational weirdness than push that pain to users.
>
> Thanks,
> Danny
>
> On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> Sure, we could definitely include things as a submodule for stuff like
>> testing multi-language, though I think there's actually a cleaner way just
>> using the Swift package manager's test facilities to access the swift sdk
>> repo.
>>
>>  That would also be consistent with the user-side experience and let us
>> test things like build-time integrations with multi-language as well (which
>> is possible in Swift through compiler plugins) in the same way as a
>> pipeline author would. You also maybe get backwards compatibility testing
>> as a side effect in that case as well.
>>
>>
>>
>>
>>
>>
>> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>>
>>>
>>>
>>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I've chatted with a couple of people offline about this and my
>>>> impression is that folks are generally amenable to a separate repo to match
>>>> the target community? I have no idea what the next steps would be though
>>>> other than guessing that there's probably some sort of PMC thing involved?
>>>> Should I write something up somewhere?
>>>>
>>>
>>> I think the process should be similar to other code/design reviews for
>>> large contributions. I don't think you need a PMC involvement here.
>>>
>>>
>>>>
>>>> Best,
>>>> B
>>>>
>>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I've been on vacation, but mostly working on getting External
>>>>> Transform support going (which in turn basically requires Schema support as
>>>>> well). It also looks like macros landed in Swift 5.9 for Linux so we'll be
>>>>> able to use those to do some compile-time automation. In particular, this
>>>>> lets us do something similar to what Java does with ByteBuddy for
>>>>> generating schema coders though it has to be ahead of time so not quite the
>>>>> same. (As far as I can tell this is a reason why macros got added to the
>>>>> language in the first place---Apple's SwiftData library makes heavy use of
>>>>> the feature).
>>>>>
>>>>> I do have one question for the group though: should the Swift SDK
>>>>> distribution take on Beam community properties or Swift community
>>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>>> that there's overhead in setting up the various github actions and other
>>>>> CI/CD bits and bobs.
>>>>>
>>>>>
>>>
>>>> The alternative would be to keep it in the beam repo itself like it is
>>>>> now, but we'd probably want to move Package.swift to the root since for
>>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>>> chosen to have packages only really able to live at the top of a repo. This
>>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>>> happens a lot.
>>>>>
>>>>> There's a third option which is basically "do both" but honestly that
>>>>> just seems like the worst of both worlds as it would require constant
>>>>> syncing if we wanted to make it possible for Swift users to target
>>>>> unreleased SDKs for development and testing.
>>>>>
>>>>> Personally, I would lean towards the former option (and would
>>>>> volunteer to set up & document the various automations) as it is lighter
>>>>> for the actual users of the SDK and more consistent with the community
>>>>> experience they expect. The CI/CD stuff is mostly a "do it once" whereas
>>>>> checking out the entire repo with many updates the user doesn't care about
>>>>> is something they will be doing all the time. FWIW some of our dependencies
>>>>> also chose this route---most notably GRPC which started with the latter
>>>>> approach and has moved to the former.
>>>>>
>>>>
>>> I believe existing SDKs benefit from living in the same repo. For
>>> example, it's easier to keep them consistent with any model/proto changes
>>> and it's easier to manage distributions/tags. Also it's easier to keep
>>> components consistent for multi-lang. If we add Swift to a separate repo,
>>> we'll probably have to add tooling/scripts to keep things consistent.
>>> Is it possible to create a separate repo, but also add a reference (and
>>> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
>>> sure that things stay consistent ?
>>>
>>> Thanks,
>>> Cham
>>>
>>>
>>>>
>>>>> Interested to hear any feedback on the subject since I'm guessing it
>>>>> probably came up with the Go SDK back in the day?
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>
>>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>>> more forgiving.
>>>>>>
>>>>>> Also added support for bounded vs unbounded pcollections through the
>>>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>>>> believe?
>>>>>>
>>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>>
>>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>>> means the function signature you use for closures would match full-fledged
>>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Okay, I tried a couple of different things.
>>>>>>>>
>>>>>>>> Implicitly passing the timestamp and window during iteration did
>>>>>>>> not go well. While physically possible it introduces an invisible side
>>>>>>>> effect into loop iteration which confused me when I tried to use it and I
>>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>>> race condition nightmare continuing down that path.
>>>>>>>>
>>>>>>>> What I decided to do instead was the following:
>>>>>>>>
>>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>>>> directly.
>>>>>>>>
>>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>>>> it will take the timestamp and/or window of the input.
>>>>>>>>
>>>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>>>
>>>>>>>> Code has been pushed to the branch/PR if you're interested in
>>>>>>>> taking a look.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>>> use @DoFn as a macro)
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I would like to figure out a way to get the stream-y interface
>>>>>>>>>>>> to work, as I think it's more natural overall.
>>>>>>>>>>>>
>>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> Related to this, we could enforce that the only
>>>>>>>>>>>> (user-accessible) way to get such a timestamped value is to start with one,
>>>>>>>>>>>> e.g. a WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the
>>>>>>>>>>>> same metadata but a new value. Thus a user wanting to do anything "fancy"
>>>>>>>>>>>> would have to explicitly request iteration over these windowed values
>>>>>>>>>>>> rather than over the raw elements. (This is also forward compatible with
>>>>>>>>>>>> expanding the metadata that can get attached, e.g. pane infos, and makes
>>>>>>>>>>>> the right thing the easiest/most natural.)
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Ah, that is a good point—being element-wise would make
>>>>>>>>>>>>> managing windows and time stamps easier for the user. Fortunately it’s a
>>>>>>>>>>>>> fairly easy change to make and maybe even less typing for the user. I was
>>>>>>>>>>>>> originally thinking side inputs and metrics would happen outside the loop,
>>>>>>>>>>>>> but I think you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> filenames is the input stream and then output and errors
>>>>>>>>>>>>>>>> are both output streams. In theory you can have as many output streams as
>>>>>>>>>>>>>>>> you like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing
>>>>>>>>>>>>>>>> implementing composite and external transforms for the reasons you suggest.
>>>>>>>>>>>>>>>> :-) I've got the basics of a composite transform (there's an equivalent
>>>>>>>>>>>>>>>> wordcount example) and am hooking it into the pipeline generation, which
>>>>>>>>>>>>>>>> should also give me everything I need to successfully hook in external
>>>>>>>>>>>>>>>> transforms as well. That will give me the jump on IOs as you say. I can
>>>>>>>>>>>>>>>> also treat the pipeline itself as a composite transform which lets me get
>>>>>>>>>>>>>>>> rid of the Pipeline { pipeline in ... } and just instead have things attach
>>>>>>>>>>>>>>>> themselves to the pipeline implicitly.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute
>>>>>>>>>>>>>>>> here and there.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how
>>>>>>>>>>>>>>>>> the FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get
>>>>>>>>>>>>>>>>>>>>> a minimal FnApi integration going you end up writing a fair bit of an SDK.
>>>>>>>>>>>>>>>>>>>>> So I decided to take things to a point where I had an SDK that could
>>>>>>>>>>>>>>>>>>>>> execute a word count example via a portable runner backend. I've now
>>>>>>>>>>>>>>>>>>>>> reached that point and would like to submit my prototype SDK to the list
>>>>>>>>>>>>>>>>>>>>> for feedback.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR
>>>>>>>>>>>>>>>>>>>>> if folks are interested, though the "Swift Way" would be to have it in its
>>>>>>>>>>>>>>>>>>>>> own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially).
>>>>>>>>>>>>>>>>>>>> Also it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Danny McCormick via user <us...@beam.apache.org>.
> I think the process should be similar to other code/design reviews for
large contributions. I don't think you need a PMC involvement here.

I think it does require PMC involvement to create the actual repo once we
have public consensus. I tried the flow at
https://infra.apache.org/version-control.html#create but it seems like its
PMC only. It's unclear to me if consensus has been achieved, maybe a
dedicated voting thread with implied lazy consensus would help here.

> Sure, we could definitely include things as a submodule for stuff like
testing multi-language, though I think there's actually a cleaner way just
using the Swift package manager's test facilities to access the swift sdk
repo.

+1 on avoiding submodules. If needed we could also use multi-repo checkout
with GitHub Actions. I think my biggest question is what we'd actually be
enforcing though. In general, I'd expect the normal update flow to be

1) Update Beam protos and/or multi-lang components (though the set of
things that needs updated for multi-lang is unclear to me)
2) Mirror those changes to the Swift SDK.

The thing that is most likely to be forgotten is the 2nd step, and that is
hard to enforce with automation since the automation would either be on the
first step which doesn't have anything to enforce or on some sort of
schedule in the swift repo, which is less likely to be visible. I'm a
little worried we wouldn't notice breakages until release time.

I wonder how much stuff happens outside of the proto directory that needs
to be mirrored. Could we just create scheduled automation to exactly copy
changes in the proto directory and version changes for multi-lang stuff to
the swift SDK repo?

---------------------------------------------------------------------

Regardless, I'm +1 on a dedicated repo; I'd rather we take on some
organizational weirdness than push that pain to users.

Thanks,
Danny

On Wed, Sep 20, 2023 at 1:38 PM Byron Ellis via user <us...@beam.apache.org>
wrote:

> Sure, we could definitely include things as a submodule for stuff like
> testing multi-language, though I think there's actually a cleaner way just
> using the Swift package manager's test facilities to access the swift sdk
> repo.
>
>  That would also be consistent with the user-side experience and let us
> test things like build-time integrations with multi-language as well (which
> is possible in Swift through compiler plugins) in the same way as a
> pipeline author would. You also maybe get backwards compatibility testing
> as a side effect in that case as well.
>
>
>
>
>
>
> On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>>
>>
>>
>> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> I've chatted with a couple of people offline about this and my
>>> impression is that folks are generally amenable to a separate repo to match
>>> the target community? I have no idea what the next steps would be though
>>> other than guessing that there's probably some sort of PMC thing involved?
>>> Should I write something up somewhere?
>>>
>>
>> I think the process should be similar to other code/design reviews for
>> large contributions. I don't think you need a PMC involvement here.
>>
>>
>>>
>>> Best,
>>> B
>>>
>>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I've been on vacation, but mostly working on getting External Transform
>>>> support going (which in turn basically requires Schema support as well). It
>>>> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
>>>> use those to do some compile-time automation. In particular, this lets us
>>>> do something similar to what Java does with ByteBuddy for generating schema
>>>> coders though it has to be ahead of time so not quite the same. (As far as
>>>> I can tell this is a reason why macros got added to the language in the
>>>> first place---Apple's SwiftData library makes heavy use of the feature).
>>>>
>>>> I do have one question for the group though: should the Swift SDK
>>>> distribution take on Beam community properties or Swift community
>>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>>> its own repo (beam-swift for example), which allows it to be most easily
>>>> consumed and keeps the checkout size under control for users. "Releases" in
>>>> the Swift world (much like Go) are just repo tags. The downside here is
>>>> that there's overhead in setting up the various github actions and other
>>>> CI/CD bits and bobs.
>>>>
>>>>
>>
>>> The alternative would be to keep it in the beam repo itself like it is
>>>> now, but we'd probably want to move Package.swift to the root since for
>>>> whatever reason the Swift community (much to some people's annoyance) has
>>>> chosen to have packages only really able to live at the top of a repo. This
>>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>>> happens a lot.
>>>>
>>>> There's a third option which is basically "do both" but honestly that
>>>> just seems like the worst of both worlds as it would require constant
>>>> syncing if we wanted to make it possible for Swift users to target
>>>> unreleased SDKs for development and testing.
>>>>
>>>> Personally, I would lean towards the former option (and would volunteer
>>>> to set up & document the various automations) as it is lighter for the
>>>> actual users of the SDK and more consistent with the community experience
>>>> they expect. The CI/CD stuff is mostly a "do it once" whereas checking out
>>>> the entire repo with many updates the user doesn't care about is something
>>>> they will be doing all the time. FWIW some of our dependencies also chose
>>>> this route---most notably GRPC which started with the latter approach and
>>>> has moved to the former.
>>>>
>>>
>> I believe existing SDKs benefit from living in the same repo. For
>> example, it's easier to keep them consistent with any model/proto changes
>> and it's easier to manage distributions/tags. Also it's easier to keep
>> components consistent for multi-lang. If we add Swift to a separate repo,
>> we'll probably have to add tooling/scripts to keep things consistent.
>> Is it possible to create a separate repo, but also add a reference (and
>> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
>> sure that things stay consistent ?
>>
>> Thanks,
>> Cham
>>
>>
>>>
>>>> Interested to hear any feedback on the subject since I'm guessing it
>>>> probably came up with the Go SDK back in the day?
>>>>
>>>> Best,
>>>> B
>>>>
>>>>
>>>>
>>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> After a couple of iterations (thanks rebo!) we've also gotten the
>>>>> Swift SDK working with the new Prism runner. The fact that it doesn't do
>>>>> fusion caught a couple of configuration bugs (e.g. that the grpc message
>>>>> receiver buffer should be fairly large). It would seem that at the moment
>>>>> Prism and the Flink runner have similar orders of strictness when
>>>>> interpreting the pipeline graph while the Python portable runner is far
>>>>> more forgiving.
>>>>>
>>>>> Also added support for bounded vs unbounded pcollections through the
>>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>>> believe?
>>>>>
>>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>>> Portable Runner" I think I have something pretty workable.
>>>>>>
>>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>>> means the function signature you use for closures would match full-fledged
>>>>>> DoFn definitions for the most part which is satisfying.
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Okay, I tried a couple of different things.
>>>>>>>
>>>>>>> Implicitly passing the timestamp and window during iteration did not
>>>>>>> go well. While physically possible it introduces an invisible side effect
>>>>>>> into loop iteration which confused me when I tried to use it and I
>>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>>> race condition nightmare continuing down that path.
>>>>>>>
>>>>>>> What I decided to do instead was the following:
>>>>>>>
>>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>>> directly.
>>>>>>>
>>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>>> it will take the timestamp and/or window of the input.
>>>>>>>
>>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>>
>>>>>>> Code has been pushed to the branch/PR if you're interested in taking
>>>>>>> a look.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>>> use @DoFn as a macro)
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> I would like to figure out a way to get the stream-y interface
>>>>>>>>>>> to work, as I think it's more natural overall.
>>>>>>>>>>>
>>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>>>>> the easiest/most natural.)
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole
>>>>>>>>>>>>> bundle rather than start/finish bundle callbacks, but one of the questions
>>>>>>>>>>>>> is how that would impact implicit passing of the timestamp (and other)
>>>>>>>>>>>>> metadata from input elements to output elements. (You can of course attach
>>>>>>>>>>>>> the metadata to any output that happens in the loop body, but it's very
>>>>>>>>>>>>> easy to implicitly to break the 1:1 relationship here (e.g. by doing
>>>>>>>>>>>>> buffering or otherwise modifying local state) and this would be hard to
>>>>>>>>>>>>> detect. (I suppose trying to output after the loop finishes could require
>>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>>>>> and there.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to
>>>>>>>>>>>>>>>>>>>> really understand how the Beam FnApi works and how it interacts with the
>>>>>>>>>>>>>>>>>>>> Portable Runner. For me at least that usually means I need to write some
>>>>>>>>>>>>>>>>>>>> code so I can see things happening in a debugger and to really prove to
>>>>>>>>>>>>>>>>>>>> myself I understood what was going on I decided I couldn't use an existing
>>>>>>>>>>>>>>>>>>>> SDK language to do it since there would be the temptation to read some code
>>>>>>>>>>>>>>>>>>>> and convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR
>>>>>>>>>>>>>>>>>>>> if folks are interested, though the "Swift Way" would be to have it in its
>>>>>>>>>>>>>>>>>>>> own repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Sure, we could definitely include things as a submodule for stuff like
testing multi-language, though I think there's actually a cleaner way just
using the Swift package manager's test facilities to access the swift sdk
repo.

 That would also be consistent with the user-side experience and let us
test things like build-time integrations with multi-language as well (which
is possible in Swift through compiler plugins) in the same way as a
pipeline author would. You also maybe get backwards compatibility testing
as a side effect in that case as well.






On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <ch...@google.com>
wrote:

>
>
>
> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com> wrote:
>
>> Hi all,
>>
>> I've chatted with a couple of people offline about this and my impression
>> is that folks are generally amenable to a separate repo to match the target
>> community? I have no idea what the next steps would be though other than
>> guessing that there's probably some sort of PMC thing involved? Should I
>> write something up somewhere?
>>
>
> I think the process should be similar to other code/design reviews for
> large contributions. I don't think you need a PMC involvement here.
>
>
>>
>> Best,
>> B
>>
>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> I've been on vacation, but mostly working on getting External Transform
>>> support going (which in turn basically requires Schema support as well). It
>>> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
>>> use those to do some compile-time automation. In particular, this lets us
>>> do something similar to what Java does with ByteBuddy for generating schema
>>> coders though it has to be ahead of time so not quite the same. (As far as
>>> I can tell this is a reason why macros got added to the language in the
>>> first place---Apple's SwiftData library makes heavy use of the feature).
>>>
>>> I do have one question for the group though: should the Swift SDK
>>> distribution take on Beam community properties or Swift community
>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>> its own repo (beam-swift for example), which allows it to be most easily
>>> consumed and keeps the checkout size under control for users. "Releases" in
>>> the Swift world (much like Go) are just repo tags. The downside here is
>>> that there's overhead in setting up the various github actions and other
>>> CI/CD bits and bobs.
>>>
>>>
>
>> The alternative would be to keep it in the beam repo itself like it is
>>> now, but we'd probably want to move Package.swift to the root since for
>>> whatever reason the Swift community (much to some people's annoyance) has
>>> chosen to have packages only really able to live at the top of a repo. This
>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>> happens a lot.
>>>
>>> There's a third option which is basically "do both" but honestly that
>>> just seems like the worst of both worlds as it would require constant
>>> syncing if we wanted to make it possible for Swift users to target
>>> unreleased SDKs for development and testing.
>>>
>>> Personally, I would lean towards the former option (and would volunteer
>>> to set up & document the various automations) as it is lighter for the
>>> actual users of the SDK and more consistent with the community experience
>>> they expect. The CI/CD stuff is mostly a "do it once" whereas checking out
>>> the entire repo with many updates the user doesn't care about is something
>>> they will be doing all the time. FWIW some of our dependencies also chose
>>> this route---most notably GRPC which started with the latter approach and
>>> has moved to the former.
>>>
>>
> I believe existing SDKs benefit from living in the same repo. For example,
> it's easier to keep them consistent with any model/proto changes and it's
> easier to manage distributions/tags. Also it's easier to keep components
> consistent for multi-lang. If we add Swift to a separate repo, we'll
> probably have to add tooling/scripts to keep things consistent.
> Is it possible to create a separate repo, but also add a reference (and
> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
> sure that things stay consistent ?
>
> Thanks,
> Cham
>
>
>>
>>> Interested to hear any feedback on the subject since I'm guessing it
>>> probably came up with the Go SDK back in the day?
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>>>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>>>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>>>> buffer should be fairly large). It would seem that at the moment Prism and
>>>> the Flink runner have similar orders of strictness when interpreting the
>>>> pipeline graph while the Python portable runner is far more forgiving.
>>>>
>>>> Also added support for bounded vs unbounded pcollections through the
>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>> believe?
>>>>
>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>> Portable Runner" I think I have something pretty workable.
>>>>>
>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>> means the function signature you use for closures would match full-fledged
>>>>> DoFn definitions for the most part which is satisfying.
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Okay, I tried a couple of different things.
>>>>>>
>>>>>> Implicitly passing the timestamp and window during iteration did not
>>>>>> go well. While physically possible it introduces an invisible side effect
>>>>>> into loop iteration which confused me when I tried to use it and I
>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>> race condition nightmare continuing down that path.
>>>>>>
>>>>>> What I decided to do instead was the following:
>>>>>>
>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>> directly.
>>>>>>
>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>> it will take the timestamp and/or window of the input.
>>>>>>
>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>
>>>>>> Code has been pushed to the branch/PR if you're interested in taking
>>>>>> a look.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>> use @DoFn as a macro)
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>>>
>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>
>>>>>>>>>>
>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>>>> the easiest/most natural.)
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>
>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>
>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>>>> and there.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Sure, we could definitely include things as a submodule for stuff like
testing multi-language, though I think there's actually a cleaner way just
using the Swift package manager's test facilities to access the swift sdk
repo.

 That would also be consistent with the user-side experience and let us
test things like build-time integrations with multi-language as well (which
is possible in Swift through compiler plugins) in the same way as a
pipeline author would. You also maybe get backwards compatibility testing
as a side effect in that case as well.






On Wed, Sep 20, 2023 at 10:20 AM Chamikara Jayalath <ch...@google.com>
wrote:

>
>
>
> On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com> wrote:
>
>> Hi all,
>>
>> I've chatted with a couple of people offline about this and my impression
>> is that folks are generally amenable to a separate repo to match the target
>> community? I have no idea what the next steps would be though other than
>> guessing that there's probably some sort of PMC thing involved? Should I
>> write something up somewhere?
>>
>
> I think the process should be similar to other code/design reviews for
> large contributions. I don't think you need a PMC involvement here.
>
>
>>
>> Best,
>> B
>>
>> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> I've been on vacation, but mostly working on getting External Transform
>>> support going (which in turn basically requires Schema support as well). It
>>> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
>>> use those to do some compile-time automation. In particular, this lets us
>>> do something similar to what Java does with ByteBuddy for generating schema
>>> coders though it has to be ahead of time so not quite the same. (As far as
>>> I can tell this is a reason why macros got added to the language in the
>>> first place---Apple's SwiftData library makes heavy use of the feature).
>>>
>>> I do have one question for the group though: should the Swift SDK
>>> distribution take on Beam community properties or Swift community
>>> properties? Specifically, in the Swift world the Swift SDK would live in
>>> its own repo (beam-swift for example), which allows it to be most easily
>>> consumed and keeps the checkout size under control for users. "Releases" in
>>> the Swift world (much like Go) are just repo tags. The downside here is
>>> that there's overhead in setting up the various github actions and other
>>> CI/CD bits and bobs.
>>>
>>>
>
>> The alternative would be to keep it in the beam repo itself like it is
>>> now, but we'd probably want to move Package.swift to the root since for
>>> whatever reason the Swift community (much to some people's annoyance) has
>>> chosen to have packages only really able to live at the top of a repo. This
>>> has less overhead from a CI/CD perspective, but lots of overhead for users
>>> as they'd be checking out the entire Beam repo to use the SDK, which
>>> happens a lot.
>>>
>>> There's a third option which is basically "do both" but honestly that
>>> just seems like the worst of both worlds as it would require constant
>>> syncing if we wanted to make it possible for Swift users to target
>>> unreleased SDKs for development and testing.
>>>
>>> Personally, I would lean towards the former option (and would volunteer
>>> to set up & document the various automations) as it is lighter for the
>>> actual users of the SDK and more consistent with the community experience
>>> they expect. The CI/CD stuff is mostly a "do it once" whereas checking out
>>> the entire repo with many updates the user doesn't care about is something
>>> they will be doing all the time. FWIW some of our dependencies also chose
>>> this route---most notably GRPC which started with the latter approach and
>>> has moved to the former.
>>>
>>
> I believe existing SDKs benefit from living in the same repo. For example,
> it's easier to keep them consistent with any model/proto changes and it's
> easier to manage distributions/tags. Also it's easier to keep components
> consistent for multi-lang. If we add Swift to a separate repo, we'll
> probably have to add tooling/scripts to keep things consistent.
> Is it possible to create a separate repo, but also add a reference (and
> Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
> sure that things stay consistent ?
>
> Thanks,
> Cham
>
>
>>
>>> Interested to hear any feedback on the subject since I'm guessing it
>>> probably came up with the Go SDK back in the day?
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>>>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>>>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>>>> buffer should be fairly large). It would seem that at the moment Prism and
>>>> the Flink runner have similar orders of strictness when interpreting the
>>>> pipeline graph while the Python portable runner is far more forgiving.
>>>>
>>>> Also added support for bounded vs unbounded pcollections through the
>>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>>> believe?
>>>>
>>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Okay, after a brief detour through "get this working in the Flink
>>>>> Portable Runner" I think I have something pretty workable.
>>>>>
>>>>> PInput and POutput can actually be structs rather than protocols,
>>>>> which simplifies things quite a bit. It also allows us to use them with
>>>>> property wrappers for a SwiftUI-like experience if we want when defining
>>>>> DoFns (which is what I was originally intending to use them for). That also
>>>>> means the function signature you use for closures would match full-fledged
>>>>> DoFn definitions for the most part which is satisfying.
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Okay, I tried a couple of different things.
>>>>>>
>>>>>> Implicitly passing the timestamp and window during iteration did not
>>>>>> go well. While physically possible it introduces an invisible side effect
>>>>>> into loop iteration which confused me when I tried to use it and I
>>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>>> race condition nightmare continuing down that path.
>>>>>>
>>>>>> What I decided to do instead was the following:
>>>>>>
>>>>>> 1. Rename the existing "pardo" functions to "pstream" and require
>>>>>> that they always emit a window and timestamp along with their value. This
>>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>>> directly.
>>>>>>
>>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>>> it will take the timestamp and/or window of the input.
>>>>>>
>>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>>
>>>>>> Code has been pushed to the branch/PR if you're interested in taking
>>>>>> a look.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>>> use @DoFn as a macro)
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>>>
>>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Are you assuming that the same stream (or overlapping sets of
>>>>>>>>> data) are pushed to multiple workers ? I thought that the set of data
>>>>>>>>> streamed here are the data that belong to the current bundle (hence already
>>>>>>>>> assigned to the current worker) so any output from the current bundle
>>>>>>>>> invocation would be a valid output of that bundle.
>>>>>>>>>
>>>>>>>>>>
>>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>>> correlation for automatically propagating metadata.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>>>> the easiest/most natural.)
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>>
>>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>>>> something more explicit).
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>>
>>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> }
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If you had parameterization you wanted to send that would
>>>>>>>>>>>>>> look like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>>>> and there.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Chamikara Jayalath via dev <de...@beam.apache.org>.
On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com> wrote:

> Hi all,
>
> I've chatted with a couple of people offline about this and my impression
> is that folks are generally amenable to a separate repo to match the target
> community? I have no idea what the next steps would be though other than
> guessing that there's probably some sort of PMC thing involved? Should I
> write something up somewhere?
>

I think the process should be similar to other code/design reviews for
large contributions. I don't think you need a PMC involvement here.


>
> Best,
> B
>
> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com> wrote:
>
>> Hi all,
>>
>> I've been on vacation, but mostly working on getting External Transform
>> support going (which in turn basically requires Schema support as well). It
>> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
>> use those to do some compile-time automation. In particular, this lets us
>> do something similar to what Java does with ByteBuddy for generating schema
>> coders though it has to be ahead of time so not quite the same. (As far as
>> I can tell this is a reason why macros got added to the language in the
>> first place---Apple's SwiftData library makes heavy use of the feature).
>>
>> I do have one question for the group though: should the Swift SDK
>> distribution take on Beam community properties or Swift community
>> properties? Specifically, in the Swift world the Swift SDK would live in
>> its own repo (beam-swift for example), which allows it to be most easily
>> consumed and keeps the checkout size under control for users. "Releases" in
>> the Swift world (much like Go) are just repo tags. The downside here is
>> that there's overhead in setting up the various github actions and other
>> CI/CD bits and bobs.
>>
>>

> The alternative would be to keep it in the beam repo itself like it is
>> now, but we'd probably want to move Package.swift to the root since for
>> whatever reason the Swift community (much to some people's annoyance) has
>> chosen to have packages only really able to live at the top of a repo. This
>> has less overhead from a CI/CD perspective, but lots of overhead for users
>> as they'd be checking out the entire Beam repo to use the SDK, which
>> happens a lot.
>>
>> There's a third option which is basically "do both" but honestly that
>> just seems like the worst of both worlds as it would require constant
>> syncing if we wanted to make it possible for Swift users to target
>> unreleased SDKs for development and testing.
>>
>> Personally, I would lean towards the former option (and would volunteer
>> to set up & document the various automations) as it is lighter for the
>> actual users of the SDK and more consistent with the community experience
>> they expect. The CI/CD stuff is mostly a "do it once" whereas checking out
>> the entire repo with many updates the user doesn't care about is something
>> they will be doing all the time. FWIW some of our dependencies also chose
>> this route---most notably GRPC which started with the latter approach and
>> has moved to the former.
>>
>
I believe existing SDKs benefit from living in the same repo. For example,
it's easier to keep them consistent with any model/proto changes and it's
easier to manage distributions/tags. Also it's easier to keep components
consistent for multi-lang. If we add Swift to a separate repo, we'll
probably have to add tooling/scripts to keep things consistent.
Is it possible to create a separate repo, but also add a reference (and
Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
sure that things stay consistent ?

Thanks,
Cham


>
>> Interested to hear any feedback on the subject since I'm guessing it
>> probably came up with the Go SDK back in the day?
>>
>> Best,
>> B
>>
>>
>>
>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>>> buffer should be fairly large). It would seem that at the moment Prism and
>>> the Flink runner have similar orders of strictness when interpreting the
>>> pipeline graph while the Python portable runner is far more forgiving.
>>>
>>> Also added support for bounded vs unbounded pcollections through the
>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>> believe?
>>>
>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Okay, after a brief detour through "get this working in the Flink
>>>> Portable Runner" I think I have something pretty workable.
>>>>
>>>> PInput and POutput can actually be structs rather than protocols, which
>>>> simplifies things quite a bit. It also allows us to use them with property
>>>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>>>> (which is what I was originally intending to use them for). That also means
>>>> the function signature you use for closures would match full-fledged DoFn
>>>> definitions for the most part which is satisfying.
>>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Okay, I tried a couple of different things.
>>>>>
>>>>> Implicitly passing the timestamp and window during iteration did not
>>>>> go well. While physically possible it introduces an invisible side effect
>>>>> into loop iteration which confused me when I tried to use it and I
>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>> race condition nightmare continuing down that path.
>>>>>
>>>>> What I decided to do instead was the following:
>>>>>
>>>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>>>> they always emit a window and timestamp along with their value. This
>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>> directly.
>>>>>
>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>> it will take the timestamp and/or window of the input.
>>>>>
>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>
>>>>> Code has been pushed to the branch/PR if you're interested in taking a
>>>>> look.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>> use @DoFn as a macro)
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>> chamikara@google.com> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>>
>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>>>> would be a valid output of that bundle.
>>>>>>>>
>>>>>>>>>
>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>> correlation for automatically propagating metadata.
>>>>>>>
>>>>>>>
>>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>>> the easiest/most natural.)
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>
>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>>> something more explicit).
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>
>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>
>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>
>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>
>>>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>
>>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>
>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>>> and there.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Chamikara Jayalath via user <us...@beam.apache.org>.
On Wed, Sep 20, 2023 at 9:54 AM Byron Ellis <by...@google.com> wrote:

> Hi all,
>
> I've chatted with a couple of people offline about this and my impression
> is that folks are generally amenable to a separate repo to match the target
> community? I have no idea what the next steps would be though other than
> guessing that there's probably some sort of PMC thing involved? Should I
> write something up somewhere?
>

I think the process should be similar to other code/design reviews for
large contributions. I don't think you need a PMC involvement here.


>
> Best,
> B
>
> On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com> wrote:
>
>> Hi all,
>>
>> I've been on vacation, but mostly working on getting External Transform
>> support going (which in turn basically requires Schema support as well). It
>> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
>> use those to do some compile-time automation. In particular, this lets us
>> do something similar to what Java does with ByteBuddy for generating schema
>> coders though it has to be ahead of time so not quite the same. (As far as
>> I can tell this is a reason why macros got added to the language in the
>> first place---Apple's SwiftData library makes heavy use of the feature).
>>
>> I do have one question for the group though: should the Swift SDK
>> distribution take on Beam community properties or Swift community
>> properties? Specifically, in the Swift world the Swift SDK would live in
>> its own repo (beam-swift for example), which allows it to be most easily
>> consumed and keeps the checkout size under control for users. "Releases" in
>> the Swift world (much like Go) are just repo tags. The downside here is
>> that there's overhead in setting up the various github actions and other
>> CI/CD bits and bobs.
>>
>>

> The alternative would be to keep it in the beam repo itself like it is
>> now, but we'd probably want to move Package.swift to the root since for
>> whatever reason the Swift community (much to some people's annoyance) has
>> chosen to have packages only really able to live at the top of a repo. This
>> has less overhead from a CI/CD perspective, but lots of overhead for users
>> as they'd be checking out the entire Beam repo to use the SDK, which
>> happens a lot.
>>
>> There's a third option which is basically "do both" but honestly that
>> just seems like the worst of both worlds as it would require constant
>> syncing if we wanted to make it possible for Swift users to target
>> unreleased SDKs for development and testing.
>>
>> Personally, I would lean towards the former option (and would volunteer
>> to set up & document the various automations) as it is lighter for the
>> actual users of the SDK and more consistent with the community experience
>> they expect. The CI/CD stuff is mostly a "do it once" whereas checking out
>> the entire repo with many updates the user doesn't care about is something
>> they will be doing all the time. FWIW some of our dependencies also chose
>> this route---most notably GRPC which started with the latter approach and
>> has moved to the former.
>>
>
I believe existing SDKs benefit from living in the same repo. For example,
it's easier to keep them consistent with any model/proto changes and it's
easier to manage distributions/tags. Also it's easier to keep components
consistent for multi-lang. If we add Swift to a separate repo, we'll
probably have to add tooling/scripts to keep things consistent.
Is it possible to create a separate repo, but also add a reference (and
Gradle tasks) under "beam/sdks/swift" so that we can add Beam tests to make
sure that things stay consistent ?

Thanks,
Cham


>
>> Interested to hear any feedback on the subject since I'm guessing it
>> probably came up with the Go SDK back in the day?
>>
>> Best,
>> B
>>
>>
>>
>> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>>> buffer should be fairly large). It would seem that at the moment Prism and
>>> the Flink runner have similar orders of strictness when interpreting the
>>> pipeline graph while the Python portable runner is far more forgiving.
>>>
>>> Also added support for bounded vs unbounded pcollections through the
>>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>>> believe?
>>>
>>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Okay, after a brief detour through "get this working in the Flink
>>>> Portable Runner" I think I have something pretty workable.
>>>>
>>>> PInput and POutput can actually be structs rather than protocols, which
>>>> simplifies things quite a bit. It also allows us to use them with property
>>>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>>>> (which is what I was originally intending to use them for). That also means
>>>> the function signature you use for closures would match full-fledged DoFn
>>>> definitions for the most part which is satisfying.
>>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Okay, I tried a couple of different things.
>>>>>
>>>>> Implicitly passing the timestamp and window during iteration did not
>>>>> go well. While physically possible it introduces an invisible side effect
>>>>> into loop iteration which confused me when I tried to use it and I
>>>>> implemented it. Also, I'm pretty sure there'd end up being some sort of
>>>>> race condition nightmare continuing down that path.
>>>>>
>>>>> What I decided to do instead was the following:
>>>>>
>>>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>>>> they always emit a window and timestamp along with their value. This
>>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>>> of pstream, but the expectation here is that if you're using pstream
>>>>> functions You Know What You Are Doing and most people won't be using it
>>>>> directly.
>>>>>
>>>>> 2. Introduce a new set of pardo functions (I didn't do all of them
>>>>> yet, but enough to test the functionality and decide I liked it) which take
>>>>> a function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>>> it will take the timestamp and/or window of the input.
>>>>>
>>>>> Trying to use that was pretty pleasant to use so I think we should
>>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>>
>>>>> Code has been pushed to the branch/PR if you're interested in taking a
>>>>> look.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>>> use @DoFn as a macro)
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>>> chamikara@google.com> wrote:
>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>>
>>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>>
>>>>>>>>
>>>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>>>> would be a valid output of that bundle.
>>>>>>>>
>>>>>>>>>
>>>>>>> Yes, the content of the stream is exactly the contents of the
>>>>>>> bundle. The question is how to do the input_element:output_element
>>>>>>> correlation for automatically propagating metadata.
>>>>>>>
>>>>>>>
>>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>>> the easiest/most natural.)
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <
>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Ah, I see.
>>>>>>>>>>>
>>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>>> something more explicit).
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>>
>>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>>
>>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>>
>>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>>   ...
>>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>>
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>>
>>>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you
>>>>>>>>>>>>> have in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>>> it yet.
>>>>>>>>>>>>>
>>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>>
>>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>>
>>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>>> and there.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet
>>>>>>>>>>>>>>> though (there's a good chance there are a few places that need to properly
>>>>>>>>>>>>>>> address endianness. Specifically timestamps in windowed values and length
>>>>>>>>>>>>>>> in iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta
>>>>>>>>>>>>>>>>>> using Swift 5.9 on Intel Macs, but should also work using beta builds of
>>>>>>>>>>>>>>>>>> 5.9 for Linux running on Intel hardware. I haven't had a chance to try it
>>>>>>>>>>>>>>>>>> on ARM hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Hi all,

I've chatted with a couple of people offline about this and my impression
is that folks are generally amenable to a separate repo to match the target
community? I have no idea what the next steps would be though other than
guessing that there's probably some sort of PMC thing involved? Should I
write something up somewhere?

Best,
B

On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com> wrote:

> Hi all,
>
> I've been on vacation, but mostly working on getting External Transform
> support going (which in turn basically requires Schema support as well). It
> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
> use those to do some compile-time automation. In particular, this lets us
> do something similar to what Java does with ByteBuddy for generating schema
> coders though it has to be ahead of time so not quite the same. (As far as
> I can tell this is a reason why macros got added to the language in the
> first place---Apple's SwiftData library makes heavy use of the feature).
>
> I do have one question for the group though: should the Swift SDK
> distribution take on Beam community properties or Swift community
> properties? Specifically, in the Swift world the Swift SDK would live in
> its own repo (beam-swift for example), which allows it to be most easily
> consumed and keeps the checkout size under control for users. "Releases" in
> the Swift world (much like Go) are just repo tags. The downside here is
> that there's overhead in setting up the various github actions and other
> CI/CD bits and bobs.
>
> The alternative would be to keep it in the beam repo itself like it is
> now, but we'd probably want to move Package.swift to the root since for
> whatever reason the Swift community (much to some people's annoyance) has
> chosen to have packages only really able to live at the top of a repo. This
> has less overhead from a CI/CD perspective, but lots of overhead for users
> as they'd be checking out the entire Beam repo to use the SDK, which
> happens a lot.
>
> There's a third option which is basically "do both" but honestly that just
> seems like the worst of both worlds as it would require constant syncing if
> we wanted to make it possible for Swift users to target unreleased SDKs for
> development and testing.
>
> Personally, I would lean towards the former option (and would volunteer to
> set up & document the various automations) as it is lighter for the actual
> users of the SDK and more consistent with the community experience they
> expect. The CI/CD stuff is mostly a "do it once" whereas checking out the
> entire repo with many updates the user doesn't care about is something they
> will be doing all the time. FWIW some of our dependencies also chose this
> route---most notably GRPC which started with the latter approach and has
> moved to the former.
>
> Interested to hear any feedback on the subject since I'm guessing it
> probably came up with the Go SDK back in the day?
>
> Best,
> B
>
>
>
> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com> wrote:
>
>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>> buffer should be fairly large). It would seem that at the moment Prism and
>> the Flink runner have similar orders of strictness when interpreting the
>> pipeline graph while the Python portable runner is far more forgiving.
>>
>> Also added support for bounded vs unbounded pcollections through the
>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>> believe?
>>
>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Okay, after a brief detour through "get this working in the Flink
>>> Portable Runner" I think I have something pretty workable.
>>>
>>> PInput and POutput can actually be structs rather than protocols, which
>>> simplifies things quite a bit. It also allows us to use them with property
>>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>>> (which is what I was originally intending to use them for). That also means
>>> the function signature you use for closures would match full-fledged DoFn
>>> definitions for the most part which is satisfying.
>>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Okay, I tried a couple of different things.
>>>>
>>>> Implicitly passing the timestamp and window during iteration did not go
>>>> well. While physically possible it introduces an invisible side effect into
>>>> loop iteration which confused me when I tried to use it and I implemented
>>>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>>>> nightmare continuing down that path.
>>>>
>>>> What I decided to do instead was the following:
>>>>
>>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>>> they always emit a window and timestamp along with their value. This
>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>> of pstream, but the expectation here is that if you're using pstream
>>>> functions You Know What You Are Doing and most people won't be using it
>>>> directly.
>>>>
>>>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>>>> but enough to test the functionality and decide I liked it) which take a
>>>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>> it will take the timestamp and/or window of the input.
>>>>
>>>> Trying to use that was pretty pleasant to use so I think we should
>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>
>>>> Code has been pushed to the branch/PR if you're interested in taking a
>>>> look.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>> use @DoFn as a macro)
>>>>>
>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>> chamikara@google.com> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>>
>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>
>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>
>>>>>>>
>>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>>> would be a valid output of that bundle.
>>>>>>>
>>>>>>>>
>>>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>>>> The question is how to do the input_element:output_element correlation for
>>>>>> automatically propagating metadata.
>>>>>>
>>>>>>
>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>> the easiest/most natural.)
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Ah, I see.
>>>>>>>>>>
>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>> something more explicit).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>
>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>
>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>
>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>   ...
>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>
>>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>
>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>> it yet.
>>>>>>>>>>>>
>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>
>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>
>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>> and there.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Hi all,

I've chatted with a couple of people offline about this and my impression
is that folks are generally amenable to a separate repo to match the target
community? I have no idea what the next steps would be though other than
guessing that there's probably some sort of PMC thing involved? Should I
write something up somewhere?

Best,
B

On Thu, Sep 14, 2023 at 9:00 AM Byron Ellis <by...@google.com> wrote:

> Hi all,
>
> I've been on vacation, but mostly working on getting External Transform
> support going (which in turn basically requires Schema support as well). It
> also looks like macros landed in Swift 5.9 for Linux so we'll be able to
> use those to do some compile-time automation. In particular, this lets us
> do something similar to what Java does with ByteBuddy for generating schema
> coders though it has to be ahead of time so not quite the same. (As far as
> I can tell this is a reason why macros got added to the language in the
> first place---Apple's SwiftData library makes heavy use of the feature).
>
> I do have one question for the group though: should the Swift SDK
> distribution take on Beam community properties or Swift community
> properties? Specifically, in the Swift world the Swift SDK would live in
> its own repo (beam-swift for example), which allows it to be most easily
> consumed and keeps the checkout size under control for users. "Releases" in
> the Swift world (much like Go) are just repo tags. The downside here is
> that there's overhead in setting up the various github actions and other
> CI/CD bits and bobs.
>
> The alternative would be to keep it in the beam repo itself like it is
> now, but we'd probably want to move Package.swift to the root since for
> whatever reason the Swift community (much to some people's annoyance) has
> chosen to have packages only really able to live at the top of a repo. This
> has less overhead from a CI/CD perspective, but lots of overhead for users
> as they'd be checking out the entire Beam repo to use the SDK, which
> happens a lot.
>
> There's a third option which is basically "do both" but honestly that just
> seems like the worst of both worlds as it would require constant syncing if
> we wanted to make it possible for Swift users to target unreleased SDKs for
> development and testing.
>
> Personally, I would lean towards the former option (and would volunteer to
> set up & document the various automations) as it is lighter for the actual
> users of the SDK and more consistent with the community experience they
> expect. The CI/CD stuff is mostly a "do it once" whereas checking out the
> entire repo with many updates the user doesn't care about is something they
> will be doing all the time. FWIW some of our dependencies also chose this
> route---most notably GRPC which started with the latter approach and has
> moved to the former.
>
> Interested to hear any feedback on the subject since I'm guessing it
> probably came up with the Go SDK back in the day?
>
> Best,
> B
>
>
>
> On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com> wrote:
>
>> After a couple of iterations (thanks rebo!) we've also gotten the Swift
>> SDK working with the new Prism runner. The fact that it doesn't do fusion
>> caught a couple of configuration bugs (e.g. that the grpc message receiver
>> buffer should be fairly large). It would seem that at the moment Prism and
>> the Flink runner have similar orders of strictness when interpreting the
>> pipeline graph while the Python portable runner is far more forgiving.
>>
>> Also added support for bounded vs unbounded pcollections through the
>> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
>> believe?
>>
>> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Okay, after a brief detour through "get this working in the Flink
>>> Portable Runner" I think I have something pretty workable.
>>>
>>> PInput and POutput can actually be structs rather than protocols, which
>>> simplifies things quite a bit. It also allows us to use them with property
>>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>>> (which is what I was originally intending to use them for). That also means
>>> the function signature you use for closures would match full-fledged DoFn
>>> definitions for the most part which is satisfying.
>>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Okay, I tried a couple of different things.
>>>>
>>>> Implicitly passing the timestamp and window during iteration did not go
>>>> well. While physically possible it introduces an invisible side effect into
>>>> loop iteration which confused me when I tried to use it and I implemented
>>>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>>>> nightmare continuing down that path.
>>>>
>>>> What I decided to do instead was the following:
>>>>
>>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>>> they always emit a window and timestamp along with their value. This
>>>> eliminates the side effect but lets us keep iteration in a bundle where
>>>> that might be convenient. For example, in my cheesy GCS implementation it
>>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>>> as a local variable, which is convenient. It's a bit more typing for users
>>>> of pstream, but the expectation here is that if you're using pstream
>>>> functions You Know What You Are Doing and most people won't be using it
>>>> directly.
>>>>
>>>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>>>> but enough to test the functionality and decide I liked it) which take a
>>>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>>> struct with friendlier names. Not strictly necessary, but makes the code
>>>> nicer to read I think. POutput introduces emit functions that optionally
>>>> allow you to specify a timestamp and a window. If you don't for either one
>>>> it will take the timestamp and/or window of the input.
>>>>
>>>> Trying to use that was pretty pleasant to use so I think we should
>>>> continue down that path. If you'd like to see it in use, I reimplemented
>>>> map() and flatMap() in terms of this new pardo functionality.
>>>>
>>>> Code has been pushed to the branch/PR if you're interested in taking a
>>>> look.
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Gotcha, I think there's a fairly easy solution to link input and
>>>>> output streams.... Let me try it out... might even be possible to have both
>>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>>> use @DoFn as a macro)
>>>>>
>>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>>> chamikara@google.com> wrote:
>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>>
>>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>>> work, as I think it's more natural overall.
>>>>>>>>
>>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>>
>>>>>>>
>>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>>> would be a valid output of that bundle.
>>>>>>>
>>>>>>>>
>>>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>>>> The question is how to do the input_element:output_element correlation for
>>>>>> automatically propagating metadata.
>>>>>>
>>>>>>
>>>>>>> Related to this, we could enforce that the only (user-accessible)
>>>>>>>> way to get such a timestamped value is to start with one, e.g. a
>>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>>> the easiest/most natural.)
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>>
>>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Ah, I see.
>>>>>>>>>>
>>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>>> something more explicit).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>>
>>>>>>>>>>> I just happened to push some "IO primitives" that uses map
>>>>>>>>>>> rather than pardo in a couple of places to do a true wordcount using good
>>>>>>>>>>> ol' Shakespeare and very very primitive GCS IO.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax
>>>>>>>>>>>> quite a bit before settling on where I ended up. Ultimately I decided to go
>>>>>>>>>>>> with something that felt more Swift-y than anything else which means that
>>>>>>>>>>>> rather than dealing with a single element like you do in the other SDKs
>>>>>>>>>>>> you're dealing with a stream of elements (which of course will often be of
>>>>>>>>>>>> size 1). That's a really natural paradigm in the Swift world especially
>>>>>>>>>>>> with the async / await structures. So when you see something like:
>>>>>>>>>>>>
>>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>>
>>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>>   ...
>>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>>
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>>
>>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>>> it somehow.
>>>>>>>>>>>>
>>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>>> it yet.
>>>>>>>>>>>>
>>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>>
>>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>>> work with GCS.
>>>>>>>>>>>>
>>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here
>>>>>>>>>>>> and there.
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Neat.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and
>>>>>>>>>>>>>>>>> incorporated some of that feedback already (for example pardo was
>>>>>>>>>>>>>>>>> originally called dofn when defining pipelines). In general I've tried to
>>>>>>>>>>>>>>>>> make the API as "Swift-y" as possible, hence the heavy reliance on closures
>>>>>>>>>>>>>>>>> and while there aren't yet composite PTransforms there's the beginnings of
>>>>>>>>>>>>>>>>> what would be needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Hi all,

I've been on vacation, but mostly working on getting External Transform
support going (which in turn basically requires Schema support as well). It
also looks like macros landed in Swift 5.9 for Linux so we'll be able to
use those to do some compile-time automation. In particular, this lets us
do something similar to what Java does with ByteBuddy for generating schema
coders though it has to be ahead of time so not quite the same. (As far as
I can tell this is a reason why macros got added to the language in the
first place---Apple's SwiftData library makes heavy use of the feature).

I do have one question for the group though: should the Swift SDK
distribution take on Beam community properties or Swift community
properties? Specifically, in the Swift world the Swift SDK would live in
its own repo (beam-swift for example), which allows it to be most easily
consumed and keeps the checkout size under control for users. "Releases" in
the Swift world (much like Go) are just repo tags. The downside here is
that there's overhead in setting up the various github actions and other
CI/CD bits and bobs.

The alternative would be to keep it in the beam repo itself like it is now,
but we'd probably want to move Package.swift to the root since for whatever
reason the Swift community (much to some people's annoyance) has chosen to
have packages only really able to live at the top of a repo. This has less
overhead from a CI/CD perspective, but lots of overhead for users as they'd
be checking out the entire Beam repo to use the SDK, which happens a lot.

There's a third option which is basically "do both" but honestly that just
seems like the worst of both worlds as it would require constant syncing if
we wanted to make it possible for Swift users to target unreleased SDKs for
development and testing.

Personally, I would lean towards the former option (and would volunteer to
set up & document the various automations) as it is lighter for the actual
users of the SDK and more consistent with the community experience they
expect. The CI/CD stuff is mostly a "do it once" whereas checking out the
entire repo with many updates the user doesn't care about is something they
will be doing all the time. FWIW some of our dependencies also chose this
route---most notably GRPC which started with the latter approach and has
moved to the former.

Interested to hear any feedback on the subject since I'm guessing it
probably came up with the Go SDK back in the day?

Best,
B



On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com> wrote:

> After a couple of iterations (thanks rebo!) we've also gotten the Swift
> SDK working with the new Prism runner. The fact that it doesn't do fusion
> caught a couple of configuration bugs (e.g. that the grpc message receiver
> buffer should be fairly large). It would seem that at the moment Prism and
> the Flink runner have similar orders of strictness when interpreting the
> pipeline graph while the Python portable runner is far more forgiving.
>
> Also added support for bounded vs unbounded pcollections through the
> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
> believe?
>
> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com> wrote:
>
>> Okay, after a brief detour through "get this working in the Flink
>> Portable Runner" I think I have something pretty workable.
>>
>> PInput and POutput can actually be structs rather than protocols, which
>> simplifies things quite a bit. It also allows us to use them with property
>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>> (which is what I was originally intending to use them for). That also means
>> the function signature you use for closures would match full-fledged DoFn
>> definitions for the most part which is satisfying.
>>
>>
>>
>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Okay, I tried a couple of different things.
>>>
>>> Implicitly passing the timestamp and window during iteration did not go
>>> well. While physically possible it introduces an invisible side effect into
>>> loop iteration which confused me when I tried to use it and I implemented
>>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>>> nightmare continuing down that path.
>>>
>>> What I decided to do instead was the following:
>>>
>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>> they always emit a window and timestamp along with their value. This
>>> eliminates the side effect but lets us keep iteration in a bundle where
>>> that might be convenient. For example, in my cheesy GCS implementation it
>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>> as a local variable, which is convenient. It's a bit more typing for users
>>> of pstream, but the expectation here is that if you're using pstream
>>> functions You Know What You Are Doing and most people won't be using it
>>> directly.
>>>
>>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>>> but enough to test the functionality and decide I liked it) which take a
>>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>> struct with friendlier names. Not strictly necessary, but makes the code
>>> nicer to read I think. POutput introduces emit functions that optionally
>>> allow you to specify a timestamp and a window. If you don't for either one
>>> it will take the timestamp and/or window of the input.
>>>
>>> Trying to use that was pretty pleasant to use so I think we should
>>> continue down that path. If you'd like to see it in use, I reimplemented
>>> map() and flatMap() in terms of this new pardo functionality.
>>>
>>> Code has been pushed to the branch/PR if you're interested in taking a
>>> look.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Gotcha, I think there's a fairly easy solution to link input and output
>>>> streams.... Let me try it out... might even be possible to have both
>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>> use @DoFn as a macro)
>>>>
>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>> work, as I think it's more natural overall.
>>>>>>>
>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>
>>>>>>
>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>> would be a valid output of that bundle.
>>>>>>
>>>>>>>
>>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>>> The question is how to do the input_element:output_element correlation for
>>>>> automatically propagating metadata.
>>>>>
>>>>>
>>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>> the easiest/most natural.)
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> Ah, I see.
>>>>>>>>>
>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>> something more explicit).
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>
>>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>>
>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>
>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>   ...
>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>
>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>> it somehow.
>>>>>>>>>>>
>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>> it yet.
>>>>>>>>>>>
>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>
>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>> work with GCS.
>>>>>>>>>>>
>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>>> there.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Neat.
>>>>>>>>>>>>
>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>
>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Hi all,

I've been on vacation, but mostly working on getting External Transform
support going (which in turn basically requires Schema support as well). It
also looks like macros landed in Swift 5.9 for Linux so we'll be able to
use those to do some compile-time automation. In particular, this lets us
do something similar to what Java does with ByteBuddy for generating schema
coders though it has to be ahead of time so not quite the same. (As far as
I can tell this is a reason why macros got added to the language in the
first place---Apple's SwiftData library makes heavy use of the feature).

I do have one question for the group though: should the Swift SDK
distribution take on Beam community properties or Swift community
properties? Specifically, in the Swift world the Swift SDK would live in
its own repo (beam-swift for example), which allows it to be most easily
consumed and keeps the checkout size under control for users. "Releases" in
the Swift world (much like Go) are just repo tags. The downside here is
that there's overhead in setting up the various github actions and other
CI/CD bits and bobs.

The alternative would be to keep it in the beam repo itself like it is now,
but we'd probably want to move Package.swift to the root since for whatever
reason the Swift community (much to some people's annoyance) has chosen to
have packages only really able to live at the top of a repo. This has less
overhead from a CI/CD perspective, but lots of overhead for users as they'd
be checking out the entire Beam repo to use the SDK, which happens a lot.

There's a third option which is basically "do both" but honestly that just
seems like the worst of both worlds as it would require constant syncing if
we wanted to make it possible for Swift users to target unreleased SDKs for
development and testing.

Personally, I would lean towards the former option (and would volunteer to
set up & document the various automations) as it is lighter for the actual
users of the SDK and more consistent with the community experience they
expect. The CI/CD stuff is mostly a "do it once" whereas checking out the
entire repo with many updates the user doesn't care about is something they
will be doing all the time. FWIW some of our dependencies also chose this
route---most notably GRPC which started with the latter approach and has
moved to the former.

Interested to hear any feedback on the subject since I'm guessing it
probably came up with the Go SDK back in the day?

Best,
B



On Tue, Aug 29, 2023 at 7:59 AM Byron Ellis <by...@google.com> wrote:

> After a couple of iterations (thanks rebo!) we've also gotten the Swift
> SDK working with the new Prism runner. The fact that it doesn't do fusion
> caught a couple of configuration bugs (e.g. that the grpc message receiver
> buffer should be fairly large). It would seem that at the moment Prism and
> the Flink runner have similar orders of strictness when interpreting the
> pipeline graph while the Python portable runner is far more forgiving.
>
> Also added support for bounded vs unbounded pcollections through the
> "type" parameter when adding a pardo. Impulse is a bounded pcollection I
> believe?
>
> On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com> wrote:
>
>> Okay, after a brief detour through "get this working in the Flink
>> Portable Runner" I think I have something pretty workable.
>>
>> PInput and POutput can actually be structs rather than protocols, which
>> simplifies things quite a bit. It also allows us to use them with property
>> wrappers for a SwiftUI-like experience if we want when defining DoFns
>> (which is what I was originally intending to use them for). That also means
>> the function signature you use for closures would match full-fledged DoFn
>> definitions for the most part which is satisfying.
>>
>>
>>
>> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Okay, I tried a couple of different things.
>>>
>>> Implicitly passing the timestamp and window during iteration did not go
>>> well. While physically possible it introduces an invisible side effect into
>>> loop iteration which confused me when I tried to use it and I implemented
>>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>>> nightmare continuing down that path.
>>>
>>> What I decided to do instead was the following:
>>>
>>> 1. Rename the existing "pardo" functions to "pstream" and require that
>>> they always emit a window and timestamp along with their value. This
>>> eliminates the side effect but lets us keep iteration in a bundle where
>>> that might be convenient. For example, in my cheesy GCS implementation it
>>> means that I can keep an OAuth token around for the lifetime of the bundle
>>> as a local variable, which is convenient. It's a bit more typing for users
>>> of pstream, but the expectation here is that if you're using pstream
>>> functions You Know What You Are Doing and most people won't be using it
>>> directly.
>>>
>>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>>> but enough to test the functionality and decide I liked it) which take a
>>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>>> struct with friendlier names. Not strictly necessary, but makes the code
>>> nicer to read I think. POutput introduces emit functions that optionally
>>> allow you to specify a timestamp and a window. If you don't for either one
>>> it will take the timestamp and/or window of the input.
>>>
>>> Trying to use that was pretty pleasant to use so I think we should
>>> continue down that path. If you'd like to see it in use, I reimplemented
>>> map() and flatMap() in terms of this new pardo functionality.
>>>
>>> Code has been pushed to the branch/PR if you're interested in taking a
>>> look.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Gotcha, I think there's a fairly easy solution to link input and output
>>>> streams.... Let me try it out... might even be possible to have both
>>>> element and stream-wise closure pardos. Definitely possible to have that at
>>>> the DoFn level (called SerializableFn in the SDK because I want to
>>>> use @DoFn as a macro)
>>>>
>>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>>> work, as I think it's more natural overall.
>>>>>>>
>>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>>> one to emit timestamp-windowed-values.
>>>>>>>
>>>>>>
>>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>>> to the current worker) so any output from the current bundle invocation
>>>>>> would be a valid output of that bundle.
>>>>>>
>>>>>>>
>>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>>> The question is how to do the input_element:output_element correlation for
>>>>> automatically propagating metadata.
>>>>>
>>>>>
>>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>>> the easiest/most natural.)
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>>
>>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> Ah, I see.
>>>>>>>>>
>>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>>> something more explicit).
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>>
>>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>>
>>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>>
>>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>>   ...
>>>>>>>>>>>   output.emit(data)
>>>>>>>>>>>
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> filenames is the input stream and then output and errors are
>>>>>>>>>>> both output streams. In theory you can have as many output streams as you
>>>>>>>>>>> like though at the moment there's a compiler bug in the new type pack
>>>>>>>>>>> feature that limits it to "as many as I felt like supporting". Presumably
>>>>>>>>>>> this will get fixed before the official 5.9 release which will probably be
>>>>>>>>>>> in the October timeframe if history is any guide)
>>>>>>>>>>>
>>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>>> it somehow.
>>>>>>>>>>>
>>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>>> it yet.
>>>>>>>>>>>
>>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>>
>>>>>>>>>>> That said, there are some interesting IO possibilities that
>>>>>>>>>>> would be Swift native. In particularly, I've been looking at the native
>>>>>>>>>>> Swift binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>>> work with GCS.
>>>>>>>>>>>
>>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>>> there.
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Neat.
>>>>>>>>>>>>
>>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>>> other SDKs.
>>>>>>>>>>>>
>>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>>
>>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
After a couple of iterations (thanks rebo!) we've also gotten the Swift SDK
working with the new Prism runner. The fact that it doesn't do fusion
caught a couple of configuration bugs (e.g. that the grpc message receiver
buffer should be fairly large). It would seem that at the moment Prism and
the Flink runner have similar orders of strictness when interpreting the
pipeline graph while the Python portable runner is far more forgiving.

Also added support for bounded vs unbounded pcollections through the "type"
parameter when adding a pardo. Impulse is a bounded pcollection I believe?

On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com> wrote:

> Okay, after a brief detour through "get this working in the Flink Portable
> Runner" I think I have something pretty workable.
>
> PInput and POutput can actually be structs rather than protocols, which
> simplifies things quite a bit. It also allows us to use them with property
> wrappers for a SwiftUI-like experience if we want when defining DoFns
> (which is what I was originally intending to use them for). That also means
> the function signature you use for closures would match full-fledged DoFn
> definitions for the most part which is satisfying.
>
>
>
> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com> wrote:
>
>> Okay, I tried a couple of different things.
>>
>> Implicitly passing the timestamp and window during iteration did not go
>> well. While physically possible it introduces an invisible side effect into
>> loop iteration which confused me when I tried to use it and I implemented
>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>> nightmare continuing down that path.
>>
>> What I decided to do instead was the following:
>>
>> 1. Rename the existing "pardo" functions to "pstream" and require that
>> they always emit a window and timestamp along with their value. This
>> eliminates the side effect but lets us keep iteration in a bundle where
>> that might be convenient. For example, in my cheesy GCS implementation it
>> means that I can keep an OAuth token around for the lifetime of the bundle
>> as a local variable, which is convenient. It's a bit more typing for users
>> of pstream, but the expectation here is that if you're using pstream
>> functions You Know What You Are Doing and most people won't be using it
>> directly.
>>
>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>> but enough to test the functionality and decide I liked it) which take a
>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>> struct with friendlier names. Not strictly necessary, but makes the code
>> nicer to read I think. POutput introduces emit functions that optionally
>> allow you to specify a timestamp and a window. If you don't for either one
>> it will take the timestamp and/or window of the input.
>>
>> Trying to use that was pretty pleasant to use so I think we should
>> continue down that path. If you'd like to see it in use, I reimplemented
>> map() and flatMap() in terms of this new pardo functionality.
>>
>> Code has been pushed to the branch/PR if you're interested in taking a
>> look.
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Gotcha, I think there's a fairly easy solution to link input and output
>>> streams.... Let me try it out... might even be possible to have both
>>> element and stream-wise closure pardos. Definitely possible to have that at
>>> the DoFn level (called SerializableFn in the SDK because I want to
>>> use @DoFn as a macro)
>>>
>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>> work, as I think it's more natural overall.
>>>>>>
>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>> one to emit timestamp-windowed-values.
>>>>>>
>>>>>
>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>> to the current worker) so any output from the current bundle invocation
>>>>> would be a valid output of that bundle.
>>>>>
>>>>>>
>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>> The question is how to do the input_element:output_element correlation for
>>>> automatically propagating metadata.
>>>>
>>>>
>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>> the easiest/most natural.)
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>>
>>>>>>>> Ah, I see.
>>>>>>>>
>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>> something more explicit).
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>
>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>
>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>
>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>   ...
>>>>>>>>>>   output.emit(data)
>>>>>>>>>>
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>>>> October timeframe if history is any guide)
>>>>>>>>>>
>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>> it somehow.
>>>>>>>>>>
>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>> it yet.
>>>>>>>>>>
>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>
>>>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>> work with GCS.
>>>>>>>>>>
>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>> there.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Neat.
>>>>>>>>>>>
>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>> other SDKs.
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>
>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>
>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
After a couple of iterations (thanks rebo!) we've also gotten the Swift SDK
working with the new Prism runner. The fact that it doesn't do fusion
caught a couple of configuration bugs (e.g. that the grpc message receiver
buffer should be fairly large). It would seem that at the moment Prism and
the Flink runner have similar orders of strictness when interpreting the
pipeline graph while the Python portable runner is far more forgiving.

Also added support for bounded vs unbounded pcollections through the "type"
parameter when adding a pardo. Impulse is a bounded pcollection I believe?

On Fri, Aug 25, 2023 at 2:04 PM Byron Ellis <by...@google.com> wrote:

> Okay, after a brief detour through "get this working in the Flink Portable
> Runner" I think I have something pretty workable.
>
> PInput and POutput can actually be structs rather than protocols, which
> simplifies things quite a bit. It also allows us to use them with property
> wrappers for a SwiftUI-like experience if we want when defining DoFns
> (which is what I was originally intending to use them for). That also means
> the function signature you use for closures would match full-fledged DoFn
> definitions for the most part which is satisfying.
>
>
>
> On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com> wrote:
>
>> Okay, I tried a couple of different things.
>>
>> Implicitly passing the timestamp and window during iteration did not go
>> well. While physically possible it introduces an invisible side effect into
>> loop iteration which confused me when I tried to use it and I implemented
>> it. Also, I'm pretty sure there'd end up being some sort of race condition
>> nightmare continuing down that path.
>>
>> What I decided to do instead was the following:
>>
>> 1. Rename the existing "pardo" functions to "pstream" and require that
>> they always emit a window and timestamp along with their value. This
>> eliminates the side effect but lets us keep iteration in a bundle where
>> that might be convenient. For example, in my cheesy GCS implementation it
>> means that I can keep an OAuth token around for the lifetime of the bundle
>> as a local variable, which is convenient. It's a bit more typing for users
>> of pstream, but the expectation here is that if you're using pstream
>> functions You Know What You Are Doing and most people won't be using it
>> directly.
>>
>> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
>> but enough to test the functionality and decide I liked it) which take a
>> function signature of (any PInput<InputType>,any POutput<OutputType>).
>> PInput takes the (InputType,Date,Window) tuple and converts it into a
>> struct with friendlier names. Not strictly necessary, but makes the code
>> nicer to read I think. POutput introduces emit functions that optionally
>> allow you to specify a timestamp and a window. If you don't for either one
>> it will take the timestamp and/or window of the input.
>>
>> Trying to use that was pretty pleasant to use so I think we should
>> continue down that path. If you'd like to see it in use, I reimplemented
>> map() and flatMap() in terms of this new pardo functionality.
>>
>> Code has been pushed to the branch/PR if you're interested in taking a
>> look.
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Gotcha, I think there's a fairly easy solution to link input and output
>>> streams.... Let me try it out... might even be possible to have both
>>> element and stream-wise closure pardos. Definitely possible to have that at
>>> the DoFn level (called SerializableFn in the SDK because I want to
>>> use @DoFn as a macro)
>>>
>>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>>
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I would like to figure out a way to get the stream-y interface to
>>>>>> work, as I think it's more natural overall.
>>>>>>
>>>>>> One hypothesis is that if any elements are carried over loop
>>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>>> one to emit timestamp-windowed-values.
>>>>>>
>>>>>
>>>>> Are you assuming that the same stream (or overlapping sets of data)
>>>>> are pushed to multiple workers ? I thought that the set of data streamed
>>>>> here are the data that belong to the current bundle (hence already assigned
>>>>> to the current worker) so any output from the current bundle invocation
>>>>> would be a valid output of that bundle.
>>>>>
>>>>>>
>>>> Yes, the content of the stream is exactly the contents of the bundle.
>>>> The question is how to do the input_element:output_element correlation for
>>>> automatically propagating metadata.
>>>>
>>>>
>>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>>> have to explicitly request iteration over these windowed values rather than
>>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>>> the easiest/most natural.)
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>>
>>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <
>>>>>>> robertwb@google.com> wrote:
>>>>>>>
>>>>>>>> Ah, I see.
>>>>>>>>
>>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>>> something more explicit).
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>>
>>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite
>>>>>>>>>> a bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>>
>>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>>
>>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>>   ...
>>>>>>>>>>   output.emit(data)
>>>>>>>>>>
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>>>> October timeframe if history is any guide)
>>>>>>>>>>
>>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have
>>>>>>>>>> in ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>>> it somehow.
>>>>>>>>>>
>>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>>> it yet.
>>>>>>>>>>
>>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>>
>>>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>>> work with GCS.
>>>>>>>>>>
>>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>>> there.
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Neat.
>>>>>>>>>>>
>>>>>>>>>>> Nothing like writing and SDK to actually understand how the
>>>>>>>>>>> FnAPI works :). I like the use of groupBy. I have to admit I'm a bit
>>>>>>>>>>> mystified by the syntax for parDo (I don't know swift at all which is
>>>>>>>>>>> probably tripping me up). The addition of external (cross-language)
>>>>>>>>>>> transforms could let you steal everything (e.g. IOs) pretty quickly from
>>>>>>>>>>> other SDKs.
>>>>>>>>>>>
>>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>>
>>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>>
>>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>>
>>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is
>>>>>>>>>>>>>> interest in Swift SDK from folks currently subscribed to the
>>>>>>>>>>>>>> +user <us...@beam.apache.org> list.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> This should be fine and we can get the code documented
>>>>>>>>>>>>>> without these features. I think support for composites and adding an
>>>>>>>>>>>>>> external transform (see, Java
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>>> Python
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>>> Go
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also
>>>>>>>>>>>>>> it'll be easier to comment on a PR :)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> [1]
>>>>>>>>>>>>>> [2]
>>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Okay, after a brief detour through "get this working in the Flink Portable
Runner" I think I have something pretty workable.

PInput and POutput can actually be structs rather than protocols, which
simplifies things quite a bit. It also allows us to use them with property
wrappers for a SwiftUI-like experience if we want when defining DoFns
(which is what I was originally intending to use them for). That also means
the function signature you use for closures would match full-fledged DoFn
definitions for the most part which is satisfying.



On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com> wrote:

> Okay, I tried a couple of different things.
>
> Implicitly passing the timestamp and window during iteration did not go
> well. While physically possible it introduces an invisible side effect into
> loop iteration which confused me when I tried to use it and I implemented
> it. Also, I'm pretty sure there'd end up being some sort of race condition
> nightmare continuing down that path.
>
> What I decided to do instead was the following:
>
> 1. Rename the existing "pardo" functions to "pstream" and require that
> they always emit a window and timestamp along with their value. This
> eliminates the side effect but lets us keep iteration in a bundle where
> that might be convenient. For example, in my cheesy GCS implementation it
> means that I can keep an OAuth token around for the lifetime of the bundle
> as a local variable, which is convenient. It's a bit more typing for users
> of pstream, but the expectation here is that if you're using pstream
> functions You Know What You Are Doing and most people won't be using it
> directly.
>
> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
> but enough to test the functionality and decide I liked it) which take a
> function signature of (any PInput<InputType>,any POutput<OutputType>).
> PInput takes the (InputType,Date,Window) tuple and converts it into a
> struct with friendlier names. Not strictly necessary, but makes the code
> nicer to read I think. POutput introduces emit functions that optionally
> allow you to specify a timestamp and a window. If you don't for either one
> it will take the timestamp and/or window of the input.
>
> Trying to use that was pretty pleasant to use so I think we should
> continue down that path. If you'd like to see it in use, I reimplemented
> map() and flatMap() in terms of this new pardo functionality.
>
> Code has been pushed to the branch/PR if you're interested in taking a
> look.
>
>
>
>
>
>
>
>
>
> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com> wrote:
>
>> Gotcha, I think there's a fairly easy solution to link input and output
>> streams.... Let me try it out... might even be possible to have both
>> element and stream-wise closure pardos. Definitely possible to have that at
>> the DoFn level (called SerializableFn in the SDK because I want to
>> use @DoFn as a macro)
>>
>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>> chamikara@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> I would like to figure out a way to get the stream-y interface to
>>>>> work, as I think it's more natural overall.
>>>>>
>>>>> One hypothesis is that if any elements are carried over loop
>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>> one to emit timestamp-windowed-values.
>>>>>
>>>>
>>>> Are you assuming that the same stream (or overlapping sets of data) are
>>>> pushed to multiple workers ? I thought that the set of data streamed here
>>>> are the data that belong to the current bundle (hence already assigned to
>>>> the current worker) so any output from the current bundle invocation would
>>>> be a valid output of that bundle.
>>>>
>>>>>
>>> Yes, the content of the stream is exactly the contents of the bundle.
>>> The question is how to do the input_element:output_element correlation for
>>> automatically propagating metadata.
>>>
>>>
>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>> have to explicitly request iteration over these windowed values rather than
>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>> the easiest/most natural.)
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah, I see.
>>>>>>>
>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>> something more explicit).
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>
>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>
>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>
>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>   ...
>>>>>>>>>   output.emit(data)
>>>>>>>>>
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>>> October timeframe if history is any guide)
>>>>>>>>>
>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>> it somehow.
>>>>>>>>>
>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>> it yet.
>>>>>>>>>
>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>
>>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>> work with GCS.
>>>>>>>>>
>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>> there.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Neat.
>>>>>>>>>>
>>>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>>>
>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>
>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>
>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>
>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>
>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>>>> transform (see, Java
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>> Python
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>> Go
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>>>
>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Okay, after a brief detour through "get this working in the Flink Portable
Runner" I think I have something pretty workable.

PInput and POutput can actually be structs rather than protocols, which
simplifies things quite a bit. It also allows us to use them with property
wrappers for a SwiftUI-like experience if we want when defining DoFns
(which is what I was originally intending to use them for). That also means
the function signature you use for closures would match full-fledged DoFn
definitions for the most part which is satisfying.



On Thu, Aug 24, 2023 at 5:55 PM Byron Ellis <by...@google.com> wrote:

> Okay, I tried a couple of different things.
>
> Implicitly passing the timestamp and window during iteration did not go
> well. While physically possible it introduces an invisible side effect into
> loop iteration which confused me when I tried to use it and I implemented
> it. Also, I'm pretty sure there'd end up being some sort of race condition
> nightmare continuing down that path.
>
> What I decided to do instead was the following:
>
> 1. Rename the existing "pardo" functions to "pstream" and require that
> they always emit a window and timestamp along with their value. This
> eliminates the side effect but lets us keep iteration in a bundle where
> that might be convenient. For example, in my cheesy GCS implementation it
> means that I can keep an OAuth token around for the lifetime of the bundle
> as a local variable, which is convenient. It's a bit more typing for users
> of pstream, but the expectation here is that if you're using pstream
> functions You Know What You Are Doing and most people won't be using it
> directly.
>
> 2. Introduce a new set of pardo functions (I didn't do all of them yet,
> but enough to test the functionality and decide I liked it) which take a
> function signature of (any PInput<InputType>,any POutput<OutputType>).
> PInput takes the (InputType,Date,Window) tuple and converts it into a
> struct with friendlier names. Not strictly necessary, but makes the code
> nicer to read I think. POutput introduces emit functions that optionally
> allow you to specify a timestamp and a window. If you don't for either one
> it will take the timestamp and/or window of the input.
>
> Trying to use that was pretty pleasant to use so I think we should
> continue down that path. If you'd like to see it in use, I reimplemented
> map() and flatMap() in terms of this new pardo functionality.
>
> Code has been pushed to the branch/PR if you're interested in taking a
> look.
>
>
>
>
>
>
>
>
>
> On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com> wrote:
>
>> Gotcha, I think there's a fairly easy solution to link input and output
>> streams.... Let me try it out... might even be possible to have both
>> element and stream-wise closure pardos. Definitely possible to have that at
>> the DoFn level (called SerializableFn in the SDK because I want to
>> use @DoFn as a macro)
>>
>> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <
>>> chamikara@google.com> wrote:
>>>
>>>>
>>>>
>>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> I would like to figure out a way to get the stream-y interface to
>>>>> work, as I think it's more natural overall.
>>>>>
>>>>> One hypothesis is that if any elements are carried over loop
>>>>> iterations, there will likely be some that are carried over beyond the loop
>>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>>> could reject "plain" elements that are emitted after this point, requiring
>>>>> one to emit timestamp-windowed-values.
>>>>>
>>>>
>>>> Are you assuming that the same stream (or overlapping sets of data) are
>>>> pushed to multiple workers ? I thought that the set of data streamed here
>>>> are the data that belong to the current bundle (hence already assigned to
>>>> the current worker) so any output from the current bundle invocation would
>>>> be a valid output of that bundle.
>>>>
>>>>>
>>> Yes, the content of the stream is exactly the contents of the bundle.
>>> The question is how to do the input_element:output_element correlation for
>>> automatically propagating metadata.
>>>
>>>
>>>> Related to this, we could enforce that the only (user-accessible) way
>>>>> to get such a timestamped value is to start with one, e.g. a
>>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>>> have to explicitly request iteration over these windowed values rather than
>>>>> over the raw elements. (This is also forward compatible with expanding the
>>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>>> the easiest/most natural.)
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah, that is a good point—being element-wise would make managing
>>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>>> change to make and maybe even less typing for the user. I was originally
>>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>>> you want a class and not a closure at that point for sanity.
>>>>>>
>>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Ah, I see.
>>>>>>>
>>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>>> from input elements to output elements. (You can of course attach the
>>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>>> suppose trying to output after the loop finishes could require
>>>>>>> something more explicit).
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Oh, I also forgot to mention that I included element-wise
>>>>>>>> collection operations like "map" that eliminate the need for pardo in many
>>>>>>>> cases. the groupBy command is actually a map + groupByKey under the hood.
>>>>>>>> That was to be more consistent with Swift's collection protocol (and is
>>>>>>>> also why PCollection and PCollectionStream are different types...
>>>>>>>> PCollection implements map and friends as pipeline construction operations
>>>>>>>> whereas PCollectionStream is an actual stream)
>>>>>>>>
>>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>>
>>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>>
>>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>>   ...
>>>>>>>>>   output.emit(data)
>>>>>>>>>
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>>> October timeframe if history is any guide)
>>>>>>>>>
>>>>>>>>> If you had parameterization you wanted to send that would look
>>>>>>>>> like pardo("Parameter") { param,filenames,output,error in ... } where
>>>>>>>>> "param" would take on the value of "Parameter." All of this is being
>>>>>>>>> typechecked at compile time BTW.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>>> it somehow.
>>>>>>>>>
>>>>>>>>> That said it would also be natural to provide elementwise
>>>>>>>>> pardos--- that would probably mean having explicit type signatures in the
>>>>>>>>> closure. I had that at one point, but it felt less natural the more I used
>>>>>>>>> it. I'm also slowly working towards adding a more "traditional" DoFn
>>>>>>>>> implementation approach where you implement the DoFn as an object type. In
>>>>>>>>> that case it would be very very easy to support both by having a default
>>>>>>>>> stream implementation call the equivalent of processElement. To make that
>>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>>> it yet.
>>>>>>>>>
>>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>>> to the pipeline implicitly.
>>>>>>>>>
>>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>>> work with GCS.
>>>>>>>>>
>>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>>> there.
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Neat.
>>>>>>>>>>
>>>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>>>
>>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>>
>>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>>
>>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>>
>>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>>
>>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>>> time.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>>>> transform (see, Java
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>>> Python
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>>> Go
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>>> TypeScript
>>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if
>>>>>>>>>>>>>> folks are interested, though the "Swift Way" would be to have it in its own
>>>>>>>>>>>>>> repo so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>>>
>>>>>>>>>>>>> - Cham
>>>>>>>>>>>>>
>>>>>>>>>>>>> [1]
>>>>>>>>>>>>> [2]
>>>>>>>>>>>>> [3]
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Best,
>>>>>>>>>>>>>> B
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Okay, I tried a couple of different things.

Implicitly passing the timestamp and window during iteration did not go
well. While physically possible it introduces an invisible side effect into
loop iteration which confused me when I tried to use it and I implemented
it. Also, I'm pretty sure there'd end up being some sort of race condition
nightmare continuing down that path.

What I decided to do instead was the following:

1. Rename the existing "pardo" functions to "pstream" and require that they
always emit a window and timestamp along with their value. This eliminates
the side effect but lets us keep iteration in a bundle where that might be
convenient. For example, in my cheesy GCS implementation it means that I
can keep an OAuth token around for the lifetime of the bundle as a local
variable, which is convenient. It's a bit more typing for users of pstream,
but the expectation here is that if you're using pstream functions You Know
What You Are Doing and most people won't be using it directly.

2. Introduce a new set of pardo functions (I didn't do all of them yet, but
enough to test the functionality and decide I liked it) which take a
function signature of (any PInput<InputType>,any POutput<OutputType>).
PInput takes the (InputType,Date,Window) tuple and converts it into a
struct with friendlier names. Not strictly necessary, but makes the code
nicer to read I think. POutput introduces emit functions that optionally
allow you to specify a timestamp and a window. If you don't for either one
it will take the timestamp and/or window of the input.

Trying to use that was pretty pleasant to use so I think we should continue
down that path. If you'd like to see it in use, I reimplemented map() and
flatMap() in terms of this new pardo functionality.

Code has been pushed to the branch/PR if you're interested in taking a look.









On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com> wrote:

> Gotcha, I think there's a fairly easy solution to link input and output
> streams.... Let me try it out... might even be possible to have both
> element and stream-wise closure pardos. Definitely possible to have that at
> the DoFn level (called SerializableFn in the SDK because I want to
> use @DoFn as a macro)
>
> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> I would like to figure out a way to get the stream-y interface to work,
>>>> as I think it's more natural overall.
>>>>
>>>> One hypothesis is that if any elements are carried over loop
>>>> iterations, there will likely be some that are carried over beyond the loop
>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>> could reject "plain" elements that are emitted after this point, requiring
>>>> one to emit timestamp-windowed-values.
>>>>
>>>
>>> Are you assuming that the same stream (or overlapping sets of data) are
>>> pushed to multiple workers ? I thought that the set of data streamed here
>>> are the data that belong to the current bundle (hence already assigned to
>>> the current worker) so any output from the current bundle invocation would
>>> be a valid output of that bundle.
>>>
>>>>
>> Yes, the content of the stream is exactly the contents of the bundle. The
>> question is how to do the input_element:output_element correlation for
>> automatically propagating metadata.
>>
>>
>>> Related to this, we could enforce that the only (user-accessible) way to
>>>> get such a timestamped value is to start with one, e.g. a
>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>> have to explicitly request iteration over these windowed values rather than
>>>> over the raw elements. (This is also forward compatible with expanding the
>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>> the easiest/most natural.)
>>>>
>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Ah, that is a good point—being element-wise would make managing
>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>> change to make and maybe even less typing for the user. I was originally
>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>> you want a class and not a closure at that point for sanity.
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah, I see.
>>>>>>
>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>> from input elements to output elements. (You can of course attach the
>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>> suppose trying to output after the loop finishes could require
>>>>>> something more explicit).
>>>>>>
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>>>> implements map and friends as pipeline construction operations whereas
>>>>>>> PCollectionStream is an actual stream)
>>>>>>>
>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>
>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>
>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>   ...
>>>>>>>>   output.emit(data)
>>>>>>>>
>>>>>>>> }
>>>>>>>>
>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>> October timeframe if history is any guide)
>>>>>>>>
>>>>>>>> If you had parameterization you wanted to send that would look like
>>>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>>>> compile time BTW.
>>>>>>>>
>>>>>>>>
>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>> it somehow.
>>>>>>>>
>>>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>>>> would be very very easy to support both by having a default stream
>>>>>>>> implementation call the equivalent of processElement. To make that
>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>> it yet.
>>>>>>>>
>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>> to the pipeline implicitly.
>>>>>>>>
>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>> work with GCS.
>>>>>>>>
>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>> there.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> Neat.
>>>>>>>>>
>>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>>
>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>
>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>
>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>
>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>
>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>
>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>
>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>> time.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>>> transform (see, Java
>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>> Python
>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>> Go
>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>> TypeScript
>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>>
>>>>>>>>>>>> - Cham
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> [2]
>>>>>>>>>>>> [3]
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Okay, I tried a couple of different things.

Implicitly passing the timestamp and window during iteration did not go
well. While physically possible it introduces an invisible side effect into
loop iteration which confused me when I tried to use it and I implemented
it. Also, I'm pretty sure there'd end up being some sort of race condition
nightmare continuing down that path.

What I decided to do instead was the following:

1. Rename the existing "pardo" functions to "pstream" and require that they
always emit a window and timestamp along with their value. This eliminates
the side effect but lets us keep iteration in a bundle where that might be
convenient. For example, in my cheesy GCS implementation it means that I
can keep an OAuth token around for the lifetime of the bundle as a local
variable, which is convenient. It's a bit more typing for users of pstream,
but the expectation here is that if you're using pstream functions You Know
What You Are Doing and most people won't be using it directly.

2. Introduce a new set of pardo functions (I didn't do all of them yet, but
enough to test the functionality and decide I liked it) which take a
function signature of (any PInput<InputType>,any POutput<OutputType>).
PInput takes the (InputType,Date,Window) tuple and converts it into a
struct with friendlier names. Not strictly necessary, but makes the code
nicer to read I think. POutput introduces emit functions that optionally
allow you to specify a timestamp and a window. If you don't for either one
it will take the timestamp and/or window of the input.

Trying to use that was pretty pleasant to use so I think we should continue
down that path. If you'd like to see it in use, I reimplemented map() and
flatMap() in terms of this new pardo functionality.

Code has been pushed to the branch/PR if you're interested in taking a look.









On Thu, Aug 24, 2023 at 2:15 PM Byron Ellis <by...@google.com> wrote:

> Gotcha, I think there's a fairly easy solution to link input and output
> streams.... Let me try it out... might even be possible to have both
> element and stream-wise closure pardos. Definitely possible to have that at
> the DoFn level (called SerializableFn in the SDK because I want to
> use @DoFn as a macro)
>
> On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>>
>>>
>>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> I would like to figure out a way to get the stream-y interface to work,
>>>> as I think it's more natural overall.
>>>>
>>>> One hypothesis is that if any elements are carried over loop
>>>> iterations, there will likely be some that are carried over beyond the loop
>>>> (after all the callee doesn't know when the loop is supposed to end). We
>>>> could reject "plain" elements that are emitted after this point, requiring
>>>> one to emit timestamp-windowed-values.
>>>>
>>>
>>> Are you assuming that the same stream (or overlapping sets of data) are
>>> pushed to multiple workers ? I thought that the set of data streamed here
>>> are the data that belong to the current bundle (hence already assigned to
>>> the current worker) so any output from the current bundle invocation would
>>> be a valid output of that bundle.
>>>
>>>>
>> Yes, the content of the stream is exactly the contents of the bundle. The
>> question is how to do the input_element:output_element correlation for
>> automatically propagating metadata.
>>
>>
>>> Related to this, we could enforce that the only (user-accessible) way to
>>>> get such a timestamped value is to start with one, e.g. a
>>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>>> have to explicitly request iteration over these windowed values rather than
>>>> over the raw elements. (This is also forward compatible with expanding the
>>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>>> the easiest/most natural.)
>>>>
>>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Ah, that is a good point—being element-wise would make managing
>>>>> windows and time stamps easier for the user. Fortunately it’s a fairly easy
>>>>> change to make and maybe even less typing for the user. I was originally
>>>>> thinking side inputs and metrics would happen outside the loop, but I think
>>>>> you want a class and not a closure at that point for sanity.
>>>>>
>>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Ah, I see.
>>>>>>
>>>>>> Yeah, I've thought about using an iterable for the whole bundle
>>>>>> rather than start/finish bundle callbacks, but one of the questions is how
>>>>>> that would impact implicit passing of the timestamp (and other) metadata
>>>>>> from input elements to output elements. (You can of course attach the
>>>>>> metadata to any output that happens in the loop body, but it's very easy to
>>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>>> suppose trying to output after the loop finishes could require
>>>>>> something more explicit).
>>>>>>
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>>>> implements map and friends as pipeline construction operations whereas
>>>>>>> PCollectionStream is an actual stream)
>>>>>>>
>>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>>> async / await structures. So when you see something like:
>>>>>>>>
>>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>>
>>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>>   ...
>>>>>>>>   output.emit(data)
>>>>>>>>
>>>>>>>> }
>>>>>>>>
>>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>>> October timeframe if history is any guide)
>>>>>>>>
>>>>>>>> If you had parameterization you wanted to send that would look like
>>>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>>>> compile time BTW.
>>>>>>>>
>>>>>>>>
>>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>>> it somehow.
>>>>>>>>
>>>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>>>> would be very very easy to support both by having a default stream
>>>>>>>> implementation call the equivalent of processElement. To make that
>>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>>> it yet.
>>>>>>>>
>>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>>> to the pipeline implicitly.
>>>>>>>>
>>>>>>>> That said, there are some interesting IO possibilities that would
>>>>>>>> be Swift native. In particularly, I've been looking at the native Swift
>>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>>> work with GCS.
>>>>>>>>
>>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>>> there.
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <
>>>>>>>> robertwb@google.com> wrote:
>>>>>>>>
>>>>>>>>> Neat.
>>>>>>>>>
>>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>>
>>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>>
>>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>>
>>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <
>>>>>>>>>> byronellis@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Cham,
>>>>>>>>>>>
>>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>>> than defining a composite.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>>
>>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>>
>>>>>>>>>>>>> One thing led to another and it turns out that to get a
>>>>>>>>>>>>> minimal FnApi integration going you end up writing a fair bit of an SDK. So
>>>>>>>>>>>>> I decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>>
>>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>>
>>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>>
>>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>>> time.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>>
>>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>>> transform (see, Java
>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>>> Python
>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>>> Go
>>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>>> TypeScript
>>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>>
>>>>>>>>>>>> - Cham
>>>>>>>>>>>>
>>>>>>>>>>>> [1]
>>>>>>>>>>>> [2]
>>>>>>>>>>>> [3]
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Best,
>>>>>>>>>>>>> B
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Gotcha, I think there's a fairly easy solution to link input and output
streams.... Let me try it out... might even be possible to have both
element and stream-wise closure pardos. Definitely possible to have that at
the DoFn level (called SerializableFn in the SDK because I want to
use @DoFn as a macro)

On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com> wrote:

> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>>
>>
>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> I would like to figure out a way to get the stream-y interface to work,
>>> as I think it's more natural overall.
>>>
>>> One hypothesis is that if any elements are carried over loop iterations,
>>> there will likely be some that are carried over beyond the loop (after all
>>> the callee doesn't know when the loop is supposed to end). We could reject
>>> "plain" elements that are emitted after this point, requiring one to emit
>>> timestamp-windowed-values.
>>>
>>
>> Are you assuming that the same stream (or overlapping sets of data) are
>> pushed to multiple workers ? I thought that the set of data streamed here
>> are the data that belong to the current bundle (hence already assigned to
>> the current worker) so any output from the current bundle invocation would
>> be a valid output of that bundle.
>>
>>>
> Yes, the content of the stream is exactly the contents of the bundle. The
> question is how to do the input_element:output_element correlation for
> automatically propagating metadata.
>
>
>> Related to this, we could enforce that the only (user-accessible) way to
>>> get such a timestamped value is to start with one, e.g. a
>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>> have to explicitly request iteration over these windowed values rather than
>>> over the raw elements. (This is also forward compatible with expanding the
>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>> the easiest/most natural.)
>>>
>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Ah, that is a good point—being element-wise would make managing windows
>>>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>>>> to make and maybe even less typing for the user. I was originally thinking
>>>> side inputs and metrics would happen outside the loop, but I think you want
>>>> a class and not a closure at that point for sanity.
>>>>
>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> Ah, I see.
>>>>>
>>>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>>>> than start/finish bundle callbacks, but one of the questions is how that
>>>>> would impact implicit passing of the timestamp (and other) metadata from
>>>>> input elements to output elements. (You can of course attach the metadata
>>>>> to any output that happens in the loop body, but it's very easy to
>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>> suppose trying to output after the loop finishes could require
>>>>> something more explicit).
>>>>>
>>>>>
>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>>> implements map and friends as pipeline construction operations whereas
>>>>>> PCollectionStream is an actual stream)
>>>>>>
>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>> async / await structures. So when you see something like:
>>>>>>>
>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>
>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>   ...
>>>>>>>   output.emit(data)
>>>>>>>
>>>>>>> }
>>>>>>>
>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>> October timeframe if history is any guide)
>>>>>>>
>>>>>>> If you had parameterization you wanted to send that would look like
>>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>>> compile time BTW.
>>>>>>>
>>>>>>>
>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>> it somehow.
>>>>>>>
>>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>>> would be very very easy to support both by having a default stream
>>>>>>> implementation call the equivalent of processElement. To make that
>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>> it yet.
>>>>>>>
>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>> to the pipeline implicitly.
>>>>>>>
>>>>>>> That said, there are some interesting IO possibilities that would be
>>>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>> work with GCS.
>>>>>>>
>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>> there.
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Neat.
>>>>>>>>
>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>
>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>
>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>
>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>
>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Cham,
>>>>>>>>>>
>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>> than defining a composite.
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>
>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>
>>>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>
>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>
>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>
>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>> time.
>>>>>>>>>>>>
>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>
>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>> transform (see, Java
>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>> Python
>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>> Go
>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>> TypeScript
>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>
>>>>>>>>>>> - Cham
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> [2]
>>>>>>>>>>> [3]
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Gotcha, I think there's a fairly easy solution to link input and output
streams.... Let me try it out... might even be possible to have both
element and stream-wise closure pardos. Definitely possible to have that at
the DoFn level (called SerializableFn in the SDK because I want to
use @DoFn as a macro)

On Thu, Aug 24, 2023 at 1:09 PM Robert Bradshaw <ro...@google.com> wrote:

> On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>>
>>
>> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> I would like to figure out a way to get the stream-y interface to work,
>>> as I think it's more natural overall.
>>>
>>> One hypothesis is that if any elements are carried over loop iterations,
>>> there will likely be some that are carried over beyond the loop (after all
>>> the callee doesn't know when the loop is supposed to end). We could reject
>>> "plain" elements that are emitted after this point, requiring one to emit
>>> timestamp-windowed-values.
>>>
>>
>> Are you assuming that the same stream (or overlapping sets of data) are
>> pushed to multiple workers ? I thought that the set of data streamed here
>> are the data that belong to the current bundle (hence already assigned to
>> the current worker) so any output from the current bundle invocation would
>> be a valid output of that bundle.
>>
>>>
> Yes, the content of the stream is exactly the contents of the bundle. The
> question is how to do the input_element:output_element correlation for
> automatically propagating metadata.
>
>
>> Related to this, we could enforce that the only (user-accessible) way to
>>> get such a timestamped value is to start with one, e.g. a
>>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>>> have to explicitly request iteration over these windowed values rather than
>>> over the raw elements. (This is also forward compatible with expanding the
>>> metadata that can get attached, e.g. pane infos, and makes the right thing
>>> the easiest/most natural.)
>>>
>>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Ah, that is a good point—being element-wise would make managing windows
>>>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>>>> to make and maybe even less typing for the user. I was originally thinking
>>>> side inputs and metrics would happen outside the loop, but I think you want
>>>> a class and not a closure at that point for sanity.
>>>>
>>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> Ah, I see.
>>>>>
>>>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>>>> than start/finish bundle callbacks, but one of the questions is how that
>>>>> would impact implicit passing of the timestamp (and other) metadata from
>>>>> input elements to output elements. (You can of course attach the metadata
>>>>> to any output that happens in the loop body, but it's very easy to
>>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>>> suppose trying to output after the loop finishes could require
>>>>> something more explicit).
>>>>>
>>>>>
>>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>>> implements map and friends as pipeline construction operations whereas
>>>>>> PCollectionStream is an actual stream)
>>>>>>
>>>>>> I just happened to push some "IO primitives" that uses map rather
>>>>>> than pardo in a couple of places to do a true wordcount using good ol'
>>>>>> Shakespeare and very very primitive GCS IO.
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>>> async / await structures. So when you see something like:
>>>>>>>
>>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>>
>>>>>>> for try await (filename,_,_) in filenames {
>>>>>>>   ...
>>>>>>>   output.emit(data)
>>>>>>>
>>>>>>> }
>>>>>>>
>>>>>>> filenames is the input stream and then output and errors are both
>>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>>> October timeframe if history is any guide)
>>>>>>>
>>>>>>> If you had parameterization you wanted to send that would look like
>>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>>> compile time BTW.
>>>>>>>
>>>>>>>
>>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>>> it somehow.
>>>>>>>
>>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>>> would be very very easy to support both by having a default stream
>>>>>>> implementation call the equivalent of processElement. To make that
>>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>>> it yet.
>>>>>>>
>>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>>> to the pipeline implicitly.
>>>>>>>
>>>>>>> That said, there are some interesting IO possibilities that would be
>>>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>>> statements on individual files with projection pushdown supported for
>>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>>> work with GCS.
>>>>>>>
>>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>>> there.
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Neat.
>>>>>>>>
>>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>>
>>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>>> user@beam.apache.org> wrote:
>>>>>>>>
>>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>>
>>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>>
>>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Cham,
>>>>>>>>>>
>>>>>>>>>> Definitely happy to open a draft PR so folks can
>>>>>>>>>> comment---there's not as much code as it looks like since most of the LOC
>>>>>>>>>> is just generated protobuf. As for the support, I definitely want to add
>>>>>>>>>> external transforms and may actually add that support before adding the
>>>>>>>>>> ability to make composites in the language itself. With the way the SDK is
>>>>>>>>>> laid out adding composites to the pipeline graph is a separate operation
>>>>>>>>>> than defining a composite.
>>>>>>>>>>
>>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>>
>>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest
>>>>>>>>>>> in Swift SDK from folks currently subscribed to the +user
>>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>>
>>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>>
>>>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>>
>>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>>
>>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>>
>>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using
>>>>>>>>>>>> Swift 5.9 on Intel Macs, but should also work using beta builds of 5.9 for
>>>>>>>>>>>> Linux running on Intel hardware. I haven't had a chance to try it on ARM
>>>>>>>>>>>> hardware and make sure all of the endian checks are complete. The
>>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>>> time.
>>>>>>>>>>>>
>>>>>>>>>>>> I've shown it to a couple of folks already and incorporated
>>>>>>>>>>>> some of that feedback already (for example pardo was originally called dofn
>>>>>>>>>>>> when defining pipelines). In general I've tried to make the API as
>>>>>>>>>>>> "Swift-y" as possible, hence the heavy reliance on closures and while there
>>>>>>>>>>>> aren't yet composite PTransforms there's the beginnings of what would be
>>>>>>>>>>>> needed for a SwiftUI-like declarative API for creating them.
>>>>>>>>>>>>
>>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>>> transform (see, Java
>>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>>> Python
>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>>> Go
>>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>>> TypeScript
>>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll
>>>>>>>>>>> be easier to comment on a PR :)
>>>>>>>>>>>
>>>>>>>>>>> - Cham
>>>>>>>>>>>
>>>>>>>>>>> [1]
>>>>>>>>>>> [2]
>>>>>>>>>>> [3]
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Best,
>>>>>>>>>>>> B
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via dev <de...@beam.apache.org>.
On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
wrote:

>
>
> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> I would like to figure out a way to get the stream-y interface to work,
>> as I think it's more natural overall.
>>
>> One hypothesis is that if any elements are carried over loop iterations,
>> there will likely be some that are carried over beyond the loop (after all
>> the callee doesn't know when the loop is supposed to end). We could reject
>> "plain" elements that are emitted after this point, requiring one to emit
>> timestamp-windowed-values.
>>
>
> Are you assuming that the same stream (or overlapping sets of data) are
> pushed to multiple workers ? I thought that the set of data streamed here
> are the data that belong to the current bundle (hence already assigned to
> the current worker) so any output from the current bundle invocation would
> be a valid output of that bundle.
>
>>
Yes, the content of the stream is exactly the contents of the bundle. The
question is how to do the input_element:output_element correlation for
automatically propagating metadata.


> Related to this, we could enforce that the only (user-accessible) way to
>> get such a timestamped value is to start with one, e.g. a
>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>> have to explicitly request iteration over these windowed values rather than
>> over the raw elements. (This is also forward compatible with expanding the
>> metadata that can get attached, e.g. pane infos, and makes the right thing
>> the easiest/most natural.)
>>
>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Ah, that is a good point—being element-wise would make managing windows
>>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>>> to make and maybe even less typing for the user. I was originally thinking
>>> side inputs and metrics would happen outside the loop, but I think you want
>>> a class and not a closure at that point for sanity.
>>>
>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> Ah, I see.
>>>>
>>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>>> than start/finish bundle callbacks, but one of the questions is how that
>>>> would impact implicit passing of the timestamp (and other) metadata from
>>>> input elements to output elements. (You can of course attach the metadata
>>>> to any output that happens in the loop body, but it's very easy to
>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>> suppose trying to output after the loop finishes could require
>>>> something more explicit).
>>>>
>>>>
>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>> implements map and friends as pipeline construction operations whereas
>>>>> PCollectionStream is an actual stream)
>>>>>
>>>>> I just happened to push some "IO primitives" that uses map rather than
>>>>> pardo in a couple of places to do a true wordcount using good ol'
>>>>> Shakespeare and very very primitive GCS IO.
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>> async / await structures. So when you see something like:
>>>>>>
>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>
>>>>>> for try await (filename,_,_) in filenames {
>>>>>>   ...
>>>>>>   output.emit(data)
>>>>>>
>>>>>> }
>>>>>>
>>>>>> filenames is the input stream and then output and errors are both
>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>> October timeframe if history is any guide)
>>>>>>
>>>>>> If you had parameterization you wanted to send that would look like
>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>> compile time BTW.
>>>>>>
>>>>>>
>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>> it somehow.
>>>>>>
>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>> would be very very easy to support both by having a default stream
>>>>>> implementation call the equivalent of processElement. To make that
>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>> it yet.
>>>>>>
>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>> to the pipeline implicitly.
>>>>>>
>>>>>> That said, there are some interesting IO possibilities that would be
>>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>> statements on individual files with projection pushdown supported for
>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>> work with GCS.
>>>>>>
>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>> there.
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Neat.
>>>>>>>
>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>
>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>> user@beam.apache.org> wrote:
>>>>>>>
>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>
>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>
>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Thanks Cham,
>>>>>>>>>
>>>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>>>> and may actually add that support before adding the ability to make
>>>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>>>> composite.
>>>>>>>>>
>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>
>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>
>>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>
>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>
>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>
>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>> time.
>>>>>>>>>>>
>>>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>>>
>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>> transform (see, Java
>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>> Python
>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>> Go
>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>> TypeScript
>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>>>> easier to comment on a PR :)
>>>>>>>>>>
>>>>>>>>>> - Cham
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>> [2]
>>>>>>>>>> [3]
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via user <us...@beam.apache.org>.
On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath <ch...@google.com>
wrote:

>
>
> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> I would like to figure out a way to get the stream-y interface to work,
>> as I think it's more natural overall.
>>
>> One hypothesis is that if any elements are carried over loop iterations,
>> there will likely be some that are carried over beyond the loop (after all
>> the callee doesn't know when the loop is supposed to end). We could reject
>> "plain" elements that are emitted after this point, requiring one to emit
>> timestamp-windowed-values.
>>
>
> Are you assuming that the same stream (or overlapping sets of data) are
> pushed to multiple workers ? I thought that the set of data streamed here
> are the data that belong to the current bundle (hence already assigned to
> the current worker) so any output from the current bundle invocation would
> be a valid output of that bundle.
>
>>
Yes, the content of the stream is exactly the contents of the bundle. The
question is how to do the input_element:output_element correlation for
automatically propagating metadata.


> Related to this, we could enforce that the only (user-accessible) way to
>> get such a timestamped value is to start with one, e.g. a
>> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
>> metadata but a new value. Thus a user wanting to do anything "fancy" would
>> have to explicitly request iteration over these windowed values rather than
>> over the raw elements. (This is also forward compatible with expanding the
>> metadata that can get attached, e.g. pane infos, and makes the right thing
>> the easiest/most natural.)
>>
>> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Ah, that is a good point—being element-wise would make managing windows
>>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>>> to make and maybe even less typing for the user. I was originally thinking
>>> side inputs and metrics would happen outside the loop, but I think you want
>>> a class and not a closure at that point for sanity.
>>>
>>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> Ah, I see.
>>>>
>>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>>> than start/finish bundle callbacks, but one of the questions is how that
>>>> would impact implicit passing of the timestamp (and other) metadata from
>>>> input elements to output elements. (You can of course attach the metadata
>>>> to any output that happens in the loop body, but it's very easy to
>>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>>> otherwise modifying local state) and this would be hard to detect. (I
>>>> suppose trying to output after the loop finishes could require
>>>> something more explicit).
>>>>
>>>>
>>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Oh, I also forgot to mention that I included element-wise collection
>>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>>> be more consistent with Swift's collection protocol (and is also why
>>>>> PCollection and PCollectionStream are different types... PCollection
>>>>> implements map and friends as pipeline construction operations whereas
>>>>> PCollectionStream is an actual stream)
>>>>>
>>>>> I just happened to push some "IO primitives" that uses map rather than
>>>>> pardo in a couple of places to do a true wordcount using good ol'
>>>>> Shakespeare and very very primitive GCS IO.
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a
>>>>>> bit before settling on where I ended up. Ultimately I decided to go with
>>>>>> something that felt more Swift-y than anything else which means that rather
>>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>>> dealing with a stream of elements (which of course will often be of size
>>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>>> async / await structures. So when you see something like:
>>>>>>
>>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>>
>>>>>> for try await (filename,_,_) in filenames {
>>>>>>   ...
>>>>>>   output.emit(data)
>>>>>>
>>>>>> }
>>>>>>
>>>>>> filenames is the input stream and then output and errors are both
>>>>>> output streams. In theory you can have as many output streams as you like
>>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>>> October timeframe if history is any guide)
>>>>>>
>>>>>> If you had parameterization you wanted to send that would look like
>>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>>> compile time BTW.
>>>>>>
>>>>>>
>>>>>> the (filename,_,_) is a tuple spreading construct like you have in
>>>>>> ES6 and other things where "_" is Swift for "ignore." In this case
>>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>>> it somehow.
>>>>>>
>>>>>> That said it would also be natural to provide elementwise pardos---
>>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>>> would be very very easy to support both by having a default stream
>>>>>> implementation call the equivalent of processElement. To make that
>>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>>> it yet.
>>>>>>
>>>>>> It's a bit more work and I've been prioritizing implementing
>>>>>> composite and external transforms for the reasons you suggest. :-) I've got
>>>>>> the basics of a composite transform (there's an equivalent wordcount
>>>>>> example) and am hooking it into the pipeline generation, which should also
>>>>>> give me everything I need to successfully hook in external transforms as
>>>>>> well. That will give me the jump on IOs as you say. I can also treat the
>>>>>> pipeline itself as a composite transform which lets me get rid of the
>>>>>> Pipeline { pipeline in ... } and just instead have things attach themselves
>>>>>> to the pipeline implicitly.
>>>>>>
>>>>>> That said, there are some interesting IO possibilities that would be
>>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>>> statements on individual files with projection pushdown supported for
>>>>>> things like Parquet which could have some cool and performant data lake
>>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>>> work with GCS.
>>>>>>
>>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>>> there.
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Neat.
>>>>>>>
>>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>>
>>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>>> user@beam.apache.org> wrote:
>>>>>>>
>>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>>
>>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>>
>>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>>
>>>>>>>>
>>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Thanks Cham,
>>>>>>>>>
>>>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>>>> and may actually add that support before adding the ability to make
>>>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>>>> composite.
>>>>>>>>>
>>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>>
>>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hello everyone,
>>>>>>>>>>>
>>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>>
>>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>>
>>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>>
>>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>>
>>>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>>> time.
>>>>>>>>>>>
>>>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>>>
>>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>>> transform (see, Java
>>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>>> Python
>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>>> Go
>>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>>> TypeScript
>>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>>> I/O connectors) for free.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>>>> easier to comment on a PR :)
>>>>>>>>>>
>>>>>>>>>> - Cham
>>>>>>>>>>
>>>>>>>>>> [1]
>>>>>>>>>> [2]
>>>>>>>>>> [3]
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Best,
>>>>>>>>>>> B
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Chamikara Jayalath via user <us...@beam.apache.org>.
On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
wrote:

> I would like to figure out a way to get the stream-y interface to work, as
> I think it's more natural overall.
>
> One hypothesis is that if any elements are carried over loop iterations,
> there will likely be some that are carried over beyond the loop (after all
> the callee doesn't know when the loop is supposed to end). We could reject
> "plain" elements that are emitted after this point, requiring one to emit
> timestamp-windowed-values.
>

Are you assuming that the same stream (or overlapping sets of data) are
pushed to multiple workers ? I thought that the set of data streamed here
are the data that belong to the current bundle (hence already assigned to
the current worker) so any output from the current bundle invocation would
be a valid output of that bundle.


> Related to this, we could enforce that the only (user-accessible) way to
> get such a timestamped value is to start with one, e.g. a
> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
> metadata but a new value. Thus a user wanting to do anything "fancy" would
> have to explicitly request iteration over these windowed values rather than
> over the raw elements. (This is also forward compatible with expanding the
> metadata that can get attached, e.g. pane infos, and makes the right thing
> the easiest/most natural.)
>
> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
> wrote:
>
>> Ah, that is a good point—being element-wise would make managing windows
>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>> to make and maybe even less typing for the user. I was originally thinking
>> side inputs and metrics would happen outside the loop, but I think you want
>> a class and not a closure at that point for sanity.
>>
>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Ah, I see.
>>>
>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>> than start/finish bundle callbacks, but one of the questions is how that
>>> would impact implicit passing of the timestamp (and other) metadata from
>>> input elements to output elements. (You can of course attach the metadata
>>> to any output that happens in the loop body, but it's very easy to
>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>> otherwise modifying local state) and this would be hard to detect. (I
>>> suppose trying to output after the loop finishes could require
>>> something more explicit).
>>>
>>>
>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Oh, I also forgot to mention that I included element-wise collection
>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>> be more consistent with Swift's collection protocol (and is also why
>>>> PCollection and PCollectionStream are different types... PCollection
>>>> implements map and friends as pipeline construction operations whereas
>>>> PCollectionStream is an actual stream)
>>>>
>>>> I just happened to push some "IO primitives" that uses map rather than
>>>> pardo in a couple of places to do a true wordcount using good ol'
>>>> Shakespeare and very very primitive GCS IO.
>>>>
>>>> Best,
>>>> B
>>>>
>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>>>> before settling on where I ended up. Ultimately I decided to go with
>>>>> something that felt more Swift-y than anything else which means that rather
>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>> dealing with a stream of elements (which of course will often be of size
>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>> async / await structures. So when you see something like:
>>>>>
>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>
>>>>> for try await (filename,_,_) in filenames {
>>>>>   ...
>>>>>   output.emit(data)
>>>>>
>>>>> }
>>>>>
>>>>> filenames is the input stream and then output and errors are both
>>>>> output streams. In theory you can have as many output streams as you like
>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>> October timeframe if history is any guide)
>>>>>
>>>>> If you had parameterization you wanted to send that would look like
>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>> compile time BTW.
>>>>>
>>>>>
>>>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>>>> and other things where "_" is Swift for "ignore." In this case
>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>> it somehow.
>>>>>
>>>>> That said it would also be natural to provide elementwise pardos---
>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>> would be very very easy to support both by having a default stream
>>>>> implementation call the equivalent of processElement. To make that
>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>> it yet.
>>>>>
>>>>> It's a bit more work and I've been prioritizing implementing composite
>>>>> and external transforms for the reasons you suggest. :-) I've got the
>>>>> basics of a composite transform (there's an equivalent wordcount example)
>>>>> and am hooking it into the pipeline generation, which should also give me
>>>>> everything I need to successfully hook in external transforms as well. That
>>>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>>>> itself as a composite transform which lets me get rid of the Pipeline {
>>>>> pipeline in ... } and just instead have things attach themselves to the
>>>>> pipeline implicitly.
>>>>>
>>>>> That said, there are some interesting IO possibilities that would be
>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>> statements on individual files with projection pushdown supported for
>>>>> things like Parquet which could have some cool and performant data lake
>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>> work with GCS.
>>>>>
>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>> there.
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Neat.
>>>>>>
>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>
>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>> user@beam.apache.org> wrote:
>>>>>>
>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>
>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>
>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks Cham,
>>>>>>>>
>>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>>> and may actually add that support before adding the ability to make
>>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>>> composite.
>>>>>>>>
>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>
>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>
>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> Hello everyone,
>>>>>>>>>>
>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>
>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>
>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>
>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>
>>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>> time.
>>>>>>>>>>
>>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>>
>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>> transform (see, Java
>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>> Python
>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>> Go
>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>> TypeScript
>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>> I/O connectors) for free.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>>> easier to comment on a PR :)
>>>>>>>>>
>>>>>>>>> - Cham
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>> [2]
>>>>>>>>> [3]
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Chamikara Jayalath via dev <de...@beam.apache.org>.
On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw <ro...@google.com>
wrote:

> I would like to figure out a way to get the stream-y interface to work, as
> I think it's more natural overall.
>
> One hypothesis is that if any elements are carried over loop iterations,
> there will likely be some that are carried over beyond the loop (after all
> the callee doesn't know when the loop is supposed to end). We could reject
> "plain" elements that are emitted after this point, requiring one to emit
> timestamp-windowed-values.
>

Are you assuming that the same stream (or overlapping sets of data) are
pushed to multiple workers ? I thought that the set of data streamed here
are the data that belong to the current bundle (hence already assigned to
the current worker) so any output from the current bundle invocation would
be a valid output of that bundle.


> Related to this, we could enforce that the only (user-accessible) way to
> get such a timestamped value is to start with one, e.g. a
> WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
> metadata but a new value. Thus a user wanting to do anything "fancy" would
> have to explicitly request iteration over these windowed values rather than
> over the raw elements. (This is also forward compatible with expanding the
> metadata that can get attached, e.g. pane infos, and makes the right thing
> the easiest/most natural.)
>
> On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com>
> wrote:
>
>> Ah, that is a good point—being element-wise would make managing windows
>> and time stamps easier for the user. Fortunately it’s a fairly easy change
>> to make and maybe even less typing for the user. I was originally thinking
>> side inputs and metrics would happen outside the loop, but I think you want
>> a class and not a closure at that point for sanity.
>>
>> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Ah, I see.
>>>
>>> Yeah, I've thought about using an iterable for the whole bundle rather
>>> than start/finish bundle callbacks, but one of the questions is how that
>>> would impact implicit passing of the timestamp (and other) metadata from
>>> input elements to output elements. (You can of course attach the metadata
>>> to any output that happens in the loop body, but it's very easy to
>>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>>> otherwise modifying local state) and this would be hard to detect. (I
>>> suppose trying to output after the loop finishes could require
>>> something more explicit).
>>>
>>>
>>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Oh, I also forgot to mention that I included element-wise collection
>>>> operations like "map" that eliminate the need for pardo in many cases. the
>>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>>> be more consistent with Swift's collection protocol (and is also why
>>>> PCollection and PCollectionStream are different types... PCollection
>>>> implements map and friends as pipeline construction operations whereas
>>>> PCollectionStream is an actual stream)
>>>>
>>>> I just happened to push some "IO primitives" that uses map rather than
>>>> pardo in a couple of places to do a true wordcount using good ol'
>>>> Shakespeare and very very primitive GCS IO.
>>>>
>>>> Best,
>>>> B
>>>>
>>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>>>> before settling on where I ended up. Ultimately I decided to go with
>>>>> something that felt more Swift-y than anything else which means that rather
>>>>> than dealing with a single element like you do in the other SDKs you're
>>>>> dealing with a stream of elements (which of course will often be of size
>>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>>> async / await structures. So when you see something like:
>>>>>
>>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>>
>>>>> for try await (filename,_,_) in filenames {
>>>>>   ...
>>>>>   output.emit(data)
>>>>>
>>>>> }
>>>>>
>>>>> filenames is the input stream and then output and errors are both
>>>>> output streams. In theory you can have as many output streams as you like
>>>>> though at the moment there's a compiler bug in the new type pack feature
>>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>>> get fixed before the official 5.9 release which will probably be in the
>>>>> October timeframe if history is any guide)
>>>>>
>>>>> If you had parameterization you wanted to send that would look like
>>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>>> compile time BTW.
>>>>>
>>>>>
>>>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>>>> and other things where "_" is Swift for "ignore." In this case
>>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>>> optionally extract the timestamp and the window if you want to manipulate
>>>>> it somehow.
>>>>>
>>>>> That said it would also be natural to provide elementwise pardos---
>>>>> that would probably mean having explicit type signatures in the closure. I
>>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>>> approach where you implement the DoFn as an object type. In that case it
>>>>> would be very very easy to support both by having a default stream
>>>>> implementation call the equivalent of processElement. To make that
>>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>>> it yet.
>>>>>
>>>>> It's a bit more work and I've been prioritizing implementing composite
>>>>> and external transforms for the reasons you suggest. :-) I've got the
>>>>> basics of a composite transform (there's an equivalent wordcount example)
>>>>> and am hooking it into the pipeline generation, which should also give me
>>>>> everything I need to successfully hook in external transforms as well. That
>>>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>>>> itself as a composite transform which lets me get rid of the Pipeline {
>>>>> pipeline in ... } and just instead have things attach themselves to the
>>>>> pipeline implicitly.
>>>>>
>>>>> That said, there are some interesting IO possibilities that would be
>>>>> Swift native. In particularly, I've been looking at the native Swift
>>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>>> statements on individual files with projection pushdown supported for
>>>>> things like Parquet which could have some cool and performant data lake
>>>>> applications. I'll probably do a couple of the simpler IOs as
>>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>>> work with GCS.
>>>>>
>>>>> In any case, I'm updating the branch as I find a minute here and
>>>>> there.
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Neat.
>>>>>>
>>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>>
>>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>>> user@beam.apache.org> wrote:
>>>>>>
>>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>>
>>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>>
>>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>>
>>>>>>>
>>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Thanks Cham,
>>>>>>>>
>>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>>> and may actually add that support before adding the ability to make
>>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>>> composite.
>>>>>>>>
>>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>>> chamikara@google.com> wrote:
>>>>>>>>
>>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>>
>>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>>
>>>>>>>>>> Hello everyone,
>>>>>>>>>>
>>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>>
>>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>>
>>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>>
>>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>>
>>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>>> time.
>>>>>>>>>>
>>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>>
>>>>>>>>>> There are of course a ton of missing bits still to be
>>>>>>>>>> implemented, like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>>> transform (see, Java
>>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>>> Python
>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>>> Go
>>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>>> TypeScript
>>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>>> I/O connectors) for free.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks
>>>>>>>>>> are interested, though the "Swift Way" would be to have it in its own repo
>>>>>>>>>> so that it can easily be used from the Swift Package Manager.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>>> easier to comment on a PR :)
>>>>>>>>>
>>>>>>>>> - Cham
>>>>>>>>>
>>>>>>>>> [1]
>>>>>>>>> [2]
>>>>>>>>> [3]
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Best,
>>>>>>>>>> B
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via dev <de...@beam.apache.org>.
I would like to figure out a way to get the stream-y interface to work, as
I think it's more natural overall.

One hypothesis is that if any elements are carried over loop iterations,
there will likely be some that are carried over beyond the loop (after all
the callee doesn't know when the loop is supposed to end). We could reject
"plain" elements that are emitted after this point, requiring one to emit
timestamp-windowed-values.

Related to this, we could enforce that the only (user-accessible) way to
get such a timestamped value is to start with one, e.g. a
WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
metadata but a new value. Thus a user wanting to do anything "fancy" would
have to explicitly request iteration over these windowed values rather than
over the raw elements. (This is also forward compatible with expanding the
metadata that can get attached, e.g. pane infos, and makes the right thing
the easiest/most natural.)

On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com> wrote:

> Ah, that is a good point—being element-wise would make managing windows
> and time stamps easier for the user. Fortunately it’s a fairly easy change
> to make and maybe even less typing for the user. I was originally thinking
> side inputs and metrics would happen outside the loop, but I think you want
> a class and not a closure at that point for sanity.
>
> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Ah, I see.
>>
>> Yeah, I've thought about using an iterable for the whole bundle rather
>> than start/finish bundle callbacks, but one of the questions is how that
>> would impact implicit passing of the timestamp (and other) metadata from
>> input elements to output elements. (You can of course attach the metadata
>> to any output that happens in the loop body, but it's very easy to
>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>> otherwise modifying local state) and this would be hard to detect. (I
>> suppose trying to output after the loop finishes could require
>> something more explicit).
>>
>>
>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Oh, I also forgot to mention that I included element-wise collection
>>> operations like "map" that eliminate the need for pardo in many cases. the
>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>> be more consistent with Swift's collection protocol (and is also why
>>> PCollection and PCollectionStream are different types... PCollection
>>> implements map and friends as pipeline construction operations whereas
>>> PCollectionStream is an actual stream)
>>>
>>> I just happened to push some "IO primitives" that uses map rather than
>>> pardo in a couple of places to do a true wordcount using good ol'
>>> Shakespeare and very very primitive GCS IO.
>>>
>>> Best,
>>> B
>>>
>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>>> before settling on where I ended up. Ultimately I decided to go with
>>>> something that felt more Swift-y than anything else which means that rather
>>>> than dealing with a single element like you do in the other SDKs you're
>>>> dealing with a stream of elements (which of course will often be of size
>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>> async / await structures. So when you see something like:
>>>>
>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>
>>>> for try await (filename,_,_) in filenames {
>>>>   ...
>>>>   output.emit(data)
>>>>
>>>> }
>>>>
>>>> filenames is the input stream and then output and errors are both
>>>> output streams. In theory you can have as many output streams as you like
>>>> though at the moment there's a compiler bug in the new type pack feature
>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>> get fixed before the official 5.9 release which will probably be in the
>>>> October timeframe if history is any guide)
>>>>
>>>> If you had parameterization you wanted to send that would look like
>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>> compile time BTW.
>>>>
>>>>
>>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>>> and other things where "_" is Swift for "ignore." In this case
>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>> optionally extract the timestamp and the window if you want to manipulate
>>>> it somehow.
>>>>
>>>> That said it would also be natural to provide elementwise pardos---
>>>> that would probably mean having explicit type signatures in the closure. I
>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>> approach where you implement the DoFn as an object type. In that case it
>>>> would be very very easy to support both by having a default stream
>>>> implementation call the equivalent of processElement. To make that
>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>> it yet.
>>>>
>>>> It's a bit more work and I've been prioritizing implementing composite
>>>> and external transforms for the reasons you suggest. :-) I've got the
>>>> basics of a composite transform (there's an equivalent wordcount example)
>>>> and am hooking it into the pipeline generation, which should also give me
>>>> everything I need to successfully hook in external transforms as well. That
>>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>>> itself as a composite transform which lets me get rid of the Pipeline {
>>>> pipeline in ... } and just instead have things attach themselves to the
>>>> pipeline implicitly.
>>>>
>>>> That said, there are some interesting IO possibilities that would be
>>>> Swift native. In particularly, I've been looking at the native Swift
>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>> statements on individual files with projection pushdown supported for
>>>> things like Parquet which could have some cool and performant data lake
>>>> applications. I'll probably do a couple of the simpler IOs as
>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>> work with GCS.
>>>>
>>>> In any case, I'm updating the branch as I find a minute here and there.
>>>>
>>>> Best,
>>>> B
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> Neat.
>>>>>
>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>
>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>> user@beam.apache.org> wrote:
>>>>>
>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>
>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>
>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks Cham,
>>>>>>>
>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>> and may actually add that support before adding the ability to make
>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>> composite.
>>>>>>>
>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>> chamikara@google.com> wrote:
>>>>>>>
>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>
>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>
>>>>>>>>> Hello everyone,
>>>>>>>>>
>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>
>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>
>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>
>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>
>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>> time.
>>>>>>>>>
>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>
>>>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>
>>>>>>>>
>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>> transform (see, Java
>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>> Python
>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>> Go
>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>> TypeScript
>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>> I/O connectors) for free.
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>>>
>>>>>>>>
>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>> easier to comment on a PR :)
>>>>>>>>
>>>>>>>> - Cham
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> [2]
>>>>>>>> [3]
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via user <us...@beam.apache.org>.
I would like to figure out a way to get the stream-y interface to work, as
I think it's more natural overall.

One hypothesis is that if any elements are carried over loop iterations,
there will likely be some that are carried over beyond the loop (after all
the callee doesn't know when the loop is supposed to end). We could reject
"plain" elements that are emitted after this point, requiring one to emit
timestamp-windowed-values.

Related to this, we could enforce that the only (user-accessible) way to
get such a timestamped value is to start with one, e.g. a
WindowedValue<T>.withValue(O) produces a WindowedValue<O> with the same
metadata but a new value. Thus a user wanting to do anything "fancy" would
have to explicitly request iteration over these windowed values rather than
over the raw elements. (This is also forward compatible with expanding the
metadata that can get attached, e.g. pane infos, and makes the right thing
the easiest/most natural.)

On Thu, Aug 24, 2023 at 12:10 PM Byron Ellis <by...@google.com> wrote:

> Ah, that is a good point—being element-wise would make managing windows
> and time stamps easier for the user. Fortunately it’s a fairly easy change
> to make and maybe even less typing for the user. I was originally thinking
> side inputs and metrics would happen outside the loop, but I think you want
> a class and not a closure at that point for sanity.
>
> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Ah, I see.
>>
>> Yeah, I've thought about using an iterable for the whole bundle rather
>> than start/finish bundle callbacks, but one of the questions is how that
>> would impact implicit passing of the timestamp (and other) metadata from
>> input elements to output elements. (You can of course attach the metadata
>> to any output that happens in the loop body, but it's very easy to
>> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
>> otherwise modifying local state) and this would be hard to detect. (I
>> suppose trying to output after the loop finishes could require
>> something more explicit).
>>
>>
>> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Oh, I also forgot to mention that I included element-wise collection
>>> operations like "map" that eliminate the need for pardo in many cases. the
>>> groupBy command is actually a map + groupByKey under the hood. That was to
>>> be more consistent with Swift's collection protocol (and is also why
>>> PCollection and PCollectionStream are different types... PCollection
>>> implements map and friends as pipeline construction operations whereas
>>> PCollectionStream is an actual stream)
>>>
>>> I just happened to push some "IO primitives" that uses map rather than
>>> pardo in a couple of places to do a true wordcount using good ol'
>>> Shakespeare and very very primitive GCS IO.
>>>
>>> Best,
>>> B
>>>
>>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>>> before settling on where I ended up. Ultimately I decided to go with
>>>> something that felt more Swift-y than anything else which means that rather
>>>> than dealing with a single element like you do in the other SDKs you're
>>>> dealing with a stream of elements (which of course will often be of size
>>>> 1). That's a really natural paradigm in the Swift world especially with the
>>>> async / await structures. So when you see something like:
>>>>
>>>> pardo(name:"Read Files") { filenames,output,errors in
>>>>
>>>> for try await (filename,_,_) in filenames {
>>>>   ...
>>>>   output.emit(data)
>>>>
>>>> }
>>>>
>>>> filenames is the input stream and then output and errors are both
>>>> output streams. In theory you can have as many output streams as you like
>>>> though at the moment there's a compiler bug in the new type pack feature
>>>> that limits it to "as many as I felt like supporting". Presumably this will
>>>> get fixed before the official 5.9 release which will probably be in the
>>>> October timeframe if history is any guide)
>>>>
>>>> If you had parameterization you wanted to send that would look like
>>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>>> would take on the value of "Parameter." All of this is being typechecked at
>>>> compile time BTW.
>>>>
>>>>
>>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>>> and other things where "_" is Swift for "ignore." In this case
>>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>>> optionally extract the timestamp and the window if you want to manipulate
>>>> it somehow.
>>>>
>>>> That said it would also be natural to provide elementwise pardos---
>>>> that would probably mean having explicit type signatures in the closure. I
>>>> had that at one point, but it felt less natural the more I used it. I'm
>>>> also slowly working towards adding a more "traditional" DoFn implementation
>>>> approach where you implement the DoFn as an object type. In that case it
>>>> would be very very easy to support both by having a default stream
>>>> implementation call the equivalent of processElement. To make that
>>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>>> it yet.
>>>>
>>>> It's a bit more work and I've been prioritizing implementing composite
>>>> and external transforms for the reasons you suggest. :-) I've got the
>>>> basics of a composite transform (there's an equivalent wordcount example)
>>>> and am hooking it into the pipeline generation, which should also give me
>>>> everything I need to successfully hook in external transforms as well. That
>>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>>> itself as a composite transform which lets me get rid of the Pipeline {
>>>> pipeline in ... } and just instead have things attach themselves to the
>>>> pipeline implicitly.
>>>>
>>>> That said, there are some interesting IO possibilities that would be
>>>> Swift native. In particularly, I've been looking at the native Swift
>>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>>> statements on individual files with projection pushdown supported for
>>>> things like Parquet which could have some cool and performant data lake
>>>> applications. I'll probably do a couple of the simpler IOs as
>>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>>> work with GCS.
>>>>
>>>> In any case, I'm updating the branch as I find a minute here and there.
>>>>
>>>> Best,
>>>> B
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> Neat.
>>>>>
>>>>> Nothing like writing and SDK to actually understand how the FnAPI
>>>>> works :). I like the use of groupBy. I have to admit I'm a bit mystified by
>>>>> the syntax for parDo (I don't know swift at all which is probably tripping
>>>>> me up). The addition of external (cross-language) transforms could let you
>>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>>
>>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>>> user@beam.apache.org> wrote:
>>>>>
>>>>>> For everyone who is interested, here's the draft PR:
>>>>>>
>>>>>> https://github.com/apache/beam/pull/28062
>>>>>>
>>>>>> I haven't had a chance to test it on my M1 machine yet though
>>>>>> (there's a good chance there are a few places that need to properly address
>>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>>
>>>>>>
>>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Thanks Cham,
>>>>>>>
>>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>>> and may actually add that support before adding the ability to make
>>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>>> composite.
>>>>>>>
>>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>>> chamikara@google.com> wrote:
>>>>>>>
>>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>>> <us...@beam.apache.org> list.
>>>>>>>>
>>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>>
>>>>>>>>> Hello everyone,
>>>>>>>>>
>>>>>>>>> A couple of months ago I decided that I wanted to really
>>>>>>>>> understand how the Beam FnApi works and how it interacts with the Portable
>>>>>>>>> Runner. For me at least that usually means I need to write some code so I
>>>>>>>>> can see things happening in a debugger and to really prove to myself I
>>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>>
>>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>>
>>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>>
>>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>>
>>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>>> time.
>>>>>>>>>
>>>>>>>>> I've shown it to a couple of folks already and incorporated some
>>>>>>>>> of that feedback already (for example pardo was originally called dofn when
>>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>>
>>>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>>>
>>>>>>>>
>>>>>>>> This should be fine and we can get the code documented without
>>>>>>>> these features. I think support for composites and adding an external
>>>>>>>> transform (see, Java
>>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>>> Python
>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>>> Go
>>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>>> TypeScript
>>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>>> I/O connectors) for free.
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>>>
>>>>>>>>
>>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>>> easier to comment on a PR :)
>>>>>>>>
>>>>>>>> - Cham
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> [2]
>>>>>>>> [3]
>>>>>>>>
>>>>>>>>
>>>>>>>>>
>>>>>>>>> Best,
>>>>>>>>> B
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Ah, that is a good point—being element-wise would make managing windows and
time stamps easier for the user. Fortunately it’s a fairly easy change to
make and maybe even less typing for the user. I was originally thinking
side inputs and metrics would happen outside the loop, but I think you want
a class and not a closure at that point for sanity.

On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
wrote:

> Ah, I see.
>
> Yeah, I've thought about using an iterable for the whole bundle rather
> than start/finish bundle callbacks, but one of the questions is how that
> would impact implicit passing of the timestamp (and other) metadata from
> input elements to output elements. (You can of course attach the metadata
> to any output that happens in the loop body, but it's very easy to
> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
> otherwise modifying local state) and this would be hard to detect. (I
> suppose trying to output after the loop finishes could require
> something more explicit).
>
>
> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com> wrote:
>
>> Oh, I also forgot to mention that I included element-wise collection
>> operations like "map" that eliminate the need for pardo in many cases. the
>> groupBy command is actually a map + groupByKey under the hood. That was to
>> be more consistent with Swift's collection protocol (and is also why
>> PCollection and PCollectionStream are different types... PCollection
>> implements map and friends as pipeline construction operations whereas
>> PCollectionStream is an actual stream)
>>
>> I just happened to push some "IO primitives" that uses map rather than
>> pardo in a couple of places to do a true wordcount using good ol'
>> Shakespeare and very very primitive GCS IO.
>>
>> Best,
>> B
>>
>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>> before settling on where I ended up. Ultimately I decided to go with
>>> something that felt more Swift-y than anything else which means that rather
>>> than dealing with a single element like you do in the other SDKs you're
>>> dealing with a stream of elements (which of course will often be of size
>>> 1). That's a really natural paradigm in the Swift world especially with the
>>> async / await structures. So when you see something like:
>>>
>>> pardo(name:"Read Files") { filenames,output,errors in
>>>
>>> for try await (filename,_,_) in filenames {
>>>   ...
>>>   output.emit(data)
>>>
>>> }
>>>
>>> filenames is the input stream and then output and errors are both output
>>> streams. In theory you can have as many output streams as you like though
>>> at the moment there's a compiler bug in the new type pack feature that
>>> limits it to "as many as I felt like supporting". Presumably this will get
>>> fixed before the official 5.9 release which will probably be in the October
>>> timeframe if history is any guide)
>>>
>>> If you had parameterization you wanted to send that would look like
>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>> would take on the value of "Parameter." All of this is being typechecked at
>>> compile time BTW.
>>>
>>>
>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>> and other things where "_" is Swift for "ignore." In this case
>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>> optionally extract the timestamp and the window if you want to manipulate
>>> it somehow.
>>>
>>> That said it would also be natural to provide elementwise pardos--- that
>>> would probably mean having explicit type signatures in the closure. I had
>>> that at one point, but it felt less natural the more I used it. I'm also
>>> slowly working towards adding a more "traditional" DoFn implementation
>>> approach where you implement the DoFn as an object type. In that case it
>>> would be very very easy to support both by having a default stream
>>> implementation call the equivalent of processElement. To make that
>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>> it yet.
>>>
>>> It's a bit more work and I've been prioritizing implementing composite
>>> and external transforms for the reasons you suggest. :-) I've got the
>>> basics of a composite transform (there's an equivalent wordcount example)
>>> and am hooking it into the pipeline generation, which should also give me
>>> everything I need to successfully hook in external transforms as well. That
>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>> itself as a composite transform which lets me get rid of the Pipeline {
>>> pipeline in ... } and just instead have things attach themselves to the
>>> pipeline implicitly.
>>>
>>> That said, there are some interesting IO possibilities that would be
>>> Swift native. In particularly, I've been looking at the native Swift
>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>> statements on individual files with projection pushdown supported for
>>> things like Parquet which could have some cool and performant data lake
>>> applications. I'll probably do a couple of the simpler IOs as
>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>> work with GCS.
>>>
>>> In any case, I'm updating the branch as I find a minute here and there.
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> Neat.
>>>>
>>>> Nothing like writing and SDK to actually understand how the FnAPI works
>>>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>>>> syntax for parDo (I don't know swift at all which is probably tripping me
>>>> up). The addition of external (cross-language) transforms could let you
>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>
>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>> user@beam.apache.org> wrote:
>>>>
>>>>> For everyone who is interested, here's the draft PR:
>>>>>
>>>>> https://github.com/apache/beam/pull/28062
>>>>>
>>>>> I haven't had a chance to test it on my M1 machine yet though (there's
>>>>> a good chance there are a few places that need to properly address
>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>
>>>>>
>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Thanks Cham,
>>>>>>
>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>> and may actually add that support before adding the ability to make
>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>> composite.
>>>>>>
>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>> chamikara@google.com> wrote:
>>>>>>
>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>> <us...@beam.apache.org> list.
>>>>>>>
>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>
>>>>>>>> Hello everyone,
>>>>>>>>
>>>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>>>> me at least that usually means I need to write some code so I can see
>>>>>>>> things happening in a debugger and to really prove to myself I
>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>
>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>
>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>
>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>
>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>> time.
>>>>>>>>
>>>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>
>>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>>
>>>>>>>
>>>>>>> This should be fine and we can get the code documented without these
>>>>>>> features. I think support for composites and adding an external transform
>>>>>>> (see, Java
>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>> Python
>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>> Go
>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>> TypeScript
>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>> I/O connectors) for free.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>>
>>>>>>>
>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>> easier to comment on a PR :)
>>>>>>>
>>>>>>> - Cham
>>>>>>>
>>>>>>> [1]
>>>>>>> [2]
>>>>>>> [3]
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>>
>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Ah, that is a good point—being element-wise would make managing windows and
time stamps easier for the user. Fortunately it’s a fairly easy change to
make and maybe even less typing for the user. I was originally thinking
side inputs and metrics would happen outside the loop, but I think you want
a class and not a closure at that point for sanity.

On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw <ro...@google.com>
wrote:

> Ah, I see.
>
> Yeah, I've thought about using an iterable for the whole bundle rather
> than start/finish bundle callbacks, but one of the questions is how that
> would impact implicit passing of the timestamp (and other) metadata from
> input elements to output elements. (You can of course attach the metadata
> to any output that happens in the loop body, but it's very easy to
> implicitly to break the 1:1 relationship here (e.g. by doing buffering or
> otherwise modifying local state) and this would be hard to detect. (I
> suppose trying to output after the loop finishes could require
> something more explicit).
>
>
> On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com> wrote:
>
>> Oh, I also forgot to mention that I included element-wise collection
>> operations like "map" that eliminate the need for pardo in many cases. the
>> groupBy command is actually a map + groupByKey under the hood. That was to
>> be more consistent with Swift's collection protocol (and is also why
>> PCollection and PCollectionStream are different types... PCollection
>> implements map and friends as pipeline construction operations whereas
>> PCollectionStream is an actual stream)
>>
>> I just happened to push some "IO primitives" that uses map rather than
>> pardo in a couple of places to do a true wordcount using good ol'
>> Shakespeare and very very primitive GCS IO.
>>
>> Best,
>> B
>>
>> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>>> before settling on where I ended up. Ultimately I decided to go with
>>> something that felt more Swift-y than anything else which means that rather
>>> than dealing with a single element like you do in the other SDKs you're
>>> dealing with a stream of elements (which of course will often be of size
>>> 1). That's a really natural paradigm in the Swift world especially with the
>>> async / await structures. So when you see something like:
>>>
>>> pardo(name:"Read Files") { filenames,output,errors in
>>>
>>> for try await (filename,_,_) in filenames {
>>>   ...
>>>   output.emit(data)
>>>
>>> }
>>>
>>> filenames is the input stream and then output and errors are both output
>>> streams. In theory you can have as many output streams as you like though
>>> at the moment there's a compiler bug in the new type pack feature that
>>> limits it to "as many as I felt like supporting". Presumably this will get
>>> fixed before the official 5.9 release which will probably be in the October
>>> timeframe if history is any guide)
>>>
>>> If you had parameterization you wanted to send that would look like
>>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>>> would take on the value of "Parameter." All of this is being typechecked at
>>> compile time BTW.
>>>
>>>
>>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>>> and other things where "_" is Swift for "ignore." In this case
>>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>>> optionally extract the timestamp and the window if you want to manipulate
>>> it somehow.
>>>
>>> That said it would also be natural to provide elementwise pardos--- that
>>> would probably mean having explicit type signatures in the closure. I had
>>> that at one point, but it felt less natural the more I used it. I'm also
>>> slowly working towards adding a more "traditional" DoFn implementation
>>> approach where you implement the DoFn as an object type. In that case it
>>> would be very very easy to support both by having a default stream
>>> implementation call the equivalent of processElement. To make that
>>> performant I need to implement an @DoFn macro and I just haven't gotten to
>>> it yet.
>>>
>>> It's a bit more work and I've been prioritizing implementing composite
>>> and external transforms for the reasons you suggest. :-) I've got the
>>> basics of a composite transform (there's an equivalent wordcount example)
>>> and am hooking it into the pipeline generation, which should also give me
>>> everything I need to successfully hook in external transforms as well. That
>>> will give me the jump on IOs as you say. I can also treat the pipeline
>>> itself as a composite transform which lets me get rid of the Pipeline {
>>> pipeline in ... } and just instead have things attach themselves to the
>>> pipeline implicitly.
>>>
>>> That said, there are some interesting IO possibilities that would be
>>> Swift native. In particularly, I've been looking at the native Swift
>>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>>> statements on individual files with projection pushdown supported for
>>> things like Parquet which could have some cool and performant data lake
>>> applications. I'll probably do a couple of the simpler IOs as
>>> well---there's a Swift AWS SDK binding that's pretty good that would give
>>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>>> work with GCS.
>>>
>>> In any case, I'm updating the branch as I find a minute here and there.
>>>
>>> Best,
>>> B
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> Neat.
>>>>
>>>> Nothing like writing and SDK to actually understand how the FnAPI works
>>>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>>>> syntax for parDo (I don't know swift at all which is probably tripping me
>>>> up). The addition of external (cross-language) transforms could let you
>>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>>
>>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>>> user@beam.apache.org> wrote:
>>>>
>>>>> For everyone who is interested, here's the draft PR:
>>>>>
>>>>> https://github.com/apache/beam/pull/28062
>>>>>
>>>>> I haven't had a chance to test it on my M1 machine yet though (there's
>>>>> a good chance there are a few places that need to properly address
>>>>> endianness. Specifically timestamps in windowed values and length in
>>>>> iterable coders as those both use specifically bigendian representations)
>>>>>
>>>>>
>>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Thanks Cham,
>>>>>>
>>>>>> Definitely happy to open a draft PR so folks can comment---there's
>>>>>> not as much code as it looks like since most of the LOC is just generated
>>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>>> and may actually add that support before adding the ability to make
>>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>>> composite.
>>>>>>
>>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>>> chamikara@google.com> wrote:
>>>>>>
>>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>>> <us...@beam.apache.org> list.
>>>>>>>
>>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>>> dev@beam.apache.org> wrote:
>>>>>>>
>>>>>>>> Hello everyone,
>>>>>>>>
>>>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>>>> me at least that usually means I need to write some code so I can see
>>>>>>>> things happening in a debugger and to really prove to myself I
>>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>>
>>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>>
>>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>>
>>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>>
>>>>>>>> At the moment it runs via the most recent XCode Beta using Swift
>>>>>>>> 5.9 on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>>> time.
>>>>>>>>
>>>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>>
>>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>>
>>>>>>>
>>>>>>> This should be fine and we can get the code documented without these
>>>>>>> features. I think support for composites and adding an external transform
>>>>>>> (see, Java
>>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>>> Python
>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>>> Go
>>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>>> TypeScript
>>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>>> I/O connectors) for free.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>>
>>>>>>>
>>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>>> easier to comment on a PR :)
>>>>>>>
>>>>>>> - Cham
>>>>>>>
>>>>>>> [1]
>>>>>>> [2]
>>>>>>> [3]
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> B
>>>>>>>>
>>>>>>>>
>>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via user <us...@beam.apache.org>.
Ah, I see.

Yeah, I've thought about using an iterable for the whole bundle rather than
start/finish bundle callbacks, but one of the questions is how that would
impact implicit passing of the timestamp (and other) metadata from
input elements to output elements. (You can of course attach the metadata
to any output that happens in the loop body, but it's very easy to
implicitly to break the 1:1 relationship here (e.g. by doing buffering or
otherwise modifying local state) and this would be hard to detect. (I
suppose trying to output after the loop finishes could require
something more explicit).


On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com> wrote:

> Oh, I also forgot to mention that I included element-wise collection
> operations like "map" that eliminate the need for pardo in many cases. the
> groupBy command is actually a map + groupByKey under the hood. That was to
> be more consistent with Swift's collection protocol (and is also why
> PCollection and PCollectionStream are different types... PCollection
> implements map and friends as pipeline construction operations whereas
> PCollectionStream is an actual stream)
>
> I just happened to push some "IO primitives" that uses map rather than
> pardo in a couple of places to do a true wordcount using good ol'
> Shakespeare and very very primitive GCS IO.
>
> Best,
> B
>
> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com> wrote:
>
>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>> before settling on where I ended up. Ultimately I decided to go with
>> something that felt more Swift-y than anything else which means that rather
>> than dealing with a single element like you do in the other SDKs you're
>> dealing with a stream of elements (which of course will often be of size
>> 1). That's a really natural paradigm in the Swift world especially with the
>> async / await structures. So when you see something like:
>>
>> pardo(name:"Read Files") { filenames,output,errors in
>>
>> for try await (filename,_,_) in filenames {
>>   ...
>>   output.emit(data)
>>
>> }
>>
>> filenames is the input stream and then output and errors are both output
>> streams. In theory you can have as many output streams as you like though
>> at the moment there's a compiler bug in the new type pack feature that
>> limits it to "as many as I felt like supporting". Presumably this will get
>> fixed before the official 5.9 release which will probably be in the October
>> timeframe if history is any guide)
>>
>> If you had parameterization you wanted to send that would look like
>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>> would take on the value of "Parameter." All of this is being typechecked at
>> compile time BTW.
>>
>>
>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>> and other things where "_" is Swift for "ignore." In this case
>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>> optionally extract the timestamp and the window if you want to manipulate
>> it somehow.
>>
>> That said it would also be natural to provide elementwise pardos--- that
>> would probably mean having explicit type signatures in the closure. I had
>> that at one point, but it felt less natural the more I used it. I'm also
>> slowly working towards adding a more "traditional" DoFn implementation
>> approach where you implement the DoFn as an object type. In that case it
>> would be very very easy to support both by having a default stream
>> implementation call the equivalent of processElement. To make that
>> performant I need to implement an @DoFn macro and I just haven't gotten to
>> it yet.
>>
>> It's a bit more work and I've been prioritizing implementing composite
>> and external transforms for the reasons you suggest. :-) I've got the
>> basics of a composite transform (there's an equivalent wordcount example)
>> and am hooking it into the pipeline generation, which should also give me
>> everything I need to successfully hook in external transforms as well. That
>> will give me the jump on IOs as you say. I can also treat the pipeline
>> itself as a composite transform which lets me get rid of the Pipeline {
>> pipeline in ... } and just instead have things attach themselves to the
>> pipeline implicitly.
>>
>> That said, there are some interesting IO possibilities that would be
>> Swift native. In particularly, I've been looking at the native Swift
>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>> statements on individual files with projection pushdown supported for
>> things like Parquet which could have some cool and performant data lake
>> applications. I'll probably do a couple of the simpler IOs as
>> well---there's a Swift AWS SDK binding that's pretty good that would give
>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>> work with GCS.
>>
>> In any case, I'm updating the branch as I find a minute here and there.
>>
>> Best,
>> B
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Neat.
>>>
>>> Nothing like writing and SDK to actually understand how the FnAPI works
>>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>>> syntax for parDo (I don't know swift at all which is probably tripping me
>>> up). The addition of external (cross-language) transforms could let you
>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>
>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>> user@beam.apache.org> wrote:
>>>
>>>> For everyone who is interested, here's the draft PR:
>>>>
>>>> https://github.com/apache/beam/pull/28062
>>>>
>>>> I haven't had a chance to test it on my M1 machine yet though (there's
>>>> a good chance there are a few places that need to properly address
>>>> endianness. Specifically timestamps in windowed values and length in
>>>> iterable coders as those both use specifically bigendian representations)
>>>>
>>>>
>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Thanks Cham,
>>>>>
>>>>> Definitely happy to open a draft PR so folks can comment---there's not
>>>>> as much code as it looks like since most of the LOC is just generated
>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>> and may actually add that support before adding the ability to make
>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>> composite.
>>>>>
>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>> <us...@beam.apache.org> list.
>>>>>>
>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>> dev@beam.apache.org> wrote:
>>>>>>
>>>>>>> Hello everyone,
>>>>>>>
>>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>>> me at least that usually means I need to write some code so I can see
>>>>>>> things happening in a debugger and to really prove to myself I
>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>
>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>
>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>
>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>
>>>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>> time.
>>>>>>>
>>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>
>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>
>>>>>>
>>>>>> This should be fine and we can get the code documented without these
>>>>>> features. I think support for composites and adding an external transform
>>>>>> (see, Java
>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>> Python
>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>> Go
>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>> TypeScript
>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>> I/O connectors) for free.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>
>>>>>>
>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>> easier to comment on a PR :)
>>>>>>
>>>>>> - Cham
>>>>>>
>>>>>> [1]
>>>>>> [2]
>>>>>> [3]
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>>
>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via dev <de...@beam.apache.org>.
Ah, I see.

Yeah, I've thought about using an iterable for the whole bundle rather than
start/finish bundle callbacks, but one of the questions is how that would
impact implicit passing of the timestamp (and other) metadata from
input elements to output elements. (You can of course attach the metadata
to any output that happens in the loop body, but it's very easy to
implicitly to break the 1:1 relationship here (e.g. by doing buffering or
otherwise modifying local state) and this would be hard to detect. (I
suppose trying to output after the loop finishes could require
something more explicit).


On Wed, Aug 23, 2023 at 6:56 PM Byron Ellis <by...@google.com> wrote:

> Oh, I also forgot to mention that I included element-wise collection
> operations like "map" that eliminate the need for pardo in many cases. the
> groupBy command is actually a map + groupByKey under the hood. That was to
> be more consistent with Swift's collection protocol (and is also why
> PCollection and PCollectionStream are different types... PCollection
> implements map and friends as pipeline construction operations whereas
> PCollectionStream is an actual stream)
>
> I just happened to push some "IO primitives" that uses map rather than
> pardo in a couple of places to do a true wordcount using good ol'
> Shakespeare and very very primitive GCS IO.
>
> Best,
> B
>
> On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com> wrote:
>
>> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
>> before settling on where I ended up. Ultimately I decided to go with
>> something that felt more Swift-y than anything else which means that rather
>> than dealing with a single element like you do in the other SDKs you're
>> dealing with a stream of elements (which of course will often be of size
>> 1). That's a really natural paradigm in the Swift world especially with the
>> async / await structures. So when you see something like:
>>
>> pardo(name:"Read Files") { filenames,output,errors in
>>
>> for try await (filename,_,_) in filenames {
>>   ...
>>   output.emit(data)
>>
>> }
>>
>> filenames is the input stream and then output and errors are both output
>> streams. In theory you can have as many output streams as you like though
>> at the moment there's a compiler bug in the new type pack feature that
>> limits it to "as many as I felt like supporting". Presumably this will get
>> fixed before the official 5.9 release which will probably be in the October
>> timeframe if history is any guide)
>>
>> If you had parameterization you wanted to send that would look like
>> pardo("Parameter") { param,filenames,output,error in ... } where "param"
>> would take on the value of "Parameter." All of this is being typechecked at
>> compile time BTW.
>>
>>
>> the (filename,_,_) is a tuple spreading construct like you have in ES6
>> and other things where "_" is Swift for "ignore." In this case
>> PCollectionStreams have an element signature of (Of,Date,Window) so you can
>> optionally extract the timestamp and the window if you want to manipulate
>> it somehow.
>>
>> That said it would also be natural to provide elementwise pardos--- that
>> would probably mean having explicit type signatures in the closure. I had
>> that at one point, but it felt less natural the more I used it. I'm also
>> slowly working towards adding a more "traditional" DoFn implementation
>> approach where you implement the DoFn as an object type. In that case it
>> would be very very easy to support both by having a default stream
>> implementation call the equivalent of processElement. To make that
>> performant I need to implement an @DoFn macro and I just haven't gotten to
>> it yet.
>>
>> It's a bit more work and I've been prioritizing implementing composite
>> and external transforms for the reasons you suggest. :-) I've got the
>> basics of a composite transform (there's an equivalent wordcount example)
>> and am hooking it into the pipeline generation, which should also give me
>> everything I need to successfully hook in external transforms as well. That
>> will give me the jump on IOs as you say. I can also treat the pipeline
>> itself as a composite transform which lets me get rid of the Pipeline {
>> pipeline in ... } and just instead have things attach themselves to the
>> pipeline implicitly.
>>
>> That said, there are some interesting IO possibilities that would be
>> Swift native. In particularly, I've been looking at the native Swift
>> binding for DuckDB (which is C++ based). DuckDB is SQL based but not
>> distributed in the same was as, say, Beam SQL... but it would allow for SQL
>> statements on individual files with projection pushdown supported for
>> things like Parquet which could have some cool and performant data lake
>> applications. I'll probably do a couple of the simpler IOs as
>> well---there's a Swift AWS SDK binding that's pretty good that would give
>> me S3 and there's a Cloud auth library as well that makes it pretty easy to
>> work with GCS.
>>
>> In any case, I'm updating the branch as I find a minute here and there.
>>
>> Best,
>> B
>>
>>
>>
>>
>>
>>
>>
>>
>>
>> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> Neat.
>>>
>>> Nothing like writing and SDK to actually understand how the FnAPI works
>>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>>> syntax for parDo (I don't know swift at all which is probably tripping me
>>> up). The addition of external (cross-language) transforms could let you
>>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>>
>>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>>> user@beam.apache.org> wrote:
>>>
>>>> For everyone who is interested, here's the draft PR:
>>>>
>>>> https://github.com/apache/beam/pull/28062
>>>>
>>>> I haven't had a chance to test it on my M1 machine yet though (there's
>>>> a good chance there are a few places that need to properly address
>>>> endianness. Specifically timestamps in windowed values and length in
>>>> iterable coders as those both use specifically bigendian representations)
>>>>
>>>>
>>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>>> wrote:
>>>>
>>>>> Thanks Cham,
>>>>>
>>>>> Definitely happy to open a draft PR so folks can comment---there's not
>>>>> as much code as it looks like since most of the LOC is just generated
>>>>> protobuf. As for the support, I definitely want to add external transforms
>>>>> and may actually add that support before adding the ability to make
>>>>> composites in the language itself. With the way the SDK is laid out adding
>>>>> composites to the pipeline graph is a separate operation than defining a
>>>>> composite.
>>>>>
>>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>>> chamikara@google.com> wrote:
>>>>>
>>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>>> Swift SDK from folks currently subscribed to the +user
>>>>>> <us...@beam.apache.org> list.
>>>>>>
>>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>>> dev@beam.apache.org> wrote:
>>>>>>
>>>>>>> Hello everyone,
>>>>>>>
>>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>>> me at least that usually means I need to write some code so I can see
>>>>>>> things happening in a debugger and to really prove to myself I
>>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>>> language to do it since there would be the temptation to read some code and
>>>>>>> convince myself that I actually understood what was going on.
>>>>>>>
>>>>>>> One thing led to another and it turns out that to get a minimal
>>>>>>> FnApi integration going you end up writing a fair bit of an SDK. So I
>>>>>>> decided to take things to a point where I had an SDK that could execute a
>>>>>>> word count example via a portable runner backend. I've now reached that
>>>>>>> point and would like to submit my prototype SDK to the list for feedback.
>>>>>>>
>>>>>>> It's currently living in a branch on my fork here:
>>>>>>>
>>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>>
>>>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>>> and make sure all of the endian checks are complete. The
>>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>>> output counts through two separate group by operations to get it past the
>>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>>> time.
>>>>>>>
>>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>>
>>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>>
>>>>>>
>>>>>> This should be fine and we can get the code documented without these
>>>>>> features. I think support for composites and adding an external transform
>>>>>> (see, Java
>>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>>> Python
>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>>> Go
>>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>>> TypeScript
>>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>>> I/O connectors) for free.
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>>
>>>>>>
>>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>>> easier to comment on a PR :)
>>>>>>
>>>>>> - Cham
>>>>>>
>>>>>> [1]
>>>>>> [2]
>>>>>> [3]
>>>>>>
>>>>>>
>>>>>>>
>>>>>>> Best,
>>>>>>> B
>>>>>>>
>>>>>>>
>>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Oh, I also forgot to mention that I included element-wise collection
operations like "map" that eliminate the need for pardo in many cases. the
groupBy command is actually a map + groupByKey under the hood. That was to
be more consistent with Swift's collection protocol (and is also why
PCollection and PCollectionStream are different types... PCollection
implements map and friends as pipeline construction operations whereas
PCollectionStream is an actual stream)

I just happened to push some "IO primitives" that uses map rather than
pardo in a couple of places to do a true wordcount using good ol'
Shakespeare and very very primitive GCS IO.

Best,
B

On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com> wrote:

> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
> before settling on where I ended up. Ultimately I decided to go with
> something that felt more Swift-y than anything else which means that rather
> than dealing with a single element like you do in the other SDKs you're
> dealing with a stream of elements (which of course will often be of size
> 1). That's a really natural paradigm in the Swift world especially with the
> async / await structures. So when you see something like:
>
> pardo(name:"Read Files") { filenames,output,errors in
>
> for try await (filename,_,_) in filenames {
>   ...
>   output.emit(data)
>
> }
>
> filenames is the input stream and then output and errors are both output
> streams. In theory you can have as many output streams as you like though
> at the moment there's a compiler bug in the new type pack feature that
> limits it to "as many as I felt like supporting". Presumably this will get
> fixed before the official 5.9 release which will probably be in the October
> timeframe if history is any guide)
>
> If you had parameterization you wanted to send that would look like
> pardo("Parameter") { param,filenames,output,error in ... } where "param"
> would take on the value of "Parameter." All of this is being typechecked at
> compile time BTW.
>
>
> the (filename,_,_) is a tuple spreading construct like you have in ES6 and
> other things where "_" is Swift for "ignore." In this case
> PCollectionStreams have an element signature of (Of,Date,Window) so you can
> optionally extract the timestamp and the window if you want to manipulate
> it somehow.
>
> That said it would also be natural to provide elementwise pardos--- that
> would probably mean having explicit type signatures in the closure. I had
> that at one point, but it felt less natural the more I used it. I'm also
> slowly working towards adding a more "traditional" DoFn implementation
> approach where you implement the DoFn as an object type. In that case it
> would be very very easy to support both by having a default stream
> implementation call the equivalent of processElement. To make that
> performant I need to implement an @DoFn macro and I just haven't gotten to
> it yet.
>
> It's a bit more work and I've been prioritizing implementing composite and
> external transforms for the reasons you suggest. :-) I've got the basics of
> a composite transform (there's an equivalent wordcount example) and am
> hooking it into the pipeline generation, which should also give me
> everything I need to successfully hook in external transforms as well. That
> will give me the jump on IOs as you say. I can also treat the pipeline
> itself as a composite transform which lets me get rid of the Pipeline {
> pipeline in ... } and just instead have things attach themselves to the
> pipeline implicitly.
>
> That said, there are some interesting IO possibilities that would be Swift
> native. In particularly, I've been looking at the native Swift binding for
> DuckDB (which is C++ based). DuckDB is SQL based but not distributed in the
> same was as, say, Beam SQL... but it would allow for SQL statements on
> individual files with projection pushdown supported for things like Parquet
> which could have some cool and performant data lake applications. I'll
> probably do a couple of the simpler IOs as well---there's a Swift AWS SDK
> binding that's pretty good that would give me S3 and there's a Cloud auth
> library as well that makes it pretty easy to work with GCS.
>
> In any case, I'm updating the branch as I find a minute here and there.
>
> Best,
> B
>
>
>
>
>
>
>
>
>
> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Neat.
>>
>> Nothing like writing and SDK to actually understand how the FnAPI works
>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>> syntax for parDo (I don't know swift at all which is probably tripping me
>> up). The addition of external (cross-language) transforms could let you
>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>
>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>> user@beam.apache.org> wrote:
>>
>>> For everyone who is interested, here's the draft PR:
>>>
>>> https://github.com/apache/beam/pull/28062
>>>
>>> I haven't had a chance to test it on my M1 machine yet though (there's a
>>> good chance there are a few places that need to properly address
>>> endianness. Specifically timestamps in windowed values and length in
>>> iterable coders as those both use specifically bigendian representations)
>>>
>>>
>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Thanks Cham,
>>>>
>>>> Definitely happy to open a draft PR so folks can comment---there's not
>>>> as much code as it looks like since most of the LOC is just generated
>>>> protobuf. As for the support, I definitely want to add external transforms
>>>> and may actually add that support before adding the ability to make
>>>> composites in the language itself. With the way the SDK is laid out adding
>>>> composites to the pipeline graph is a separate operation than defining a
>>>> composite.
>>>>
>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>> Swift SDK from folks currently subscribed to the +user
>>>>> <us...@beam.apache.org> list.
>>>>>
>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>> dev@beam.apache.org> wrote:
>>>>>
>>>>>> Hello everyone,
>>>>>>
>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>> me at least that usually means I need to write some code so I can see
>>>>>> things happening in a debugger and to really prove to myself I
>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>> language to do it since there would be the temptation to read some code and
>>>>>> convince myself that I actually understood what was going on.
>>>>>>
>>>>>> One thing led to another and it turns out that to get a minimal FnApi
>>>>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>>>>> take things to a point where I had an SDK that could execute a word count
>>>>>> example via a portable runner backend. I've now reached that point and
>>>>>> would like to submit my prototype SDK to the list for feedback.
>>>>>>
>>>>>> It's currently living in a branch on my fork here:
>>>>>>
>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>
>>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>> and make sure all of the endian checks are complete. The
>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>> output counts through two separate group by operations to get it past the
>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>> time.
>>>>>>
>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>
>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>
>>>>>
>>>>> This should be fine and we can get the code documented without these
>>>>> features. I think support for composites and adding an external transform
>>>>> (see, Java
>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>> Python
>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>> Go
>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>> TypeScript
>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>> I/O connectors) for free.
>>>>>
>>>>>
>>>>>>
>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>
>>>>>
>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>> easier to comment on a PR :)
>>>>>
>>>>> - Cham
>>>>>
>>>>> [1]
>>>>> [2]
>>>>> [3]
>>>>>
>>>>>
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>>
>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Oh, I also forgot to mention that I included element-wise collection
operations like "map" that eliminate the need for pardo in many cases. the
groupBy command is actually a map + groupByKey under the hood. That was to
be more consistent with Swift's collection protocol (and is also why
PCollection and PCollectionStream are different types... PCollection
implements map and friends as pipeline construction operations whereas
PCollectionStream is an actual stream)

I just happened to push some "IO primitives" that uses map rather than
pardo in a couple of places to do a true wordcount using good ol'
Shakespeare and very very primitive GCS IO.

Best,
B

On Wed, Aug 23, 2023 at 6:08 PM Byron Ellis <by...@google.com> wrote:

> Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
> before settling on where I ended up. Ultimately I decided to go with
> something that felt more Swift-y than anything else which means that rather
> than dealing with a single element like you do in the other SDKs you're
> dealing with a stream of elements (which of course will often be of size
> 1). That's a really natural paradigm in the Swift world especially with the
> async / await structures. So when you see something like:
>
> pardo(name:"Read Files") { filenames,output,errors in
>
> for try await (filename,_,_) in filenames {
>   ...
>   output.emit(data)
>
> }
>
> filenames is the input stream and then output and errors are both output
> streams. In theory you can have as many output streams as you like though
> at the moment there's a compiler bug in the new type pack feature that
> limits it to "as many as I felt like supporting". Presumably this will get
> fixed before the official 5.9 release which will probably be in the October
> timeframe if history is any guide)
>
> If you had parameterization you wanted to send that would look like
> pardo("Parameter") { param,filenames,output,error in ... } where "param"
> would take on the value of "Parameter." All of this is being typechecked at
> compile time BTW.
>
>
> the (filename,_,_) is a tuple spreading construct like you have in ES6 and
> other things where "_" is Swift for "ignore." In this case
> PCollectionStreams have an element signature of (Of,Date,Window) so you can
> optionally extract the timestamp and the window if you want to manipulate
> it somehow.
>
> That said it would also be natural to provide elementwise pardos--- that
> would probably mean having explicit type signatures in the closure. I had
> that at one point, but it felt less natural the more I used it. I'm also
> slowly working towards adding a more "traditional" DoFn implementation
> approach where you implement the DoFn as an object type. In that case it
> would be very very easy to support both by having a default stream
> implementation call the equivalent of processElement. To make that
> performant I need to implement an @DoFn macro and I just haven't gotten to
> it yet.
>
> It's a bit more work and I've been prioritizing implementing composite and
> external transforms for the reasons you suggest. :-) I've got the basics of
> a composite transform (there's an equivalent wordcount example) and am
> hooking it into the pipeline generation, which should also give me
> everything I need to successfully hook in external transforms as well. That
> will give me the jump on IOs as you say. I can also treat the pipeline
> itself as a composite transform which lets me get rid of the Pipeline {
> pipeline in ... } and just instead have things attach themselves to the
> pipeline implicitly.
>
> That said, there are some interesting IO possibilities that would be Swift
> native. In particularly, I've been looking at the native Swift binding for
> DuckDB (which is C++ based). DuckDB is SQL based but not distributed in the
> same was as, say, Beam SQL... but it would allow for SQL statements on
> individual files with projection pushdown supported for things like Parquet
> which could have some cool and performant data lake applications. I'll
> probably do a couple of the simpler IOs as well---there's a Swift AWS SDK
> binding that's pretty good that would give me S3 and there's a Cloud auth
> library as well that makes it pretty easy to work with GCS.
>
> In any case, I'm updating the branch as I find a minute here and there.
>
> Best,
> B
>
>
>
>
>
>
>
>
>
> On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> Neat.
>>
>> Nothing like writing and SDK to actually understand how the FnAPI works
>> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
>> syntax for parDo (I don't know swift at all which is probably tripping me
>> up). The addition of external (cross-language) transforms could let you
>> steal everything (e.g. IOs) pretty quickly from other SDKs.
>>
>> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <
>> user@beam.apache.org> wrote:
>>
>>> For everyone who is interested, here's the draft PR:
>>>
>>> https://github.com/apache/beam/pull/28062
>>>
>>> I haven't had a chance to test it on my M1 machine yet though (there's a
>>> good chance there are a few places that need to properly address
>>> endianness. Specifically timestamps in windowed values and length in
>>> iterable coders as those both use specifically bigendian representations)
>>>
>>>
>>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>>> wrote:
>>>
>>>> Thanks Cham,
>>>>
>>>> Definitely happy to open a draft PR so folks can comment---there's not
>>>> as much code as it looks like since most of the LOC is just generated
>>>> protobuf. As for the support, I definitely want to add external transforms
>>>> and may actually add that support before adding the ability to make
>>>> composites in the language itself. With the way the SDK is laid out adding
>>>> composites to the pipeline graph is a separate operation than defining a
>>>> composite.
>>>>
>>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <
>>>> chamikara@google.com> wrote:
>>>>
>>>>> Thanks Byron. This sounds great. I wonder if there is interest in
>>>>> Swift SDK from folks currently subscribed to the +user
>>>>> <us...@beam.apache.org> list.
>>>>>
>>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>>> dev@beam.apache.org> wrote:
>>>>>
>>>>>> Hello everyone,
>>>>>>
>>>>>> A couple of months ago I decided that I wanted to really understand
>>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>>> me at least that usually means I need to write some code so I can see
>>>>>> things happening in a debugger and to really prove to myself I
>>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>>> language to do it since there would be the temptation to read some code and
>>>>>> convince myself that I actually understood what was going on.
>>>>>>
>>>>>> One thing led to another and it turns out that to get a minimal FnApi
>>>>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>>>>> take things to a point where I had an SDK that could execute a word count
>>>>>> example via a portable runner backend. I've now reached that point and
>>>>>> would like to submit my prototype SDK to the list for feedback.
>>>>>>
>>>>>> It's currently living in a branch on my fork here:
>>>>>>
>>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>>
>>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>>> and make sure all of the endian checks are complete. The
>>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>>> output counts through two separate group by operations to get it past the
>>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>>> time.
>>>>>>
>>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>>> that feedback already (for example pardo was originally called dofn when
>>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>>> SwiftUI-like declarative API for creating them.
>>>>>>
>>>>>> There are of course a ton of missing bits still to be implemented,
>>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>>
>>>>>
>>>>> This should be fine and we can get the code documented without these
>>>>> features. I think support for composites and adding an external transform
>>>>> (see, Java
>>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>>> Python
>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>>> Go
>>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>>> TypeScript
>>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>>> I/O connectors) for free.
>>>>>
>>>>>
>>>>>>
>>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>>> that it can easily be used from the Swift Package Manager.
>>>>>>
>>>>>
>>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>>> easier to comment on a PR :)
>>>>>
>>>>> - Cham
>>>>>
>>>>> [1]
>>>>> [2]
>>>>> [3]
>>>>>
>>>>>
>>>>>>
>>>>>> Best,
>>>>>> B
>>>>>>
>>>>>>
>>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
before settling on where I ended up. Ultimately I decided to go with
something that felt more Swift-y than anything else which means that rather
than dealing with a single element like you do in the other SDKs you're
dealing with a stream of elements (which of course will often be of size
1). That's a really natural paradigm in the Swift world especially with the
async / await structures. So when you see something like:

pardo(name:"Read Files") { filenames,output,errors in

for try await (filename,_,_) in filenames {
  ...
  output.emit(data)

}

filenames is the input stream and then output and errors are both output
streams. In theory you can have as many output streams as you like though
at the moment there's a compiler bug in the new type pack feature that
limits it to "as many as I felt like supporting". Presumably this will get
fixed before the official 5.9 release which will probably be in the October
timeframe if history is any guide)

If you had parameterization you wanted to send that would look like
pardo("Parameter") { param,filenames,output,error in ... } where "param"
would take on the value of "Parameter." All of this is being typechecked at
compile time BTW.


the (filename,_,_) is a tuple spreading construct like you have in ES6 and
other things where "_" is Swift for "ignore." In this case
PCollectionStreams have an element signature of (Of,Date,Window) so you can
optionally extract the timestamp and the window if you want to manipulate
it somehow.

That said it would also be natural to provide elementwise pardos--- that
would probably mean having explicit type signatures in the closure. I had
that at one point, but it felt less natural the more I used it. I'm also
slowly working towards adding a more "traditional" DoFn implementation
approach where you implement the DoFn as an object type. In that case it
would be very very easy to support both by having a default stream
implementation call the equivalent of processElement. To make that
performant I need to implement an @DoFn macro and I just haven't gotten to
it yet.

It's a bit more work and I've been prioritizing implementing composite and
external transforms for the reasons you suggest. :-) I've got the basics of
a composite transform (there's an equivalent wordcount example) and am
hooking it into the pipeline generation, which should also give me
everything I need to successfully hook in external transforms as well. That
will give me the jump on IOs as you say. I can also treat the pipeline
itself as a composite transform which lets me get rid of the Pipeline {
pipeline in ... } and just instead have things attach themselves to the
pipeline implicitly.

That said, there are some interesting IO possibilities that would be Swift
native. In particularly, I've been looking at the native Swift binding for
DuckDB (which is C++ based). DuckDB is SQL based but not distributed in the
same was as, say, Beam SQL... but it would allow for SQL statements on
individual files with projection pushdown supported for things like Parquet
which could have some cool and performant data lake applications. I'll
probably do a couple of the simpler IOs as well---there's a Swift AWS SDK
binding that's pretty good that would give me S3 and there's a Cloud auth
library as well that makes it pretty easy to work with GCS.

In any case, I'm updating the branch as I find a minute here and there.

Best,
B









On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com> wrote:

> Neat.
>
> Nothing like writing and SDK to actually understand how the FnAPI works
> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
> syntax for parDo (I don't know swift at all which is probably tripping me
> up). The addition of external (cross-language) transforms could let you
> steal everything (e.g. IOs) pretty quickly from other SDKs.
>
> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> For everyone who is interested, here's the draft PR:
>>
>> https://github.com/apache/beam/pull/28062
>>
>> I haven't had a chance to test it on my M1 machine yet though (there's a
>> good chance there are a few places that need to properly address
>> endianness. Specifically timestamps in windowed values and length in
>> iterable coders as those both use specifically bigendian representations)
>>
>>
>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Thanks Cham,
>>>
>>> Definitely happy to open a draft PR so folks can comment---there's not
>>> as much code as it looks like since most of the LOC is just generated
>>> protobuf. As for the support, I definitely want to add external transforms
>>> and may actually add that support before adding the ability to make
>>> composites in the language itself. With the way the SDK is laid out adding
>>> composites to the pipeline graph is a separate operation than defining a
>>> composite.
>>>
>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
>>> wrote:
>>>
>>>> Thanks Byron. This sounds great. I wonder if there is interest in Swift
>>>> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>>>>  list.
>>>>
>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>> dev@beam.apache.org> wrote:
>>>>
>>>>> Hello everyone,
>>>>>
>>>>> A couple of months ago I decided that I wanted to really understand
>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>> me at least that usually means I need to write some code so I can see
>>>>> things happening in a debugger and to really prove to myself I
>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>> language to do it since there would be the temptation to read some code and
>>>>> convince myself that I actually understood what was going on.
>>>>>
>>>>> One thing led to another and it turns out that to get a minimal FnApi
>>>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>>>> take things to a point where I had an SDK that could execute a word count
>>>>> example via a portable runner backend. I've now reached that point and
>>>>> would like to submit my prototype SDK to the list for feedback.
>>>>>
>>>>> It's currently living in a branch on my fork here:
>>>>>
>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>
>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>> and make sure all of the endian checks are complete. The
>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>> output counts through two separate group by operations to get it past the
>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>> time.
>>>>>
>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>> that feedback already (for example pardo was originally called dofn when
>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>> SwiftUI-like declarative API for creating them.
>>>>>
>>>>> There are of course a ton of missing bits still to be implemented,
>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>
>>>>
>>>> This should be fine and we can get the code documented without these
>>>> features. I think support for composites and adding an external transform
>>>> (see, Java
>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>> Python
>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>> Go
>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>> TypeScript
>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>> I/O connectors) for free.
>>>>
>>>>
>>>>>
>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>> that it can easily be used from the Swift Package Manager.
>>>>>
>>>>
>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>> easier to comment on a PR :)
>>>>
>>>> - Cham
>>>>
>>>> [1]
>>>> [2]
>>>> [3]
>>>>
>>>>
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Indeed :-) Yeah, I went back and forth on the pardo syntax quite a bit
before settling on where I ended up. Ultimately I decided to go with
something that felt more Swift-y than anything else which means that rather
than dealing with a single element like you do in the other SDKs you're
dealing with a stream of elements (which of course will often be of size
1). That's a really natural paradigm in the Swift world especially with the
async / await structures. So when you see something like:

pardo(name:"Read Files") { filenames,output,errors in

for try await (filename,_,_) in filenames {
  ...
  output.emit(data)

}

filenames is the input stream and then output and errors are both output
streams. In theory you can have as many output streams as you like though
at the moment there's a compiler bug in the new type pack feature that
limits it to "as many as I felt like supporting". Presumably this will get
fixed before the official 5.9 release which will probably be in the October
timeframe if history is any guide)

If you had parameterization you wanted to send that would look like
pardo("Parameter") { param,filenames,output,error in ... } where "param"
would take on the value of "Parameter." All of this is being typechecked at
compile time BTW.


the (filename,_,_) is a tuple spreading construct like you have in ES6 and
other things where "_" is Swift for "ignore." In this case
PCollectionStreams have an element signature of (Of,Date,Window) so you can
optionally extract the timestamp and the window if you want to manipulate
it somehow.

That said it would also be natural to provide elementwise pardos--- that
would probably mean having explicit type signatures in the closure. I had
that at one point, but it felt less natural the more I used it. I'm also
slowly working towards adding a more "traditional" DoFn implementation
approach where you implement the DoFn as an object type. In that case it
would be very very easy to support both by having a default stream
implementation call the equivalent of processElement. To make that
performant I need to implement an @DoFn macro and I just haven't gotten to
it yet.

It's a bit more work and I've been prioritizing implementing composite and
external transforms for the reasons you suggest. :-) I've got the basics of
a composite transform (there's an equivalent wordcount example) and am
hooking it into the pipeline generation, which should also give me
everything I need to successfully hook in external transforms as well. That
will give me the jump on IOs as you say. I can also treat the pipeline
itself as a composite transform which lets me get rid of the Pipeline {
pipeline in ... } and just instead have things attach themselves to the
pipeline implicitly.

That said, there are some interesting IO possibilities that would be Swift
native. In particularly, I've been looking at the native Swift binding for
DuckDB (which is C++ based). DuckDB is SQL based but not distributed in the
same was as, say, Beam SQL... but it would allow for SQL statements on
individual files with projection pushdown supported for things like Parquet
which could have some cool and performant data lake applications. I'll
probably do a couple of the simpler IOs as well---there's a Swift AWS SDK
binding that's pretty good that would give me S3 and there's a Cloud auth
library as well that makes it pretty easy to work with GCS.

In any case, I'm updating the branch as I find a minute here and there.

Best,
B









On Wed, Aug 23, 2023 at 5:02 PM Robert Bradshaw <ro...@google.com> wrote:

> Neat.
>
> Nothing like writing and SDK to actually understand how the FnAPI works
> :). I like the use of groupBy. I have to admit I'm a bit mystified by the
> syntax for parDo (I don't know swift at all which is probably tripping me
> up). The addition of external (cross-language) transforms could let you
> steal everything (e.g. IOs) pretty quickly from other SDKs.
>
> On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <us...@beam.apache.org>
> wrote:
>
>> For everyone who is interested, here's the draft PR:
>>
>> https://github.com/apache/beam/pull/28062
>>
>> I haven't had a chance to test it on my M1 machine yet though (there's a
>> good chance there are a few places that need to properly address
>> endianness. Specifically timestamps in windowed values and length in
>> iterable coders as those both use specifically bigendian representations)
>>
>>
>> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com>
>> wrote:
>>
>>> Thanks Cham,
>>>
>>> Definitely happy to open a draft PR so folks can comment---there's not
>>> as much code as it looks like since most of the LOC is just generated
>>> protobuf. As for the support, I definitely want to add external transforms
>>> and may actually add that support before adding the ability to make
>>> composites in the language itself. With the way the SDK is laid out adding
>>> composites to the pipeline graph is a separate operation than defining a
>>> composite.
>>>
>>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
>>> wrote:
>>>
>>>> Thanks Byron. This sounds great. I wonder if there is interest in Swift
>>>> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>>>>  list.
>>>>
>>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <
>>>> dev@beam.apache.org> wrote:
>>>>
>>>>> Hello everyone,
>>>>>
>>>>> A couple of months ago I decided that I wanted to really understand
>>>>> how the Beam FnApi works and how it interacts with the Portable Runner. For
>>>>> me at least that usually means I need to write some code so I can see
>>>>> things happening in a debugger and to really prove to myself I
>>>>> understood what was going on I decided I couldn't use an existing SDK
>>>>> language to do it since there would be the temptation to read some code and
>>>>> convince myself that I actually understood what was going on.
>>>>>
>>>>> One thing led to another and it turns out that to get a minimal FnApi
>>>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>>>> take things to a point where I had an SDK that could execute a word count
>>>>> example via a portable runner backend. I've now reached that point and
>>>>> would like to submit my prototype SDK to the list for feedback.
>>>>>
>>>>> It's currently living in a branch on my fork here:
>>>>>
>>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>>
>>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9
>>>>> on Intel Macs, but should also work using beta builds of 5.9 for Linux
>>>>> running on Intel hardware. I haven't had a chance to try it on ARM hardware
>>>>> and make sure all of the endian checks are complete. The
>>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>>> output counts through two separate group by operations to get it past the
>>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>>> time.
>>>>>
>>>>> I've shown it to a couple of folks already and incorporated some of
>>>>> that feedback already (for example pardo was originally called dofn when
>>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>>> SwiftUI-like declarative API for creating them.
>>>>>
>>>>> There are of course a ton of missing bits still to be implemented,
>>>>> like counters, metrics, windowing, state, timers, etc.
>>>>>
>>>>
>>>> This should be fine and we can get the code documented without these
>>>> features. I think support for composites and adding an external transform
>>>> (see, Java
>>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>>> Python
>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>>> Go
>>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>>> TypeScript
>>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>>> to add support for multi-lang will bring in a lot of features (for example,
>>>> I/O connectors) for free.
>>>>
>>>>
>>>>>
>>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>>> that it can easily be used from the Swift Package Manager.
>>>>>
>>>>
>>>> +1 for creating a PR (may be as a draft initially). Also it'll be
>>>> easier to comment on a PR :)
>>>>
>>>> - Cham
>>>>
>>>> [1]
>>>> [2]
>>>> [3]
>>>>
>>>>
>>>>>
>>>>> Best,
>>>>> B
>>>>>
>>>>>
>>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Robert Bradshaw via user <us...@beam.apache.org>.
Neat.

Nothing like writing and SDK to actually understand how the FnAPI works :).
I like the use of groupBy. I have to admit I'm a bit mystified by the
syntax for parDo (I don't know swift at all which is probably tripping me
up). The addition of external (cross-language) transforms could let you
steal everything (e.g. IOs) pretty quickly from other SDKs.

On Fri, Aug 18, 2023 at 7:55 AM Byron Ellis via user <us...@beam.apache.org>
wrote:

> For everyone who is interested, here's the draft PR:
>
> https://github.com/apache/beam/pull/28062
>
> I haven't had a chance to test it on my M1 machine yet though (there's a
> good chance there are a few places that need to properly address
> endianness. Specifically timestamps in windowed values and length in
> iterable coders as those both use specifically bigendian representations)
>
>
> On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com> wrote:
>
>> Thanks Cham,
>>
>> Definitely happy to open a draft PR so folks can comment---there's not as
>> much code as it looks like since most of the LOC is just generated
>> protobuf. As for the support, I definitely want to add external transforms
>> and may actually add that support before adding the ability to make
>> composites in the language itself. With the way the SDK is laid out adding
>> composites to the pipeline graph is a separate operation than defining a
>> composite.
>>
>> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>> Thanks Byron. This sounds great. I wonder if there is interest in Swift
>>> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>>>  list.
>>>
>>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
>>> wrote:
>>>
>>>> Hello everyone,
>>>>
>>>> A couple of months ago I decided that I wanted to really understand how
>>>> the Beam FnApi works and how it interacts with the Portable Runner. For me
>>>> at least that usually means I need to write some code so I can see things
>>>> happening in a debugger and to really prove to myself I understood what was
>>>> going on I decided I couldn't use an existing SDK language to do it since
>>>> there would be the temptation to read some code and convince myself that I
>>>> actually understood what was going on.
>>>>
>>>> One thing led to another and it turns out that to get a minimal FnApi
>>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>>> take things to a point where I had an SDK that could execute a word count
>>>> example via a portable runner backend. I've now reached that point and
>>>> would like to submit my prototype SDK to the list for feedback.
>>>>
>>>> It's currently living in a branch on my fork here:
>>>>
>>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>>
>>>> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
>>>> Intel Macs, but should also work using beta builds of 5.9 for Linux running
>>>> on Intel hardware. I haven't had a chance to try it on ARM hardware and
>>>> make sure all of the endian checks are complete. The
>>>> "IntegrationTests.swift" file contains a word count example that reads some
>>>> local files (as well as a missing file to exercise DLQ functionality) and
>>>> output counts through two separate group by operations to get it past the
>>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>>> time.
>>>>
>>>> I've shown it to a couple of folks already and incorporated some of
>>>> that feedback already (for example pardo was originally called dofn when
>>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>>> possible, hence the heavy reliance on closures and while there aren't yet
>>>> composite PTransforms there's the beginnings of what would be needed for a
>>>> SwiftUI-like declarative API for creating them.
>>>>
>>>> There are of course a ton of missing bits still to be implemented, like
>>>> counters, metrics, windowing, state, timers, etc.
>>>>
>>>
>>> This should be fine and we can get the code documented without these
>>> features. I think support for composites and adding an external transform
>>> (see, Java
>>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>>> Python
>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>>> Go
>>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>>> TypeScript
>>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>>> to add support for multi-lang will bring in a lot of features (for example,
>>> I/O connectors) for free.
>>>
>>>
>>>>
>>>> Any and all feedback welcome and happy to submit a PR if folks are
>>>> interested, though the "Swift Way" would be to have it in its own repo so
>>>> that it can easily be used from the Swift Package Manager.
>>>>
>>>
>>> +1 for creating a PR (may be as a draft initially). Also it'll be easier
>>> to comment on a PR :)
>>>
>>> - Cham
>>>
>>> [1]
>>> [2]
>>> [3]
>>>
>>>
>>>>
>>>> Best,
>>>> B
>>>>
>>>>
>>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
For everyone who is interested, here's the draft PR:

https://github.com/apache/beam/pull/28062

I haven't had a chance to test it on my M1 machine yet though (there's a
good chance there are a few places that need to properly address
endianness. Specifically timestamps in windowed values and length in
iterable coders as those both use specifically bigendian representations)


On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com> wrote:

> Thanks Cham,
>
> Definitely happy to open a draft PR so folks can comment---there's not as
> much code as it looks like since most of the LOC is just generated
> protobuf. As for the support, I definitely want to add external transforms
> and may actually add that support before adding the ability to make
> composites in the language itself. With the way the SDK is laid out adding
> composites to the pipeline graph is a separate operation than defining a
> composite.
>
> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>> Thanks Byron. This sounds great. I wonder if there is interest in Swift
>> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>>  list.
>>
>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
>> wrote:
>>
>>> Hello everyone,
>>>
>>> A couple of months ago I decided that I wanted to really understand how
>>> the Beam FnApi works and how it interacts with the Portable Runner. For me
>>> at least that usually means I need to write some code so I can see things
>>> happening in a debugger and to really prove to myself I understood what was
>>> going on I decided I couldn't use an existing SDK language to do it since
>>> there would be the temptation to read some code and convince myself that I
>>> actually understood what was going on.
>>>
>>> One thing led to another and it turns out that to get a minimal FnApi
>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>> take things to a point where I had an SDK that could execute a word count
>>> example via a portable runner backend. I've now reached that point and
>>> would like to submit my prototype SDK to the list for feedback.
>>>
>>> It's currently living in a branch on my fork here:
>>>
>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>
>>> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
>>> Intel Macs, but should also work using beta builds of 5.9 for Linux running
>>> on Intel hardware. I haven't had a chance to try it on ARM hardware and
>>> make sure all of the endian checks are complete. The
>>> "IntegrationTests.swift" file contains a word count example that reads some
>>> local files (as well as a missing file to exercise DLQ functionality) and
>>> output counts through two separate group by operations to get it past the
>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>> time.
>>>
>>> I've shown it to a couple of folks already and incorporated some of that
>>> feedback already (for example pardo was originally called dofn when
>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>> possible, hence the heavy reliance on closures and while there aren't yet
>>> composite PTransforms there's the beginnings of what would be needed for a
>>> SwiftUI-like declarative API for creating them.
>>>
>>> There are of course a ton of missing bits still to be implemented, like
>>> counters, metrics, windowing, state, timers, etc.
>>>
>>
>> This should be fine and we can get the code documented without these
>> features. I think support for composites and adding an external transform
>> (see, Java
>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>> Python
>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>> Go
>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>> TypeScript
>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>> to add support for multi-lang will bring in a lot of features (for example,
>> I/O connectors) for free.
>>
>>
>>>
>>> Any and all feedback welcome and happy to submit a PR if folks are
>>> interested, though the "Swift Way" would be to have it in its own repo so
>>> that it can easily be used from the Swift Package Manager.
>>>
>>
>> +1 for creating a PR (may be as a draft initially). Also it'll be easier
>> to comment on a PR :)
>>
>> - Cham
>>
>> [1]
>> [2]
>> [3]
>>
>>
>>>
>>> Best,
>>> B
>>>
>>>
>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
For everyone who is interested, here's the draft PR:

https://github.com/apache/beam/pull/28062

I haven't had a chance to test it on my M1 machine yet though (there's a
good chance there are a few places that need to properly address
endianness. Specifically timestamps in windowed values and length in
iterable coders as those both use specifically bigendian representations)


On Thu, Aug 17, 2023 at 8:57 PM Byron Ellis <by...@google.com> wrote:

> Thanks Cham,
>
> Definitely happy to open a draft PR so folks can comment---there's not as
> much code as it looks like since most of the LOC is just generated
> protobuf. As for the support, I definitely want to add external transforms
> and may actually add that support before adding the ability to make
> composites in the language itself. With the way the SDK is laid out adding
> composites to the pipeline graph is a separate operation than defining a
> composite.
>
> On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>> Thanks Byron. This sounds great. I wonder if there is interest in Swift
>> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>>  list.
>>
>> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
>> wrote:
>>
>>> Hello everyone,
>>>
>>> A couple of months ago I decided that I wanted to really understand how
>>> the Beam FnApi works and how it interacts with the Portable Runner. For me
>>> at least that usually means I need to write some code so I can see things
>>> happening in a debugger and to really prove to myself I understood what was
>>> going on I decided I couldn't use an existing SDK language to do it since
>>> there would be the temptation to read some code and convince myself that I
>>> actually understood what was going on.
>>>
>>> One thing led to another and it turns out that to get a minimal FnApi
>>> integration going you end up writing a fair bit of an SDK. So I decided to
>>> take things to a point where I had an SDK that could execute a word count
>>> example via a portable runner backend. I've now reached that point and
>>> would like to submit my prototype SDK to the list for feedback.
>>>
>>> It's currently living in a branch on my fork here:
>>>
>>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>>
>>> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
>>> Intel Macs, but should also work using beta builds of 5.9 for Linux running
>>> on Intel hardware. I haven't had a chance to try it on ARM hardware and
>>> make sure all of the endian checks are complete. The
>>> "IntegrationTests.swift" file contains a word count example that reads some
>>> local files (as well as a missing file to exercise DLQ functionality) and
>>> output counts through two separate group by operations to get it past the
>>> "map reduce" size of pipeline. I've tested it against the Python Portable
>>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>>> time.
>>>
>>> I've shown it to a couple of folks already and incorporated some of that
>>> feedback already (for example pardo was originally called dofn when
>>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>>> possible, hence the heavy reliance on closures and while there aren't yet
>>> composite PTransforms there's the beginnings of what would be needed for a
>>> SwiftUI-like declarative API for creating them.
>>>
>>> There are of course a ton of missing bits still to be implemented, like
>>> counters, metrics, windowing, state, timers, etc.
>>>
>>
>> This should be fine and we can get the code documented without these
>> features. I think support for composites and adding an external transform
>> (see, Java
>> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
>> Python
>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
>> Go
>> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
>> TypeScript
>> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
>> to add support for multi-lang will bring in a lot of features (for example,
>> I/O connectors) for free.
>>
>>
>>>
>>> Any and all feedback welcome and happy to submit a PR if folks are
>>> interested, though the "Swift Way" would be to have it in its own repo so
>>> that it can easily be used from the Swift Package Manager.
>>>
>>
>> +1 for creating a PR (may be as a draft initially). Also it'll be easier
>> to comment on a PR :)
>>
>> - Cham
>>
>> [1]
>> [2]
>> [3]
>>
>>
>>>
>>> Best,
>>> B
>>>
>>>
>>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via dev <de...@beam.apache.org>.
Thanks Cham,

Definitely happy to open a draft PR so folks can comment---there's not as
much code as it looks like since most of the LOC is just generated
protobuf. As for the support, I definitely want to add external transforms
and may actually add that support before adding the ability to make
composites in the language itself. With the way the SDK is laid out adding
composites to the pipeline graph is a separate operation than defining a
composite.

On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
wrote:

> Thanks Byron. This sounds great. I wonder if there is interest in Swift
> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>  list.
>
> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
> wrote:
>
>> Hello everyone,
>>
>> A couple of months ago I decided that I wanted to really understand how
>> the Beam FnApi works and how it interacts with the Portable Runner. For me
>> at least that usually means I need to write some code so I can see things
>> happening in a debugger and to really prove to myself I understood what was
>> going on I decided I couldn't use an existing SDK language to do it since
>> there would be the temptation to read some code and convince myself that I
>> actually understood what was going on.
>>
>> One thing led to another and it turns out that to get a minimal FnApi
>> integration going you end up writing a fair bit of an SDK. So I decided to
>> take things to a point where I had an SDK that could execute a word count
>> example via a portable runner backend. I've now reached that point and
>> would like to submit my prototype SDK to the list for feedback.
>>
>> It's currently living in a branch on my fork here:
>>
>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>
>> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
>> Intel Macs, but should also work using beta builds of 5.9 for Linux running
>> on Intel hardware. I haven't had a chance to try it on ARM hardware and
>> make sure all of the endian checks are complete. The
>> "IntegrationTests.swift" file contains a word count example that reads some
>> local files (as well as a missing file to exercise DLQ functionality) and
>> output counts through two separate group by operations to get it past the
>> "map reduce" size of pipeline. I've tested it against the Python Portable
>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>> time.
>>
>> I've shown it to a couple of folks already and incorporated some of that
>> feedback already (for example pardo was originally called dofn when
>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>> possible, hence the heavy reliance on closures and while there aren't yet
>> composite PTransforms there's the beginnings of what would be needed for a
>> SwiftUI-like declarative API for creating them.
>>
>> There are of course a ton of missing bits still to be implemented, like
>> counters, metrics, windowing, state, timers, etc.
>>
>
> This should be fine and we can get the code documented without these
> features. I think support for composites and adding an external transform
> (see, Java
> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
> Python
> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
> Go
> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
> TypeScript
> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
> to add support for multi-lang will bring in a lot of features (for example,
> I/O connectors) for free.
>
>
>>
>> Any and all feedback welcome and happy to submit a PR if folks are
>> interested, though the "Swift Way" would be to have it in its own repo so
>> that it can easily be used from the Swift Package Manager.
>>
>
> +1 for creating a PR (may be as a draft initially). Also it'll be easier
> to comment on a PR :)
>
> - Cham
>
> [1]
> [2]
> [3]
>
>
>>
>> Best,
>> B
>>
>>
>>

Re: [Request for Feedback] Swift SDK Prototype

Posted by Byron Ellis via user <us...@beam.apache.org>.
Thanks Cham,

Definitely happy to open a draft PR so folks can comment---there's not as
much code as it looks like since most of the LOC is just generated
protobuf. As for the support, I definitely want to add external transforms
and may actually add that support before adding the ability to make
composites in the language itself. With the way the SDK is laid out adding
composites to the pipeline graph is a separate operation than defining a
composite.

On Thu, Aug 17, 2023 at 4:28 PM Chamikara Jayalath <ch...@google.com>
wrote:

> Thanks Byron. This sounds great. I wonder if there is interest in Swift
> SDK from folks currently subscribed to the +user <us...@beam.apache.org>
>  list.
>
> On Wed, Aug 16, 2023 at 6:53 PM Byron Ellis via dev <de...@beam.apache.org>
> wrote:
>
>> Hello everyone,
>>
>> A couple of months ago I decided that I wanted to really understand how
>> the Beam FnApi works and how it interacts with the Portable Runner. For me
>> at least that usually means I need to write some code so I can see things
>> happening in a debugger and to really prove to myself I understood what was
>> going on I decided I couldn't use an existing SDK language to do it since
>> there would be the temptation to read some code and convince myself that I
>> actually understood what was going on.
>>
>> One thing led to another and it turns out that to get a minimal FnApi
>> integration going you end up writing a fair bit of an SDK. So I decided to
>> take things to a point where I had an SDK that could execute a word count
>> example via a portable runner backend. I've now reached that point and
>> would like to submit my prototype SDK to the list for feedback.
>>
>> It's currently living in a branch on my fork here:
>>
>> https://github.com/byronellis/beam/tree/swift-sdk/sdks/swift
>>
>> At the moment it runs via the most recent XCode Beta using Swift 5.9 on
>> Intel Macs, but should also work using beta builds of 5.9 for Linux running
>> on Intel hardware. I haven't had a chance to try it on ARM hardware and
>> make sure all of the endian checks are complete. The
>> "IntegrationTests.swift" file contains a word count example that reads some
>> local files (as well as a missing file to exercise DLQ functionality) and
>> output counts through two separate group by operations to get it past the
>> "map reduce" size of pipeline. I've tested it against the Python Portable
>> Runner. Since my goal was to learn FnApi there is no Direct Runner at this
>> time.
>>
>> I've shown it to a couple of folks already and incorporated some of that
>> feedback already (for example pardo was originally called dofn when
>> defining pipelines). In general I've tried to make the API as "Swift-y" as
>> possible, hence the heavy reliance on closures and while there aren't yet
>> composite PTransforms there's the beginnings of what would be needed for a
>> SwiftUI-like declarative API for creating them.
>>
>> There are of course a ton of missing bits still to be implemented, like
>> counters, metrics, windowing, state, timers, etc.
>>
>
> This should be fine and we can get the code documented without these
> features. I think support for composites and adding an external transform
> (see, Java
> <https://github.com/apache/beam/blob/master/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java>,
> Python
> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/python/apache_beam/transforms/external.py#L556>,
> Go
> <https://github.com/apache/beam/blob/c7b7921185686da573f76ce7320817c32375c7d0/sdks/go/pkg/beam/xlang.go#L155>,
> TypeScript
> <https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/transforms/external.ts>)
> to add support for multi-lang will bring in a lot of features (for example,
> I/O connectors) for free.
>
>
>>
>> Any and all feedback welcome and happy to submit a PR if folks are
>> interested, though the "Swift Way" would be to have it in its own repo so
>> that it can easily be used from the Swift Package Manager.
>>
>
> +1 for creating a PR (may be as a draft initially). Also it'll be easier
> to comment on a PR :)
>
> - Cham
>
> [1]
> [2]
> [3]
>
>
>>
>> Best,
>> B
>>
>>
>>