blob: cc4b61147627d64de4564624d172c303c96bd22c [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.coders.row_coder &mdash; Apache Beam 2.47.0 documentation</title>
<script type="text/javascript" src="../../../_static/js/modernizr.min.js"></script>
<script type="text/javascript" id="documentation_options" data-url_root="../../../" src="../../../_static/documentation_options.js"></script>
<script type="text/javascript" src="../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../_static/language_data.js"></script>
<script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/javascript" src="../../../_static/js/theme.js"></script>
<link rel="stylesheet" href="../../../_static/css/theme.css" type="text/css" />
<link rel="stylesheet" href="../../../_static/pygments.css" type="text/css" />
<link rel="index" title="Index" href="../../../genindex.html" />
<link rel="search" title="Search" href="../../../search.html" />
</head>
<body class="wy-body-for-nav">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search" >
<a href="../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div class="version">
2.47.0
</div>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.ml.html">apache_beam.ml package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.yaml.html">apache_beam.yaml package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.coders.row_coder</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.coders.row_coder</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">from</span> <span class="nn">google.protobuf</span> <span class="kn">import</span> <span class="n">json_format</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders</span> <span class="kn">import</span> <span class="n">typecoders</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coder_impl</span> <span class="kn">import</span> <span class="n">LogicalTypeCoderImpl</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coder_impl</span> <span class="kn">import</span> <span class="n">RowCoderImpl</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">BigEndianShortCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">BooleanCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">BytesCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">Coder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">DecimalCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">FastCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">FloatCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">IterableCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">MapCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">NullableCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">SinglePrecisionFloatCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">StrUtf8Coder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">TimestampCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders.coders</span> <span class="kn">import</span> <span class="n">VarIntCoder</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">common_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">schema_pb2</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">row_type</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">PYTHON_ANY_URN</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">LogicalType</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">named_tuple_from_schema</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">schema_from_element_type</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">proto_utils</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;RowCoder&quot;</span><span class="p">]</span>
<div class="viewcode-block" id="RowCoder"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder">[docs]</a><span class="k">class</span> <span class="nc">RowCoder</span><span class="p">(</span><span class="n">FastCoder</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot; Coder for `typing.NamedTuple` instances.</span>
<span class="sd"> Implements the beam:coder:row:v1 standard coder spec.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<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">schema</span><span class="p">,</span> <span class="n">force_deterministic</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a :class:`RowCoder`.</span>
<span class="sd"> Args:</span>
<span class="sd"> schema (apache_beam.portability.api.schema_pb2.Schema): The protobuf</span>
<span class="sd"> representation of the schema of the data that the RowCoder will be used</span>
<span class="sd"> to encode/decode.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">=</span> <span class="n">schema</span>
<span class="c1"># Eagerly generate type hint to escalate any issues with the Schema proto</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_type_hint</span> <span class="o">=</span> <span class="n">named_tuple_from_schema</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">)</span>
<span class="c1"># Use non-null coders because null values are represented separately</span>
<span class="bp">self</span><span class="o">.</span><span class="n">components</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">_nonnull_coder_from_type</span><span class="p">(</span><span class="n">field</span><span class="o">.</span><span class="n">type</span><span class="p">)</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="o">.</span><span class="n">fields</span>
<span class="p">]</span>
<span class="k">if</span> <span class="n">force_deterministic</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">components</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="n">as_deterministic_coder</span><span class="p">(</span><span class="n">force_deterministic</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">components</span>
<span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">forced_deterministic</span> <span class="o">=</span> <span class="nb">bool</span><span class="p">(</span><span class="n">force_deterministic</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_create_impl</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">RowCoderImpl</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">components</span><span class="p">)</span>
<div class="viewcode-block" id="RowCoder.is_deterministic"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.is_deterministic">[docs]</a> <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 class="k">return</span> <span class="nb">all</span><span class="p">(</span><span class="n">c</span><span class="o">.</span><span class="n">is_deterministic</span><span class="p">()</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">components</span><span class="p">)</span></div>
<div class="viewcode-block" id="RowCoder.as_deterministic_coder"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.as_deterministic_coder">[docs]</a> <span class="k">def</span> <span class="nf">as_deterministic_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">step_label</span><span class="p">,</span> <span class="n">error_message</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_deterministic</span><span class="p">():</span>
<span class="k">return</span> <span class="bp">self</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">RowCoder</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span> <span class="n">error_message</span> <span class="ow">or</span> <span class="n">step_label</span><span class="p">)</span></div>
<div class="viewcode-block" id="RowCoder.to_type_hint"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.to_type_hint">[docs]</a> <span class="k">def</span> <span class="nf">to_type_hint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_type_hint</span></div>
<div class="viewcode-block" id="RowCoder.as_cloud_object"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.as_cloud_object">[docs]</a> <span class="k">def</span> <span class="nf">as_cloud_object</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">coders_context</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="n">value</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">as_cloud_object</span><span class="p">(</span><span class="n">coders_context</span><span class="p">)</span>
<span class="n">value</span><span class="p">[</span><span class="s1">&#39;schema&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">json_format</span><span class="o">.</span><span class="n">MessageToJson</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</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 class="k">return</span> <span class="n">value</span></div>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">())</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">type</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">==</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">schema</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">forced_deterministic</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">forced_deterministic</span><span class="p">)</span>
<div class="viewcode-block" id="RowCoder.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">ROW</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span> <span class="p">[])</span></div>
<div class="viewcode-block" id="RowCoder.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@Coder</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">ROW</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">Schema</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="n">components</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">RowCoder</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span></div>
<div class="viewcode-block" id="RowCoder.from_type_hint"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.from_type_hint">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_type_hint</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">type_hint</span><span class="p">,</span> <span class="n">registry</span><span class="p">):</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/21541): Remove once all</span>
<span class="c1"># runners are portable.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">type_hint</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="kn">import</span> <span class="nn">importlib</span>
<span class="n">main_module</span> <span class="o">=</span> <span class="n">importlib</span><span class="o">.</span><span class="n">import_module</span><span class="p">(</span><span class="s1">&#39;__main__&#39;</span><span class="p">)</span>
<span class="n">type_hint</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">main_module</span><span class="p">,</span> <span class="n">type_hint</span><span class="p">,</span> <span class="n">type_hint</span><span class="p">)</span>
<span class="n">schema</span> <span class="o">=</span> <span class="n">schema_from_element_type</span><span class="p">(</span><span class="n">type_hint</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span></div>
<div class="viewcode-block" id="RowCoder.from_payload"><a class="viewcode-back" href="../../../apache_beam.coders.row_coder.html#apache_beam.coders.row_coder.RowCoder.from_payload">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_payload</span><span class="p">(</span><span class="n">payload</span><span class="p">):</span>
<span class="c1"># type: (bytes) -&gt; RowCoder</span>
<span class="k">return</span> <span class="n">RowCoder</span><span class="p">(</span><span class="n">proto_utils</span><span class="o">.</span><span class="n">parse_Bytes</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">Schema</span><span class="p">))</span></div>
<span class="k">def</span> <span class="nf">__reduce__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># when pickling, use bytes representation of the schema. schema_pb2.Schema</span>
<span class="c1"># objects cannot be pickled.</span>
<span class="k">return</span> <span class="p">(</span><span class="n">RowCoder</span><span class="o">.</span><span class="n">from_payload</span><span class="p">,</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">(),</span> <span class="p">))</span></div>
<span class="n">typecoders</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">row_type</span><span class="o">.</span><span class="n">RowTypeConstraint</span><span class="p">,</span> <span class="n">RowCoder</span><span class="p">)</span>
<span class="n">typecoders</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">row_type</span><span class="o">.</span><span class="n">GeneratedClassRowTypeConstraint</span><span class="p">,</span> <span class="n">RowCoder</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="p">):</span>
<span class="n">coder</span> <span class="o">=</span> <span class="n">_nonnull_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="p">)</span>
<span class="k">if</span> <span class="n">field_type</span><span class="o">.</span><span class="n">nullable</span><span class="p">:</span>
<span class="k">return</span> <span class="n">NullableCoder</span><span class="p">(</span><span class="n">coder</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">coder</span>
<span class="k">def</span> <span class="nf">_nonnull_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="p">):</span>
<span class="n">type_info</span> <span class="o">=</span> <span class="n">field_type</span><span class="o">.</span><span class="n">WhichOneof</span><span class="p">(</span><span class="s2">&quot;type_info&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">type_info</span> <span class="o">==</span> <span class="s2">&quot;atomic_type&quot;</span><span class="p">:</span>
<span class="k">if</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="ow">in</span> <span class="p">(</span><span class="n">schema_pb2</span><span class="o">.</span><span class="n">INT32</span><span class="p">,</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">INT64</span><span class="p">):</span>
<span class="k">return</span> <span class="n">VarIntCoder</span><span class="p">()</span>
<span class="k">if</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">INT16</span><span class="p">:</span>
<span class="k">return</span> <span class="n">BigEndianShortCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">FLOAT</span><span class="p">:</span>
<span class="k">return</span> <span class="n">SinglePrecisionFloatCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">DOUBLE</span><span class="p">:</span>
<span class="k">return</span> <span class="n">FloatCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">STRING</span><span class="p">:</span>
<span class="k">return</span> <span class="n">StrUtf8Coder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">BOOLEAN</span><span class="p">:</span>
<span class="k">return</span> <span class="n">BooleanCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">atomic_type</span> <span class="o">==</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">BYTES</span><span class="p">:</span>
<span class="k">return</span> <span class="n">BytesCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">type_info</span> <span class="o">==</span> <span class="s2">&quot;array_type&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">IterableCoder</span><span class="p">(</span><span class="n">_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">array_type</span><span class="o">.</span><span class="n">element_type</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">type_info</span> <span class="o">==</span> <span class="s2">&quot;map_type&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">MapCoder</span><span class="p">(</span>
<span class="n">_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">map_type</span><span class="o">.</span><span class="n">key_type</span><span class="p">),</span>
<span class="n">_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">map_type</span><span class="o">.</span><span class="n">value_type</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">type_info</span> <span class="o">==</span> <span class="s2">&quot;logical_type&quot;</span><span class="p">:</span>
<span class="k">if</span> <span class="n">field_type</span><span class="o">.</span><span class="n">logical_type</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">PYTHON_ANY_URN</span><span class="p">:</span>
<span class="c1"># Special case for the Any logical type. Just use the default coder for an</span>
<span class="c1"># unknown Python object.</span>
<span class="k">return</span> <span class="n">typecoders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="nb">object</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">logical_type</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">millis_instant</span><span class="o">.</span><span class="n">urn</span><span class="p">:</span>
<span class="c1"># Special case for millis instant logical type used to handle Java sdk&#39;s</span>
<span class="c1"># millis Instant. It explicitly uses TimestampCoder which deals with fix</span>
<span class="c1"># length 8-bytes big-endian-long instead of VarInt coder.</span>
<span class="k">return</span> <span class="n">TimestampCoder</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">field_type</span><span class="o">.</span><span class="n">logical_type</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="s1">&#39;beam:logical_type:decimal:v1&#39;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DecimalCoder</span><span class="p">()</span>
<span class="n">logical_type</span> <span class="o">=</span> <span class="n">LogicalType</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">logical_type</span><span class="p">)</span>
<span class="k">return</span> <span class="n">LogicalTypeCoder</span><span class="p">(</span>
<span class="n">logical_type</span><span class="p">,</span> <span class="n">_coder_from_type</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">logical_type</span><span class="o">.</span><span class="n">representation</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">type_info</span> <span class="o">==</span> <span class="s2">&quot;row_type&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">RowCoder</span><span class="p">(</span><span class="n">field_type</span><span class="o">.</span><span class="n">row_type</span><span class="o">.</span><span class="n">schema</span><span class="p">)</span>
<span class="c1"># The Java SDK supports several more types, but the coders are not yet</span>
<span class="c1"># standard, and are not implemented in Python.</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Encountered a type that is not currently supported by RowCoder: </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span>
<span class="n">field_type</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">LogicalTypeCoder</span><span class="p">(</span><span class="n">FastCoder</span><span class="p">):</span>
<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">logical_type</span><span class="p">,</span> <span class="n">representation_coder</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">logical_type</span> <span class="o">=</span> <span class="n">logical_type</span>
<span class="bp">self</span><span class="o">.</span><span class="n">representation_coder</span> <span class="o">=</span> <span class="n">representation_coder</span>
<span class="k">def</span> <span class="nf">_create_impl</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">LogicalTypeCoderImpl</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">logical_type</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">representation_coder</span><span class="p">)</span>
<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 class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">representation_coder</span><span class="o">.</span><span class="n">is_deterministic</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">to_type_hint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">logical_type</span><span class="o">.</span><span class="n">language_type</span><span class="p">()</span>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>