github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/interactive/README.md (about) 1 <!-- 2 Licensed to the Apache Software Foundation (ASF) under one 3 or more contributor license agreements. See the NOTICE file 4 distributed with this work for additional information 5 regarding copyright ownership. The ASF licenses this file 6 to you under the Apache License, Version 2.0 (the 7 "License"); you may not use this file except in compliance 8 with the License. You may obtain a copy of the License at 9 10 http://www.apache.org/licenses/LICENSE-2.0 11 12 Unless required by applicable law or agreed to in writing, 13 software distributed under the License is distributed on an 14 "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 KIND, either express or implied. See the License for the 16 specific language governing permissions and limitations 17 under the License. 18 --> 19 20 # Interactive Beam 21 22 ## Overview 23 24 Interactive Beam is aimed at integrating Apache Beam with 25 [Jupyter notebook](http://jupyter.org/) to make pipeline prototyping and data 26 exploration much faster and easier. It provides nice features including 27 28 1. Graphical representation 29 30 Visualize the Pipeline DAG: 31 32 ```python 33 import apache_beam.runners.interactive.interactive_beam as ib 34 from apache_beam.runners.interactive.interactive_runner import InteractiveRunner 35 36 p = beam.Pipeline(InteractiveRunner()) 37 # ... add transforms 38 ib.show_graph(pipeline) 39 ``` 40 41 Visualize elements in a PCollection: 42 43 ```python 44 pcoll = p | beam.Create([1, 2, 3]) 45 # include_window_info displays windowing information 46 # visualize_data visualizes data with https://pair-code.github.io/facets/ 47 ib.show(pcoll, include_window_info=True, visualize_data=True) 48 ``` 49 More details see the docstrings of `interactive_beam` module. 50 51 2. Support of streaming record/replay and dynamic visualization 52 53 For streaming pipelines, Interactive Beam records a subset of unbounded 54 sources in the pipeline automatically so that they can be replayed for 55 pipeline changes during prototyping. 56 57 There are a few knobs to tune the source recording: 58 59 ```python 60 # Set the amount of time recording data from unbounded sources. 61 ib.options.recording_duration = '10m' 62 63 # Set the recording size limit to 1 GB. 64 ib.options.recording_size_limit = 1e9 65 66 # Visualization is dynamic as data streamed in real time. 67 # n=100 indicates that displays at most 100 elements. 68 # duration=60 indicates that displays at most 60 seconds worth of unbounded 69 # source generated data. 70 ib.show(pcoll, include_window_info=True, n=100, duration=60) 71 72 # duration can also be strings. 73 ib.show(pcoll, include_window_info=True, duration='1m') 74 75 # If neither n nor duration is provided, the display is indefinitely until 76 # the current machine's recording usage hits the threadshold set by 77 # ib.options. 78 ib.show(pcoll, include_window_info=True) 79 ``` 80 More details see the docstrings of `interactive_beam` module. 81 82 3. Fetching PCollections as pandas.DataFrame 83 84 PCollections can be collected as a pandas.DataFrame: 85 86 ```python 87 pcoll_df = ib.collect(pcoll) # This returns a pandas.DataFrame! 88 ``` 89 90 4. Faster execution and re-execution 91 92 Interactive Beam analyzes the pipeline graph depending on what PCollection 93 you want to inspect and builds a pipeline fragment to only compute 94 necessary data. 95 96 ```python 97 pcoll = p | PTransformA | PTransformB 98 pcoll2 = p | PTransformC | PTransformD 99 100 ib.collect(pcoll) # <- only executes PTransformA and PTransformB 101 ib.collect(pcoll2) # <- only executes PTransformC and PTransformD 102 ``` 103 104 Interactive Beam caches PCollection inspected previously and re-uses it 105 when the data is still in scope. 106 107 ```python 108 pcoll = p | PTransformA 109 # pcoll2 depends on pcoll 110 pcoll2 = pcoll | PTransformB 111 ib.collect(pcoll2) # <- caches data for both pcoll and pcoll2 112 113 pcoll3 = pcoll2 | PTransformC 114 ib.collect(pcoll3) # <- reuses data of pcoll2 and only executes PTransformC 115 116 pcoll4 = pcoll | PTransformD 117 ib.collect(pcoll4) # <- reuses data of pcoll and only executes PTransformD 118 ``` 119 120 5. Supports global and local scopes 121 122 Interactive Beam automatically watches the `__main__` scope for pipeline and 123 PCollection definitions to implicitly do magic under the hood. 124 125 ```python 126 # In a script or in a notebook 127 p = beam.Pipeline(InteractiveRunner()) 128 pcoll = beam | SomeTransform 129 pcoll2 = pcoll | SomeOtherTransform 130 131 # p, pcoll and pcoll2 are all known to Interactive Beam. 132 ib.collect(pcoll) 133 ib.collect(pcoll2) 134 ib.show_graph(p) 135 ``` 136 137 You have to explicitly watch pipelines and PCollections in your local scope. 138 Otherwise, Interactive Beam doesn't know about them and won't handle them 139 with interactive features. 140 141 ```python 142 def a_func(): 143 p = beam.Pipeline(InteractiveRunner()) 144 pcoll = beam | SomeTransform 145 pcoll2 = pcoll | SomeOtherTransform 146 147 # Watch everything defined locally before this line. 148 ib.watch(locals()) 149 # Or explicitly watch them. 150 ib.watch({ 151 'p': p, 152 'pcoll': pcoll, 153 'pcoll2': pcoll2}) 154 155 # p, pcoll and pcoll2 are all known to Interactive Beam. 156 ib.collect(pcoll) 157 ib.collect(pcoll2) 158 ib.show_graph(p) 159 160 return p, pcoll, pcoll2 161 162 # Or return them to main scope 163 p, pcoll, pcoll2 = a_func() 164 ib.collect(pcoll) # Also works! 165 ``` 166 167 ## Status 168 169 * Supported languages: Python 170 * Supported platforms and caching location 171 172 | | Caching locally | Caching on GCS | 173 | ------------------------ | --------------- | -------------- | 174 | Running on local machine | supported | supported | 175 | Running on Flink | supported | supported | 176 177 ## Getting Started 178 179 **Note**: This guide assumes that you are somewhat familiar with key concepts in 180 Beam programming model including `Pipeline`, `PCollection`, `PTransform` and 181 `PipelineRunner` (see 182 [The Beam Model](https://github.com/apache/beam/tree/master#the-beam-model) for 183 a quick reference). For a more general and complete getting started guide, see 184 [Apache Beam Python SDK Quickstart](https://beam.apache.org/get-started/quickstart-py/). 185 186 ### Pre-requisites 187 188 * Install [GraphViz](https://www.graphviz.org/download/) with your favorite 189 system package manager. 190 191 * Install, create and activate your [venv](https://docs.python.org/3/library/venv.html). 192 (optional but recommended) 193 194 ```bash 195 python3 -m venv /path/to/beam_venv_dir 196 source /path/to/beam_venv_dir/bin/activate 197 pip install --upgrade pip setuptools wheel 198 ``` 199 200 If you are using shells other than bash (e.g. fish, csh), check 201 `beam_venv_dir/bin` for other scripts that activates the virtual 202 environment. 203 204 **CHECK** that the virtual environment is activated by running 205 206 ```bash 207 which python # This sould point to beam_venv_dir/bin/python 208 ``` 209 210 * Install [JupyterLab](https://jupyter.org/install.html). You can use 211 either **conda** or **pip**. 212 213 * conda 214 ```bash 215 conda install -c conda-forge jupyterlab 216 ``` 217 * pip 218 ```bash 219 pip install jupyterlab 220 ``` 221 222 * Set up Apache Beam Python. **Make sure the virtual environment is activated 223 when you run `setup.py`** 224 225 * ```bash 226 git clone https://github.com/apache/beam 227 cd beam/sdks/python 228 python setup.py install 229 ``` 230 231 * Install an IPython kernel for the virtual environment you've just created. 232 **Make sure the virtual environment is activate when you do this.** You can 233 skip this step if not using venv. 234 235 ```bash 236 pip install ipykernel 237 python -m ipykernel install --user --name beam_venv_kernel --display-name "Python3 (beam_venv)" 238 ``` 239 240 **CHECK** that IPython kernel `beam_venv_kernel` is available for Jupyter to 241 use. 242 243 ```bash 244 jupyter kernelspec list 245 ``` 246 247 * Extend JupyterLab through labextension. **Note**: labextension is different from nbextension 248 from pre-lab jupyter notebooks. 249 250 All jupyter labextensions need nodejs 251 252 ```bash 253 # Homebrew users do 254 brew install node 255 # Or Conda users do 256 conda install -c conda-forge nodejs 257 ``` 258 259 Enable ipywidgets 260 261 ```bash 262 pip install ipywidgets 263 jupyter labextension install @jupyter-widgets/jupyterlab-manager 264 ``` 265 266 ### Start the notebook 267 268 To start the notebook, simply run 269 270 ```bash 271 jupyter lab 272 ``` 273 274 Optionally increase the iopub broadcast data rate limit of jupyterlab 275 276 ```bash 277 jupyter lab --NotebookApp.iopub_data_rate_limit=10000000 278 ``` 279 280 281 This automatically opens your default web browser pointing to 282 http://localhost:8888/lab. 283 284 You can create a new notebook file by clicking `Python3 (beam_venv)` from the launcher 285 page of jupyterlab. 286 287 Or after you've already opened a notebook, change the kernel by clicking 288 `Kernel` > `Change Kernel` > `Python3 (beam_venv)`. 289 290 Voila! You can now run Beam pipelines interactively in your Jupyter notebook! 291 292 In the notebook, you can use `tab` key on the keyboard for auto-completion. 293 To turn on greedy auto-completion, you can run such ipython magic 294 295 ``` 296 %config IPCompleter.greedy=True 297 ``` 298 299 You can also use `shift` + `tab` keys on the keyboard for a popup of docstrings at the 300 current cursor position. 301 302 **See [Interactive Beam Example.ipynb](examples/Interactive%20Beam%20Example.ipynb) 303 for more examples.** 304 305 ## Portability 306 307 ### Portability across Storage Systems 308 309 By default, the caches are kept on the local file system of the machine in 310 `/tmp` directory. 311 312 You can specify the caching directory as follows 313 314 ```python 315 ib.options.cache_root = 'some/path/to/dir' 316 ``` 317 318 When using an `InteractiveRunner(underlying_runner=...)` that is running remotely 319 and distributed, a distributed file system such as Cloud Storage 320 (`ib.options.cache_root = gs://bucket/obj`) is necessary. 321 322 #### Caching PCollection on Google Cloud Storage 323 324 You can choose to cache PCollections on Google Cloud Storage with a few 325 credential settings. 326 327 * Install [Google Cloud SDK](cloud.google.com/sdk), and set your project, 328 account and other configurations with the following command. 329 330 ```bash 331 $ gcloud init 332 $ gcloud auth login 333 ``` 334 335 * Install the following google cloud modules. **Make sure the virtual 336 environment is activated when you do this.** 337 338 * ```bash 339 $ python -m pip install --upgrade apache-beam[gcp] 340 $ python -m pip install --upgrade google-cloud-storage 341 $ python -m pip install --upgrade google-cloud-dataflow 342 ``` 343 344 * Make sure you have **read and write access to that bucket** when you specify 345 to use that directory as caching directory. 346 347 * You may configure a cache directory to be used by all pipelines created afterward with 348 an `InteractiveRunner`. 349 350 * ```python 351 ib.options.cache_root = 'gs://bucket-name/obj' 352 ``` 353 354 ### Portability across Execution Platforms 355 356 The platform where the pipeline is executed is decided by the underlying runner 357 of InteractiveRunner. The default configuration runs on local machine with 358 [`direct_runner.DirectRunner()`](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/direct/direct_runner.py) 359 as the underlying runner. 360 361 #### Running Interactive Beam on Flink 362 363 You can choose to run Interactive Beam on Flink with the following settings. 364 365 * Use 366 [`flink_runner.FlinkRunner()`](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/runners/portability/flink_runner.py) 367 as the underlying runner. 368 369 ```python 370 p = beam.Pipeline(interactive_runner.InteractiveRunner(underlying_runner=flink_runner.FlinkRunner())) 371 ``` 372 373 * Alternatively, if the runtime environment is configured with a Google Cloud project, you can run Interactive Beam with Flink on Cloud Dataproc. To do so, configure the pipeline with a Google Cloud project. If using dev versioned Beam built from source code, it is necessary to specify an `environment_config` option to configure a containerized Beam SDK (you can choose a released container or build one yourself). 374 375 * ```python 376 ib.options.cache_root = 'gs://bucket-name/obj' 377 options = PipelineOptions([ 378 # The project can be attained simply from running the following commands: 379 # import google.auth 380 # project = google.auth.default()[1] 381 '--project={}'.format(project), 382 # The following environment_config only needs to be used when using a development kernel. 383 # Users do not need to use the 2.35.0 SDK, but the chosen release must be compatible with 384 # the Flink version used by the Dataproc image used by Interactive Beam. The current Flink 385 # version used is 1.12.5. 386 '--environment_config=apache/beam_python3.7_sdk:2.35.0', 387 ]) 388 ``` 389 390 **Note**: This guide and 391 [Interactive Beam Running on Flink.ipynb](examples/Interactive%20Beam%20Running%20on%20Flink.ipynb) 392 capture the status of the world when it's last updated. 393 394 ## More Information 395 396 * [Apache Beam Python SDK Quickstart](https://beam.apache.org/get-started/quickstart-py/) 397 * [Interactive Beam Design Doc V2](https://docs.google.com/document/d/1DYWrT6GL_qDCXhRMoxpjinlVAfHeVilK5Mtf8gO6zxQ/edit?usp=sharing) 398 * [Interactive Beam Design Doc V1](https://docs.google.com/document/d/10bTc97GN5Wk-nhwncqNq9_XkJFVVy0WLT4gPFqP6Kmw/edit?usp=sharing)