On Wed, Aug 14, 2019 at 11:29 AM Ning Kang <ni...@google.com> wrote:

> Ahmet, thanks for forwarding!
>
>
>> My main concern at this point is the introduction of new concepts, even
>> though these are not changing other parts of the Beam SDKs. It would be
>> good to see at least an alternative option covered in the design document.
>> The reason is each additional concept adds to the mental load of users. And
>> also concepts from interactive Beam will shift user's expectations of Beam
>> even though there are not direct SDK modifications.
>
>
> Hi Robert. About the concern, I think I have a few points:
>
>    1. *Interactive Beam (or Interactive Runner) is already an existing
>    "new concept" that normal Beam user could opt-in if they want an
>    interactive Beam experience.* They need to do lots of setup steps and
>    learn new things such as Jupyter notebook and at least interactive_runner
>    module to make it work and make use of it.
>
> I think we should start with the perspective that most users interested in
using Beam interactively already know about Jupyter notebooks, or at least
ipython, and would want to use it to learn (and more effectively use) Beam.

>
>    1. *The behavior of existing interactive Beam is different from normal
>    Beam because of the interactive nature and the users would expect that.* 
> And
>    the users wouldn't shift their expectation of normal Beam. Just like
>    running Python scripts might result in different behavior than running all
>    of them in an interactive Python session.
>
> I'm not quite following this. One of the advantages strengths of Python is
that lack of the difference between the interactive vs. non-interactive
behavior. (The fact that a script's execution is always in top to bottom
order, unlike a notebook, is the primary difference.)

>
>    1. Or if a user runs a Beam pipeline with direct runner, they should
>    expect the behavior be different from running it on Dataflow while a user
>    needs GCP account. I think the users are aware of the difference when they
>    choose to use Interactive Beam.
>
>  The central, defining tenant of Beam is that behavior should be
consistent across different runners. Of course there are operational
details that are difficult, or perhaps even undesirable, to align (like, as
you mention, needing a GCP account for running on Dataflow, or providing
the location of the master when running Flink). But even these should be
minimized (see the recent efforts to make the temp location a standard
rather than dataflow-specific option).

We should, however, attempt to minimize gratuitous differences. In
particular, we should make it as easy as possible to transition (in terms
of code, docs, and developers) between interactive and non-interactive.

>
>    1. *Our design actually reduces the mental load of interactive Beam
>    users with intuitive interactive features*: create pipeline, visualize
>    intermediate PCollection, run pipeline at some point with other runners and
>    etc. For example, right now, the user needs to use a more complicated set
>    of libraries, like creating a Beam pipeline with interactive runner that
>    needs an underlying runner fed in.  We are getting rid of it. An
>    interactive Beam user shouldn't be concerned about the underlying
>    interactive magic.
>
> I agree a user shouldn't be concerned about the implementation details,
but I fail to see how

    p = interactive_module.create_pipeline()

is significantly simpler than, or preferable to,

   p = Pipeline(interactive_module.InteractiveRunner())

especially as the latter is in line with non-interactive pipelines (and all
our examples, docs, etc.) Now, perhaps an argument could be made that
interactivity is not a property of the runner, but something orthogonal to
that, e.g. one should write

    p = InteractivePipeline()

or

    p = Pipeline(options, interactive=True)

or similar. (It was introduced as a runner because, conceptually, a runner
is something that takes a pipeline and executes it.)

Similarly, why introduce a special run_pipline(p) rather than p.run()?


>    1. The interactive experience should be tailored for different
>    underlying runners. There is no portability of interactivity and users
>    opt-in interactive Beam using notebook would naturally expect something
>    similar to the direct runner.
>
> This concerns me a lot. Again, the core tenant of beam is that one can
choose the execution environment (runner) completely independently with how
one writes the pipeline. We should figure out what, if anything, needs to
be supported by a runner to support interactivity (the only thing that
comes to mind is a place to read and write temporary/cached data), but I
very strongly feel we should not go down the road of having different
interactive apis/experiences for different runners. In particular, the many
instances to the DirectRunner are worrisome--what's special about the
DirectRunner that other runners cannot provide that's needed for
interactive? If we can't come up with a good answer to that, we should not
impose this restriction.

>
>    1. *When users run pipeline built from interactive runner in a
>    non-interactive environment, it's direct runner like any other Beam
>    tutorial demonstrates*. It's even easier because the user doesn't need
>    to specify the runner nor pass in options.
>
>  So is the idea to have code like

    if is_ipython() or is_jupyter() or is ...:
      do_something()
    else:
      do_another_thing()

