blob: 5e58fc069aa4a7eaad8a23c9f827223ab1685675 [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.io.gcp.pubsub &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../../genindex.html"/>
<link rel="search" title="Search" href="../../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../../index.html"/>
<link rel="up" title="Module code" href="../../../index.html"/>
<script src="../../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<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 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.internal.html">apache_beam.internal 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.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.tools.html">apache_beam.tools 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>
</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>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" 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.io.gcp.pubsub</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.io.gcp.pubsub</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="sd">&quot;&quot;&quot;Google Cloud PubSub sources and sinks.</span>
<span class="sd">Cloud Pub/Sub sources and sinks are currently supported only in streaming</span>
<span class="sd">pipelines, during remote execution.</span>
<span class="sd">This API is currently under development and is subject to change.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">import</span> <span class="nn">re</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">iteritems</span>
<span class="kn">from</span> <span class="nn">past.builtins</span> <span class="k">import</span> <span class="n">unicode</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">coders</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.iobase</span> <span class="k">import</span> <span class="n">Read</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.iobase</span> <span class="k">import</span> <span class="n">Write</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.native_io</span> <span class="k">import</span> <span class="n">iobase</span> <span class="k">as</span> <span class="n">dataflow_io</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">Map</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">PTransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.display</span> <span class="k">import</span> <span class="n">DisplayDataItem</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.annotations</span> <span class="k">import</span> <span class="n">deprecated</span>
<span class="k">try</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">google.cloud</span> <span class="k">import</span> <span class="n">pubsub</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="n">pubsub</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;PubsubMessage&#39;</span><span class="p">,</span> <span class="s1">&#39;ReadFromPubSub&#39;</span><span class="p">,</span> <span class="s1">&#39;ReadStringsFromPubSub&#39;</span><span class="p">,</span>
<span class="s1">&#39;WriteStringsToPubSub&#39;</span><span class="p">,</span> <span class="s1">&#39;WriteToPubSub&#39;</span><span class="p">]</span>
<div class="viewcode-block" id="PubsubMessage"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.PubsubMessage">[docs]</a><span class="k">class</span> <span class="nc">PubsubMessage</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Represents a Cloud Pub/Sub message.</span>
<span class="sd"> Message payload includes the data and attributes fields. For the payload to be</span>
<span class="sd"> valid, at least one of its fields must be non-empty.</span>
<span class="sd"> This interface is experimental. No backwards compatibility guarantees.</span>
<span class="sd"> Attributes:</span>
<span class="sd"> data: (bytes) Message data. May be None.</span>
<span class="sd"> attributes: (dict) Key-value map of str to str, containing both user-defined</span>
<span class="sd"> and service generated attributes (such as id_label and</span>
<span class="sd"> timestamp_attribute). May be None.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">data</span><span class="p">,</span> <span class="n">attributes</span><span class="p">):</span>
<span class="k">if</span> <span class="n">data</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">attributes</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Either data (</span><span class="si">%r</span><span class="s1">) or attributes (</span><span class="si">%r</span><span class="s1">) must be set.&#39;</span><span class="p">,</span>
<span class="n">data</span><span class="p">,</span> <span class="n">attributes</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="n">data</span>
<span class="bp">self</span><span class="o">.</span><span class="n">attributes</span> <span class="o">=</span> <span class="n">attributes</span>
<span class="k">def</span> <span class="nf">__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">data</span><span class="p">,</span> <span class="nb">frozenset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">attributes</span><span class="o">.</span><span class="n">items</span><span class="p">())))</span>
<span class="k">def</span> <span class="nf">__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="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">PubsubMessage</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">data</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">attributes</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">attributes</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">__ne__</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="c1"># TODO(BEAM-5949): Needed for Python 2 compatibility.</span>
<span class="k">return</span> <span class="ow">not</span> <span class="bp">self</span> <span class="o">==</span> <span class="n">other</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;PubsubMessage(</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="bp">self</span><span class="o">.</span><span class="n">data</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">attributes</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_proto_str</span><span class="p">(</span><span class="n">proto_msg</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Construct from serialized form of ``PubsubMessage``.</span>
<span class="sd"> Args:</span>
<span class="sd"> proto_msg: String containing a serialized protobuf of type</span>
<span class="sd"> https://cloud.google.com/pubsub/docs/reference/rpc/google.pubsub.v1#google.pubsub.v1.PubsubMessage</span>
<span class="sd"> Returns:</span>
<span class="sd"> A new PubsubMessage object.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">msg</span> <span class="o">=</span> <span class="n">pubsub</span><span class="o">.</span><span class="n">types</span><span class="o">.</span><span class="n">pubsub_pb2</span><span class="o">.</span><span class="n">PubsubMessage</span><span class="p">()</span>
<span class="n">msg</span><span class="o">.</span><span class="n">ParseFromString</span><span class="p">(</span><span class="n">proto_msg</span><span class="p">)</span>
<span class="c1"># Convert ScalarMapContainer to dict.</span>
<span class="n">attributes</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">msg</span><span class="o">.</span><span class="n">attributes</span><span class="p">[</span><span class="n">key</span><span class="p">])</span> <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">msg</span><span class="o">.</span><span class="n">attributes</span><span class="p">)</span>
<span class="k">return</span> <span class="n">PubsubMessage</span><span class="p">(</span><span class="n">msg</span><span class="o">.</span><span class="n">data</span><span class="p">,</span> <span class="n">attributes</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_to_proto_str</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Get serialized form of ``PubsubMessage``.</span>
<span class="sd"> Args:</span>
<span class="sd"> proto_msg: str containing a serialized protobuf.</span>
<span class="sd"> Returns:</span>
<span class="sd"> A str containing a serialized protobuf of type</span>
<span class="sd"> https://cloud.google.com/pubsub/docs/reference/rpc/google.pubsub.v1#google.pubsub.v1.PubsubMessage</span>
<span class="sd"> containing the payload of this object.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">msg</span> <span class="o">=</span> <span class="n">pubsub</span><span class="o">.</span><span class="n">types</span><span class="o">.</span><span class="n">pubsub_pb2</span><span class="o">.</span><span class="n">PubsubMessage</span><span class="p">()</span>
<span class="n">msg</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">data</span>
<span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">iteritems</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">attributes</span><span class="p">):</span>
<span class="n">msg</span><span class="o">.</span><span class="n">attributes</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">return</span> <span class="n">msg</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">()</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_message</span><span class="p">(</span><span class="n">msg</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Construct from ``google.cloud.pubsub_v1.subscriber.message.Message``.</span>
<span class="sd"> https://googleapis.github.io/google-cloud-python/latest/pubsub/subscriber/api/message.html</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Convert ScalarMapContainer to dict.</span>
<span class="n">attributes</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">msg</span><span class="o">.</span><span class="n">attributes</span><span class="p">[</span><span class="n">key</span><span class="p">])</span> <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">msg</span><span class="o">.</span><span class="n">attributes</span><span class="p">)</span>
<span class="k">return</span> <span class="n">PubsubMessage</span><span class="p">(</span><span class="n">msg</span><span class="o">.</span><span class="n">data</span><span class="p">,</span> <span class="n">attributes</span><span class="p">)</span></div>
<div class="viewcode-block" id="ReadFromPubSub"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.ReadFromPubSub">[docs]</a><span class="k">class</span> <span class="nc">ReadFromPubSub</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A ``PTransform`` for reading from Cloud Pub/Sub.&quot;&quot;&quot;</span>
<span class="c1"># Implementation note: This ``PTransform`` is overridden by Directrunner.</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">subscription</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">with_attributes</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">timestamp_attribute</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes ``ReadFromPubSub``.</span>
<span class="sd"> Args:</span>
<span class="sd"> topic: Cloud Pub/Sub topic in the form</span>
<span class="sd"> &quot;projects/&lt;project&gt;/topics/&lt;topic&gt;&quot;. If provided, subscription must be</span>
<span class="sd"> None.</span>
<span class="sd"> subscription: Existing Cloud Pub/Sub subscription to use in the</span>
<span class="sd"> form &quot;projects/&lt;project&gt;/subscriptions/&lt;subscription&gt;&quot;. If not</span>
<span class="sd"> specified, a temporary subscription will be created from the specified</span>
<span class="sd"> topic. If provided, topic must be None.</span>
<span class="sd"> id_label: The attribute on incoming Pub/Sub messages to use as a unique</span>
<span class="sd"> record identifier. When specified, the value of this attribute (which</span>
<span class="sd"> can be any string that uniquely identifies the record) will be used for</span>
<span class="sd"> deduplication of messages. If not provided, we cannot guarantee</span>
<span class="sd"> that no duplicate data will be delivered on the Pub/Sub stream. In this</span>
<span class="sd"> case, deduplication of the stream will be strictly best effort.</span>
<span class="sd"> with_attributes:</span>
<span class="sd"> True - output elements will be :class:`~PubsubMessage` objects.</span>
<span class="sd"> False - output elements will be of type ``bytes`` (message</span>
<span class="sd"> data only).</span>
<span class="sd"> timestamp_attribute: Message value to use as element timestamp. If None,</span>
<span class="sd"> uses message publishing time as the timestamp.</span>
<span class="sd"> Timestamp values should be in one of two formats:</span>
<span class="sd"> - A numerical value representing the number of milliseconds since the</span>
<span class="sd"> Unix epoch.</span>
<span class="sd"> - A string in RFC 3339 format, UTC timezone. Example:</span>
<span class="sd"> ``2015-10-29T23:41:41.123Z``. The sub-second component of the</span>
<span class="sd"> timestamp is optional, and digits beyond the first three (i.e., time</span>
<span class="sd"> units smaller than milliseconds) may be ignored.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">(</span><span class="n">ReadFromPubSub</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span> <span class="o">=</span> <span class="n">with_attributes</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_source</span> <span class="o">=</span> <span class="n">_PubSubSource</span><span class="p">(</span>
<span class="n">topic</span><span class="o">=</span><span class="n">topic</span><span class="p">,</span>
<span class="n">subscription</span><span class="o">=</span><span class="n">subscription</span><span class="p">,</span>
<span class="n">id_label</span><span class="o">=</span><span class="n">id_label</span><span class="p">,</span>
<span class="n">with_attributes</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">,</span>
<span class="n">timestamp_attribute</span><span class="o">=</span><span class="n">timestamp_attribute</span><span class="p">)</span>
<div class="viewcode-block" id="ReadFromPubSub.expand"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.ReadFromPubSub.expand">[docs]</a> <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">pvalue</span><span class="p">):</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">|</span> <span class="n">Read</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_source</span><span class="p">)</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="nb">bytes</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">:</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">PubsubMessage</span><span class="o">.</span><span class="n">_from_proto_str</span><span class="p">)</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">PubsubMessage</span>
<span class="k">return</span> <span class="n">pcoll</span></div>
<div class="viewcode-block" id="ReadFromPubSub.to_runner_api_parameter"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.ReadFromPubSub.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">context</span><span class="p">):</span>
<span class="c1"># Required as this is identified by type in PTransformOverrides.</span>
<span class="c1"># TODO(BEAM-3812): Use an actual URN here.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_runner_api_pickled</span><span class="p">(</span><span class="n">context</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ReadStringsFromPubSub"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.ReadStringsFromPubSub">[docs]</a><span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.7.0&#39;</span><span class="p">,</span> <span class="n">extra_message</span><span class="o">=</span><span class="s1">&#39;Use ReadFromPubSub instead.&#39;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">ReadStringsFromPubSub</span><span class="p">(</span><span class="n">topic</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">subscription</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_ReadStringsFromPubSub</span><span class="p">(</span><span class="n">topic</span><span class="p">,</span> <span class="n">subscription</span><span class="p">,</span> <span class="n">id_label</span><span class="p">)</span></div>
<span class="k">class</span> <span class="nc">_ReadStringsFromPubSub</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;This class is deprecated. Use ``ReadFromPubSub`` instead.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">subscription</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_ReadStringsFromPubSub</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">topic</span> <span class="o">=</span> <span class="n">topic</span>
<span class="bp">self</span><span class="o">.</span><span class="n">subscription</span> <span class="o">=</span> <span class="n">subscription</span>
<span class="bp">self</span><span class="o">.</span><span class="n">id_label</span> <span class="o">=</span> <span class="n">id_label</span>
<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">pvalue</span><span class="p">):</span>
<span class="n">p</span> <span class="o">=</span> <span class="p">(</span><span class="n">pvalue</span><span class="o">.</span><span class="n">pipeline</span>
<span class="o">|</span> <span class="n">ReadFromPubSub</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">topic</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">subscription</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">id_label</span><span class="p">,</span>
<span class="n">with_attributes</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span>
<span class="o">|</span> <span class="s1">&#39;DecodeString&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">b</span><span class="p">:</span> <span class="n">b</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="n">p</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">unicode</span>
<span class="k">return</span> <span class="n">p</span>
<div class="viewcode-block" id="WriteStringsToPubSub"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.WriteStringsToPubSub">[docs]</a><span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.7.0&#39;</span><span class="p">,</span> <span class="n">extra_message</span><span class="o">=</span><span class="s1">&#39;Use WriteToPubSub instead.&#39;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">WriteStringsToPubSub</span><span class="p">(</span><span class="n">topic</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_WriteStringsToPubSub</span><span class="p">(</span><span class="n">topic</span><span class="p">)</span></div>
<span class="k">class</span> <span class="nc">_WriteStringsToPubSub</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;This class is deprecated. Use ``WriteToPubSub`` instead.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes ``_WriteStringsToPubSub``.</span>
<span class="sd"> Attributes:</span>
<span class="sd"> topic: Cloud Pub/Sub topic in the form &quot;/topics/&lt;project&gt;/&lt;topic&gt;&quot;.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_WriteStringsToPubSub</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_sink</span> <span class="o">=</span> <span class="n">_PubSubSink</span><span class="p">(</span><span class="n">topic</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">with_attributes</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">timestamp_attribute</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span>
<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">pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;EncodeString&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</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="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="nb">bytes</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Write</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sink</span><span class="p">)</span>
<div class="viewcode-block" id="WriteToPubSub"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.WriteToPubSub">[docs]</a><span class="k">class</span> <span class="nc">WriteToPubSub</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A ``PTransform`` for writing messages to Cloud Pub/Sub.&quot;&quot;&quot;</span>
<span class="c1"># Implementation note: This ``PTransform`` is overridden by Directrunner.</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="p">,</span> <span class="n">with_attributes</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">timestamp_attribute</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes ``WriteToPubSub``.</span>
<span class="sd"> Args:</span>
<span class="sd"> topic: Cloud Pub/Sub topic in the form &quot;/topics/&lt;project&gt;/&lt;topic&gt;&quot;.</span>
<span class="sd"> with_attributes:</span>
<span class="sd"> True - input elements will be :class:`~PubsubMessage` objects.</span>
<span class="sd"> False - input elements will be of type ``bytes`` (message</span>
<span class="sd"> data only).</span>
<span class="sd"> id_label: If set, will set an attribute for each Cloud Pub/Sub message</span>
<span class="sd"> with the given name and a unique value. This attribute can then be used</span>
<span class="sd"> in a ReadFromPubSub PTransform to deduplicate messages.</span>
<span class="sd"> timestamp_attribute: If set, will set an attribute for each Cloud Pub/Sub</span>
<span class="sd"> message with the given name and the message&#39;s publish time as the value.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">(</span><span class="n">WriteToPubSub</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span> <span class="o">=</span> <span class="n">with_attributes</span>
<span class="bp">self</span><span class="o">.</span><span class="n">id_label</span> <span class="o">=</span> <span class="n">id_label</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="o">=</span> <span class="n">timestamp_attribute</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_sink</span> <span class="o">=</span> <span class="n">_PubSubSink</span><span class="p">(</span><span class="n">topic</span><span class="p">,</span> <span class="n">id_label</span><span class="p">,</span> <span class="n">with_attributes</span><span class="p">,</span>
<span class="n">timestamp_attribute</span><span class="p">)</span>
<div class="viewcode-block" id="WriteToPubSub.to_proto_str"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.WriteToPubSub.to_proto_str">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">to_proto_str</span><span class="p">(</span><span class="n">element</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">PubsubMessage</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Unexpected element. Type: </span><span class="si">%s</span><span class="s1"> (expected: PubsubMessage), &#39;</span>
<span class="s1">&#39;value: </span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">element</span><span class="p">),</span> <span class="n">element</span><span class="p">))</span>
<span class="k">return</span> <span class="n">element</span><span class="o">.</span><span class="n">_to_proto_str</span><span class="p">()</span></div>
<div class="viewcode-block" id="WriteToPubSub.expand"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.WriteToPubSub.expand">[docs]</a> <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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">:</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;ToProtobuf&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">to_proto_str</span><span class="p">)</span>
<span class="c1"># Without attributes, message data is written as-is. With attributes,</span>
<span class="c1"># message data + attributes are passed as a serialized protobuf string (see</span>
<span class="c1"># ``PubsubMessage._to_proto_str`` for exact protobuf message type).</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="nb">bytes</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Write</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sink</span><span class="p">)</span></div>
<div class="viewcode-block" id="WriteToPubSub.to_runner_api_parameter"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.pubsub.html#apache_beam.io.gcp.pubsub.WriteToPubSub.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">context</span><span class="p">):</span>
<span class="c1"># Required as this is identified by type in PTransformOverrides.</span>
<span class="c1"># TODO(BEAM-3812): Use an actual URN here.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_runner_api_pickled</span><span class="p">(</span><span class="n">context</span><span class="p">)</span></div></div>
<span class="n">PROJECT_ID_REGEXP</span> <span class="o">=</span> <span class="s1">&#39;[a-z][-a-z0-9:.]{4,61}[a-z0-9]&#39;</span>
<span class="n">SUBSCRIPTION_REGEXP</span> <span class="o">=</span> <span class="s1">&#39;projects/([^/]+)/subscriptions/(.+)&#39;</span>
<span class="n">TOPIC_REGEXP</span> <span class="o">=</span> <span class="s1">&#39;projects/([^/]+)/topics/(.+)&#39;</span>
<span class="k">def</span> <span class="nf">parse_topic</span><span class="p">(</span><span class="n">full_topic</span><span class="p">):</span>
<span class="n">match</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">TOPIC_REGEXP</span><span class="p">,</span> <span class="n">full_topic</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">match</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;PubSub topic must be in the form &quot;projects/&lt;project&gt;/topics&#39;</span>
<span class="s1">&#39;/&lt;topic&gt;&quot; (got </span><span class="si">%r</span><span class="s1">).&#39;</span> <span class="o">%</span> <span class="n">full_topic</span><span class="p">)</span>
<span class="n">project</span><span class="p">,</span> <span class="n">topic_name</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">1</span><span class="p">),</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">2</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">PROJECT_ID_REGEXP</span><span class="p">,</span> <span class="n">project</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Invalid PubSub project name: </span><span class="si">%r</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">project</span><span class="p">)</span>
<span class="k">return</span> <span class="n">project</span><span class="p">,</span> <span class="n">topic_name</span>
<span class="k">def</span> <span class="nf">parse_subscription</span><span class="p">(</span><span class="n">full_subscription</span><span class="p">):</span>
<span class="n">match</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">SUBSCRIPTION_REGEXP</span><span class="p">,</span> <span class="n">full_subscription</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">match</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;PubSub subscription must be in the form &quot;projects/&lt;project&gt;&#39;</span>
<span class="s1">&#39;/subscriptions/&lt;subscription&gt;&quot; (got </span><span class="si">%r</span><span class="s1">).&#39;</span> <span class="o">%</span> <span class="n">full_subscription</span><span class="p">)</span>
<span class="n">project</span><span class="p">,</span> <span class="n">subscription_name</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">1</span><span class="p">),</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">2</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">PROJECT_ID_REGEXP</span><span class="p">,</span> <span class="n">project</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Invalid PubSub project name: </span><span class="si">%r</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">project</span><span class="p">)</span>
<span class="k">return</span> <span class="n">project</span><span class="p">,</span> <span class="n">subscription_name</span>
<span class="k">class</span> <span class="nc">_PubSubSource</span><span class="p">(</span><span class="n">dataflow_io</span><span class="o">.</span><span class="n">NativeSource</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Source for a Cloud Pub/Sub topic or subscription.</span>
<span class="sd"> This ``NativeSource`` is overridden by a native Pubsub implementation.</span>
<span class="sd"> Attributes:</span>
<span class="sd"> with_attributes: If False, will fetch just message data. Otherwise,</span>
<span class="sd"> fetches ``PubsubMessage`` protobufs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">subscription</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">id_label</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">with_attributes</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">timestamp_attribute</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">full_topic</span> <span class="o">=</span> <span class="n">topic</span>
<span class="bp">self</span><span class="o">.</span><span class="n">full_subscription</span> <span class="o">=</span> <span class="n">subscription</span>
<span class="bp">self</span><span class="o">.</span><span class="n">topic_name</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">subscription_name</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">id_label</span> <span class="o">=</span> <span class="n">id_label</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span> <span class="o">=</span> <span class="n">with_attributes</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="o">=</span> <span class="n">timestamp_attribute</span>
<span class="c1"># Perform some validation on the topic and subscription.</span>
<span class="k">if</span> <span class="ow">not</span> <span class="p">(</span><span class="n">topic</span> <span class="ow">or</span> <span class="n">subscription</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Either a topic or subscription must be provided.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">topic</span> <span class="ow">and</span> <span class="n">subscription</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Only one of topic or subscription should be provided.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">topic</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">project</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">topic_name</span> <span class="o">=</span> <span class="n">parse_topic</span><span class="p">(</span><span class="n">topic</span><span class="p">)</span>
<span class="k">if</span> <span class="n">subscription</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">project</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">subscription_name</span> <span class="o">=</span> <span class="n">parse_subscription</span><span class="p">(</span><span class="n">subscription</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">format</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Source format name required for remote execution.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="s1">&#39;pubsub&#39;</span>
<span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span><span class="s1">&#39;id_label&#39;</span><span class="p">:</span>
<span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">id_label</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;ID Label Attribute&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="s1">&#39;topic&#39;</span><span class="p">:</span>
<span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">full_topic</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;Pubsub Topic&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="s1">&#39;subscription&#39;</span><span class="p">:</span>
<span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">full_subscription</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;Pubsub Subscription&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="s1">&#39;with_attributes&#39;</span><span class="p">:</span>
<span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;With Attributes&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="s1">&#39;timestamp_attribute&#39;</span><span class="p">:</span>
<span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;Timestamp Attribute&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">reader</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="k">def</span> <span class="nf">is_bounded</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">class</span> <span class="nc">_PubSubSink</span><span class="p">(</span><span class="n">dataflow_io</span><span class="o">.</span><span class="n">NativeSink</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Sink for a Cloud Pub/Sub topic.</span>
<span class="sd"> This ``NativeSource`` is overridden by a native Pubsub implementation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">topic</span><span class="p">,</span> <span class="n">id_label</span><span class="p">,</span> <span class="n">with_attributes</span><span class="p">,</span> <span class="n">timestamp_attribute</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">full_topic</span> <span class="o">=</span> <span class="n">topic</span>
<span class="bp">self</span><span class="o">.</span><span class="n">id_label</span> <span class="o">=</span> <span class="n">id_label</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span> <span class="o">=</span> <span class="n">with_attributes</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="o">=</span> <span class="n">timestamp_attribute</span>
<span class="bp">self</span><span class="o">.</span><span class="n">project</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">topic_name</span> <span class="o">=</span> <span class="n">parse_topic</span><span class="p">(</span><span class="n">topic</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">format</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Sink format name required for remote execution.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="s1">&#39;pubsub&#39;</span>
<span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span>
<span class="s1">&#39;topic&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">full_topic</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Pubsub Topic&#39;</span><span class="p">),</span>
<span class="s1">&#39;id_label&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">id_label</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;ID Label Attribute&#39;</span><span class="p">),</span>
<span class="s1">&#39;with_attributes&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;With Attributes&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">(),</span>
<span class="s1">&#39;timestamp_attribute&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Timestamp Attribute&#39;</span><span class="p">),</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">writer</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
</pre></div>
</div>
<div class="articleComments">
</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/snide/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">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</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/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>