[ 
https://issues.apache.org/jira/browse/BEAM-7760?focusedWorklogId=312395&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-312395
 ]

ASF GitHub Bot logged work on BEAM-7760:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 13/Sep/19 22:07
            Start Date: 13/Sep/19 22:07
    Worklog Time Spent: 10m 
      Work Description: rosetn commented on pull request #9278: [BEAM-7760] 
Added Interactive Beam module
URL: https://github.com/apache/beam/pull/9278#discussion_r324374632
 
 

 ##########
 File path: sdks/python/apache_beam/runners/interactive/interactive_beam.py
 ##########
 @@ -0,0 +1,88 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Module of Interactive Beam features that can be used in notebook.
+
+The purpose of the module is to reduce the learning curve of Interactive Beam
+users, provide a single place for importing and add sugar syntax for all
+Interactive Beam components. It gives users capability to interact with 
existing
+environment/session/context for Interactive Beam and visualize PCollections as
+bounded dataset. In the meantime, it hides the interactivity implementation
+from users so that users can focus on developing Beam pipeline without worrying
+about how hidden states in the interactive session are managed.
+
+Note: Backward-compatibility of Interactive Beam is only guaranteed within this
+module. Please only invoke interfaces provided by this module in your notebook
+or application code if you want backward-compatibility.
+"""
+
+from apache_beam.runners.interactive import interactive_environment as ie
+
+
+def watch(watchable):
+  """Watches a watchable so that Interactive Beam understands your pipeline.
+
+  If you write Beam pipeline in a notebook or __main__ module directly, since
 
 Review comment:
   Do you only mean a Jupyter notebook?
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 312395)
    Time Spent: 5.5h  (was: 5h 20m)

> Interactive Beam Caching PCollections bound to user defined vars in notebook
> ----------------------------------------------------------------------------
>
>                 Key: BEAM-7760
>                 URL: https://issues.apache.org/jira/browse/BEAM-7760
>             Project: Beam
>          Issue Type: New Feature
>          Components: examples-python
>            Reporter: Ning Kang
>            Assignee: Ning Kang
>            Priority: Major
>          Time Spent: 5.5h
>  Remaining Estimate: 0h
>
> Cache only PCollections bound to user defined variables in a pipeline when 
> running pipeline with interactive runner in jupyter notebooks.
> [Interactive 
> Beam|[https://github.com/apache/beam/tree/master/sdks/python/apache_beam/runners/interactive]]
>  has been caching and using caches of "leaf" PCollections for interactive 
> execution in jupyter notebooks.
> The interactive execution is currently supported so that when appending new 
> transforms to existing pipeline for a new run, executed part of the pipeline 
> doesn't need to be re-executed. 
> A PCollection is "leaf" when it is never used as input in any PTransform in 
> the pipeline.
> The problem with building caches and pipeline to execute around "leaf" is 
> that when a PCollection is consumed by a sink with no output, the pipeline to 
> execute built will miss the subgraph generating and consuming that 
> PCollection.
> An example, "ReadFromPubSub --> WirteToPubSub" will result in an empty 
> pipeline.
> Caching around PCollections bound to user defined variables and replacing 
> transforms with source and sink of caches could resolve the pipeline to 
> execute properly under the interactive execution scenario. Also, cached 
> PCollection now can trace back to user code and can be used for user data 
> visualization if user wants to do it.
> E.g.,
> {code:java}
> // ...
> p = beam.Pipeline(interactive_runner.InteractiveRunner(),
>                   options=pipeline_options)
> messages = p | "Read" >> beam.io.ReadFromPubSub(subscription='...')
> messages | "Write" >> beam.io.WriteToPubSub(topic_path)
> result = p.run()
> // ...
> visualize(messages){code}
>  The interactive runner automatically figures out that PCollection
> {code:java}
> messages{code}
> created by
> {code:java}
> p | "Read" >> beam.io.ReadFromPubSub(subscription='...'){code}
> should be cached and reused if the notebook user appends more transforms.
>  And once the pipeline gets executed, the user could use any 
> visualize(PCollection) module to visualize the data statically (batch) or 
> dynamically (stream)



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

Reply via email to