I'd really like to avoid this as it means one will (quite surprisingly, and
possibly for subtle reasons) not copy code from a notebook elsewhere. Or
did you mean something else here?

>
>    1. *Interactive Beam is solving an orthogonal set of problems than
>    Beam*. You can think of it as a wrapper of Beam that enables
>    interactivity and it's not even a real runner. It doesn't change the Beam
>    model such as how you build a pipeline. And with the Beam portability, you
>    get the capability to run the pipeline built from interactive runner with
>    other runners for free. It adds the interactive behavior that a user
>    expects.
>    2. *We want to open source it though we can iterate faster without
>    doing it*. The whole project can be encapsulated in a completely
>    irrelevant repository and from a developer's perspective, I want to hide
>    all the implementation details from the interactive Beam user. However, as
>    there is more and more desire for interactive Beam (+Mehran Nazir
>    <mna...@google.com> for more details), we want to share the
>    implementation with others who want to contribute and explore the
>    interactive world.
>
> I would much rather see interactivity as part of the Beam project. With
good APIs the implementations don't have to be tightly coupled (e.g. the
underlying runner delegation) but I think it will be a better user
experience if interactive was a mode rather than a wrapper with different
entry points.


I think watch() is a really good solution to knowing which collections to
cache, and visualize() will be very useful.

One thing I don't see tackled at all yet is the fact that pipelines are
only ever mutated by appending on new operations, so some design needs to
be done in terms of how to remove (possibly error-causing) operations or
replace bad ones with fixed ones. This is where most of the unsolved
problems lie.

