Thanks for the input, Robert!

> On Aug 21, 2019, at 11:49 AM, Robert Bradshaw <[email protected]> wrote:
> 
> On Wed, Aug 14, 2019 at 11:29 AM Ning Kang <[email protected] 
> <mailto:[email protected]>> 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:
> 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. 
Yes, I agree with the perspective for users who are familiar with notebook. Yet 
it doesn’t prevent us from creating ready-to-use containers (such as binder 
<https://github.com/jupyterhub/binderhub>)  for users who want to try Beam 
interactively without setting up a environment with all the dependencies 
interactive Beam introduces. I agree that experienced users understand how to 
set up additional dependencies and read examples, it’s just we are also 
targeting other entry level audiences.
But back to the original topic, the design is not trying to add new concept, 
but fixing some rough edges of existing Interactive Beam features. We can 
discuss whether a factory of create_pipeline() is really desired and decide 
whether to expose it later. We hope the interactive_beam module to be the only 
module an Interactive Beam user would directly invoke in their notebook.
> 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.)
Sorry for the confusion. What I’m saying is about the hidden states. Running 
several Python scripts from top to bottom in an IPython session might generate 
different effects than running them in the same order normally. Say if you have 
an in-memory global configuration that is shared among all the scripts and if 
it’s missing, a script initializes one. Running the scripts in IPython will 
pass the initialization and modification of configuration along the scripts. 
While running the scripts one by one will initialize different configurations. 
Running cells in a notebook is equivalent to appending the cells into a script 
and run it. The interactivity is not about the order, but if there is hidden 
states preserved between each statement or each script execution. And the users 
should expect that there might be hidden states when they are in an interactive 
environment because that is exactly the interactivity they expect. However, 
they don’t hold the hidden states, the session does it for them. A user 
wouldn’t need to explicitly say “preserve the variable x I’ve defined in this 
cell because I want to reuse it in some other cells I’m going to execute”. The 
user can directly access variable x once the cell defining x is executed. And 
even if the user deletes the cell defining x, x still exists. At that stage, no 
one would know there is a variable x in memory by just looking at the notebook. 
One would see a missing execution sequence (on top left of each executed cell) 
and wonder where the piece of code executed goes.
Above is just for interactivity, not Beam.

Another interesting thing is that a named PTransform cannot be applied to the 
same pipeline more than once. It means a cell with named PTransform:  p | 
“Name” >> NamedPTransform() cannot be re-executed. We might want to support 
such re-execution if the pipeline is in an interactive mode. In that case, the 
Beam behavior might be different from non-interactive Beam. 
> 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. 
Yes, I agree that runners should be consistent. The interactive runner we 
currently have is a wrapper of its underlying runner. Thus we don’t intend to 
brand it as a standalone runner. And that’s why we want to create an 
interactive_beam module for end users to avoid confusing them with a new runner 
called InteractiveRunner. And I want to focus on direct runner as underlying 
runner for now. Because with the portability of Beam, when applying 
interactivity to original pipeline, we can always do: pipeline_with_runner_A -> 
portable proto -> pipeline_with_direct_runner -> apply interactivity and get a 
new pipeline -> new portable proto -> pipeline_with_runner_A with interactivity.

And the run_pipeline() feature is exactly the transition from interactive to 
non-interactive.
With p.run(), one has to re-built/re-execute the pipeline p from the beginning. 
While they can do p -> portable proto -> 
pipeline_with_their_desired_runner_no_interactivity in a subsequent cell.
The run_pipeline() is a shorthand for the process. It’s about the same thing we 
want to decide for create_pipeline(): do we leave the boilerplate one-liner to 
the users and what would the level of the users be for their first time playing 
with Beam?
We never mutate the p defined by the user so p by itself is always 
non-interactive. Only when p.run() is invoked, interactivity is applied.

> 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.)
Yeah, this is debatable. I think the difference is, we want to introduce 
something that is Interactive Beam but not Interactive Runner.
I really wish that we have a 4runner standard thingy that can be implemented to 
intercept the pipeline for any runner before the execution. In that case, the 
wrapped interactive logic can be just a function that is applicable to existing 
runner in some interactive mode with “interactive=True”. But we don’t intend to 
introduce the new concept.
>> p = interactive_module.create_pipeline()
is saying the pipeline is created with interactivity and runnable in notebook. 
You can still convert the pipeline for other runners without interactivity.
There is no guarantee that the interactivity is provided by (direct) runner nor 
any of the underlying implementation is backward compatible.
There is no implementation details exposed in the API. You can even treat it as 
using this library, one can create a pipeline with direct runner but with 
interactivity as additional feature.
I would favor the composition way.
>> p = Pipeline(interactive_module.InteractiveRunner())
is saying you can create a pipeline with this new runner thingy and we’ll 
maintain the runner forever like other runners. 
Additionally it can even take in an underlying runner. But does it make sense 
for a runner to take another runner? Is this applicable to all runners?
The implementation details is exposed in the API itself while users don’t care 
about and we don’t want to maintain.
>> p = Pipeline(options, interactive=True)
is saying Pipeline can have an interactive mode. Like discussed above, it’ll be 
a new concept for the concept “pipeline”. It’s basically saying anything 
pipeline can have an interactive mode. That’s the inheritance way.
Does this apply to pipeline created by other runner? Is it needed when 
interactivity can be portable through doing it just for the direct runner and 
convert to pipelines for all other runners with Beam’s portability.

