On Wed, Jul 27, 2022 at 11:10 AM Lina Mårtensson <lina@camus.energy> wrote:

> Thanks Cham!
>
> Could you provide some more detail on your preference for developing a
> Python wrapper rather than implementing a source purely in Python?
>

I've mentioned the main advantages of developing a cross-language transform
over natively implementing this in Python below.

* Reduced cost of development

It's much easier to  develop a cross-language wrapper of the Java  source
than re-implementing the source in Python. Sources are some of the most
complex
code we have in Beam and sources control the parallelization of the
pipeline (for example, splitting and dynamic work rebalancing for supported
runners). So getting this code wrong can result in hard to track data
loss/duplication related issues.
Additionally, based on my experience, it's very hard to get a source
implementation correct and performant on the first try. It could take
additional benchmarks/user feedback over time to get the source production
ready.
Java BT source is already battle tested well (actually we have two Java
implementations [1][2] currently). So I would rather use a Java BT
connector as a cross-language transform than re-implementing sources for
other SDKs.

* Minimal maintenance cost

Developing a source/sink is just a part of the story. We (as a community)
have to maintain it over time and make sure that ongoing issues/feature
requests are adequately handled. In the past, we have had cases where
sources/sinks are available for multiple SDKs but one
is significantly better than others when it comes to the feature set (for
example, BigQuery). Cross-language will make this easier and will allow us
to maintain key logic in a single place.


>
> If I look at the instructions for using the x-language Spanner
> connector, then using this - from the user's perspective - would
> involve installing a Java runtime.
> That's not terrible, but I fear that getting this to work with bazel
> might end up being more trouble than expected. (That has often
> happened here, and we have enough trouble with getting Python 3.9 and
> 3.10 to co-exist.)
>

>From an end user perspective, all they should have to do is make sure that
Java is available in the machine where the job is submitted from. Beam has
features to allow starting up cross-language expansion services (that is
needed during job submission) automatically so users should not have to do
anything other than that.

At job execution, Beam (portable) uses Docker-based SDK harness containers
and we already release appropriate containers for each SDK. The runners
should seamlessly download containers needed to execute the job.

That said, the main downside of cross-language today is runner support.
Cross-language transform support is only available for portable Beam
runners (for example, Dataflow Runner v2) but this is the direction Beam
runners are going anyway.


>
> There are a few of us at our small start-up that have written
> MapReduces and similar in the past and are completely convinced by the
> Beam/Dataflow model. But many others have no previous experience and
> are skeptical, and see this new tool we're introducing as something
> that's more trouble than it's worth, and something they'd rather avoid
> - even when we see how lots of their use cases could be made much
> easier using Beam. I'm worried that every extra hoop to jump through
> will make it less likely to be widely used for us. Because of that, my
> bias would be towards having a Python connector rather than
> x-language, and I would find it really helpful to learn about why you
> both favor the x-language option.
>

I understand your concerns. It's certainly possible to develop the same
connector in multiple SDKs (and we provide SDF source framework support in
all SDK languages). But hopefully my comments above will give you an idea
of the downsides of this approach :).

Thanks,
Cham

[1]
https://github.com/apache/beam/blob/master/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
[2] https://cloud.google.com/bigtable/docs/hbase-dataflow-java


>
> Thanks!
> -Lina
>
> On Tue, Jul 26, 2022 at 6:11 PM Chamikara Jayalath <chamik...@google.com>
> wrote:
> >
> >
> >
> > On Mon, Jul 25, 2022 at 12:53 PM Lina Mårtensson via dev <
> dev@beam.apache.org> wrote:
> >>
> >> Hi dev,
> >>
> >> We're starting to incorporate BigTable in our stack and I've delighted
> >> my co-workers with how easy it was to create some BigTables with
> >> Beam... but there doesn't appear to be a reader for BigTable in
> >> Python.
> >>
> >> First off, is there a good reason why not/any reason why it would be
> difficult?
> >
> >
> > There's was a previous effort to implement a Python BT source but that
> was not completed:
> https://github.com/apache/beam/pull/11295#issuecomment-646378304
> >
> >>
> >>
> >> I could write one, but before I start, I'd love some input to make it
> easier.
> >>
> >> It appears that there would be two options: either write one in
> >> Python, or try to set one up with x-language from Java which I see is
> >> done e.g. with the Spanner IO Connector.
> >> Any recommendation on which one to pick or potential pitfalls in either
> choice?
> >>
> >> If I write one in Python, what should I think about?
> >> It is not obvious to me how to achieve parallelization, so any tips
> >> here would be welcome.
> >
> >
> > I would strongly prefer developing a  Python wrapper for the existing
> Java BT source using Beam's Multi-language Pipelines framework over
> developing a new Python source.
> >
> https://beam.apache.org/documentation/programming-guide/#multi-language-pipelines
> >
> > Thanks,
> > Cham
> >
> >
> >>
> >>
> >> Thanks!
> >> -Lina
>

Reply via email to