blob: ed48610ac6fa56017458b7361ea74ae13cfe1c93 [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.bigquery module &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="apache_beam.io.gcp package" href="apache_beam.io.gcp.html"/>
<link rel="next" title="apache_beam.io.gcp.bigquery_file_loads module" href="apache_beam.io.gcp.bigquery_file_loads.html"/>
<link rel="prev" title="apache_beam.io.gcp.big_query_query_to_table_pipeline module" href="apache_beam.io.gcp.big_query_query_to_table_pipeline.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 class="current">
<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 current"><a class="reference internal" href="apache_beam.io.html">apache_beam.io package</a><ul class="current">
<li class="toctree-l2 current"><a class="reference internal" href="apache_beam.io.html#subpackages">Subpackages</a><ul class="current">
<li class="toctree-l3"><a class="reference internal" href="apache_beam.io.external.html">apache_beam.io.external package</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.io.flink.html">apache_beam.io.flink package</a></li>
<li class="toctree-l3 current"><a class="reference internal" href="apache_beam.io.gcp.html">apache_beam.io.gcp package</a><ul class="current">
<li class="toctree-l4"><a class="reference internal" href="apache_beam.io.gcp.html#subpackages">Subpackages</a></li>
<li class="toctree-l4 current"><a class="reference internal" href="apache_beam.io.gcp.html#submodules">Submodules</a></li>
</ul>
</li>
</ul>
</li>
<li class="toctree-l2"><a class="reference internal" href="apache_beam.io.html#submodules">Submodules</a></li>
</ul>
</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="apache_beam.io.html">apache_beam.io package</a> &raquo;</li>
<li><a href="apache_beam.io.gcp.html">apache_beam.io.gcp package</a> &raquo;</li>
<li>apache_beam.io.gcp.bigquery module</li>
<li class="wy-breadcrumbs-aside">
<a href="_sources/apache_beam.io.gcp.bigquery.rst.txt" rel="nofollow"> View page source</a>
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<div class="section" id="module-apache_beam.io.gcp.bigquery">
<span id="apache-beam-io-gcp-bigquery-module"></span><h1>apache_beam.io.gcp.bigquery module<a class="headerlink" href="#module-apache_beam.io.gcp.bigquery" title="Permalink to this headline"></a></h1>
<p>BigQuery sources and sinks.</p>
<p>This module implements reading from and writing to BigQuery tables. It relies
on several classes exposed by the BigQuery API: TableSchema, TableFieldSchema,
TableRow, and TableCell. The default mode is to return table rows read from a
BigQuery source as dictionaries. Similarly a Write transform to a BigQuerySink
accepts PCollections of dictionaries. This is done for more convenient
programming. If desired, the native TableRow objects can be used throughout to
represent rows (use an instance of TableRowJsonCoder as a coder argument when
creating the sources or sinks respectively).</p>
<p>Also, for programming convenience, instances of TableReference and TableSchema
have a string representation that can be used for the corresponding arguments:</p>
<blockquote>
<div><ul class="simple">
<li>TableReference can be a PROJECT:DATASET.TABLE or DATASET.TABLE string.</li>
<li>TableSchema can be a NAME:TYPE{,NAME:TYPE}* string
(e.g. ‘month:STRING,event_count:INTEGER’).</li>
</ul>
</div></blockquote>
<p>The syntax supported is described here:
<a class="reference external" href="https://cloud.google.com/bigquery/bq-command-line-tool-quickstart">https://cloud.google.com/bigquery/bq-command-line-tool-quickstart</a></p>
<p>BigQuery sources can be used as main inputs or side inputs. A main input
(common case) is expected to be massive and will be split into manageable chunks
and processed in parallel. Side inputs are expected to be small and will be read
completely every time a ParDo DoFn gets executed. In the example below the
lambda function implementing the DoFn for the Map transform will get on each
call <em>one</em> row of the main table and <em>all</em> rows of the side table. The runner
may use some caching techniques to share the side inputs between calls in order
to avoid excessive reading::</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="n">main_table</span> <span class="o">=</span> <span class="n">pipeline</span> <span class="o">|</span> <span class="s1">&#39;VeryBig&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Read</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">BigQuerySource</span><span class="p">()</span>
<span class="n">side_table</span> <span class="o">=</span> <span class="n">pipeline</span> <span class="o">|</span> <span class="s1">&#39;NotBig&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Read</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">BigQuerySource</span><span class="p">()</span>
<span class="n">results</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">main_table</span>
<span class="o">|</span> <span class="s1">&#39;ProcessData&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">element</span><span class="p">,</span> <span class="n">side_input</span><span class="p">:</span> <span class="o">...</span><span class="p">,</span> <span class="n">AsList</span><span class="p">(</span><span class="n">side_table</span><span class="p">)))</span>
</pre></div>
</div>
<p>There is no difference in how main and side inputs are read. What makes the
side_table a ‘side input’ is the AsList wrapper used when passing the table
as a parameter to the Map transform. AsList signals to the execution framework
that its input should be made available whole.</p>
<p>The main and side inputs are implemented differently. Reading a BigQuery table
as main input entails exporting the table to a set of GCS files (currently in
JSON format) and then processing those files. Reading the same table as a side
input entails querying the table for all its rows. The coder argument on
BigQuerySource controls the reading of the lines in the export files (i.e.,
transform a JSON object into a PCollection element). The coder is not involved
when the same table is read as a side input since there is no intermediate
format involved. We get the table rows directly from the BigQuery service with
a query.</p>
<p>Users may provide a query to read from rather than reading all of a BigQuery
table. If specified, the result obtained by executing the specified query will
be used as the data of the input transform.:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="n">query_results</span> <span class="o">=</span> <span class="n">pipeline</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Read</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">BigQuerySource</span><span class="p">(</span>
<span class="n">query</span><span class="o">=</span><span class="s1">&#39;SELECT year, mean_temp FROM samples.weather_stations&#39;</span><span class="p">))</span>
</pre></div>
</div>
<p>When creating a BigQuery input transform, users should provide either a query
or a table. Pipeline construction will fail with a validation error if neither
or both are specified.</p>
<div class="section" id="writing-data-to-bigquery">
<h2>Writing Data to BigQuery<a class="headerlink" href="#writing-data-to-bigquery" title="Permalink to this headline"></a></h2>
<p>The <cite>WriteToBigQuery</cite> transform is the recommended way of writing data to
BigQuery. It supports a large set of parameters to customize how you’d like to
write to BigQuery.</p>
<div class="section" id="table-references">
<h3>Table References<a class="headerlink" href="#table-references" title="Permalink to this headline"></a></h3>
<p>This transform allows you to provide static <cite>project</cite>, <cite>dataset</cite> and <cite>table</cite>
parameters which point to a specific BigQuery table to be created. The <cite>table</cite>
parameter can also be a dynamic parameter (i.e. a callable), which receives an
element to be written to BigQuery, and returns the table that that element
should be sent to.</p>
<p>You may also provide a tuple of PCollectionView elements to be passed as side
inputs to your callable. For example, suppose that one wishes to send
events of different types to different tables, and the table names are
computed at pipeline runtime, one may do something like the following:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="k">with</span> <span class="n">Pipeline</span><span class="p">()</span> <span class="k">as</span> <span class="n">p</span><span class="p">:</span>
<span class="n">elements</span> <span class="o">=</span> <span class="p">(</span><span class="n">p</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span>
<span class="p">{</span><span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;error&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:56&#39;</span><span class="p">,</span> <span class="s1">&#39;message&#39;</span><span class="p">:</span> <span class="s1">&#39;bad&#39;</span><span class="p">},</span>
<span class="p">{</span><span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;user_log&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:59&#39;</span><span class="p">,</span> <span class="s1">&#39;query&#39;</span><span class="p">:</span> <span class="s1">&#39;flu symptom&#39;</span><span class="p">},</span>
<span class="p">]))</span>
<span class="n">table_names</span> <span class="o">=</span> <span class="p">(</span><span class="n">p</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span>
<span class="p">(</span><span class="s1">&#39;error&#39;</span><span class="p">,</span> <span class="s1">&#39;my_project.dataset1.error_table_for_today&#39;</span><span class="p">),</span>
<span class="p">(</span><span class="s1">&#39;user_log&#39;</span><span class="p">,</span> <span class="s1">&#39;my_project.dataset1.query_table_for_today&#39;</span><span class="p">),</span>
<span class="p">])</span>
<span class="n">table_names_dict</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsDict</span><span class="p">(</span><span class="n">table_names</span><span class="p">)</span>
<span class="n">elements</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">gcp</span><span class="o">.</span><span class="n">WriteToBigQuery</span><span class="p">(</span>
<span class="n">table</span><span class="o">=</span><span class="k">lambda</span> <span class="n">row</span><span class="p">,</span> <span class="n">table_dict</span><span class="p">:</span> <span class="n">table_dict</span><span class="p">[</span><span class="n">row</span><span class="p">[</span><span class="s1">&#39;type&#39;</span><span class="p">]],</span>
<span class="n">table_side_inputs</span><span class="o">=</span><span class="p">(</span><span class="n">table_names_dict</span><span class="p">,))</span>
</pre></div>
</div>
<p>In the example above, the <cite>table_dict</cite> argument passed to the function in
<cite>table_dict</cite> is the side input coming from <cite>table_names_dict</cite>, which is passed
as part of the <cite>table_side_inputs</cite> argument.</p>
</div>
<div class="section" id="schemas">
<h3>Schemas<a class="headerlink" href="#schemas" title="Permalink to this headline"></a></h3>
<p>This transform also allows you to provide a static or dynamic <cite>schema</cite>
parameter (i.e. a callable).</p>
<p>If providing a callable, this should take in a table reference (as returned by
the <cite>table</cite> parameter), and return the corresponding schema for that table.
This allows to provide different schemas for different tables:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="k">def</span> <span class="nf">compute_table_name</span><span class="p">(</span><span class="n">row</span><span class="p">):</span>
<span class="o">...</span>
<span class="n">errors_schema</span> <span class="o">=</span> <span class="p">{</span><span class="s1">&#39;fields&#39;</span><span class="p">:</span> <span class="p">[</span>
<span class="p">{</span><span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="s1">&#39;type&#39;</span><span class="p">,</span> <span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;STRING&#39;</span><span class="p">,</span> <span class="s1">&#39;mode&#39;</span><span class="p">:</span> <span class="s1">&#39;NULLABLE&#39;</span><span class="p">},</span>
<span class="p">{</span><span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="s1">&#39;message&#39;</span><span class="p">,</span> <span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;STRING&#39;</span><span class="p">,</span> <span class="s1">&#39;mode&#39;</span><span class="p">:</span> <span class="s1">&#39;NULLABLE&#39;</span><span class="p">}]}</span>
<span class="n">queries_schema</span> <span class="o">=</span> <span class="p">{</span><span class="s1">&#39;fields&#39;</span><span class="p">:</span> <span class="p">[</span>
<span class="p">{</span><span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="s1">&#39;type&#39;</span><span class="p">,</span> <span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;STRING&#39;</span><span class="p">,</span> <span class="s1">&#39;mode&#39;</span><span class="p">:</span> <span class="s1">&#39;NULLABLE&#39;</span><span class="p">},</span>
<span class="p">{</span><span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="s1">&#39;query&#39;</span><span class="p">,</span> <span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;STRING&#39;</span><span class="p">,</span> <span class="s1">&#39;mode&#39;</span><span class="p">:</span> <span class="s1">&#39;NULLABLE&#39;</span><span class="p">}]}</span>
<span class="k">with</span> <span class="n">Pipeline</span><span class="p">()</span> <span class="k">as</span> <span class="n">p</span><span class="p">:</span>
<span class="n">elements</span> <span class="o">=</span> <span class="p">(</span><span class="n">p</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span>
<span class="p">{</span><span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;error&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:56&#39;</span><span class="p">,</span> <span class="s1">&#39;message&#39;</span><span class="p">:</span> <span class="s1">&#39;bad&#39;</span><span class="p">},</span>
<span class="p">{</span><span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;user_log&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:59&#39;</span><span class="p">,</span> <span class="s1">&#39;query&#39;</span><span class="p">:</span> <span class="s1">&#39;flu symptom&#39;</span><span class="p">},</span>
<span class="p">]))</span>
<span class="n">elements</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">gcp</span><span class="o">.</span><span class="n">WriteToBigQuery</span><span class="p">(</span>
<span class="n">table</span><span class="o">=</span><span class="n">compute_table_name</span><span class="p">,</span>
<span class="n">schema</span><span class="o">=</span><span class="k">lambda</span> <span class="n">table</span><span class="p">:</span> <span class="p">(</span><span class="n">errors_schema</span>
<span class="k">if</span> <span class="s1">&#39;errors&#39;</span> <span class="ow">in</span> <span class="n">table</span>
<span class="k">else</span> <span class="n">queries_schema</span><span class="p">))</span>
</pre></div>
</div>
<p>It may be the case that schemas are computed at pipeline runtime. In cases
like these, one can also provide a <cite>schema_side_inputs</cite> parameter, which is
a tuple of PCollectionViews to be passed to the schema callable (much like
the <cite>table_side_inputs</cite> parameter).</p>
</div>
<div class="section" id="additional-parameters-for-bigquery-tables">
<h3>Additional Parameters for BigQuery Tables<a class="headerlink" href="#additional-parameters-for-bigquery-tables" title="Permalink to this headline"></a></h3>
<p>This sink is able to create tables in BigQuery if they don’t already exist. It
also relies on creating temporary tables when performing file loads.</p>
<p>The WriteToBigQuery transform creates tables using the BigQuery API by
inserting a load job (see the API reference [1]), or by inserting a new table
(see the API reference for that [2][3]).</p>
<p>When creating a new BigQuery table, there are a number of extra parameters
that one may need to specify. For example, clustering, partitioning, data
encoding, etc. It is possible to provide these additional parameters by
passing a Python dictionary as <cite>additional_bq_parameters</cite> to the transform.
As an example, to create a table that has specific partitioning, and
clustering properties, one would do the following:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="n">additional_bq_parameters</span> <span class="o">=</span> <span class="p">{</span>
<span class="s1">&#39;timePartitioning&#39;</span><span class="p">:</span> <span class="p">{</span><span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="s1">&#39;DAY&#39;</span><span class="p">},</span>
<span class="s1">&#39;clustering&#39;</span><span class="p">:</span> <span class="p">{</span><span class="s1">&#39;fields&#39;</span><span class="p">:</span> <span class="p">[</span><span class="s1">&#39;country&#39;</span><span class="p">]}}</span>
<span class="k">with</span> <span class="n">Pipeline</span><span class="p">()</span> <span class="k">as</span> <span class="n">p</span><span class="p">:</span>
<span class="n">elements</span> <span class="o">=</span> <span class="p">(</span><span class="n">p</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span>
<span class="p">{</span><span class="s1">&#39;country&#39;</span><span class="p">:</span> <span class="s1">&#39;mexico&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:56&#39;</span><span class="p">,</span> <span class="s1">&#39;query&#39;</span><span class="p">:</span> <span class="s1">&#39;acapulco&#39;</span><span class="p">},</span>
<span class="p">{</span><span class="s1">&#39;country&#39;</span><span class="p">:</span> <span class="s1">&#39;canada&#39;</span><span class="p">,</span> <span class="s1">&#39;timestamp&#39;</span><span class="p">:</span> <span class="s1">&#39;12:34:59&#39;</span><span class="p">,</span> <span class="s1">&#39;query&#39;</span><span class="p">:</span> <span class="s1">&#39;influenza&#39;</span><span class="p">},</span>
<span class="p">]))</span>
<span class="n">elements</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">gcp</span><span class="o">.</span><span class="n">WriteToBigQuery</span><span class="p">(</span>
<span class="n">table</span><span class="o">=</span><span class="s1">&#39;project_name1.dataset_2.query_events_table&#39;</span><span class="p">,</span>
<span class="n">additional_bq_parameters</span><span class="o">=</span><span class="n">additional_bq_parameters</span><span class="p">)</span>
</pre></div>
</div>
<p>Much like the schema case, the parameter with <cite>additional_bq_parameters</cite> can
also take a callable that receives a table reference.</p>
<p>[1] <a class="reference external" href="https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load">https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load</a>
[2] <a class="reference external" href="https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/insert">https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/insert</a>
[3] <a class="reference external" href="https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#resource">https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#resource</a></p>
<p><strong>* Short introduction to BigQuery concepts *</strong>
Tables have rows (TableRow) and each row has cells (TableCell).
A table has a schema (TableSchema), which in turn describes the schema of each
cell (TableFieldSchema). The terms field and cell are used interchangeably.</p>
<dl class="docutils">
<dt>TableSchema: Describes the schema (types and order) for values in each row.</dt>
<dd>Has one attribute, ‘field’, which is list of TableFieldSchema objects.</dd>
<dt>TableFieldSchema: Describes the schema (type, name) for one field.</dt>
<dd>Has several attributes, including ‘name’ and ‘type’. Common values for
the type attribute are: ‘STRING’, ‘INTEGER’, ‘FLOAT’, ‘BOOLEAN’, ‘NUMERIC’,
‘GEOGRAPHY’.
All possible values are described at:
<a class="reference external" href="https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types">https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types</a></dd>
<dt>TableRow: Holds all values in a table row. Has one attribute, ‘f’, which is a</dt>
<dd>list of TableCell instances.</dd>
<dt>TableCell: Holds the value for one cell (or field). Has one attribute,</dt>
<dd>‘v’, which is a JsonValue instance. This class is defined in
apitools.base.py.extra_types.py module.</dd>
</dl>
<p>As of Beam 2.7.0, the NUMERIC data type is supported. This data type supports
high-precision decimal numbers (precision of 38 digits, scale of 9 digits).
The GEOGRAPHY data type works with Well-Known Text (See
<a class="reference external" href="https://en.wikipedia.org/wiki/Well-known_text">https://en.wikipedia.org/wiki/Well-known_text</a>) format for reading and writing
to BigQuery.
BigQuery IO requires values of BYTES datatype to be encoded using base64
encoding when writing to BigQuery. When bytes are read from BigQuery they are
returned as base64-encoded bytes.</p>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.TableRowJsonCoder">
<em class="property">class </em><code class="descclassname">apache_beam.io.gcp.bigquery.</code><code class="descname">TableRowJsonCoder</code><span class="sig-paren">(</span><em>table_schema=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#TableRowJsonCoder"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.TableRowJsonCoder" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.coders.coders.html#apache_beam.coders.coders.Coder" title="apache_beam.coders.coders.Coder"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.coders.coders.Coder</span></code></a></p>
<p>A coder for a TableRow instance to/from a JSON string.</p>
<p>Note that the encoding operation (used when writing to sinks) requires the
table schema in order to obtain the ordered list of field names. Reading from
sources on the other hand does not need the table schema.</p>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.TableRowJsonCoder.encode">
<code class="descname">encode</code><span class="sig-paren">(</span><em>table_row</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#TableRowJsonCoder.encode"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.TableRowJsonCoder.encode" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.TableRowJsonCoder.decode">
<code class="descname">decode</code><span class="sig-paren">(</span><em>encoded_table_row</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#TableRowJsonCoder.decode"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.TableRowJsonCoder.decode" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition">
<em class="property">class </em><code class="descclassname">apache_beam.io.gcp.bigquery.</code><code class="descname">BigQueryDisposition</code><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQueryDisposition"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <code class="xref py py-class docutils literal"><span class="pre">future.types.newobject.newobject</span></code></p>
<p>Class holding standard strings used for create and write dispositions.</p>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER">
<code class="descname">CREATE_NEVER</code><em class="property"> = 'CREATE_NEVER'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED">
<code class="descname">CREATE_IF_NEEDED</code><em class="property"> = 'CREATE_IF_NEEDED'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE">
<code class="descname">WRITE_TRUNCATE</code><em class="property"> = 'WRITE_TRUNCATE'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND">
<code class="descname">WRITE_APPEND</code><em class="property"> = 'WRITE_APPEND'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY">
<code class="descname">WRITE_EMPTY</code><em class="property"> = 'WRITE_EMPTY'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_create">
<em class="property">static </em><code class="descname">validate_create</code><span class="sig-paren">(</span><em>disposition</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQueryDisposition.validate_create"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_create" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_write">
<em class="property">static </em><code class="descname">validate_write</code><span class="sig-paren">(</span><em>disposition</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQueryDisposition.validate_write"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_write" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySource">
<em class="property">class </em><code class="descclassname">apache_beam.io.gcp.bigquery.</code><code class="descname">BigQuerySource</code><span class="sig-paren">(</span><em>table=None</em>, <em>dataset=None</em>, <em>project=None</em>, <em>query=None</em>, <em>validate=False</em>, <em>coder=None</em>, <em>use_standard_sql=False</em>, <em>flatten_results=True</em>, <em>kms_key=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySource"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySource" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSource" title="apache_beam.runners.dataflow.native_io.iobase.NativeSource"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.runners.dataflow.native_io.iobase.NativeSource</span></code></a></p>
<p>A source based on a BigQuery table.</p>
<p>Initialize a <a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQuerySource" title="apache_beam.io.gcp.bigquery.BigQuerySource"><code class="xref py py-class docutils literal"><span class="pre">BigQuerySource</span></code></a>.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><ul class="first simple">
<li><strong>table</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of a BigQuery table. If specified all data of the
table will be used as input of the current source. The ID must contain
only letters <code class="docutils literal"><span class="pre">a-z</span></code>, <code class="docutils literal"><span class="pre">A-Z</span></code>, numbers <code class="docutils literal"><span class="pre">0-9</span></code>, or underscores
<code class="docutils literal"><span class="pre">_</span></code>. If dataset and query arguments are <a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> then the table
argument must contain the entire table reference specified as:
<code class="docutils literal"><span class="pre">'DATASET.TABLE'</span></code> or <code class="docutils literal"><span class="pre">'PROJECT:DATASET.TABLE'</span></code>.</li>
<li><strong>dataset</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the dataset containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument or a query is specified.</li>
<li><strong>project</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the project containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument or a query is specified.</li>
<li><strong>query</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – A query to be used instead of arguments table, dataset, and
project.</li>
<li><strong>validate</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#bool" title="(in Python v2.7)"><em>bool</em></a>) – If <a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a>, various checks will be done when source
gets initialized (e.g., is table present?). This should be
<a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a> for most scenarios in order to catch errors as early as
possible (pipeline construction instead of pipeline execution). It
should be <a class="reference external" href="https://docs.python.org/2/library/constants.html#False" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">False</span></code></a> if the table is created during pipeline
execution by a previous step.</li>
<li><strong>coder</strong> (<a class="reference internal" href="apache_beam.coders.coders.html#apache_beam.coders.coders.Coder" title="apache_beam.coders.coders.Coder"><em>Coder</em></a>) – The coder for the table
rows if serialized to disk. If <a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a>, then the default coder is
<a class="reference internal" href="apache_beam.io.gcp.bigquery_tools.html#apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder" title="apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder"><code class="xref py py-class docutils literal"><span class="pre">RowAsDictJsonCoder</span></code></a>,
which will interpret every line in a file as a JSON serialized
dictionary. This argument needs a value only in special cases when
returning table rows as dictionaries is not desirable.</li>
<li><strong>use_standard_sql</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#bool" title="(in Python v2.7)"><em>bool</em></a>) – Specifies whether to use BigQuery’s standard SQL
dialect for this query. The default value is <a class="reference external" href="https://docs.python.org/2/library/constants.html#False" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">False</span></code></a>.
If set to <a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a>, the query will use BigQuery’s updated SQL
dialect with improved standards compliance.
This parameter is ignored for table inputs.</li>
<li><strong>flatten_results</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#bool" title="(in Python v2.7)"><em>bool</em></a>) – Flattens all nested and repeated fields in the
query results. The default value is <a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a>.</li>
<li><strong>kms_key</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – Experimental. Optional Cloud KMS key name for use when
creating new tables.</li>
</ul>
</td>
</tr>
<tr class="field-even field"><th class="field-name">Raises:</th><td class="field-body"><p class="first"><a class="reference external" href="https://docs.python.org/2/library/exceptions.html#exceptions.ValueError" title="(in Python v2.7)"><code class="xref py py-exc docutils literal"><span class="pre">ValueError</span></code></a> – if any of the following is true:</p>
<ol class="last arabic simple">
<li>the table reference as a string does not match the expected format</li>
<li>neither a table nor a query is specified</li>
<li>both a table and a query is specified.</li>
</ol>
</td>
</tr>
</tbody>
</table>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySource.display_data">
<code class="descname">display_data</code><span class="sig-paren">(</span><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySource.display_data"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySource.display_data" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySource.format">
<code class="descname">format</code><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySource.format" title="Permalink to this definition"></a></dt>
<dd><p>Source format name required for remote execution.</p>
</dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySource.reader">
<code class="descname">reader</code><span class="sig-paren">(</span><em>test_bigquery_client=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySource.reader"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySource.reader" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySink">
<em class="property">class </em><code class="descclassname">apache_beam.io.gcp.bigquery.</code><code class="descname">BigQuerySink</code><span class="sig-paren">(</span><em>table</em>, <em>dataset=None</em>, <em>project=None</em>, <em>schema=None</em>, <em>create_disposition='CREATE_IF_NEEDED'</em>, <em>write_disposition='WRITE_EMPTY'</em>, <em>validate=False</em>, <em>coder=None</em>, <em>kms_key=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySink"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySink" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSink" title="apache_beam.runners.dataflow.native_io.iobase.NativeSink"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.runners.dataflow.native_io.iobase.NativeSink</span></code></a></p>
<p>A sink based on a BigQuery table.</p>
<p>This BigQuery sink triggers a Dataflow native sink for BigQuery
that only supports batch pipelines.
Instead of using this sink directly, please use WriteToBigQuery
transform that works for both batch and streaming pipelines.</p>
<p>Initialize a BigQuerySink.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><ul class="first simple">
<li><strong>table</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the table. The ID must contain only letters
<code class="docutils literal"><span class="pre">a-z</span></code>, <code class="docutils literal"><span class="pre">A-Z</span></code>, numbers <code class="docutils literal"><span class="pre">0-9</span></code>, or underscores <code class="docutils literal"><span class="pre">_</span></code>. If
<strong>dataset</strong> argument is <a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> then the table argument must
contain the entire table reference specified as: <code class="docutils literal"><span class="pre">'DATASET.TABLE'</span></code> or
<code class="docutils literal"><span class="pre">'PROJECT:DATASET.TABLE'</span></code>.</li>
<li><strong>dataset</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the dataset containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument.</li>
<li><strong>project</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the project containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument.</li>
<li><strong>schema</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The schema to be used if the BigQuery table to write has
to be created. This can be either specified as a
<code class="xref py py-class docutils literal"><span class="pre">TableSchema</span></code> object or a single string of the form
<code class="docutils literal"><span class="pre">'field1:type1,field2:type2,field3:type3'</span></code> that defines a comma
separated list of fields. Here <code class="docutils literal"><span class="pre">'type'</span></code> should specify the BigQuery
type of the field. Single string based schemas do not support nested
fields, repeated fields, or specifying a BigQuery mode for fields (mode
will always be set to <code class="docutils literal"><span class="pre">'NULLABLE'</span></code>).</li>
<li><strong>create_disposition</strong> (<a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition" title="apache_beam.io.gcp.bigquery.BigQueryDisposition"><em>BigQueryDisposition</em></a>) – <p>A string describing what
happens if the table does not exist. Possible values are:</p>
<blockquote>
<div><ul>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.CREATE_IF_NEEDED</span></code></a>: create if does not
exist.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.CREATE_NEVER</span></code></a>: fail the write if does not
exist.</li>
</ul>
</div></blockquote>
</li>
<li><strong>write_disposition</strong> (<a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition" title="apache_beam.io.gcp.bigquery.BigQueryDisposition"><em>BigQueryDisposition</em></a>) – <p>A string describing what
happens if the table has already some data. Possible values are:</p>
<blockquote>
<div><ul>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_TRUNCATE</span></code></a>: delete existing rows.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_APPEND</span></code></a>: add to existing rows.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_EMPTY</span></code></a>: fail the write if table not
empty.</li>
</ul>
</div></blockquote>
</li>
<li><strong>validate</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#bool" title="(in Python v2.7)"><em>bool</em></a>) – If <a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a>, various checks will be done when sink
gets initialized (e.g., is table present given the disposition
arguments?). This should be <a class="reference external" href="https://docs.python.org/2/library/constants.html#True" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">True</span></code></a> for most scenarios in order to
catch errors as early as possible (pipeline construction instead of
pipeline execution). It should be <a class="reference external" href="https://docs.python.org/2/library/constants.html#False" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">False</span></code></a> if the table is created
during pipeline execution by a previous step.</li>
<li><strong>coder</strong> (<a class="reference internal" href="apache_beam.coders.coders.html#apache_beam.coders.coders.Coder" title="apache_beam.coders.coders.Coder"><em>Coder</em></a>) – The coder for the
table rows if serialized to disk. If <a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a>, then the default
coder is <a class="reference internal" href="apache_beam.io.gcp.bigquery_tools.html#apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder" title="apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder"><code class="xref py py-class docutils literal"><span class="pre">RowAsDictJsonCoder</span></code></a>,
which will interpret every element written to the sink as a dictionary
that will be JSON serialized as a line in a file. This argument needs a
value only in special cases when writing table rows as dictionaries is
not desirable.</li>
<li><strong>kms_key</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – Experimental. Optional Cloud KMS key name for use when
creating new tables.</li>
</ul>
</td>
</tr>
<tr class="field-even field"><th class="field-name">Raises:</th><td class="field-body"><ul class="first last simple">
<li><a class="reference external" href="https://docs.python.org/2/library/exceptions.html#exceptions.TypeError" title="(in Python v2.7)"><code class="xref py py-exc docutils literal"><span class="pre">TypeError</span></code></a> – if the schema argument is not a <a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><code class="xref py py-class docutils literal"><span class="pre">str</span></code></a> or a
<code class="xref py py-class docutils literal"><span class="pre">TableSchema</span></code> object.</li>
<li><a class="reference external" href="https://docs.python.org/2/library/exceptions.html#exceptions.ValueError" title="(in Python v2.7)"><code class="xref py py-exc docutils literal"><span class="pre">ValueError</span></code></a> – if the table reference as a string does not
match the expected format.</li>
</ul>
</td>
</tr>
</tbody>
</table>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySink.display_data">
<code class="descname">display_data</code><span class="sig-paren">(</span><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySink.display_data"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySink.display_data" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySink.schema_as_json">
<code class="descname">schema_as_json</code><span class="sig-paren">(</span><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySink.schema_as_json"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySink.schema_as_json" title="Permalink to this definition"></a></dt>
<dd><p>Returns the TableSchema associated with the sink as a JSON string.</p>
</dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySink.format">
<code class="descname">format</code><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySink.format" title="Permalink to this definition"></a></dt>
<dd><p>Sink format name required for remote execution.</p>
</dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.BigQuerySink.writer">
<code class="descname">writer</code><span class="sig-paren">(</span><em>test_bigquery_client=None</em>, <em>buffer_size=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#BigQuerySink.writer"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.BigQuerySink.writer" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery">
<em class="property">class </em><code class="descclassname">apache_beam.io.gcp.bigquery.</code><code class="descname">WriteToBigQuery</code><span class="sig-paren">(</span><em>table</em>, <em>dataset=None</em>, <em>project=None</em>, <em>schema=None</em>, <em>create_disposition='CREATE_IF_NEEDED'</em>, <em>write_disposition='WRITE_APPEND'</em>, <em>kms_key=None</em>, <em>batch_size=None</em>, <em>max_file_size=None</em>, <em>max_files_per_bundle=None</em>, <em>test_client=None</em>, <em>custom_gcs_temp_location=None</em>, <em>method=None</em>, <em>insert_retry_strategy=None</em>, <em>additional_bq_parameters=None</em>, <em>table_side_inputs=None</em>, <em>schema_side_inputs=None</em>, <em>triggering_frequency=None</em>, <em>validate=True</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>Write data to BigQuery.</p>
<p>This transform receives a PCollection of elements to be inserted into BigQuery
tables. The elements would come in as Python dictionaries, or as <cite>TableRow</cite>
instances.</p>
<dl class="docutils">
<dt>Known Limitations/Edge Cases:</dt>
<dd><ol class="first last arabic simple">
<li>[BEAM-7822] In file loads method of writing to BigQuery,
copying from temp_tables to destination_table is not atomic.
See: <a class="reference external" href="https://issues.apache.org/jira/browse/BEAM-7822">https://issues.apache.org/jira/browse/BEAM-7822</a></li>
</ol>
</dd>
</dl>
<p>Initialize a WriteToBigQuery transform.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><ul class="first last simple">
<li><strong>table</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a><em>, </em><a class="reference external" href="https://docs.python.org/2/library/functions.html#callable" title="(in Python v2.7)"><em>callable</em></a><em>, </em><a class="reference internal" href="apache_beam.options.value_provider.html#apache_beam.options.value_provider.ValueProvider" title="apache_beam.options.value_provider.ValueProvider"><em>ValueProvider</em></a>) – The ID of the table, or a callable
that returns it. The ID must contain only letters <code class="docutils literal"><span class="pre">a-z</span></code>, <code class="docutils literal"><span class="pre">A-Z</span></code>,
numbers <code class="docutils literal"><span class="pre">0-9</span></code>, or underscores <code class="docutils literal"><span class="pre">_</span></code>. If dataset argument is
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> then the table argument must contain the entire table
reference specified as: <code class="docutils literal"><span class="pre">'DATASET.TABLE'</span></code>
or <code class="docutils literal"><span class="pre">'PROJECT:DATASET.TABLE'</span></code>. If it’s a callable, it must receive one
argument representing an element to be written to BigQuery, and return
a TableReference, or a string table name as specified above.
Multiple destinations are only supported on Batch pipelines at the
moment.</li>
<li><strong>dataset</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the dataset containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument.</li>
<li><strong>project</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The ID of the project containing this table or
<a class="reference external" href="https://docs.python.org/2/library/constants.html#None" title="(in Python v2.7)"><code class="xref py py-data docutils literal"><span class="pre">None</span></code></a> if the table reference is specified entirely by the table
argument.</li>
<li><strong>schema</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a><em>,</em><a class="reference external" href="https://docs.python.org/2/library/stdtypes.html#dict" title="(in Python v2.7)"><em>dict</em></a><em>,</em><a class="reference internal" href="apache_beam.options.value_provider.html#apache_beam.options.value_provider.ValueProvider" title="apache_beam.options.value_provider.ValueProvider"><em>ValueProvider</em></a><em>,</em><a class="reference external" href="https://docs.python.org/2/library/functions.html#callable" title="(in Python v2.7)"><em>callable</em></a>) – The schema to be used if the
BigQuery table to write has to be created. This can be either specified
as a <code class="xref py py-class docutils literal"><span class="pre">TableSchema</span></code>. or a <cite>ValueProvider</cite> that has a JSON string,
or a python dictionary, or the string or dictionary itself,
object or a single string of the form
<code class="docutils literal"><span class="pre">'field1:type1,field2:type2,field3:type3'</span></code> that defines a comma
separated list of fields. Here <code class="docutils literal"><span class="pre">'type'</span></code> should specify the BigQuery
type of the field. Single string based schemas do not support nested
fields, repeated fields, or specifying a BigQuery mode for fields
(mode will always be set to <code class="docutils literal"><span class="pre">'NULLABLE'</span></code>).
If a callable, then it should receive a destination (in the form of
a TableReference or a string, and return a str, dict or TableSchema.
One may also pass <code class="docutils literal"><span class="pre">SCHEMA_AUTODETECT</span></code> here, and BigQuery will try to
infer the schema for the files that are being loaded.</li>
<li><strong>create_disposition</strong> (<a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition" title="apache_beam.io.gcp.bigquery.BigQueryDisposition"><em>BigQueryDisposition</em></a>) – <p>A string describing what
happens if the table does not exist. Possible values are:</p>
<ul>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_IF_NEEDED"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.CREATE_IF_NEEDED</span></code></a>: create if does not
exist.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.CREATE_NEVER"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.CREATE_NEVER</span></code></a>: fail the write if does not
exist.</li>
</ul>
</li>
<li><strong>write_disposition</strong> (<a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition" title="apache_beam.io.gcp.bigquery.BigQueryDisposition"><em>BigQueryDisposition</em></a>) – <p>A string describing what happens
if the table has already some data. Possible values are:</p>
<ul>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_TRUNCATE"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_TRUNCATE</span></code></a>: delete existing rows.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_APPEND"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_APPEND</span></code></a>: add to existing rows.</li>
<li><a class="reference internal" href="#apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY" title="apache_beam.io.gcp.bigquery.BigQueryDisposition.WRITE_EMPTY"><code class="xref py py-attr docutils literal"><span class="pre">BigQueryDisposition.WRITE_EMPTY</span></code></a>: fail the write if table not
empty.</li>
</ul>
<p>For streaming pipelines WriteTruncate can not be used.</p>
</li>
<li><strong>kms_key</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – Experimental. Optional Cloud KMS key name for use when
creating new tables.</li>
<li><strong>batch_size</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#int" title="(in Python v2.7)"><em>int</em></a>) – Number of rows to be written to BQ per streaming API
insert. The default is 500.
insert.</li>
<li><strong>test_client</strong> – Override the default bigquery client used for testing.</li>
<li><strong>max_file_size</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#int" title="(in Python v2.7)"><em>int</em></a>) – The maximum size for a file to be written and then
loaded into BigQuery. The default value is 4TB, which is 80% of the
limit of 5TB for BigQuery to load any file.</li>
<li><strong>max_files_per_bundle</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#int" title="(in Python v2.7)"><em>int</em></a>) – The maximum number of files to be concurrently
written by a worker. The default here is 20. Larger values will allow
writing to multiple destinations without having to reshard - but they
increase the memory burden on the workers.</li>
<li><strong>custom_gcs_temp_location</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – A GCS location to store files to be used
for file loads into BigQuery. By default, this will use the pipeline’s
temp_location, but for pipelines whose temp_location is not appropriate
for BQ File Loads, users should pass a specific one.</li>
<li><strong>method</strong> – The method to use to write to BigQuery. It may be
STREAMING_INSERTS, FILE_LOADS, or DEFAULT. An introduction on loading
data to BigQuery: <a class="reference external" href="https://cloud.google.com/bigquery/docs/loading-data">https://cloud.google.com/bigquery/docs/loading-data</a>.
DEFAULT will use STREAMING_INSERTS on Streaming pipelines and
FILE_LOADS on Batch pipelines.</li>
<li><strong>insert_retry_strategy</strong> – The strategy to use when retrying streaming inserts
into BigQuery. Options are shown in bigquery_tools.RetryStrategy attrs.</li>
<li><strong>additional_bq_parameters</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#callable" title="(in Python v2.7)"><em>callable</em></a>) – A function that returns a dictionary
with additional parameters to pass to BQ when creating / loading data
into a table. These can be ‘timePartitioning’, ‘clustering’, etc. They
are passed directly to the job load configuration. See
<a class="reference external" href="https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load">https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load</a></li>
<li><strong>table_side_inputs</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#tuple" title="(in Python v2.7)"><em>tuple</em></a>) – A tuple with <code class="docutils literal"><span class="pre">AsSideInput</span></code> PCollections to be
passed to the table callable (if one is provided).</li>
<li><strong>schema_side_inputs</strong> – A tuple with <code class="docutils literal"><span class="pre">AsSideInput</span></code> PCollections to be
passed to the schema callable (if one is provided).</li>
<li><strong>triggering_frequency</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#int" title="(in Python v2.7)"><em>int</em></a>) – Every triggering_frequency duration, a
BigQuery load job will be triggered for all the data written since
the last load job. BigQuery has limits on how many load jobs can be
triggered per day, so be careful not to set this duration too low, or
you may exceed daily quota. Often this is set to 5 or 10 minutes to
ensure that the project stays well under the BigQuery quota.
See <a class="reference external" href="https://cloud.google.com/bigquery/quota-policy">https://cloud.google.com/bigquery/quota-policy</a> for more information
about BigQuery quotas.</li>
<li><strong>validate</strong> – Indicates whether to perform validation checks on
inputs. This parameter is primarily used for testing.</li>
</ul>
</td>
</tr>
</tbody>
</table>
<dl class="class">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.Method">
<em class="property">class </em><code class="descname">Method</code><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.Method"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.Method" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <code class="xref py py-class docutils literal"><span class="pre">future.types.newobject.newobject</span></code></p>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.DEFAULT">
<code class="descname">DEFAULT</code><em class="property"> = 'DEFAULT'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.DEFAULT" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.STREAMING_INSERTS">
<code class="descname">STREAMING_INSERTS</code><em class="property"> = 'STREAMING_INSERTS'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.STREAMING_INSERTS" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="attribute">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.FILE_LOADS">
<code class="descname">FILE_LOADS</code><em class="property"> = 'FILE_LOADS'</em><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.Method.FILE_LOADS" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.get_table_schema_from_string">
<em class="property">static </em><code class="descname">get_table_schema_from_string</code><span class="sig-paren">(</span><em>schema</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.get_table_schema_from_string"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.get_table_schema_from_string" title="Permalink to this definition"></a></dt>
<dd><p>Transform the string table schema into a
<code class="xref py py-class docutils literal"><span class="pre">TableSchema</span></code> instance.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><strong>schema</strong> (<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)"><em>str</em></a>) – The sting schema to be used if the BigQuery table to write
has to be created.</td>
</tr>
<tr class="field-even field"><th class="field-name">Returns:</th><td class="field-body">The schema to be used if the BigQuery table to write has to be created
but in the <code class="xref py py-class docutils literal"><span class="pre">TableSchema</span></code> format.</td>
</tr>
<tr class="field-odd field"><th class="field-name">Return type:</th><td class="field-body">TableSchema</td>
</tr>
</tbody>
</table>
</dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.table_schema_to_dict">
<em class="property">static </em><code class="descname">table_schema_to_dict</code><span class="sig-paren">(</span><em>table_schema</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.table_schema_to_dict"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.table_schema_to_dict" title="Permalink to this definition"></a></dt>
<dd><p>Create a dictionary representation of table schema for serialization</p>
</dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema">
<em class="property">static </em><code class="descname">get_dict_table_schema</code><span class="sig-paren">(</span><em>schema</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.get_dict_table_schema"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema" title="Permalink to this definition"></a></dt>
<dd><p>Transform the table schema into a dictionary instance.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><strong>schema</strong> (<em>TableSchema</em>) – The schema to be used if the BigQuery table to write has to be created.
This can either be a dict or string or in the TableSchema format.</td>
</tr>
<tr class="field-even field"><th class="field-name">Returns:</th><td class="field-body">The schema to be used if the BigQuery table to write has
to be created but in the dictionary format.</td>
</tr>
<tr class="field-odd field"><th class="field-name">Return type:</th><td class="field-body">Dict[<a class="reference external" href="https://docs.python.org/2/library/functions.html#str" title="(in Python v2.7)">str</a>, Any]</td>
</tr>
</tbody>
</table>
</dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.io.gcp.bigquery.WriteToBigQuery.display_data">
<code class="descname">display_data</code><span class="sig-paren">(</span><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/io/gcp/bigquery.html#WriteToBigQuery.display_data"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.io.gcp.bigquery.WriteToBigQuery.display_data" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
</div>
</div>
</div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<div class="rst-footer-buttons" role="navigation" aria-label="footer navigation">
<a href="apache_beam.io.gcp.bigquery_file_loads.html" class="btn btn-neutral float-right" title="apache_beam.io.gcp.bigquery_file_loads module" accesskey="n" rel="next">Next <span class="fa fa-arrow-circle-right"></span></a>
<a href="apache_beam.io.gcp.big_query_query_to_table_pipeline.html" class="btn btn-neutral" title="apache_beam.io.gcp.big_query_query_to_table_pipeline module" accesskey="p" rel="prev"><span class="fa fa-arrow-circle-left"></span> Previous</a>
</div>
<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>