I would argue that anyone can use composition to achieve: pipeline + 
interactivity without expecting that interactivity is a standard defined by 
Beam.
And anyone can use any available interactivity component they could find online 
or implement by themselves.
Rather than Beam defining that pipeline.interactivity is a thing and anyone 
should implement and maintain it.

> 
> Similarly, why introduce a special run_pipline(p) rather than p.run()? 
> 
p.run() is interactive while run_pipeline(p) is the transition to 
non-interactive without the user’s boilerplate.
Switching runner for p.run() in notebook requires modifying existing cells and 
re-execution of cells.
Adding a line of run_pipeline(p) doesn’t require modifying existing cell nor 
re-execution of cells. It’s adhoc and a feature for interactivity.
> 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. 
We don’t intend to provide different interactive experience for different 
runners.
We want to focus on direct runner for now. In a notebook environment, direct 
runner is very intuitive. Especially for users who have not built any Beam 
pipeline nor even running pipelines with any runner. No excessive pipeline 
options required either.
And we apply the interactivity by creating a new pipeline with hidden states in 
the interactive environment from the user defined pipeline.
Runners run the new pipeline. It doesn’t matter what the runner is: 
(pipeline_with_runner_A -> portable proto ->) pipeline_with_direct_runner -> 
apply interactivity and get a new pipeline -> new portable proto -> 
pipeline_with_runner_A with interactivity.
And at the same time, user defined pipeline is never touched by interactivity 
logic. Users can further develop pipelines from it.
> 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? 
It’s still the same discussion of interactivity. We don’t have this 
specification.
Assuming you use interactive_beam module to write a piece of code:
  - When a user writes and execute codes in a notebook or in an IPython 
session, there will be hidden states. Until a kernel gets restarted, hidden 
states preserves across all statement execution.
  - When a user writes code in a script and execute it, each execution is 
equivalent to a distinct IPython session.
Interactivity is only applicable when hidden states are accessed from within 
the same interactive session/environment/context.
Thus the code is automatically non-interactive and equivalent to running the 
pipeline for the first time in a notebook.
> 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.
> 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 <mailto:[email protected]> 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 agree with it when the interactive mode is acknowledged as a 
must-have/common/standard for all runners or for Beam pipeline itself.
But this will be a new concept that is distributed top-down from Beam to its 
contributors.
Our current objective is to enable interactivity of Beam in notebook and 
introduce experimental usages/features (and we want to limit the features and 
make them easy to use).
I like the idea that anyone can write a runner or a transform. I would wish 
that anyone can add interactivity (before, during, or after pipeline execution) 
when it’s a mature concept.
Maybe in the future, interactivity will be something that is similar to 
accessibility, internationalization, testability and etc. (Like if you write a 
class, you need not only a test but also a notebook to demo it)
But we are not doing it right now until we have more feedback from the 
community and concrete CUJs.


> 
> 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. 
Thanks! And we haven’t really decided if plotting plain data is a good idea for 
PCollections. Plotting the metadata/analytics/insight might be a better option 
for users. I’m looking into facets <https://pair-code.github.io/facets/> that 
TFX notebook uses.

Yes, existing PR <https://github.com/apache/beam/pull/9278> is appending 
operations only.
We would have pruning when the appended operations severe the whole pipeline 
DAG into a set of DAGs to only execute the part of pipeline that needs to be 
re-executed (optimization). This is included in the design.

We could also have the replacing logic that allows users to re-execute a cell 
with named PTransform by replacing the PTransform.
This needs your precious feedback and is a bug-fix of existing interactive beam 
rather than the design around user-defined Collection variables.
This is actually another debatable topic. Like I said, executing cells in 
notebook is equivalent to appending the code into a script.
Users can re-execute cells with only anonymous PTransforms. It’s equivalent to 
applying the PTransform many times "in parallel".
However, users cannot re-execute cells with any named PTransform. The 
inconsistency comes.
Should the re-execution of named PTransform be supported? When supported, 
should we replace PTransform or append PTransforms "in parallel"?
I think we’ll eventually pick a route, go through it and see what the 
interactive Beam users provide as feed back.

> 
> Also +David Yan <mailto:[email protected]>  for more opinions.
> 
> Thanks!
> 
> Ning.
> 
> On Tue, Aug 13, 2019 at 6:00 PM Ahmet Altay <[email protected] 
> <mailto:[email protected]>> 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 <[email protected] 
> <mailto:[email protected]>> 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 <[email protected] 
> <mailto:[email protected]>> 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 <[email protected] 
> <mailto:[email protected]>> 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 
> <mailto:[email protected]> +Valentyn Tymofieiev 
> <mailto:[email protected]> +Sindy Li <mailto:[email protected]> +Brian 
> Hulette <mailto:[email protected]> )
> 
> Ahmet
> 
> 
> On Wed, Aug 7, 2019 at 12:36 PM Ning Kang <[email protected] 
> <mailto:[email protected]>> 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 
> <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