tree: 9993ea21d81159ceb778db96b55df50d57b6c6cf [path history] [tgz]
  1. caching/
  2. display/
  3. examples/
  4. extensions/
  5. messaging/
  6. options/
  7. testing/
  8. __init__.py
  9. background_caching_job.py
  10. background_caching_job_test.py
  11. cache_manager.py
  12. cache_manager_test.py
  13. interactive_beam.py
  14. interactive_beam_test.py
  15. interactive_environment.py
  16. interactive_environment_test.py
  17. interactive_runner.py
  18. interactive_runner_test.py
  19. pipeline_analyzer.py
  20. pipeline_analyzer_test.py
  21. pipeline_fragment.py
  22. pipeline_fragment_test.py
  23. pipeline_instrument.py
  24. pipeline_instrument_test.py
  25. README.md
  26. utils.py
  27. utils_test.py
sdks/python/apache_beam/runners/interactive/README.md

Interactive Beam

Overview

Interactive Beam is aimed at integrating Apache Beam with Jupyter notebook to make pipeline prototyping and data exploration much faster and easier. It provides nice features including

  1. Graphical representation

    When a pipeline is executed on a Jupyter notebook, it instantly displays the pipeline as a directed acyclic graph. Sampled PCollection results will be added to the graph as the pipeline execution proceeds.

  2. Fetching PCollections as list

    PCollections can be fetched as a list from the pipeline result. This unique feature of InteractiveRunner makes it much easier to integrate Beam pipeline into data analysis.

    p = beam.Pipeline(interactive_runner.InteractiveRunner())
    pcoll = p | SomePTransform | AnotherPTransform
    result = p.run().wait_until_finish()
    pcoll_list = result.get(pcoll)  # This returns a list!
    
  3. Faster re-execution

    InteractiveRunner caches PCollection results of pipeline executed previously and re-uses it when the same pipeline is submitted again.

    p = beam.Pipeline(interactive_runner.InteractiveRunner())
    pcoll = p | SomePTransform | AnotherPTransform
    result = p.run().wait_until_finish()
    
    pcoll2 = pcoll | YetAnotherPTransform
    result = p.run().wait_until_finish()  # <- only executes YetAnotherPTransform
    

Status

  • Supported languages: Python

  • Supported platforms and caching location

    Caching locallyCaching on GCS
    Running on local machinesupportedsupported
    Running on Flinksupportedsupported

Getting Started

Note: This guide assumes that you are somewhat familiar with key concepts in Beam programming model including Pipeline, PCollection, PTransform and PipelineRunner (see The Beam Model for a quick reference). For a more general and complete getting started guide, see Apache Beam Python SDK Quickstart.

Pre-requisites

  • Install GraphViz with your favorite system package manager.
  • Install JupyterLab. You can use either conda or pip.

    • conda
      conda install -c conda-forge jupyterlab
      
    • pip
      pip install jupyterlab
      
  • Install, create and activate your venv. (optional but recommended)

    python3 -m venv /path/to/beam_venv_dir
    source /path/to/beam_venv_dir/bin/activate
    

    If you are using shells other than bash (e.g. fish, csh), check beam_venv_dir/bin for other scripts that activates the virtual environment.

    CHECK that the virtual environment is activated by running

    which python  # This sould point to beam_venv_dir/bin/python
    
  • Set up Apache Beam Python. Make sure the virtual environment is activated when you run setup.py

  • git clone https://github.com/apache/beam
    cd beam/sdks/python
    python setup.py install
    
  • Install an IPython kernel for the virtual environment you've just created. Make sure the virtual environment is activate when you do this. You can skip this step if not using venv.

    pip install ipykernel
    python -m ipykernel install --user --name beam_venv_kernel --display-name "Python3 (beam_venv)"
    

    CHECK that IPython kernel beam_venv_kernel is available for Jupyter to use.

    jupyter kernelspec list
    
  • Extend JupyterLab through labextension. Note: labextension is different from nbextension from pre-lab jupyter notebooks.

    All jupyter labextensions need nodejs

    # Homebrew users do
    brew install node
    # Or Conda users do
    conda install -c conda-forge nodejs
    

    Enable ipywidgets

    pip install ipywidgets
    jupyter labextension install @jupyter-widgets/jupyterlab-manager
    

Start the notebook

To start the notebook, simply run

jupyter lab

Optionally increase the iopub broadcast data rate limit of jupyterlab

jupyter lab --NotebookApp.iopub_data_rate_limit=10000000

This automatically opens your default web browser pointing to http://localhost:8888/lab.

You can create a new notebook file by clicking Python3 (beam_venv) from the launcher page of jupyterlab.

Or after you've already opened a notebook, change the kernel by clicking Kernel > Change Kernel > Python3 (beam_venv).

Voila! You can now run Beam pipelines interactively in your Jupyter notebook!

In the notebook, you can use tab key on the keyboard for auto-completion. To turn on greedy auto-completion, you can run such ipython magic

%config IPCompleter.greedy=True

You can also use shift + tab keys on the keyboard for a popup of docstrings at the current cursor position.

See Interactive Beam Example.ipynb for more examples.

Portability

Portability across Storage Systems

By default, the caches are kept on the local file system of the machine in /tmp directory.

You can specify the caching directory as follows

cache_dir = 'some/path/to/dir'
runner = interactive_runner.InteractiveRunner(cache_dir=cache_dir)
p = beam.Pipeline(runner=runner)

Caching PCollection on Google Cloud Storage

You can choose to cache PCollections on Google Cloud Storage with a few credential settings.

  • Install Google Cloud SDK, and set your project, account and other configurations with the following command.

    $ gcloud init
    $ gcloud auth login
    
  • Install the following google cloud modules. Make sure the virtual environment is activated when you do this.

  • $ python -m pip install --upgrade apache-beam[gcp]
    $ python -m pip install --upgrade google-cloud-storage
    $ python -m pip install --upgrade google-cloud-dataflow
    
  • Make sure you have read and write access to that bucket when you specify to use that directory as caching directory.

  • cache_dir = 'gs://bucket-name/dir'
    runner = interactive_runner.InteractiveRunner(cache_dir=cache_dir)
    p = beam.Pipeline(runner=runner)
    

Portability across Execution Platforms

The platform where the pipeline is executed is decided by the underlying runner of InteractiveRunner. The default configuration runs on local machine with direct_runner.DirectRunner() as the underlying runner.

Running Interactive Beam on Flink

You can choose to run Interactive Beam on Flink with the following settings.

  • Use flink_runner.FlinkRunner() as the underlying runner.

    p = beam.Pipeline(interactive_runner.InteractiveRunner(underlying_runner=flink_runner.FlinkRunner()))
    

Note: This guide and Interactive Beam Running on Flink.ipynb capture the status of the world when it's last updated.

More Information