Also +David Yan <david...@google.com>  for more opinions.
>
> Thanks!
>
> Ning.
>
> On Tue, Aug 13, 2019 at 6:00 PM Ahmet Altay <al...@google.com> wrote:
>
>> Ning, I believe Robert's questions from his email has not been answered
>> yet.
>>
>> On Tue, Aug 13, 2019 at 5:00 PM Ning Kang <ni...@google.com> wrote:
>>
>>> Hi all, I'll leave another 3 days for design
>>> <https://docs.google.com/document/d/1DYWrT6GL_qDCXhRMoxpjinlVAfHeVilK5Mtf8gO6zxQ/edit?usp=sharing>
>>>  review.
>>> Then we can have a vote session if there is no objection.
>>>
>>> Thanks!
>>>
>>> On Fri, Aug 9, 2019 at 12:14 PM Ning Kang <ni...@google.com> wrote:
>>>
>>>> Thanks Ahmet for the introduction!
>>>>
>>>> I've composed a design overview
>>>> <https://docs.google.com/document/d/1DYWrT6GL_qDCXhRMoxpjinlVAfHeVilK5Mtf8gO6zxQ/edit?usp=sharing>
>>>> describing changes we are making to components around interactive runner.
>>>> I'll share the document in our email thread too.
>>>>
>>>> The truth is since interactive runner is not yet a recognized runner as
>>>> part of the Beam SDK (and it's fundamentally a wrapper around direct
>>>> runner), we are not touching any Beam SDK components.
>>>> We'll not change any behavior of existing Beam SDK and we'll try our
>>>> best to keep it that way in the future.
>>>>
>>>
>> My main concern at this point is the introduction of new concepts, even
>> though these are not changing other parts of the Beam SDKs. It would be
>> good to see at least an alternative option covered in the design document.
>> The reason is each additional concept adds to the mental load of users. And
>> also concepts from interactive Beam will shift user's expectations of Beam
>> even though there are not direct SDK modifications.
>>
>>
>>>
>>>> In the meantime, I'll work on other components orthogonal to Beam such
>>>> as Pipeline Display and Data Visualization I mentioned in the design
>>>> overview.
>>>>
>>>> If you have any questions, please feel free to contact me through this
>>>> email address!
>>>>
>>>> Thanks!
>>>>
>>>> Regards,
>>>> Ning.
>>>>
>>>> On Wed, Aug 7, 2019 at 5:01 PM Ahmet Altay <al...@google.com> wrote:
>>>>
>>>>> Ning, thank you for the heads up.
>>>>>
>>>>> All, this is a proposed work for improving interactive Beam
>>>>> experience. As mentioned in Ning's email, new concepts are being
>>>>> introduced. And in addition iBeam as a name is used as a new reference. I
>>>>> hope that bringing the discussion to the mailing list will give it the
>>>>> additional visibility and more people could share their feedback.
>>>>>
>>>>> (cc'ing a few folks that might be interested +Robert Bradshaw
>>>>> <rober...@google.com> +Valentyn Tymofieiev <valen...@google.com> +Sindy
>>>>> Li <qiny...@google.com> +Brian Hulette <bhule...@google.com> )
>>>>>
>>>>> Ahmet
>>>>>
>>>>>
>>>>> On Wed, Aug 7, 2019 at 12:36 PM Ning Kang <ni...@google.com> wrote:
>>>>>
>>>>>> To whom may concern,
>>>>>>
>>>>>> This is Ning from Google. We are currently making efforts to leverage
>>>>>> an interactive runner under python beam sdk.
>>>>>>
>>>>>> There is already an interactive Beam (iBeam for short) runner with
>>>>>> jupyter notebook in the repo
>>>>>> <https://github.com/apache/beam/tree/master/sdks/python/apache_beam/runners/interactive>
>>>>>> .
>>>>>> Following the instructions on that page, one can set up an
>>>>>> interactive environment to develop and execute Beam pipeline 
>>>>>> interactively.
>>>>>>
>>>>>> However, there are many issues with existing iBeam. One issue is that
>>>>>> it uses a concept of leaf PCollection to cache and materialize 
>>>>>> intermediate
>>>>>> PCollection. If the user wants to reuse/introspect a non-leaf 
>>>>>> PCollection,
>>>>>> the interactive runner will run into errors.
>>>>>>
>>>>>> Our initial effort will be fixing the existing issues. And we also
>>>>>> want to make iBeam easy to use. Since iBeam uses the same model Beam 
>>>>>> uses,
>>>>>> there isn't really any difference for users between creating a pipeline
>>>>>> with interactive runner and other runners.
>>>>>> So we want to minimize the interfaces a user needs to learn while
>>>>>> giving the user some capability to interact with the interactive
>>>>>> environment.
>>>>>>
>>>>>> See this initial PR <https://github.com/apache/beam/pull/9278>, the
>>>>>> interactive_beam module will provide mainly 4 interfaces:
>>>>>>
>>>>>>    - For advanced users who define pipeline outside __main__, let
>>>>>>    them tell current interactive environment where they define their 
>>>>>> pipeline:
>>>>>>    watch()
>>>>>>       - This is very useful for tests where pipeline can be defined
>>>>>>       in test methods.
>>>>>>       - If the user simply creates pipeline in a Jupyter notebook or
>>>>>>       a plain Python script, they don't have to know/use this feature at 
>>>>>> all.
>>>>>>    - Let users create an interactive pipeline: create_pipeline()
>>>>>>       - invoking create_pipeline(), the user gets a Pipeline object
>>>>>>       that works as any other Pipeline object created from 
>>>>>> apache_beam.Pipeline()
>>>>>>       - However, the pipeline object p, when invoking p.run(), does
>>>>>>       some extra interactive magic.
>>>>>>       - We'll support interactive execution for DirectRunner at this
>>>>>>       moment.
>>>>>>    - Let users run the interactive pipeline as a normal pipeline:
>>>>>>    run_pipeline()
>>>>>>       - In an interactive environment, a user only needs to add and
>>>>>>       execute 1 line of code run_pipeline(pipeline) to execute any 
>>>>>> existing
>>>>>>       interactive pipeline object as normal pipeline in any selected 
>>>>>> platform.
>>>>>>       - We'll probably support Dataflow only. Other implementations
>>>>>>       can be added though.
>>>>>>    - Let users introspect any intermediate PCollection they have
>>>>>>    handler to: visualize()
>>>>>>       - If a user ever writes pcoll = p | "Some Transform" >>
>>>>>>       some_transform() ..., they can visualize(pcoll) once the pipeline 
>>>>>> p is
>>>>>>       executed.
>>>>>>       - p can be batch or streaming
>>>>>>       - The visualization will be some plot graph of data for the
>>>>>>       given PCollection as if it's materialized. If the PCollection is 
>>>>>> unbounded,
>>>>>>       the graph is dynamic.
>>>>>>
>>>>>> The PR will implement 1 and 2.
>>>>>>
>>>>>> We'll use https://issues.apache.org/jira/browse/BEAM-7923 as the top
>>>>>> level JIRA and add blocking JIRAs as development goes.
>>>>>>
>>>>>> External Beam users will not worry about any of the underlying
>>>>>> implementation details.
>>>>>> Except the 4 interfaces above, they learn and write normal Beam code
>>>>>> and can execute the pipeline immediately when they are done with
>>>>>> prototyping.
>>>>>>
>>>>>> Ning.
>>>>>>
>>>>>

Reply via email to