blob: b82a6ff6c5276abf9f623c5d10eaabdddab5c4b8 [file] [log] [blame]
<!doctype html><html lang=en class=no-js><head><meta charset=utf-8><meta http-equiv=x-ua-compatible content="IE=edge"><meta name=viewport content="width=device-width,initial-scale=1"><title>Ensuring Python Type Safety</title><meta name=description content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes."><link href="https://fonts.googleapis.com/css?family=Roboto:100,300,400,500,700" rel=stylesheet><link rel=preload href=/scss/main.min.408fddfe3e8a45f87a5a8c9a839d77db667c1c534e5e5cd0d957ffc3dd6c14cf.css as=style><link href=/scss/main.min.408fddfe3e8a45f87a5a8c9a839d77db667c1c534e5e5cd0d957ffc3dd6c14cf.css rel=stylesheet integrity><script src=https://code.jquery.com/jquery-2.2.4.min.js></script><style>.body__contained img{max-width:100%}</style><script type=text/javascript src=/js/bootstrap.min.2979f9a6e32fc42c3e7406339ee9fe76b31d1b52059776a02b4a7fa6a4fd280a.js defer></script>
<script type=text/javascript src=/js/language-switch-v2.min.121952b7980b920320ab229551857669209945e39b05ba2b433a565385ca44c6.js defer></script>
<script type=text/javascript src=/js/fix-menu.min.039174b67107465f2090a493f91e126f7aa797f29420f9edab8a54d9dd4b3d2d.js defer></script>
<script type=text/javascript src=/js/section-nav.min.1405fd5e70fab5f6c54037c269b1d137487d8f3d1b3009032525f6db3fbce991.js defer></script>
<script type=text/javascript src=/js/page-nav.min.af231204c9c52c5089d53a4c02739eacbb7f939e3be1c6ffcc212e0ac4dbf879.js defer></script>
<script type=text/javascript src=/js/expandable-list.min.75a4526624a3b8898fe7fb9e3428c205b581f8b38c7926922467aef17eac69f2.js defer></script>
<script type=text/javascript src=/js/copy-to-clipboard.min.364c06423d7e8993fc42bb4abc38c03195bc8386db26d18774ce775d08d5b18d.js defer></script>
<script type=text/javascript src=/js/calendar.min.336664054fa0f52b08bbd4e3c59b5cb6d63dcfb2b4d602839746516b0817446b.js defer></script>
<script type=text/javascript src=/js/fix-playground-nested-scroll.min.0283f1037cb1b9d5074c6eaf041292b524a8148a7cdb803d5ccd6d1fc4eb3253.js defer></script>
<script type=text/javascript src=/js/anchor-content-jump-fix.min.22d3240f81632e4c11179b9d2aaf37a40da9414333c43aa97344e8b21a7df0e4.js defer></script>
<link rel=alternate type=application/rss+xml title="Apache Beam" href=/feed.xml><link rel=canonical href=/documentation/sdks/python-type-safety/ data-proofer-ignore><link rel="shortcut icon" type=image/x-icon href=/images/favicon.ico><link rel=stylesheet href=https://use.fontawesome.com/releases/v5.4.1/css/all.css integrity=sha384-5sAR7xN1Nv6T6+dT2mhtzEpVJvfS3NScPQTrOxhwjIuvcA67KV2R5Jz6kr4abQsz crossorigin=anonymous><link rel=stylesheet href=https://unpkg.com/swiper@8/swiper-bundle.min.css><script async src=https://platform.twitter.com/widgets.js></script>
<script>(function(e,t,n,s,o,i,a){e.GoogleAnalyticsObject=o,e[o]=e[o]||function(){(e[o].q=e[o].q||[]).push(arguments)},e[o].l=1*new Date,i=t.createElement(n),a=t.getElementsByTagName(n)[0],i.async=1,i.src=s,a.parentNode.insertBefore(i,a)})(window,document,"script","//www.google-analytics.com/analytics.js","ga"),ga("create","UA-73650088-1","auto"),ga("send","pageview")</script><script>(function(e,t,n,s,o,i){e.hj=e.hj||function(){(e.hj.q=e.hj.q||[]).push(arguments)},e._hjSettings={hjid:2182187,hjsv:6},o=t.getElementsByTagName("head")[0],i=t.createElement("script"),i.async=1,i.src=n+e._hjSettings.hjid+s+e._hjSettings.hjsv,o.appendChild(i)})(window,document,"https://static.hotjar.com/c/hotjar-",".js?sv=")</script></head><body class=body data-spy=scroll data-target=.page-nav data-offset=0><nav class="navigation-bar-mobile header navbar navbar-fixed-top"><div class=navbar-header><a href=/ class=navbar-brand><img alt=Brand style=height:46px;width:43px src=/images/beam_logo_navbar_mobile.png></a>
<a class=navbar-link href=/get-started/>Get Started</a>
<a class=navbar-link href=/documentation/>Documentation</a>
<button type=button class="navbar-toggle menu-open" aria-expanded=false aria-controls=navbar onclick=openMenu()>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button></div><div class="navbar-mask closed"></div><div id=navbar class="navbar-container closed"><button type=button class=navbar-toggle aria-expanded=false aria-controls=navbar id=closeMenu>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button><ul class="nav navbar-nav"><li><div class=searchBar-mobile><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search></div></li><li><a class=navbar-link href=/about>About</a></li><li><a class=navbar-link href=/get-started/>Get Started</a></li><li><span class=navbar-link>Documentation</span><ul><li><a href=/documentation/>General</a></li><li><a href=/documentation/sdks/java/>Languages</a></li><li><a href=/documentation/runners/capability-matrix/>Runners</a></li><li><a href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><li><a class=navbar-link href=/roadmap/>Roadmap</a></li><li><a class=navbar-link href=/community/>Community</a></li><li><a class=navbar-link href=/contribute/>Contribute</a></li><li><a class=navbar-link href=/blog/>Blog</a></li><li><a class=navbar-link href=/case-studies/>Case Studies</a></li></ul><ul class="nav navbar-nav navbar-right"><li><a href=https://github.com/apache/beam/edit/master/website/www/site/content/en/documentation/sdks/python-type-safety.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a></li><li class=dropdown><a href=# class=dropdown-toggle id=apache-dropdown data-toggle=dropdown role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class="dropdown-menu dropdown-menu-right"><li><a target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a target=_blank href=https://www.apache.org/security/>Security</a></li><li><a target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></ul></div></nav><nav class=navigation-bar-desktop><a href=/ class=navbar-logo><img src=/images/beam_logo_navbar.png alt="Beam Logo"></a><div class=navbar-bar-left><div class=navbar-links><a class=navbar-link href=/about>About</a>
<a class=navbar-link href=/get-started/>Get Started</a><li class="dropdown navbar-dropdown navbar-dropdown-documentation"><a href=# class="dropdown-toggle navbar-link" role=button aria-haspopup=true aria-expanded=false>Documentation
<span><svg xmlns="http://www.w3.org/2000/svg" width="12" height="11" fill="none" viewBox="0 0 12 11"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M10.666 4.535 5.847 9.108 1.444 4.535"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link href=/documentation/>General</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/sdks/java/>Languages</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/runners/capability-matrix/>Runners</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><a class=navbar-link href=/roadmap/>Roadmap</a>
<a class=navbar-link href=/community/>Community</a>
<a class=navbar-link href=/contribute/>Contribute</a>
<a class=navbar-link href=/blog/>Blog</a>
<a class=navbar-link href=/case-studies/>Case Studies</a></div><div id=iconsBar><a type=button onclick=showSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M10.191 17c3.866.0 7-3.134 7-7s-3.134-7-7-7-7 3.134-7 7 3.134 7 7 7zm11 4-6-6"/></svg></a><a target=_blank href=https://github.com/apache/beam/edit/master/website/www/site/content/en/documentation/sdks/python-type-safety.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a><li class="dropdown navbar-dropdown navbar-dropdown-apache"><a href=# class=dropdown-toggle role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/security/>Security</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></div><div class="searchBar disappear"><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search>
<a type=button onclick=endSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="25" fill="none" viewBox="0 0 25 25"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M21.122 20.827 4.727 4.432M21.122 4.43 4.727 20.827"/></svg></a></div></div></nav><div class=header-push></div><div class="top-banners swiper"><div class=swiper-wrapper><div class=swiper-slide><a href=https://tour.beam.apache.org><img class=banner-img-desktop src=/images/banners/tour-of-beam/tour-of-beam-desktop.png alt="Start Tour of Beam">
<img class=banner-img-mobile src=/images/banners/tour-of-beam/tour-of-beam-mobile.png alt="Start Tour of Beam"></a></div><div class=swiper-slide><a href=https://beam.apache.org/documentation/ml/overview/><img class=banner-img-desktop src=/images/banners/machine-learning/machine-learning-desktop.jpg alt="Machine Learning">
<img class=banner-img-mobile src=/images/banners/machine-learning/machine-learning-mobile.jpg alt="Machine Learning"></a></div></div><div class=swiper-pagination></div><div class=swiper-button-prev></div><div class=swiper-button-next></div></div><script src=/js/swiper-bundle.min.min.e0e8f81b0b15728d35ff73c07f42ddbb17a108d6f23df4953cb3e60df7ade675.js></script>
<script src=/js/sliders/top-banners.min.afa7d0a19acf7a3b28ca369490b3d401a619562a2a4c9612577be2f66a4b9855.js></script>
<script>function showSearch(){addPlaceholder();var e,t=document.querySelector(".searchBar");t.classList.remove("disappear"),e=document.querySelector("#iconsBar"),e.classList.add("disappear")}function addPlaceholder(){$("input:text").attr("placeholder","What are you looking for?")}function endSearch(){var e,t=document.querySelector(".searchBar");t.classList.add("disappear"),e=document.querySelector("#iconsBar"),e.classList.remove("disappear")}function blockScroll(){$("body").toggleClass("fixedPosition")}function openMenu(){addPlaceholder(),blockScroll()}</script><div class="clearfix container-main-content"><div class="section-nav closed" data-offset-top=90 data-offset-bottom=500><span class="section-nav-back glyphicon glyphicon-menu-left"></span><nav><ul class=section-nav-list data-section-nav><li><span class=section-nav-list-main-title>Languages</span></li><li><span class=section-nav-list-title>Java</span><ul class=section-nav-list><li><a href=/documentation/sdks/java/>Java SDK overview</a></li><li><a href=https://beam.apache.org/releases/javadoc/2.56.0/ target=_blank>Java SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=/documentation/sdks/java-dependencies/>Java SDK dependencies</a></li><li><a href=/documentation/sdks/java-extensions/>Java SDK extensions</a></li><li><a href=/documentation/sdks/java-thirdparty/>Java 3rd party extensions</a></li><li><a href=/documentation/sdks/java/testing/nexmark/>Nexmark benchmark suite</a></li><li><a href=/documentation/sdks/java/testing/tpcds/>TPC-DS benchmark suite</a></li><li><a href=/documentation/sdks/java-multi-language-pipelines/>Java multi-language pipelines quickstart</a></li></ul></li><li><span class=section-nav-list-title>Python</span><ul class=section-nav-list><li><a href=/documentation/sdks/python/>Python SDK overview</a></li><li><a href=https://beam.apache.org/releases/pydoc/2.56.0/ target=_blank>Python SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=/documentation/sdks/python-dependencies/>Python SDK dependencies</a></li><li><a href=/documentation/sdks/python-streaming/>Python streaming pipelines</a></li><li><a href=/documentation/sdks/python-type-safety/>Ensuring Python type safety</a></li><li><a href=/documentation/sdks/python-machine-learning/>Machine Learning</a></li><li><a href=/documentation/sdks/python-pipeline-dependencies/>Managing pipeline dependencies</a></li><li><a href=/documentation/sdks/python-multi-language-pipelines/>Python multi-language pipelines quickstart</a></li><li><a href=/documentation/sdks/python-unrecoverable-errors/>Python Unrecoverable Errors</a></li></ul></li><li><span class=section-nav-list-title>Go</span><ul class=section-nav-list><li><a href=/documentation/sdks/go/>Go SDK overview</a></li><li><a href=https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam target=_blank>Go SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a><li><a href=/documentation/sdks/go-dependencies/>Go SDK dependencies</a></li><li><a href=/documentation/sdks/go-cross-compilation/>Cross compilation</a></li></li></ul></li><li><span class=section-nav-list-title>Typescript</span><ul class=section-nav-list><li><a href=/documentation/sdks/typescript/>Typescript SDK overview</a></li><li><a href=https://beam.apache.org/releases/typedoc/current/ target=_blank>Typescript SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li><li><span class=section-nav-list-title>Scala</span><ul class=section-nav-list><li><a href=/documentation/sdks/scala/>Scio</a></li><li><a href=https://spotify.github.io/scio/api/com/spotify/scio/index.html target=_blank>Scio SDK API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li><li><span class=section-nav-list-title>Yaml</span><ul class=section-nav-list><li><a href=/documentation/sdks/yaml/>Yaml overview</a></li><li><a href=/documentation/sdks/yaml-udf/>Yaml User Defined Functions</a></li><li><a href=/documentation/sdks/yaml-combine/>Yaml Aggregation</a></li><li><a href=/documentation/sdks/yaml-errors/>Error handling</a></li><li><a href=/documentation/sdks/yaml-inline-python/>Inlining Python</a></li><li><a href=https://beam.apache.org/releases/yamldoc/current/ target=_blank>YAML API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></ul></li><li><span class=section-nav-list-title>SQL</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/overview/>Overview</a></li><li><a href=/documentation/dsls/sql/walkthrough/>Walkthrough</a></li><li><a href=/documentation/dsls/sql/shell/>Shell</a></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Apache Calcite dialect</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/calcite/overview/>Calcite support overview</a></li><li><a href=/documentation/dsls/sql/calcite/query-syntax/>Query syntax</a></li><li><a href=/documentation/dsls/sql/calcite/lexical/>Lexical structure</a></li><li><a href=/documentation/dsls/sql/calcite/data-types/>Data types</a></li><li><a href=/documentation/dsls/sql/calcite/scalar-functions/>Scalar functions</a></li><li><a href=/documentation/dsls/sql/calcite/aggregate-functions/>Aggregate functions</a></li></ul></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>ZetaSQL dialect</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/zetasql/overview/>ZetaSQL support overview</a></li><li><a href=/documentation/dsls/sql/zetasql/syntax/>Function call rules</a></li><li><a href=/documentation/dsls/sql/zetasql/conversion-rules/>Conversion rules</a></li><li><a href=/documentation/dsls/sql/zetasql/query-syntax/>Query syntax</a></li><li><a href=/documentation/dsls/sql/zetasql/lexical/>Lexical structure</a></li><li><a href=/documentation/dsls/sql/zetasql/data-types/>Data types</a></li><li><a href=/documentation/dsls/sql/zetasql/operators/>Operators</a></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Scalar functions</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/zetasql/string-functions/>String functions</a></li><li><a href=/documentation/dsls/sql/zetasql/math-functions/>Mathematical functions</a></li><li><a href=/documentation/dsls/sql/zetasql/conditional-expressions/>Conditional expressions</a></li></ul></li><li><a href=/documentation/dsls/sql/zetasql/aggregate-functions/>Aggregate functions</a></li></ul></li><li class=section-nav-item--collapsible><span class=section-nav-list-title>Beam SQL extensions</span><ul class=section-nav-list><li><a href=/documentation/dsls/sql/extensions/create-external-table/>CREATE EXTERNAL TABLE</a></li><li><a href=/documentation/dsls/sql/extensions/windowing-and-triggering/>Windowing & triggering</a></li><li><a href=/documentation/dsls/sql/extensions/joins/>Joins</a></li><li><a href=/documentation/dsls/sql/extensions/user-defined-functions/>User-defined functions</a></li><li><a href=/documentation/dsls/sql/extensions/set/>SET pipeline options</a></li></ul></li></ul></li><li><span class=section-nav-list-title>DataFrames</span><ul class=section-nav-list><li><a href=/documentation/dsls/dataframes/overview/>Overview</a></li><li><a href=/documentation/dsls/dataframes/differences-from-pandas/>Differences from pandas</a></li><li><a href=https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/dataframe target=_blank>Example pipelines <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li><li><a href=https://beam.apache.org/releases/pydoc/2.56.0/apache_beam.dataframe.html target=_blank>DataFrame API reference <img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></li></ul></li></ul></nav></div><nav class="page-nav clearfix" data-offset-top=90 data-offset-bottom=500><nav id=TableOfContents><ul><li><a href=#benefits-of-type-hints>Benefits of Type Hints</a></li><li><a href=#declaring-type-hints>Declaring Type Hints</a><ul><li><a href=#declaring-type-hints-using-type-annotations>Declaring Type Hints Using Type Annotations</a></li><li><a href=#declaring-type-hints-inline>Declaring Type Hints Inline</a></li><li><a href=#declaring-type-hints-using-decorators>Declaring Type Hints Using Decorators</a><ul><li><a href=#disabling-annotations-use>Disabling Annotations Use</a></li></ul></li><li><a href=#defining-generic-types>Defining Generic Types</a></li></ul></li><li><a href=#kinds-of-type-hints>Kinds of Type Hints</a><ul><li><a href=#simple-type-hints>Simple Type Hints</a></li><li><a href=#parameterized-type-hints>Parameterized Type Hints</a></li><li><a href=#special-type-hints>Special Type Hints</a></li></ul></li><li><a href=#runtime-type-checking>Runtime Type Checking</a><ul><li><a href=#faster-runtime-type-checking>Faster Runtime Type Checking</a></li></ul></li><li><a href=#use-of-type-hints-in-coders>Use of Type Hints in Coders</a><ul><li><a href=#deterministic-coders>Deterministic Coders</a></li></ul></li></ul></nav></nav><div class="body__contained body__section-nav"><h1 id=ensuring-python-type-safety>Ensuring Python Type Safety</h1><p>Python is a dynamically-typed language with no static type checking. Because of the way Python&rsquo;s type checking works, as well as the deferred nature of runner execution, developer productivity can easily become bottle-necked by time spent investigating type-related errors.</p><p>The Apache Beam SDK for Python uses <strong>type hints</strong> during pipeline construction and runtime to try to emulate the correctness guarantees achieved by true static typing. Additionally, using type hints lays some groundwork that allows the backend service to perform efficient type deduction and registration of <code>Coder</code> objects.</p><p>Python version 3.5 introduces a module called <strong>typing</strong> to provide hints for type validators in the language.
The Beam SDK for Python implements a subset of <a href=https://www.python.org/dev/peps/pep-0484/>PEP 484</a> and aims to follow it as closely as possible in its own typehints module.</p><p>These flags control Beam type safety:</p><ul><li><p><code>--no_pipeline_type_check</code></p><p>Disables type checking during pipeline construction.
Default is to perform these checks.</p></li><li><p><code>--runtime_type_check</code></p><p>Enables runtime type checking of every element.
This may affect pipeline performance, so the default is to skip these checks.</p></li><li><p><code>--type_check_additional</code></p><p>Enables additional type checks. These are no enabled by default to preserve
backwards compatibility. This flag accepts a comma-separate list of options:</p><ul><li><code>all</code>: Enable all additional checks.</li><li><code>ptransform_fn</code>: Enable type hint decorators when used with the
<code>@ptransform_fn</code> decorator.</li></ul></li></ul><h2 id=benefits-of-type-hints>Benefits of Type Hints</h2><p>When you use type hints, Beam raises exceptions during pipeline construction time, rather than runtime.
For example, Beam generates an exception if it detects that your pipeline applies mismatched <code>PTransforms</code> (where the expected outputs of one transform do not match the expected inputs of the following transform).
These exceptions are raised at pipeline construction time, regardless of where your pipeline will execute.
Introducing type hints for the <code>PTransforms</code> you define allows you to catch potential bugs up front in the local runner, rather than after minutes of execution into a deep, complex pipeline.</p><p>Consider the following example, in which <code>numbers</code> is a <code>PCollection</code> of <code>str</code> values:</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=n>p</span> <span class=o>=</span> <span class=n>TestPipeline</span><span class=p>()</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>numbers</span> <span class=o>=</span> <span class=n>p</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Create</span><span class=p>([</span><span class=s1>&#39;1&#39;</span><span class=p>,</span> <span class=s1>&#39;2&#39;</span><span class=p>,</span> <span class=s1>&#39;3&#39;</span><span class=p>])</span></span></span></code></pre></div></div></div><p>The code then applies a <code>Filter</code> transform to the <code>numbers</code> collection with a callable that retrieves the even numbers.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=n>evens</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Filter</span><span class=p>(</span><span class=k>lambda</span> <span class=n>x</span><span class=p>:</span> <span class=n>x</span> <span class=o>%</span> <span class=mi>2</span> <span class=o>==</span> <span class=mi>0</span><span class=p>)</span></span></span></code></pre></div></div></div><p>When you call <code>p.run()</code>, this code generates an error when trying to execute this transform because <code>Filter</code> expects a <code>PCollection</code> of integers, but is given a <code>PCollection</code> of strings instead.
With type hints, this error could have been caught at pipeline construction time, before the pipeline even started running.</p><p>The Beam SDK for Python includes some automatic type hinting: for example, some <code>PTransforms</code>, such as <code>Create</code> and simple <code>ParDo</code> transforms, attempt to deduce their output type given their input.
However, Beam cannot deduce types in all cases.
Therefore, the recommendation is that you declare type hints to aid you in performing your own type checks.</p><h2 id=declaring-type-hints>Declaring Type Hints</h2><p>You can declare type hints on callables, <code>DoFns</code>, or entire <code>PTransforms</code>. There are three ways to declare type hints: inline during pipeline construction, as properties of the <code>DoFn</code> or <code>PTransform</code> using decorators, or as Python 3 type annotations on certain functions.</p><p>You can always declare type hints inline, but if you need them for code that is going to be reused, declare them as annotations or decorators.
For example, if your <code>DoFn</code> requires an <code>int</code> input, it makes more sense to declare the type hint for the input as an annotation of the arguments to <code>process</code> (or a property of the <code>DoFn</code>) rather than inline.</p><p>Using Annotations has the added benefit of allowing use of a static type checker (such as mypy) to additionally type check your code.
If you already use a type checker, using annotations instead of decorators reduces code duplication.
However, annotations do not cover all the use cases that decorators and inline declarations do.
For instance, they do not work for lambda functions.</p><h3 id=declaring-type-hints-using-type-annotations>Declaring Type Hints Using Type Annotations</h3><p><em>New in version 2.21.0.</em></p><p>To specify type hints as annotations on certain functions, use them as usual and omit any decorator hints or inline hints.</p><p>Annotations are currently supported on:</p><ul><li><code>process()</code> methods on <code>DoFn</code> subclasses.</li><li><code>expand()</code> methods on <code>PTransform</code> subclasses.</li><li>Functions passed to: <code>ParDo</code>, <code>Map</code>, <code>FlatMap</code>, <code>Filter</code>.</li></ul><p>The following code declares an <code>int</code> input and a <code>str</code> output type hint on the <code>to_id</code> transform, using annotations on <code>my_fn</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=k>def</span> <span class=nf>my_fn</span><span class=p>(</span><span class=n>element</span><span class=p>:</span> <span class=nb>int</span><span class=p>)</span> <span class=o>-&gt;</span> <span class=nb>str</span><span class=p>:</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=s1>&#39;id_&#39;</span> <span class=o>+</span> <span class=nb>str</span><span class=p>(</span><span class=n>element</span><span class=p>)</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>ids</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=s1>&#39;to_id&#39;</span> <span class=o>&gt;&gt;</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=n>my_fn</span><span class=p>)</span></span></span></code></pre></div></div></div><p>The following code demonstrates how to use annotations on <code>PTransform</code> subclasses.
A valid annotation is a <code>PCollection</code> that wraps an internal (nested) type, <code>PBegin</code>, <code>PDone</code>, or <code>None</code>.
The following code declares typehints on a custom PTransform, that takes a <code>PCollection[int]</code> input
and outputs a <code>PCollection[str]</code>, using annotations.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=kn>from</span> <span class=nn>apache_beam.pvalue</span> <span class=kn>import</span> <span class=n>PCollection</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>IntToStr</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>PTransform</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>expand</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>pcoll</span><span class=p>:</span> <span class=n>PCollection</span><span class=p>[</span><span class=nb>int</span><span class=p>])</span> <span class=o>-&gt;</span> <span class=n>PCollection</span><span class=p>[</span><span class=nb>str</span><span class=p>]:</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>pcoll</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=k>lambda</span> <span class=n>elem</span><span class=p>:</span> <span class=nb>str</span><span class=p>(</span><span class=n>elem</span><span class=p>))</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>ids</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=s1>&#39;convert to str&#39;</span> <span class=o>&gt;&gt;</span> <span class=n>IntToStr</span><span class=p>()</span></span></span></code></pre></div></div></div><p>The following code declares <code>int</code> input and output type hints on <code>filter_evens</code>, using annotations on <code>FilterEvensDoFn.process</code>.
Since <code>process</code> returns a generator, the output type for a DoFn producing a <code>PCollection[int]</code> is annotated as <code>Iterable[int]</code> (<code>Generator[int, None, None]</code> would also work here).
Beam will remove the outer iterable of the return type on the <code>DoFn.process</code> method and functions passed to <code>FlatMap</code> to deduce the element type of resulting PCollection .
It is an error to have a non-iterable return type annotation for these functions.
Other supported iterable types include: <code>Iterator</code>, <code>Generator</code>, <code>Tuple</code>, <code>List</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=kn>from</span> <span class=nn>typing</span> <span class=kn>import</span> <span class=n>Iterable</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>TypedFilterEvensDoFn</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>DoFn</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>process</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>element</span><span class=p>:</span> <span class=nb>int</span><span class=p>)</span> <span class=o>-&gt;</span> <span class=n>Iterable</span><span class=p>[</span><span class=nb>int</span><span class=p>]:</span>
</span></span><span class=line><span class=cl> <span class=k>if</span> <span class=n>element</span> <span class=o>%</span> <span class=mi>2</span> <span class=o>==</span> <span class=mi>0</span><span class=p>:</span>
</span></span><span class=line><span class=cl> <span class=k>yield</span> <span class=n>element</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>evens</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=s1>&#39;filter_evens&#39;</span> <span class=o>&gt;&gt;</span> <span class=n>beam</span><span class=o>.</span><span class=n>ParDo</span><span class=p>(</span><span class=n>TypedFilterEvensDoFn</span><span class=p>())</span></span></span></code></pre></div></div></div><p>The following code declares <code>int</code> input and output type hints on <code>double_evens</code>, using annotations on <code>FilterEvensDoubleDoFn.process</code>.
Since <code>process</code> returns a <code>list</code> or <code>None</code>, the output type is annotated as <code>Optional[List[int]]</code>.
Beam will also remove the outer <code>Optional</code> and (as above) the outer iterable of the return type, only on the <code>DoFn.process</code> method and functions passed to <code>FlatMap</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=kn>from</span> <span class=nn>typing</span> <span class=kn>import</span> <span class=n>List</span><span class=p>,</span> <span class=n>Optional</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>FilterEvensDoubleDoFn</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>DoFn</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>process</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>element</span><span class=p>:</span> <span class=nb>int</span><span class=p>)</span> <span class=o>-&gt;</span> <span class=n>Optional</span><span class=p>[</span><span class=n>List</span><span class=p>[</span><span class=nb>int</span><span class=p>]]:</span>
</span></span><span class=line><span class=cl> <span class=k>if</span> <span class=n>element</span> <span class=o>%</span> <span class=mi>2</span> <span class=o>==</span> <span class=mi>0</span><span class=p>:</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=p>[</span><span class=n>element</span><span class=p>,</span> <span class=n>element</span><span class=p>]</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=kc>None</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>evens</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=s1>&#39;double_evens&#39;</span> <span class=o>&gt;&gt;</span> <span class=n>beam</span><span class=o>.</span><span class=n>ParDo</span><span class=p>(</span><span class=n>FilterEvensDoubleDoFn</span><span class=p>())</span></span></span></code></pre></div></div></div><h3 id=declaring-type-hints-inline>Declaring Type Hints Inline</h3><p>To specify type hints inline, use the methods <code>with_input_types</code> and <code>with_output_types</code>. The following example code declares an input type hint inline:</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=n>evens</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Filter</span><span class=p>(</span><span class=k>lambda</span> <span class=n>x</span><span class=p>:</span> <span class=n>x</span> <span class=o>%</span> <span class=mi>2</span> <span class=o>==</span> <span class=mi>0</span><span class=p>)</span><span class=o>.</span><span class=n>with_input_types</span><span class=p>(</span><span class=nb>int</span><span class=p>)</span></span></span></code></pre></div></div></div><p>When you apply the Filter transform to the numbers collection in the example above, you&rsquo;ll be able to catch the error during pipeline construction.</p><h3 id=declaring-type-hints-using-decorators>Declaring Type Hints Using Decorators</h3><p>To specify type hints as properties of a <code>DoFn</code> or <code>PTransform</code>, use the decorators <code>@with_input_types()</code> and <code>@with_output_types()</code>.</p><p>The following code declares an <code>int</code> type hint on <code>FilterEvensDoFn</code>, using the decorator <code>@with_input_types()</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=nd>@beam.typehints.with_input_types</span><span class=p>(</span><span class=nb>int</span><span class=p>)</span>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>FilterEvensDoFn</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>DoFn</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>process</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>element</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>if</span> <span class=n>element</span> <span class=o>%</span> <span class=mi>2</span> <span class=o>==</span> <span class=mi>0</span><span class=p>:</span>
</span></span><span class=line><span class=cl> <span class=k>yield</span> <span class=n>element</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>evens</span> <span class=o>=</span> <span class=n>numbers</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>ParDo</span><span class=p>(</span><span class=n>FilterEvensDoFn</span><span class=p>())</span></span></span></code></pre></div></div></div><p>Decorators receive an arbitrary number of positional and/or keyword arguments, typically interpreted in the context of the function they&rsquo;re wrapping. Generally the first argument is a type hint for the main input, and additional arguments are type hints for side inputs.</p><h4 id=disabling-annotations-use>Disabling Annotations Use</h4><p>Since this style of type hint declaration is enabled by default, here are some ways to disable it.</p><ol><li>Using the <code>@beam.typehints.no_annotations</code> decorator on the specific function you want Beam to ignore annotations for.</li><li>Declaring type hints using the decorator or inline methods above.
These will take precedence over annotations.</li><li>Calling <code>beam.typehints.disable_type_annotations()</code> before pipeline creation.
This will prevent Beam from looking at annotations on all functions.</li></ol><h3 id=defining-generic-types>Defining Generic Types</h3><p>You can use type hint annotations to define generic types.
The following code specifies an input type hint that asserts the generic type <code>T</code>, and an output type hint that asserts the type <code>Tuple[int, T]</code>.
If the input to <code>MyTransform</code> is of type <code>str</code>, Beam will infer the output type to be <code>Tuple[int, str]</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=kn>from</span> <span class=nn>typing</span> <span class=kn>import</span> <span class=n>Tuple</span><span class=p>,</span> <span class=n>TypeVar</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>T</span> <span class=o>=</span> <span class=n>TypeVar</span><span class=p>(</span><span class=s1>&#39;T&#39;</span><span class=p>)</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=nd>@beam.typehints.with_input_types</span><span class=p>(</span><span class=n>T</span><span class=p>)</span>
</span></span><span class=line><span class=cl><span class=nd>@beam.typehints.with_output_types</span><span class=p>(</span><span class=n>Tuple</span><span class=p>[</span><span class=nb>int</span><span class=p>,</span> <span class=n>T</span><span class=p>])</span>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>MyTransform</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>PTransform</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>expand</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>pcoll</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>pcoll</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=k>lambda</span> <span class=n>x</span><span class=p>:</span> <span class=p>(</span><span class=nb>len</span><span class=p>(</span><span class=n>x</span><span class=p>),</span> <span class=n>x</span><span class=p>))</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>words_with_lens</span> <span class=o>=</span> <span class=n>words</span> <span class=o>|</span> <span class=n>MyTransform</span><span class=p>()</span></span></span></code></pre></div></div></div><h2 id=kinds-of-type-hints>Kinds of Type Hints</h2><p>You can use type hints with any class, including Python primitive types, container classes, and user-defined classes. All classes, such as <code>int</code>, <code>float</code>, and user-defined classes, can be used to define type hints, called <strong>simple type hints</strong>. Container types such as lists, tuples, and iterables, can also be used to define type hints and are called <strong>parameterized type hints</strong>. Finally, there are some special types that don&rsquo;t correspond to any concrete Python classes, such as <code>Any</code>, <code>Optional</code>, and <code>Union</code>, that are also permitted as type hints.</p><p>Beam defines its own internal type hint types, which are still available for use for backward compatibility.
It also supports Python&rsquo;s typing module types, which are internally converted to Beam internal types.</p><blockquote><p>For new code, it is recommended to use <a href=https://docs.python.org/3/library/typing.html><strong>typing</strong></a> module types.</p></blockquote><h3 id=simple-type-hints>Simple Type Hints</h3><p>Type hints can be of any class, from <code>int</code> and <code>str</code>, to user-defined classes. If you have a class as a type hint, you may want to define a coder for it.</p><h3 id=parameterized-type-hints>Parameterized Type Hints</h3><p>Parameterized type hints are useful for hinting the types of container-like Python objects, such as <code>list</code>. These type hints further refine the elements in those container objects.</p><p>The parameters for parameterized type hints can be simple types, parameterized types, or type variables. Element types that are type variables, such as <code>T</code>, impose relationships between the inputs and outputs of an operation (for example, <code>List[T]</code> -> <code>T</code>). Type hints can be nested, allowing you to define type hints for complex types. For example, <code>List[Tuple[int, int, str]]</code>.</p><p>In order to avoid conflicting with the namespace of the built-in container types, the first letter is capitalized.</p><p>The following parameterized type hints are permitted:</p><ul><li><code>Tuple[T, U]</code></li><li><code>Tuple[T, ...]</code></li><li><code>List[T]</code></li><li><code>KV[T, U]</code></li><li><code>Dict[T, U]</code></li><li><code>Set[T]</code></li><li><code>FrozenSet[T]</code></li><li><code>Iterable[T]</code></li><li><code>Iterator[T]</code></li><li><code>Generator[T]</code></li><li><code>PCollection[T]</code></li></ul><p><strong>Note:</strong> The <code>Tuple[T, U]</code> type hint is a tuple with a fixed number of heterogeneously typed elements, while the <code>Tuple[T, ...]</code> type hint is a tuple with a variable of homogeneously typed elements.</p><h3 id=special-type-hints>Special Type Hints</h3><p>The following are special type hints that don&rsquo;t correspond to a class, but rather to special types introduced in <a href=https://www.python.org/dev/peps/pep-0484/>PEP 484</a>.</p><ul><li><code>Any</code></li><li><code>Union[T, U, V]</code></li><li><code>Optional[T]</code></li></ul><h2 id=runtime-type-checking>Runtime Type Checking</h2><p>In addition to using type hints for type checking at pipeline construction, you can enable runtime type checking to check that actual elements satisfy the declared type constraints during pipeline execution.</p><p>For example, the following pipeline emits elements of the wrong type. Depending on the runner implementation, its execution may or may not fail at runtime.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=n>p</span> <span class=o>=</span> <span class=n>TestPipeline</span><span class=p>()</span>
</span></span><span class=line><span class=cl><span class=n>p</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Create</span><span class=p>([</span><span class=s1>&#39;a&#39;</span><span class=p>])</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=k>lambda</span> <span class=n>x</span><span class=p>:</span> <span class=mi>3</span><span class=p>)</span><span class=o>.</span><span class=n>with_output_types</span><span class=p>(</span><span class=nb>str</span><span class=p>)</span></span></span></code></pre></div></div></div><p>However, if you enable runtime type checking, the code is guaranteed to fail at runtime. To enable runtime type checking, set the pipeline option <code>runtime_type_check</code> to <code>True</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=n>p</span> <span class=o>=</span> <span class=n>TestPipeline</span><span class=p>(</span><span class=n>options</span><span class=o>=</span><span class=n>PipelineOptions</span><span class=p>(</span><span class=n>runtime_type_check</span><span class=o>=</span><span class=kc>True</span><span class=p>))</span>
</span></span><span class=line><span class=cl><span class=n>p</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Create</span><span class=p>([</span><span class=s1>&#39;a&#39;</span><span class=p>])</span> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=k>lambda</span> <span class=n>x</span><span class=p>:</span> <span class=mi>3</span><span class=p>)</span><span class=o>.</span><span class=n>with_output_types</span><span class=p>(</span><span class=nb>str</span><span class=p>)</span>
</span></span><span class=line><span class=cl><span class=n>p</span><span class=o>.</span><span class=n>run</span><span class=p>()</span></span></span></code></pre></div></div></div><p>Note that because runtime type checks are done for each <code>PCollection</code> element, enabling this feature may incur a significant performance penalty. It is therefore recommended that runtime type checks are disabled for production pipelines. See the following section for a quicker, production-friendly alternative.</p><h3 id=faster-runtime-type-checking>Faster Runtime Type Checking</h3><p>You can enable faster, sampling-based runtime type checking by setting the pipeline option <code>performance_runtime_type_check</code> to <code>True</code>.</p><p>The is a Python 3 only feature that works by runtime type checking a small subset of values, called a sample, using optimized Cython code.</p><p>Currently, this feature does not support runtime type checking for side inputs or combining operations.
These are planned to be supported in a future release of Beam.</p><h2 id=use-of-type-hints-in-coders>Use of Type Hints in Coders</h2><p>When your pipeline reads, writes, or otherwise materializes its data, the elements in your <code>PCollection</code> need to be encoded and decoded to and from byte strings. Byte strings are used for intermediate storage, for comparing keys in <code>GroupByKey</code> operations, and for reading from sources and writing to sinks.</p><p>The Beam SDK for Python uses Python&rsquo;s native support for serializing objects of unknown type, a process called <strong>pickling</strong>. However, using the <code>PickleCoder</code> comes with several drawbacks: it is less efficient in time and space, and the encoding used is not deterministic, which hinders distributed partitioning, grouping, and state lookup.</p><p>To avoid these drawbacks, you can define <code>Coder</code> classes for encoding and decoding types in a more efficient way. You can specify a <code>Coder</code> to describe how the elements of a given <code>PCollection</code> should be encoded and decoded.</p><p>In order to be correct and efficient, a <code>Coder</code> needs type information and for <code>PCollections</code> to be associated with a specific type. Type hints are what make this type information available. The Beam SDK for Python provides built-in coders for the standard Python types such as <code>int</code>, <code>float</code>, <code>str</code>, <code>bytes</code>, and <code>unicode</code>.</p><h3 id=deterministic-coders>Deterministic Coders</h3><p>If you don&rsquo;t define a <code>Coder</code>, the default is a coder that falls back to pickling for unknown types. In some cases, you must specify a deterministic <code>Coder</code> or else you will get a runtime error.</p><p>For example, suppose you have a <code>PCollection</code> of key-value pairs whose keys are <code>Player</code> objects. If you apply a <code>GroupByKey</code> transform to such a collection, its key objects might be serialized differently on different machines when a nondeterministic coder, such as the default pickle coder, is used. Since <code>GroupByKey</code> uses this serialized representation to compare keys, this may result in incorrect behavior. To ensure that the elements are always encoded and decoded in the same way, you need to define a deterministic <code>Coder</code> for the <code>Player</code> class.</p><p>The following code shows the example <code>Player</code> class and how to define a <code>Coder</code> for it. When you use type hints, Beam infers which <code>Coders</code> to use, using <code>beam.coders.registry</code>. The following code registers <code>PlayerCoder</code> as a coder for the <code>Player</code> class. In the example, the input type declared for <code>CombinePerKey</code> is <code>Tuple[Player, int]</code>. In this case, Beam infers that the <code>Coder</code> objects to use are <code>TupleCoder</code>, <code>PlayerCoder</code>, and <code>IntCoder</code>.</p><div class='language-py snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-py data-lang=py><span class=line><span class=cl><span class=kn>from</span> <span class=nn>typing</span> <span class=kn>import</span> <span class=n>Tuple</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>Player</span><span class=p>(</span><span class=nb>object</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=fm>__init__</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>team</span><span class=p>,</span> <span class=n>name</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=bp>self</span><span class=o>.</span><span class=n>team</span> <span class=o>=</span> <span class=n>team</span>
</span></span><span class=line><span class=cl> <span class=bp>self</span><span class=o>.</span><span class=n>name</span> <span class=o>=</span> <span class=n>name</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>class</span> <span class=nc>PlayerCoder</span><span class=p>(</span><span class=n>beam</span><span class=o>.</span><span class=n>coders</span><span class=o>.</span><span class=n>Coder</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>encode</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>player</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=p>(</span><span class=s1>&#39;</span><span class=si>%s</span><span class=s1>:</span><span class=si>%s</span><span class=s1>&#39;</span> <span class=o>%</span> <span class=p>(</span><span class=n>player</span><span class=o>.</span><span class=n>team</span><span class=p>,</span> <span class=n>player</span><span class=o>.</span><span class=n>name</span><span class=p>))</span><span class=o>.</span><span class=n>encode</span><span class=p>(</span><span class=s1>&#39;utf-8&#39;</span><span class=p>)</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>decode</span><span class=p>(</span><span class=bp>self</span><span class=p>,</span> <span class=n>s</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>Player</span><span class=p>(</span><span class=o>*</span><span class=n>s</span><span class=o>.</span><span class=n>decode</span><span class=p>(</span><span class=s1>&#39;utf-8&#39;</span><span class=p>)</span><span class=o>.</span><span class=n>split</span><span class=p>(</span><span class=s1>&#39;:&#39;</span><span class=p>))</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=k>def</span> <span class=nf>is_deterministic</span><span class=p>(</span><span class=bp>self</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=kc>True</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>beam</span><span class=o>.</span><span class=n>coders</span><span class=o>.</span><span class=n>registry</span><span class=o>.</span><span class=n>register_coder</span><span class=p>(</span><span class=n>Player</span><span class=p>,</span> <span class=n>PlayerCoder</span><span class=p>)</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=k>def</span> <span class=nf>parse_player_and_score</span><span class=p>(</span><span class=n>csv</span><span class=p>):</span>
</span></span><span class=line><span class=cl> <span class=n>name</span><span class=p>,</span> <span class=n>team</span><span class=p>,</span> <span class=n>score</span> <span class=o>=</span> <span class=n>csv</span><span class=o>.</span><span class=n>split</span><span class=p>(</span><span class=s1>&#39;,&#39;</span><span class=p>)</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>Player</span><span class=p>(</span><span class=n>team</span><span class=p>,</span> <span class=n>name</span><span class=p>),</span> <span class=nb>int</span><span class=p>(</span><span class=n>score</span><span class=p>)</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=n>totals</span> <span class=o>=</span> <span class=p>(</span>
</span></span><span class=line><span class=cl> <span class=n>lines</span>
</span></span><span class=line><span class=cl> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>Map</span><span class=p>(</span><span class=n>parse_player_and_score</span><span class=p>)</span>
</span></span><span class=line><span class=cl> <span class=o>|</span> <span class=n>beam</span><span class=o>.</span><span class=n>CombinePerKey</span><span class=p>(</span><span class=nb>sum</span><span class=p>)</span><span class=o>.</span><span class=n>with_input_types</span><span class=p>(</span><span class=n>Tuple</span><span class=p>[</span><span class=n>Player</span><span class=p>,</span> <span class=nb>int</span><span class=p>]))</span></span></span></code></pre></div></div></div></div></div><footer class=footer><div class=footer__contained><div class=footer__cols><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col__logo><img src=/images/beam_logo_circle.svg class=footer__logo alt="Beam logo"></div><div class=footer__cols__col__logo><img src=/images/apache_logo_circle.svg class=footer__logo alt="Apache logo"></div></div><div class=footer-wrapper><div class=wrapper-grid><div class=footer__cols__col><div class=footer__cols__col__title>Start</div><div class=footer__cols__col__link><a href=/get-started/beam-overview/>Overview</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-java/>Quickstart (Java)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-py/>Quickstart (Python)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-go/>Quickstart (Go)</a></div><div class=footer__cols__col__link><a href=/get-started/downloads/>Downloads</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Docs</div><div class=footer__cols__col__link><a href=/documentation/programming-guide/>Concepts</a></div><div class=footer__cols__col__link><a href=/documentation/pipelines/design-your-pipeline/>Pipelines</a></div><div class=footer__cols__col__link><a href=/documentation/runners/capability-matrix/>Runners</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Community</div><div class=footer__cols__col__link><a href=/contribute/>Contribute</a></div><div class=footer__cols__col__link><a href=https://projects.apache.org/committee.html?beam target=_blank>Team<img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></div><div class=footer__cols__col__link><a href=/community/presentation-materials/>Media</a></div><div class=footer__cols__col__link><a href=/community/in-person/>Events/Meetups</a></div><div class=footer__cols__col__link><a href=/community/contact-us/>Contact Us</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Resources</div><div class=footer__cols__col__link><a href=/blog/>Blog</a></div><div class=footer__cols__col__link><a href=https://github.com/apache/beam>GitHub</a></div></div></div><div class=footer__bottom>&copy;
<a href=https://www.apache.org>The Apache Software Foundation</a>
| <a href=/privacy_policy>Privacy Policy</a>
| <a href=/feed.xml>RSS Feed</a><br><br>Apache Beam, Apache, Beam, the Beam logo, and the Apache feather logo are either registered trademarks or trademarks of The Apache Software Foundation. All other products or name brands are trademarks of their respective holders, including The Apache Software Foundation.</div></div><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://github.com/apache/beam><img src=/images/logos/social-icons/github-logo-150.png class=footer__logo alt="Github logo"></a></div><div class=footer__cols__col__logo><a href=https://www.linkedin.com/company/apache-beam/><img src=/images/logos/social-icons/linkedin-logo-150.png class=footer__logo alt="Linkedin logo"></a></div></div><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://twitter.com/apachebeam><img src=/images/logos/social-icons/twitter-logo-150.png class=footer__logo alt="Twitter logo"></a></div><div class=footer__cols__col__logo><a href=https://www.youtube.com/channel/UChNnb_YO_7B0HlW6FhAXZZQ><img src=/images/logos/social-icons/youtube-logo-150.png class=footer__logo alt="Youtube logo"></a></div></div></div></div></div></footer></body></html>