blob: 86bf27845a7a2941766e1d9d7a50e02b452e6999 [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 &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.bigquery</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.bigquery</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;BigQuery sources and sinks.</span>
<span class="sd">This module implements reading from and writing to BigQuery tables. It relies</span>
<span class="sd">on several classes exposed by the BigQuery API: TableSchema, TableFieldSchema,</span>
<span class="sd">TableRow, and TableCell. The default mode is to return table rows read from a</span>
<span class="sd">BigQuery source as dictionaries. Similarly a Write transform to a BigQuerySink</span>
<span class="sd">accepts PCollections of dictionaries. This is done for more convenient</span>
<span class="sd">programming. If desired, the native TableRow objects can be used throughout to</span>
<span class="sd">represent rows (use an instance of TableRowJsonCoder as a coder argument when</span>
<span class="sd">creating the sources or sinks respectively).</span>
<span class="sd">Also, for programming convenience, instances of TableReference and TableSchema</span>
<span class="sd">have a string representation that can be used for the corresponding arguments:</span>
<span class="sd"> - TableReference can be a PROJECT:DATASET.TABLE or DATASET.TABLE string.</span>
<span class="sd"> - TableSchema can be a NAME:TYPE{,NAME:TYPE}* string</span>
<span class="sd"> (e.g. &#39;month:STRING,event_count:INTEGER&#39;).</span>
<span class="sd">The syntax supported is described here:</span>
<span class="sd">https://cloud.google.com/bigquery/bq-command-line-tool-quickstart</span>
<span class="sd">BigQuery sources can be used as main inputs or side inputs. A main input</span>
<span class="sd">(common case) is expected to be massive and will be split into manageable chunks</span>
<span class="sd">and processed in parallel. Side inputs are expected to be small and will be read</span>
<span class="sd">completely every time a ParDo DoFn gets executed. In the example below the</span>
<span class="sd">lambda function implementing the DoFn for the Map transform will get on each</span>
<span class="sd">call *one* row of the main table and *all* rows of the side table. The runner</span>
<span class="sd">may use some caching techniques to share the side inputs between calls in order</span>
<span class="sd">to avoid excessive reading:::</span>
<span class="sd"> main_table = pipeline | &#39;VeryBig&#39; &gt;&gt; beam.io.Read(beam.io.BigQuerySource()</span>
<span class="sd"> side_table = pipeline | &#39;NotBig&#39; &gt;&gt; beam.io.Read(beam.io.BigQuerySource()</span>
<span class="sd"> results = (</span>
<span class="sd"> main_table</span>
<span class="sd"> | &#39;ProcessData&#39; &gt;&gt; beam.Map(</span>
<span class="sd"> lambda element, side_input: ..., AsList(side_table)))</span>
<span class="sd">There is no difference in how main and side inputs are read. What makes the</span>
<span class="sd">side_table a &#39;side input&#39; is the AsList wrapper used when passing the table</span>
<span class="sd">as a parameter to the Map transform. AsList signals to the execution framework</span>
<span class="sd">that its input should be made available whole.</span>
<span class="sd">The main and side inputs are implemented differently. Reading a BigQuery table</span>
<span class="sd">as main input entails exporting the table to a set of GCS files (currently in</span>
<span class="sd">JSON format) and then processing those files. Reading the same table as a side</span>
<span class="sd">input entails querying the table for all its rows. The coder argument on</span>
<span class="sd">BigQuerySource controls the reading of the lines in the export files (i.e.,</span>
<span class="sd">transform a JSON object into a PCollection element). The coder is not involved</span>
<span class="sd">when the same table is read as a side input since there is no intermediate</span>
<span class="sd">format involved. We get the table rows directly from the BigQuery service with</span>
<span class="sd">a query.</span>
<span class="sd">Users may provide a query to read from rather than reading all of a BigQuery</span>
<span class="sd">table. If specified, the result obtained by executing the specified query will</span>
<span class="sd">be used as the data of the input transform.::</span>
<span class="sd"> query_results = pipeline | beam.io.Read(beam.io.BigQuerySource(</span>
<span class="sd"> query=&#39;SELECT year, mean_temp FROM samples.weather_stations&#39;))</span>
<span class="sd">When creating a BigQuery input transform, users should provide either a query</span>
<span class="sd">or a table. Pipeline construction will fail with a validation error if neither</span>
<span class="sd">or both are specified.</span>
<span class="sd">Writing Data to BigQuery</span>
<span class="sd">========================</span>
<span class="sd">The `WriteToBigQuery` transform is the recommended way of writing data to</span>
<span class="sd">BigQuery. It supports a large set of parameters to customize how you&#39;d like to</span>
<span class="sd">write to BigQuery.</span>
<span class="sd">Table References</span>
<span class="sd">----------------</span>
<span class="sd">This transform allows you to provide static `project`, `dataset` and `table`</span>
<span class="sd">parameters which point to a specific BigQuery table to be created. The `table`</span>
<span class="sd">parameter can also be a dynamic parameter (i.e. a callable), which receives an</span>
<span class="sd">element to be written to BigQuery, and returns the table that that element</span>
<span class="sd">should be sent to.</span>
<span class="sd">You may also provide a tuple of PCollectionView elements to be passed as side</span>
<span class="sd">inputs to your callable. For example, suppose that one wishes to send</span>
<span class="sd">events of different types to different tables, and the table names are</span>
<span class="sd">computed at pipeline runtime, one may do something like the following::</span>
<span class="sd"> with Pipeline() as p:</span>
<span class="sd"> elements = (p | beam.Create([</span>
<span class="sd"> {&#39;type&#39;: &#39;error&#39;, &#39;timestamp&#39;: &#39;12:34:56&#39;, &#39;message&#39;: &#39;bad&#39;},</span>
<span class="sd"> {&#39;type&#39;: &#39;user_log&#39;, &#39;timestamp&#39;: &#39;12:34:59&#39;, &#39;query&#39;: &#39;flu symptom&#39;},</span>
<span class="sd"> ]))</span>
<span class="sd"> table_names = (p | beam.Create([</span>
<span class="sd"> (&#39;error&#39;, &#39;my_project.dataset1.error_table_for_today&#39;),</span>
<span class="sd"> (&#39;user_log&#39;, &#39;my_project.dataset1.query_table_for_today&#39;),</span>
<span class="sd"> ])</span>
<span class="sd"> table_names_dict = beam.pvalue.AsDict(table_names)</span>
<span class="sd"> elements | beam.io.gcp.WriteToBigQuery(</span>
<span class="sd"> table=lambda row, table_dict: table_dict[row[&#39;type&#39;]],</span>
<span class="sd"> table_side_inputs=(table_names_dict,))</span>
<span class="sd">In the example above, the `table_dict` argument passed to the function in</span>
<span class="sd">`table_dict` is the side input coming from `table_names_dict`, which is passed</span>
<span class="sd">as part of the `table_side_inputs` argument.</span>
<span class="sd">Schemas</span>
<span class="sd">---------</span>
<span class="sd">This transform also allows you to provide a static or dynamic `schema`</span>
<span class="sd">parameter (i.e. a callable).</span>
<span class="sd">If providing a callable, this should take in a table reference (as returned by</span>
<span class="sd">the `table` parameter), and return the corresponding schema for that table.</span>
<span class="sd">This allows to provide different schemas for different tables::</span>
<span class="sd"> def compute_table_name(row):</span>
<span class="sd"> ...</span>
<span class="sd"> errors_schema = {&#39;fields&#39;: [</span>
<span class="sd"> {&#39;name&#39;: &#39;type&#39;, &#39;type&#39;: &#39;STRING&#39;, &#39;mode&#39;: &#39;NULLABLE&#39;},</span>
<span class="sd"> {&#39;name&#39;: &#39;message&#39;, &#39;type&#39;: &#39;STRING&#39;, &#39;mode&#39;: &#39;NULLABLE&#39;}]}</span>
<span class="sd"> queries_schema = {&#39;fields&#39;: [</span>
<span class="sd"> {&#39;name&#39;: &#39;type&#39;, &#39;type&#39;: &#39;STRING&#39;, &#39;mode&#39;: &#39;NULLABLE&#39;},</span>
<span class="sd"> {&#39;name&#39;: &#39;query&#39;, &#39;type&#39;: &#39;STRING&#39;, &#39;mode&#39;: &#39;NULLABLE&#39;}]}</span>
<span class="sd"> with Pipeline() as p:</span>
<span class="sd"> elements = (p | beam.Create([</span>
<span class="sd"> {&#39;type&#39;: &#39;error&#39;, &#39;timestamp&#39;: &#39;12:34:56&#39;, &#39;message&#39;: &#39;bad&#39;},</span>
<span class="sd"> {&#39;type&#39;: &#39;user_log&#39;, &#39;timestamp&#39;: &#39;12:34:59&#39;, &#39;query&#39;: &#39;flu symptom&#39;},</span>
<span class="sd"> ]))</span>
<span class="sd"> elements | beam.io.gcp.WriteToBigQuery(</span>
<span class="sd"> table=compute_table_name,</span>
<span class="sd"> schema=lambda table: (errors_schema</span>
<span class="sd"> if &#39;errors&#39; in table</span>
<span class="sd"> else queries_schema))</span>
<span class="sd">It may be the case that schemas are computed at pipeline runtime. In cases</span>
<span class="sd">like these, one can also provide a `schema_side_inputs` parameter, which is</span>
<span class="sd">a tuple of PCollectionViews to be passed to the schema callable (much like</span>
<span class="sd">the `table_side_inputs` parameter).</span>
<span class="sd">Additional Parameters for BigQuery Tables</span>
<span class="sd">-----------------------------------------</span>
<span class="sd">This sink is able to create tables in BigQuery if they don&#39;t already exist. It</span>
<span class="sd">also relies on creating temporary tables when performing file loads.</span>
<span class="sd">The WriteToBigQuery transform creates tables using the BigQuery API by</span>
<span class="sd">inserting a load job (see the API reference [1]), or by inserting a new table</span>
<span class="sd">(see the API reference for that [2][3]).</span>
<span class="sd">When creating a new BigQuery table, there are a number of extra parameters</span>
<span class="sd">that one may need to specify. For example, clustering, partitioning, data</span>
<span class="sd">encoding, etc. It is possible to provide these additional parameters by</span>
<span class="sd">passing a Python dictionary as `additional_bq_parameters` to the transform.</span>
<span class="sd">As an example, to create a table that has specific partitioning, and</span>
<span class="sd">clustering properties, one would do the following::</span>
<span class="sd"> additional_bq_parameters = {</span>
<span class="sd"> &#39;timePartitioning&#39;: {&#39;type&#39;: &#39;DAY&#39;},</span>
<span class="sd"> &#39;clustering&#39;: {&#39;fields&#39;: [&#39;country&#39;]}}</span>
<span class="sd"> with Pipeline() as p:</span>
<span class="sd"> elements = (p | beam.Create([</span>
<span class="sd"> {&#39;country&#39;: &#39;mexico&#39;, &#39;timestamp&#39;: &#39;12:34:56&#39;, &#39;query&#39;: &#39;acapulco&#39;},</span>
<span class="sd"> {&#39;country&#39;: &#39;canada&#39;, &#39;timestamp&#39;: &#39;12:34:59&#39;, &#39;query&#39;: &#39;influenza&#39;},</span>
<span class="sd"> ]))</span>
<span class="sd"> elements | beam.io.gcp.WriteToBigQuery(</span>
<span class="sd"> table=&#39;project_name1.dataset_2.query_events_table&#39;,</span>
<span class="sd"> additional_bq_parameters=additional_bq_parameters)</span>
<span class="sd">Much like the schema case, the parameter with `additional_bq_parameters` can</span>
<span class="sd">also take a callable that receives a table reference.</span>
<span class="sd">[1] https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#\</span>
<span class="sd">configuration.load</span>
<span class="sd">[2] https://cloud.google.com/bigquery/docs/reference/rest/v2/tables/insert</span>
<span class="sd">[3] https://cloud.google.com/bigquery/docs/reference/rest/v2/tables#resource</span>
<span class="sd">*** Short introduction to BigQuery concepts ***</span>
<span class="sd">Tables have rows (TableRow) and each row has cells (TableCell).</span>
<span class="sd">A table has a schema (TableSchema), which in turn describes the schema of each</span>
<span class="sd">cell (TableFieldSchema). The terms field and cell are used interchangeably.</span>
<span class="sd">TableSchema: Describes the schema (types and order) for values in each row.</span>
<span class="sd"> Has one attribute, &#39;field&#39;, which is list of TableFieldSchema objects.</span>
<span class="sd">TableFieldSchema: Describes the schema (type, name) for one field.</span>
<span class="sd"> Has several attributes, including &#39;name&#39; and &#39;type&#39;. Common values for</span>
<span class="sd"> the type attribute are: &#39;STRING&#39;, &#39;INTEGER&#39;, &#39;FLOAT&#39;, &#39;BOOLEAN&#39;, &#39;NUMERIC&#39;,</span>
<span class="sd"> &#39;GEOGRAPHY&#39;.</span>
<span class="sd"> All possible values are described at:</span>
<span class="sd"> https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types</span>
<span class="sd">TableRow: Holds all values in a table row. Has one attribute, &#39;f&#39;, which is a</span>
<span class="sd"> list of TableCell instances.</span>
<span class="sd">TableCell: Holds the value for one cell (or field). Has one attribute,</span>
<span class="sd"> &#39;v&#39;, which is a JsonValue instance. This class is defined in</span>
<span class="sd"> apitools.base.py.extra_types.py module.</span>
<span class="sd">As of Beam 2.7.0, the NUMERIC data type is supported. This data type supports</span>
<span class="sd">high-precision decimal numbers (precision of 38 digits, scale of 9 digits).</span>
<span class="sd">The GEOGRAPHY data type works with Well-Known Text (See</span>
<span class="sd">https://en.wikipedia.org/wiki/Well-known_text) format for reading and writing</span>
<span class="sd">to BigQuery.</span>
<span class="sd">BigQuery IO requires values of BYTES datatype to be encoded using base64</span>
<span class="sd">encoding when writing to BigQuery. When bytes are read from BigQuery they are</span>
<span class="sd">returned as base64-encoded bytes.</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">collections</span>
<span class="kn">import</span> <span class="nn">itertools</span>
<span class="kn">import</span> <span class="nn">json</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">time</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">builtins</span> <span class="k">import</span> <span class="nb">zip</span>
<span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">itervalues</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">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</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</span> <span class="k">import</span> <span class="n">pvalue</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal.gcp.json_value</span> <span class="k">import</span> <span class="n">from_json_value</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal.gcp.json_value</span> <span class="k">import</span> <span class="n">to_json_value</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.gcp</span> <span class="k">import</span> <span class="n">bigquery_tools</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.gcp.internal.clients</span> <span class="k">import</span> <span class="n">bigquery</span>
<span class="kn">from</span> <span class="nn">apache_beam.options</span> <span class="k">import</span> <span class="n">value_provider</span> <span class="k">as</span> <span class="n">vp</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">DebugOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">GoogleCloudOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">StandardOptions</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">DoFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">ParDo</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.transforms.window</span> <span class="k">import</span> <span class="n">GlobalWindows</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="k">import</span> <span class="n">retry</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="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;TableRowJsonCoder&#39;</span><span class="p">,</span>
<span class="s1">&#39;BigQueryDisposition&#39;</span><span class="p">,</span>
<span class="s1">&#39;BigQuerySource&#39;</span><span class="p">,</span>
<span class="s1">&#39;BigQuerySink&#39;</span><span class="p">,</span>
<span class="s1">&#39;WriteToBigQuery&#39;</span><span class="p">,</span>
<span class="s1">&#39;SCHEMA_AUTODETECT&#39;</span><span class="p">,</span>
<span class="p">]</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.parse_table_reference&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_parse_table_reference</span><span class="p">(</span><span class="n">table</span><span class="p">,</span> <span class="n">dataset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">project</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span><span class="n">table</span><span class="p">,</span> <span class="n">dataset</span><span class="p">,</span> <span class="n">project</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span>
<span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.parse_table_schema_from_json&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">parse_table_schema_from_json</span><span class="p">(</span><span class="n">schema_string</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_schema_from_json</span><span class="p">(</span><span class="n">schema_string</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.default_encoder&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">default_encoder</span><span class="p">(</span><span class="n">obj</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">default_encoder</span><span class="p">(</span><span class="n">obj</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.RowAsDictJsonCoder&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">RowAsDictJsonCoder</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">RowAsDictJsonCoder</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.BigQueryReader&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">BigQueryReader</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryReader</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.BigQueryWriter&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">BigQueryWriter</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryWriter</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">&#39;2.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;bigquery_tools.BigQueryWrapper&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">BigQueryWrapper</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryWrapper</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<div class="viewcode-block" id="TableRowJsonCoder"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.TableRowJsonCoder">[docs]</a><span class="k">class</span> <span class="nc">TableRowJsonCoder</span><span class="p">(</span><span class="n">coders</span><span class="o">.</span><span class="n">Coder</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A coder for a TableRow instance to/from a JSON string.</span>
<span class="sd"> Note that the encoding operation (used when writing to sinks) requires the</span>
<span class="sd"> table schema in order to obtain the ordered list of field names. Reading from</span>
<span class="sd"> sources on the other hand does not need the table schema.</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">table_schema</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># The table schema is needed for encoding TableRows as JSON (writing to</span>
<span class="c1"># sinks) because the ordered list of field names is used in the JSON</span>
<span class="c1"># representation.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span> <span class="o">=</span> <span class="n">table_schema</span>
<span class="c1"># Precompute field names since we need them for row encoding.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">field_names</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">fs</span><span class="o">.</span><span class="n">name</span> <span class="k">for</span> <span class="n">fs</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">field_types</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">fs</span><span class="o">.</span><span class="n">type</span> <span class="k">for</span> <span class="n">fs</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="p">)</span>
<div class="viewcode-block" id="TableRowJsonCoder.encode"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.TableRowJsonCoder.encode">[docs]</a> <span class="k">def</span> <span class="nf">encode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">table_row</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">AttributeError</span><span class="p">(</span>
<span class="s1">&#39;The TableRowJsonCoder requires a table schema for &#39;</span>
<span class="s1">&#39;encoding operations. Please specify a table_schema argument.&#39;</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span>
<span class="n">collections</span><span class="o">.</span><span class="n">OrderedDict</span><span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">field_names</span><span class="p">,</span>
<span class="p">[</span><span class="n">from_json_value</span><span class="p">(</span><span class="n">f</span><span class="o">.</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">table_row</span><span class="o">.</span><span class="n">f</span><span class="p">])),</span>
<span class="n">allow_nan</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">default</span><span class="o">=</span><span class="n">bigquery_tools</span><span class="o">.</span><span class="n">default_encoder</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">ValueError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;</span><span class="si">%s</span><span class="s1">. </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">e</span><span class="p">,</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">JSON_COMPLIANCE_ERROR</span><span class="p">))</span></div>
<div class="viewcode-block" id="TableRowJsonCoder.decode"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.TableRowJsonCoder.decode">[docs]</a> <span class="k">def</span> <span class="nf">decode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">encoded_table_row</span><span class="p">):</span>
<span class="n">od</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span>
<span class="n">encoded_table_row</span><span class="p">,</span> <span class="n">object_pairs_hook</span><span class="o">=</span><span class="n">collections</span><span class="o">.</span><span class="n">OrderedDict</span><span class="p">)</span>
<span class="k">return</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableRow</span><span class="p">(</span>
<span class="n">f</span><span class="o">=</span><span class="p">[</span><span class="n">bigquery</span><span class="o">.</span><span class="n">TableCell</span><span class="p">(</span><span class="n">v</span><span class="o">=</span><span class="n">to_json_value</span><span class="p">(</span><span class="n">e</span><span class="p">))</span> <span class="k">for</span> <span class="n">e</span> <span class="ow">in</span> <span class="n">itervalues</span><span class="p">(</span><span class="n">od</span><span class="p">)])</span></div></div>
<div class="viewcode-block" id="BigQueryDisposition"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQueryDisposition">[docs]</a><span class="k">class</span> <span class="nc">BigQueryDisposition</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Class holding standard strings used for create and write dispositions.&quot;&quot;&quot;</span>
<span class="n">CREATE_NEVER</span> <span class="o">=</span> <span class="s1">&#39;CREATE_NEVER&#39;</span>
<span class="n">CREATE_IF_NEEDED</span> <span class="o">=</span> <span class="s1">&#39;CREATE_IF_NEEDED&#39;</span>
<span class="n">WRITE_TRUNCATE</span> <span class="o">=</span> <span class="s1">&#39;WRITE_TRUNCATE&#39;</span>
<span class="n">WRITE_APPEND</span> <span class="o">=</span> <span class="s1">&#39;WRITE_APPEND&#39;</span>
<span class="n">WRITE_EMPTY</span> <span class="o">=</span> <span class="s1">&#39;WRITE_EMPTY&#39;</span>
<div class="viewcode-block" id="BigQueryDisposition.validate_create"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_create">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">validate_create</span><span class="p">(</span><span class="n">disposition</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="p">(</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">CREATE_NEVER</span><span class="p">,</span>
<span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">CREATE_IF_NEEDED</span><span class="p">)</span>
<span class="k">if</span> <span class="n">disposition</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">values</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Invalid create disposition </span><span class="si">%s</span><span class="s1">. Expecting </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">disposition</span><span class="p">,</span> <span class="n">values</span><span class="p">))</span>
<span class="k">return</span> <span class="n">disposition</span></div>
<div class="viewcode-block" id="BigQueryDisposition.validate_write"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQueryDisposition.validate_write">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">validate_write</span><span class="p">(</span><span class="n">disposition</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="p">(</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_TRUNCATE</span><span class="p">,</span>
<span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_APPEND</span><span class="p">,</span>
<span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_EMPTY</span><span class="p">)</span>
<span class="k">if</span> <span class="n">disposition</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">values</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Invalid write disposition </span><span class="si">%s</span><span class="s1">. Expecting </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">disposition</span><span class="p">,</span> <span class="n">values</span><span class="p">))</span>
<span class="k">return</span> <span class="n">disposition</span></div></div>
<span class="c1"># -----------------------------------------------------------------------------</span>
<span class="c1"># BigQuerySource, BigQuerySink.</span>
<div class="viewcode-block" id="BigQuerySource"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySource">[docs]</a><span class="k">class</span> <span class="nc">BigQuerySource</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;A source based on a BigQuery table.&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">table</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">dataset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">project</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">query</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">validate</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">coder</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">use_standard_sql</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">flatten_results</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">kms_key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initialize a :class:`BigQuerySource`.</span>
<span class="sd"> Args:</span>
<span class="sd"> table (str): The ID of a BigQuery table. If specified all data of the</span>
<span class="sd"> table will be used as input of the current source. The ID must contain</span>
<span class="sd"> only letters ``a-z``, ``A-Z``, numbers ``0-9``, or underscores</span>
<span class="sd"> ``_``. If dataset and query arguments are :data:`None` then the table</span>
<span class="sd"> argument must contain the entire table reference specified as:</span>
<span class="sd"> ``&#39;DATASET.TABLE&#39;`` or ``&#39;PROJECT:DATASET.TABLE&#39;``.</span>
<span class="sd"> dataset (str): The ID of the dataset containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument or a query is specified.</span>
<span class="sd"> project (str): The ID of the project containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument or a query is specified.</span>
<span class="sd"> query (str): A query to be used instead of arguments table, dataset, and</span>
<span class="sd"> project.</span>
<span class="sd"> validate (bool): If :data:`True`, various checks will be done when source</span>
<span class="sd"> gets initialized (e.g., is table present?). This should be</span>
<span class="sd"> :data:`True` for most scenarios in order to catch errors as early as</span>
<span class="sd"> possible (pipeline construction instead of pipeline execution). It</span>
<span class="sd"> should be :data:`False` if the table is created during pipeline</span>
<span class="sd"> execution by a previous step.</span>
<span class="sd"> coder (~apache_beam.coders.coders.Coder): The coder for the table</span>
<span class="sd"> rows if serialized to disk. If :data:`None`, then the default coder is</span>
<span class="sd"> :class:`~apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder`,</span>
<span class="sd"> which will interpret every line in a file as a JSON serialized</span>
<span class="sd"> dictionary. This argument needs a value only in special cases when</span>
<span class="sd"> returning table rows as dictionaries is not desirable.</span>
<span class="sd"> use_standard_sql (bool): Specifies whether to use BigQuery&#39;s standard SQL</span>
<span class="sd"> dialect for this query. The default value is :data:`False`.</span>
<span class="sd"> If set to :data:`True`, the query will use BigQuery&#39;s updated SQL</span>
<span class="sd"> dialect with improved standards compliance.</span>
<span class="sd"> This parameter is ignored for table inputs.</span>
<span class="sd"> flatten_results (bool): Flattens all nested and repeated fields in the</span>
<span class="sd"> query results. The default value is :data:`True`.</span>
<span class="sd"> kms_key (str): Experimental. Optional Cloud KMS key name for use when</span>
<span class="sd"> creating new tables.</span>
<span class="sd"> Raises:</span>
<span class="sd"> ~exceptions.ValueError: if any of the following is true:</span>
<span class="sd"> 1) the table reference as a string does not match the expected format</span>
<span class="sd"> 2) neither a table nor a query is specified</span>
<span class="sd"> 3) both a table and a query is specified.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span> <span class="nn">apitools.base</span> <span class="k">import</span> <span class="n">py</span> <span class="c1"># pylint: disable=unused-variable</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span>
<span class="s1">&#39;Google Cloud IO not available, &#39;</span>
<span class="s1">&#39;please install apache_beam[gcp]&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">table</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">query</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Both a BigQuery table and a query were specified.&#39;</span>
<span class="s1">&#39; Please specify only one of these.&#39;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">table</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">query</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;A BigQuery table or a query must be specified&#39;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">table</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span>
<span class="n">table</span><span class="p">,</span> <span class="n">dataset</span><span class="p">,</span> <span class="n">project</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">query</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">use_legacy_sql</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">query</span> <span class="o">=</span> <span class="n">query</span>
<span class="c1"># TODO(BEAM-1082): Change the internal flag to be standard_sql</span>
<span class="bp">self</span><span class="o">.</span><span class="n">use_legacy_sql</span> <span class="o">=</span> <span class="ow">not</span> <span class="n">use_standard_sql</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">validate</span> <span class="o">=</span> <span class="n">validate</span>
<span class="bp">self</span><span class="o">.</span><span class="n">flatten_results</span> <span class="o">=</span> <span class="n">flatten_results</span>
<span class="bp">self</span><span class="o">.</span><span class="n">coder</span> <span class="o">=</span> <span class="n">coder</span> <span class="ow">or</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">RowAsDictJsonCoder</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span> <span class="o">=</span> <span class="n">kms_key</span>
<div class="viewcode-block" id="BigQuerySource.display_data"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySource.display_data">[docs]</a> <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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">query</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">res</span> <span class="o">=</span> <span class="p">{</span><span class="s1">&#39;query&#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">query</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Query&#39;</span><span class="p">)}</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span>
<span class="n">res</span> <span class="o">=</span> <span class="p">{</span><span class="s1">&#39;table&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="n">tableSpec</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Table&#39;</span><span class="p">)}</span>
<span class="n">res</span><span class="p">[</span><span class="s1">&#39;validation&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">validate</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s1">&#39;Validation Enabled&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">res</span></div>
<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;bigquery&#39;</span>
<div class="viewcode-block" id="BigQuerySource.reader"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySource.reader">[docs]</a> <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="n">test_bigquery_client</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryReader</span><span class="p">(</span>
<span class="n">source</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span>
<span class="n">test_bigquery_client</span><span class="o">=</span><span class="n">test_bigquery_client</span><span class="p">,</span>
<span class="n">use_legacy_sql</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">use_legacy_sql</span><span class="p">,</span>
<span class="n">flatten_results</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">flatten_results</span><span class="p">,</span>
<span class="n">kms_key</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="BigQuerySink"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySink">[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.11.0&#39;</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s2">&quot;WriteToBigQuery&quot;</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">BigQuerySink</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;A sink based on a BigQuery table.</span>
<span class="sd"> This BigQuery sink triggers a Dataflow native sink for BigQuery</span>
<span class="sd"> that only supports batch pipelines.</span>
<span class="sd"> Instead of using this sink directly, please use WriteToBigQuery</span>
<span class="sd"> transform that works for both batch and streaming pipelines.</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">table</span><span class="p">,</span> <span class="n">dataset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">project</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">schema</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">create_disposition</span><span class="o">=</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">CREATE_IF_NEEDED</span><span class="p">,</span>
<span class="n">write_disposition</span><span class="o">=</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_EMPTY</span><span class="p">,</span>
<span class="n">validate</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">coder</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">kms_key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initialize a BigQuerySink.</span>
<span class="sd"> Args:</span>
<span class="sd"> table (str): The ID of the table. The ID must contain only letters</span>
<span class="sd"> ``a-z``, ``A-Z``, numbers ``0-9``, or underscores ``_``. If</span>
<span class="sd"> **dataset** argument is :data:`None` then the table argument must</span>
<span class="sd"> contain the entire table reference specified as: ``&#39;DATASET.TABLE&#39;`` or</span>
<span class="sd"> ``&#39;PROJECT:DATASET.TABLE&#39;``.</span>
<span class="sd"> dataset (str): The ID of the dataset containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument.</span>
<span class="sd"> project (str): The ID of the project containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument.</span>
<span class="sd"> schema (str): The schema to be used if the BigQuery table to write has</span>
<span class="sd"> to be created. This can be either specified as a</span>
<span class="sd"> :class:`~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema` object or a single string of the form</span>
<span class="sd"> ``&#39;field1:type1,field2:type2,field3:type3&#39;`` that defines a comma</span>
<span class="sd"> separated list of fields. Here ``&#39;type&#39;`` should specify the BigQuery</span>
<span class="sd"> type of the field. Single string based schemas do not support nested</span>
<span class="sd"> fields, repeated fields, or specifying a BigQuery mode for fields (mode</span>
<span class="sd"> will always be set to ``&#39;NULLABLE&#39;``).</span>
<span class="sd"> create_disposition (BigQueryDisposition): A string describing what</span>
<span class="sd"> happens if the table does not exist. Possible values are:</span>
<span class="sd"> * :attr:`BigQueryDisposition.CREATE_IF_NEEDED`: create if does not</span>
<span class="sd"> exist.</span>
<span class="sd"> * :attr:`BigQueryDisposition.CREATE_NEVER`: fail the write if does not</span>
<span class="sd"> exist.</span>
<span class="sd"> write_disposition (BigQueryDisposition): A string describing what</span>
<span class="sd"> happens if the table has already some data. Possible values are:</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_TRUNCATE`: delete existing rows.</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_APPEND`: add to existing rows.</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_EMPTY`: fail the write if table not</span>
<span class="sd"> empty.</span>
<span class="sd"> validate (bool): If :data:`True`, various checks will be done when sink</span>
<span class="sd"> gets initialized (e.g., is table present given the disposition</span>
<span class="sd"> arguments?). This should be :data:`True` for most scenarios in order to</span>
<span class="sd"> catch errors as early as possible (pipeline construction instead of</span>
<span class="sd"> pipeline execution). It should be :data:`False` if the table is created</span>
<span class="sd"> during pipeline execution by a previous step.</span>
<span class="sd"> coder (~apache_beam.coders.coders.Coder): The coder for the</span>
<span class="sd"> table rows if serialized to disk. If :data:`None`, then the default</span>
<span class="sd"> coder is :class:`~apache_beam.io.gcp.bigquery_tools.RowAsDictJsonCoder`,</span>
<span class="sd"> which will interpret every element written to the sink as a dictionary</span>
<span class="sd"> that will be JSON serialized as a line in a file. This argument needs a</span>
<span class="sd"> value only in special cases when writing table rows as dictionaries is</span>
<span class="sd"> not desirable.</span>
<span class="sd"> kms_key (str): Experimental. Optional Cloud KMS key name for use when</span>
<span class="sd"> creating new tables.</span>
<span class="sd"> Raises:</span>
<span class="sd"> ~exceptions.TypeError: if the schema argument is not a :class:`str` or a</span>
<span class="sd"> :class:`~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema` object.</span>
<span class="sd"> ~exceptions.ValueError: if the table reference as a string does not</span>
<span class="sd"> match the expected format.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span> <span class="nn">apitools.base</span> <span class="k">import</span> <span class="n">py</span> <span class="c1"># pylint: disable=unused-variable</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span>
<span class="s1">&#39;Google Cloud IO not available, &#39;</span>
<span class="s1">&#39;please install apache_beam[gcp]&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span>
<span class="n">table</span><span class="p">,</span> <span class="n">dataset</span><span class="p">,</span> <span class="n">project</span><span class="p">)</span>
<span class="c1"># Transform the table schema into a bigquery.TableSchema instance.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">unicode</span><span class="p">)):</span>
<span class="c1"># TODO(silviuc): Should add a regex-based validation of the format.</span>
<span class="n">table_schema</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">()</span>
<span class="n">schema_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">s</span><span class="o">.</span><span class="n">strip</span><span class="p">(</span><span class="s1">&#39; &#39;</span><span class="p">)</span> <span class="k">for</span> <span class="n">s</span> <span class="ow">in</span> <span class="n">schema</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;,&#39;</span><span class="p">)]</span>
<span class="k">for</span> <span class="n">field_and_type</span> <span class="ow">in</span> <span class="n">schema_list</span><span class="p">:</span>
<span class="n">field_name</span><span class="p">,</span> <span class="n">field_type</span> <span class="o">=</span> <span class="n">field_and_type</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;:&#39;</span><span class="p">)</span>
<span class="n">field_schema</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableFieldSchema</span><span class="p">()</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">field_name</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">type</span> <span class="o">=</span> <span class="n">field_type</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">mode</span> <span class="o">=</span> <span class="s1">&#39;NULLABLE&#39;</span>
<span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">field_schema</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span> <span class="o">=</span> <span class="n">table_schema</span>
<span class="k">elif</span> <span class="n">schema</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># TODO(silviuc): Should check that table exists if no schema specified.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span> <span class="o">=</span> <span class="n">schema</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span> <span class="o">=</span> <span class="n">schema</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">schema</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span> <span class="o">=</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">validate_create</span><span class="p">(</span>
<span class="n">create_disposition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span> <span class="o">=</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">validate_write</span><span class="p">(</span>
<span class="n">write_disposition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">validate</span> <span class="o">=</span> <span class="n">validate</span>
<span class="bp">self</span><span class="o">.</span><span class="n">coder</span> <span class="o">=</span> <span class="n">coder</span> <span class="ow">or</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">RowAsDictJsonCoder</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span> <span class="o">=</span> <span class="n">kms_key</span>
<div class="viewcode-block" id="BigQuerySink.display_data"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySink.display_data">[docs]</a> <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="n">res</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="n">tableSpec</span><span class="p">)</span>
<span class="n">res</span><span class="p">[</span><span class="s1">&#39;table&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="n">tableSpec</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Table&#39;</span><span class="p">)</span>
<span class="n">res</span><span class="p">[</span><span class="s1">&#39;validation&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">validate</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="s2">&quot;Validation Enabled&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">res</span></div>
<div class="viewcode-block" id="BigQuerySink.schema_as_json"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySink.schema_as_json">[docs]</a> <span class="k">def</span> <span class="nf">schema_as_json</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns the TableSchema associated with the sink as a JSON string.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">schema_list_as_object</span><span class="p">(</span><span class="n">schema_list</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a list of TableFieldSchema objects as a list of dicts.&quot;&quot;&quot;</span>
<span class="n">fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">schema_list</span><span class="p">:</span>
<span class="n">fs</span> <span class="o">=</span> <span class="p">{</span><span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="n">f</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> <span class="s1">&#39;type&#39;</span><span class="p">:</span> <span class="n">f</span><span class="o">.</span><span class="n">type</span><span class="p">}</span>
<span class="k">if</span> <span class="n">f</span><span class="o">.</span><span class="n">description</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">fs</span><span class="p">[</span><span class="s1">&#39;description&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="n">description</span>
<span class="k">if</span> <span class="n">f</span><span class="o">.</span><span class="n">mode</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">fs</span><span class="p">[</span><span class="s1">&#39;mode&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="n">mode</span>
<span class="k">if</span> <span class="n">f</span><span class="o">.</span><span class="n">type</span><span class="o">.</span><span class="n">lower</span><span class="p">()</span> <span class="o">==</span> <span class="s1">&#39;record&#39;</span><span class="p">:</span>
<span class="n">fs</span><span class="p">[</span><span class="s1">&#39;fields&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">schema_list_as_object</span><span class="p">(</span><span class="n">f</span><span class="o">.</span><span class="n">fields</span><span class="p">)</span>
<span class="n">fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">fs</span><span class="p">)</span>
<span class="k">return</span> <span class="n">fields</span>
<span class="k">return</span> <span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span>
<span class="p">{</span><span class="s1">&#39;fields&#39;</span><span class="p">:</span> <span class="n">schema_list_as_object</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="p">)})</span></div>
<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;bigquery&#39;</span>
<div class="viewcode-block" id="BigQuerySink.writer"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.BigQuerySink.writer">[docs]</a> <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="n">test_bigquery_client</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">buffer_size</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryWriter</span><span class="p">(</span>
<span class="n">sink</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span> <span class="n">test_bigquery_client</span><span class="o">=</span><span class="n">test_bigquery_client</span><span class="p">,</span>
<span class="n">buffer_size</span><span class="o">=</span><span class="n">buffer_size</span><span class="p">)</span></div></div>
<span class="k">class</span> <span class="nc">BigQueryWriteFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A ``DoFn`` that streams writes to BigQuery once the table is created.&quot;&quot;&quot;</span>
<span class="n">DEFAULT_MAX_BUFFERED_ROWS</span> <span class="o">=</span> <span class="mi">2000</span>
<span class="n">DEFAULT_MAX_BATCH_SIZE</span> <span class="o">=</span> <span class="mi">500</span>
<span class="n">FAILED_ROWS</span> <span class="o">=</span> <span class="s1">&#39;FailedRows&#39;</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">batch_size</span><span class="p">,</span>
<span class="n">schema</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">create_disposition</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">write_disposition</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">kms_key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">test_client</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">max_buffered_rows</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">retry_strategy</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">additional_bq_parameters</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initialize a WriteToBigQuery transform.</span>
<span class="sd"> Args:</span>
<span class="sd"> batch_size: Number of rows to be written to BQ per streaming API insert.</span>
<span class="sd"> schema: The schema to be used if the BigQuery table to write has to be</span>
<span class="sd"> created. This can be either specified as a &#39;bigquery.TableSchema&#39; object</span>
<span class="sd"> or a single string of the form &#39;field1:type1,field2:type2,field3:type3&#39;</span>
<span class="sd"> that defines a comma separated list of fields. Here &#39;type&#39; should</span>
<span class="sd"> specify the BigQuery type of the field. Single string based schemas do</span>
<span class="sd"> not support nested fields, repeated fields, or specifying a BigQuery</span>
<span class="sd"> mode for fields (mode will always be set to &#39;NULLABLE&#39;).</span>
<span class="sd"> create_disposition: A string describing what happens if the table does not</span>
<span class="sd"> exist. Possible values are:</span>
<span class="sd"> - BigQueryDisposition.CREATE_IF_NEEDED: create if does not exist.</span>
<span class="sd"> - BigQueryDisposition.CREATE_NEVER: fail the write if does not exist.</span>
<span class="sd"> write_disposition: A string describing what happens if the table has</span>
<span class="sd"> already some data. Possible values are:</span>
<span class="sd"> - BigQueryDisposition.WRITE_TRUNCATE: delete existing rows.</span>
<span class="sd"> - BigQueryDisposition.WRITE_APPEND: add to existing rows.</span>
<span class="sd"> - BigQueryDisposition.WRITE_EMPTY: fail the write if table not empty.</span>
<span class="sd"> For streaming pipelines WriteTruncate can not be used.</span>
<span class="sd"> kms_key: Experimental. Optional Cloud KMS key name for use when creating</span>
<span class="sd"> new tables.</span>
<span class="sd"> test_client: Override the default bigquery client used for testing.</span>
<span class="sd"> max_buffered_rows: The maximum number of rows that are allowed to stay</span>
<span class="sd"> buffered when running dynamic destinations. When destinations are</span>
<span class="sd"> dynamic, it is important to keep caches small even when a single</span>
<span class="sd"> batch has not been completely filled up.</span>
<span class="sd"> retry_strategy: The strategy to use when retrying streaming inserts</span>
<span class="sd"> into BigQuery. Options are shown in bigquery_tools.RetryStrategy attrs.</span>
<span class="sd"> additional_bq_parameters (dict, callable): A set of additional parameters</span>
<span class="sd"> to be passed when creating a BigQuery table. These are passed when</span>
<span class="sd"> triggering a load job for FILE_LOADS, and when creating a new table for</span>
<span class="sd"> STREAMING_INSERTS.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">=</span> <span class="n">schema</span>
<span class="bp">self</span><span class="o">.</span><span class="n">test_client</span> <span class="o">=</span> <span class="n">test_client</span>
<span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span> <span class="o">=</span> <span class="n">create_disposition</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span> <span class="o">=</span> <span class="n">write_disposition</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reset_rows_buffer</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_observed_tables</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_total_buffered_rows</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span> <span class="o">=</span> <span class="n">kms_key</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span> <span class="o">=</span> <span class="n">batch_size</span> <span class="ow">or</span> <span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">DEFAULT_MAX_BATCH_SIZE</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_buffered_rows</span> <span class="o">=</span> <span class="p">(</span><span class="n">max_buffered_rows</span>
<span class="ow">or</span> <span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">DEFAULT_MAX_BUFFERED_ROWS</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_retry_strategy</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">retry_strategy</span> <span class="ow">or</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">RetryStrategy</span><span class="o">.</span><span class="n">RETRY_ON_TRANSIENT_ERROR</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span> <span class="o">=</span> <span class="n">additional_bq_parameters</span> <span class="ow">or</span> <span class="p">{}</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;max_batch_size&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">,</span>
<span class="s1">&#39;max_buffered_rows&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_buffered_rows</span><span class="p">,</span>
<span class="s1">&#39;retry_strategy&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_retry_strategy</span><span class="p">,</span>
<span class="s1">&#39;create_disposition&#39;</span><span class="p">:</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">),</span>
<span class="s1">&#39;write_disposition&#39;</span><span class="p">:</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">),</span>
<span class="s1">&#39;additional_bq_parameters&#39;</span><span class="p">:</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span><span class="p">)}</span>
<span class="k">def</span> <span class="nf">_reset_rows_buffer</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="k">lambda</span><span class="p">:</span> <span class="p">[])</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">get_table_schema</span><span class="p">(</span><span class="n">schema</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Transform the table schema into a bigquery.TableSchema instance.</span>
<span class="sd"> Args:</span>
<span class="sd"> schema: The schema to be used if the BigQuery table to write has to be</span>
<span class="sd"> created. This is a dictionary object created in the WriteToBigQuery</span>
<span class="sd"> transform.</span>
<span class="sd"> Returns:</span>
<span class="sd"> table_schema: The schema to be used if the BigQuery table to write has</span>
<span class="sd"> to be created but in the bigquery.TableSchema format.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">schema</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="n">schema</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">unicode</span><span class="p">)):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_schema_from_json</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="k">return</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_schema_from_json</span><span class="p">(</span><span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">schema</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">schema</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reset_rows_buffer</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">bigquery_wrapper</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">BigQueryWrapper</span><span class="p">(</span>
<span class="n">client</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">test_client</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_observed_tables</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_backoff_calculator</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">(</span><span class="n">retry</span><span class="o">.</span><span class="n">FuzzedExponentialIntervals</span><span class="p">(</span>
<span class="n">initial_delay_secs</span><span class="o">=</span><span class="mf">0.2</span><span class="p">,</span>
<span class="n">num_retries</span><span class="o">=</span><span class="mi">10000</span><span class="p">,</span>
<span class="n">max_delay_secs</span><span class="o">=</span><span class="mi">1500</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">_create_table_if_needed</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">table_reference</span><span class="p">,</span> <span class="n">schema</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="n">str_table_reference</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span>
<span class="k">if</span> <span class="n">str_table_reference</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_observed_tables</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span> <span class="o">==</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">CREATE_NEVER</span><span class="p">:</span>
<span class="c1"># If we never want to create the table, we assume it already exists,</span>
<span class="c1"># and avoid the get-or-create step.</span>
<span class="k">return</span>
<span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s1">&#39;Creating or getting table </span><span class="si">%s</span><span class="s1"> with schema </span><span class="si">%s</span><span class="s1">.&#39;</span><span class="p">,</span>
<span class="n">table_reference</span><span class="p">,</span> <span class="n">schema</span><span class="p">)</span>
<span class="n">table_schema</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_table_schema</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span>
<span class="k">if</span> <span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="o">=</span> <span class="n">vp</span><span class="o">.</span><span class="n">RuntimeValueProvider</span><span class="o">.</span><span class="n">get_value</span><span class="p">(</span>
<span class="s1">&#39;project&#39;</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">bigquery_wrapper</span><span class="o">.</span><span class="n">get_or_create_table</span><span class="p">(</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">,</span>
<span class="n">table_schema</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">,</span>
<span class="n">additional_create_parameters</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_observed_tables</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">str_table_reference</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="n">schema_side_inputs</span><span class="p">):</span>
<span class="n">destination</span> <span class="o">=</span> <span class="n">element</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="n">callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">):</span>
<span class="n">schema</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">(</span><span class="n">destination</span><span class="p">,</span> <span class="o">*</span><span class="n">schema_side_inputs</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span> <span class="n">vp</span><span class="o">.</span><span class="n">ValueProvider</span><span class="p">):</span>
<span class="n">schema</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="o">.</span><span class="n">get</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">schema</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_create_table_if_needed</span><span class="p">(</span>
<span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span><span class="n">destination</span><span class="p">),</span>
<span class="n">schema</span><span class="p">)</span>
<span class="n">destination</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">get_hashable_destination</span><span class="p">(</span><span class="n">destination</span><span class="p">)</span>
<span class="n">row</span> <span class="o">=</span> <span class="n">element</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">row</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_total_buffered_rows</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">])</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_flush_batch</span><span class="p">(</span><span class="n">destination</span><span class="p">)</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">_total_buffered_rows</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_buffered_rows</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_flush_all_batches</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_flush_all_batches</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_flush_all_batches</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s1">&#39;Attempting to flush to all destinations. Total buffered: </span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_total_buffered_rows</span><span class="p">)</span>
<span class="k">return</span> <span class="n">itertools</span><span class="o">.</span><span class="n">chain</span><span class="p">(</span><span class="o">*</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_flush_batch</span><span class="p">(</span><span class="n">destination</span><span class="p">)</span>
<span class="k">for</span> <span class="n">destination</span> <span class="ow">in</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">]])</span>
<span class="k">def</span> <span class="nf">_flush_batch</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">destination</span><span class="p">):</span>
<span class="c1"># Flush the current batch of rows to BigQuery.</span>
<span class="n">rows</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">]</span>
<span class="n">table_reference</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span><span class="n">destination</span><span class="p">)</span>
<span class="k">if</span> <span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="o">=</span> <span class="n">vp</span><span class="o">.</span><span class="n">RuntimeValueProvider</span><span class="o">.</span><span class="n">get_value</span><span class="p">(</span>
<span class="s1">&#39;project&#39;</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s1">&#39;Flushing data to </span><span class="si">%s</span><span class="s1">. Total </span><span class="si">%s</span><span class="s1"> rows.&#39;</span><span class="p">,</span>
<span class="n">destination</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="n">rows</span><span class="p">))</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="c1"># TODO: Figure out an insertId to make calls idempotent.</span>
<span class="n">passed</span><span class="p">,</span> <span class="n">errors</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">bigquery_wrapper</span><span class="o">.</span><span class="n">insert_rows</span><span class="p">(</span>
<span class="n">project_id</span><span class="o">=</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="n">dataset_id</span><span class="o">=</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="n">table_id</span><span class="o">=</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">,</span>
<span class="n">rows</span><span class="o">=</span><span class="n">rows</span><span class="p">,</span>
<span class="n">skip_invalid_rows</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s2">&quot;Passed: </span><span class="si">%s</span><span class="s2">. Errors are </span><span class="si">%s</span><span class="s2">&quot;</span><span class="p">,</span> <span class="n">passed</span><span class="p">,</span> <span class="n">errors</span><span class="p">)</span>
<span class="n">failed_rows</span> <span class="o">=</span> <span class="p">[</span><span class="n">rows</span><span class="p">[</span><span class="n">entry</span><span class="o">.</span><span class="n">index</span><span class="p">]</span> <span class="k">for</span> <span class="n">entry</span> <span class="ow">in</span> <span class="n">errors</span><span class="p">]</span>
<span class="n">should_retry</span> <span class="o">=</span> <span class="nb">any</span><span class="p">(</span>
<span class="n">bigquery_tools</span><span class="o">.</span><span class="n">RetryStrategy</span><span class="o">.</span><span class="n">should_retry</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_retry_strategy</span><span class="p">,</span> <span class="n">entry</span><span class="o">.</span><span class="n">errors</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">reason</span><span class="p">)</span>
<span class="k">for</span> <span class="n">entry</span> <span class="ow">in</span> <span class="n">errors</span><span class="p">)</span>
<span class="n">rows</span> <span class="o">=</span> <span class="n">failed_rows</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">should_retry</span><span class="p">:</span>
<span class="k">break</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">retry_backoff</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_backoff_calculator</span><span class="p">)</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Sleeping </span><span class="si">%s</span><span class="s1"> seconds before retrying insertion.&#39;</span><span class="p">,</span>
<span class="n">retry_backoff</span><span class="p">)</span>
<span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="n">retry_backoff</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_total_buffered_rows</span> <span class="o">-=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">])</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="p">[</span><span class="n">destination</span><span class="p">]</span>
<span class="k">return</span> <span class="p">[</span><span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span><span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">FAILED_ROWS</span><span class="p">,</span>
<span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value</span><span class="p">(</span>
<span class="p">(</span><span class="n">destination</span><span class="p">,</span> <span class="n">row</span><span class="p">)))</span> <span class="k">for</span> <span class="n">row</span> <span class="ow">in</span> <span class="n">failed_rows</span><span class="p">]</span>
<span class="c1"># Flag to be passed to WriteToBigQuery to force schema autodetection</span>
<span class="n">SCHEMA_AUTODETECT</span> <span class="o">=</span> <span class="s1">&#39;SCHEMA_AUTODETECT&#39;</span>
<div class="viewcode-block" id="WriteToBigQuery"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery">[docs]</a><span class="k">class</span> <span class="nc">WriteToBigQuery</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Write data to BigQuery.</span>
<span class="sd"> This transform receives a PCollection of elements to be inserted into BigQuery</span>
<span class="sd"> tables. The elements would come in as Python dictionaries, or as `TableRow`</span>
<span class="sd"> instances.</span>
<span class="sd"> Known Limitations/Edge Cases:</span>
<span class="sd"> 1. [BEAM-7822] In file loads method of writing to BigQuery,</span>
<span class="sd"> copying from temp_tables to destination_table is not atomic.</span>
<span class="sd"> See: https://issues.apache.org/jira/browse/BEAM-7822</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="WriteToBigQuery.Method"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.Method">[docs]</a> <span class="k">class</span> <span class="nc">Method</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="n">DEFAULT</span> <span class="o">=</span> <span class="s1">&#39;DEFAULT&#39;</span>
<span class="n">STREAMING_INSERTS</span> <span class="o">=</span> <span class="s1">&#39;STREAMING_INSERTS&#39;</span>
<span class="n">FILE_LOADS</span> <span class="o">=</span> <span class="s1">&#39;FILE_LOADS&#39;</span></div>
<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">table</span><span class="p">,</span>
<span class="n">dataset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">project</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">schema</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">create_disposition</span><span class="o">=</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">CREATE_IF_NEEDED</span><span class="p">,</span>
<span class="n">write_disposition</span><span class="o">=</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_APPEND</span><span class="p">,</span>
<span class="n">kms_key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">batch_size</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">max_file_size</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">max_files_per_bundle</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">test_client</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">custom_gcs_temp_location</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">method</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">insert_retry_strategy</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">additional_bq_parameters</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">table_side_inputs</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">schema_side_inputs</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">triggering_frequency</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">validate</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initialize a WriteToBigQuery transform.</span>
<span class="sd"> Args:</span>
<span class="sd"> table (str, callable, ValueProvider): The ID of the table, or a callable</span>
<span class="sd"> that returns it. The ID must contain only letters ``a-z``, ``A-Z``,</span>
<span class="sd"> numbers ``0-9``, or underscores ``_``. If dataset argument is</span>
<span class="sd"> :data:`None` then the table argument must contain the entire table</span>
<span class="sd"> reference specified as: ``&#39;DATASET.TABLE&#39;``</span>
<span class="sd"> or ``&#39;PROJECT:DATASET.TABLE&#39;``. If it&#39;s a callable, it must receive one</span>
<span class="sd"> argument representing an element to be written to BigQuery, and return</span>
<span class="sd"> a TableReference, or a string table name as specified above.</span>
<span class="sd"> Multiple destinations are only supported on Batch pipelines at the</span>
<span class="sd"> moment.</span>
<span class="sd"> dataset (str): The ID of the dataset containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument.</span>
<span class="sd"> project (str): The ID of the project containing this table or</span>
<span class="sd"> :data:`None` if the table reference is specified entirely by the table</span>
<span class="sd"> argument.</span>
<span class="sd"> schema (str,dict,ValueProvider,callable): The schema to be used if the</span>
<span class="sd"> BigQuery table to write has to be created. This can be either specified</span>
<span class="sd"> as a :class:`~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema`. or a `ValueProvider` that has a JSON string,</span>
<span class="sd"> or a python dictionary, or the string or dictionary itself,</span>
<span class="sd"> object or a single string of the form</span>
<span class="sd"> ``&#39;field1:type1,field2:type2,field3:type3&#39;`` that defines a comma</span>
<span class="sd"> separated list of fields. Here ``&#39;type&#39;`` should specify the BigQuery</span>
<span class="sd"> type of the field. Single string based schemas do not support nested</span>
<span class="sd"> fields, repeated fields, or specifying a BigQuery mode for fields</span>
<span class="sd"> (mode will always be set to ``&#39;NULLABLE&#39;``).</span>
<span class="sd"> If a callable, then it should receive a destination (in the form of</span>
<span class="sd"> a TableReference or a string, and return a str, dict or TableSchema.</span>
<span class="sd"> One may also pass ``SCHEMA_AUTODETECT`` here, and BigQuery will try to</span>
<span class="sd"> infer the schema for the files that are being loaded.</span>
<span class="sd"> create_disposition (BigQueryDisposition): A string describing what</span>
<span class="sd"> happens if the table does not exist. Possible values are:</span>
<span class="sd"> * :attr:`BigQueryDisposition.CREATE_IF_NEEDED`: create if does not</span>
<span class="sd"> exist.</span>
<span class="sd"> * :attr:`BigQueryDisposition.CREATE_NEVER`: fail the write if does not</span>
<span class="sd"> exist.</span>
<span class="sd"> write_disposition (BigQueryDisposition): A string describing what happens</span>
<span class="sd"> if the table has already some data. Possible values are:</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_TRUNCATE`: delete existing rows.</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_APPEND`: add to existing rows.</span>
<span class="sd"> * :attr:`BigQueryDisposition.WRITE_EMPTY`: fail the write if table not</span>
<span class="sd"> empty.</span>
<span class="sd"> For streaming pipelines WriteTruncate can not be used.</span>
<span class="sd"> kms_key (str): Experimental. Optional Cloud KMS key name for use when</span>
<span class="sd"> creating new tables.</span>
<span class="sd"> batch_size (int): Number of rows to be written to BQ per streaming API</span>
<span class="sd"> insert. The default is 500.</span>
<span class="sd"> insert.</span>
<span class="sd"> test_client: Override the default bigquery client used for testing.</span>
<span class="sd"> max_file_size (int): The maximum size for a file to be written and then</span>
<span class="sd"> loaded into BigQuery. The default value is 4TB, which is 80% of the</span>
<span class="sd"> limit of 5TB for BigQuery to load any file.</span>
<span class="sd"> max_files_per_bundle(int): The maximum number of files to be concurrently</span>
<span class="sd"> written by a worker. The default here is 20. Larger values will allow</span>
<span class="sd"> writing to multiple destinations without having to reshard - but they</span>
<span class="sd"> increase the memory burden on the workers.</span>
<span class="sd"> custom_gcs_temp_location (str): A GCS location to store files to be used</span>
<span class="sd"> for file loads into BigQuery. By default, this will use the pipeline&#39;s</span>
<span class="sd"> temp_location, but for pipelines whose temp_location is not appropriate</span>
<span class="sd"> for BQ File Loads, users should pass a specific one.</span>
<span class="sd"> method: The method to use to write to BigQuery. It may be</span>
<span class="sd"> STREAMING_INSERTS, FILE_LOADS, or DEFAULT. An introduction on loading</span>
<span class="sd"> data to BigQuery: https://cloud.google.com/bigquery/docs/loading-data.</span>
<span class="sd"> DEFAULT will use STREAMING_INSERTS on Streaming pipelines and</span>
<span class="sd"> FILE_LOADS on Batch pipelines.</span>
<span class="sd"> insert_retry_strategy: The strategy to use when retrying streaming inserts</span>
<span class="sd"> into BigQuery. Options are shown in bigquery_tools.RetryStrategy attrs.</span>
<span class="sd"> additional_bq_parameters (callable): A function that returns a dictionary</span>
<span class="sd"> with additional parameters to pass to BQ when creating / loading data</span>
<span class="sd"> into a table. These can be &#39;timePartitioning&#39;, &#39;clustering&#39;, etc. They</span>
<span class="sd"> are passed directly to the job load configuration. See</span>
<span class="sd"> https://cloud.google.com/bigquery/docs/reference/rest/v2/jobs#configuration.load</span>
<span class="sd"> table_side_inputs (tuple): A tuple with ``AsSideInput`` PCollections to be</span>
<span class="sd"> passed to the table callable (if one is provided).</span>
<span class="sd"> schema_side_inputs: A tuple with ``AsSideInput`` PCollections to be</span>
<span class="sd"> passed to the schema callable (if one is provided).</span>
<span class="sd"> triggering_frequency (int): Every triggering_frequency duration, a</span>
<span class="sd"> BigQuery load job will be triggered for all the data written since</span>
<span class="sd"> the last load job. BigQuery has limits on how many load jobs can be</span>
<span class="sd"> triggered per day, so be careful not to set this duration too low, or</span>
<span class="sd"> you may exceed daily quota. Often this is set to 5 or 10 minutes to</span>
<span class="sd"> ensure that the project stays well under the BigQuery quota.</span>
<span class="sd"> See https://cloud.google.com/bigquery/quota-policy for more information</span>
<span class="sd"> about BigQuery quotas.</span>
<span class="sd"> validate: Indicates whether to perform validation checks on</span>
<span class="sd"> inputs. This parameter is primarily used for testing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="o">=</span> <span class="n">bigquery_tools</span><span class="o">.</span><span class="n">parse_table_reference</span><span class="p">(</span>
<span class="n">table</span><span class="p">,</span> <span class="n">dataset</span><span class="p">,</span> <span class="n">project</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span> <span class="o">=</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">validate_create</span><span class="p">(</span>
<span class="n">create_disposition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span> <span class="o">=</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">validate_write</span><span class="p">(</span>
<span class="n">write_disposition</span><span class="p">)</span>
<span class="k">if</span> <span class="n">schema</span> <span class="o">==</span> <span class="n">SCHEMA_AUTODETECT</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">=</span> <span class="n">schema</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">=</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">get_dict_table_schema</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span> <span class="o">=</span> <span class="n">batch_size</span>
<span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span> <span class="o">=</span> <span class="n">kms_key</span>
<span class="bp">self</span><span class="o">.</span><span class="n">test_client</span> <span class="o">=</span> <span class="n">test_client</span>
<span class="c1"># TODO(pabloem): Consider handling ValueProvider for this location.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">custom_gcs_temp_location</span> <span class="o">=</span> <span class="n">custom_gcs_temp_location</span>
<span class="bp">self</span><span class="o">.</span><span class="n">max_file_size</span> <span class="o">=</span> <span class="n">max_file_size</span>
<span class="bp">self</span><span class="o">.</span><span class="n">max_files_per_bundle</span> <span class="o">=</span> <span class="n">max_files_per_bundle</span>
<span class="bp">self</span><span class="o">.</span><span class="n">method</span> <span class="o">=</span> <span class="n">method</span> <span class="ow">or</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">DEFAULT</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggering_frequency</span> <span class="o">=</span> <span class="n">triggering_frequency</span>
<span class="bp">self</span><span class="o">.</span><span class="n">insert_retry_strategy</span> <span class="o">=</span> <span class="n">insert_retry_strategy</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_validate</span> <span class="o">=</span> <span class="n">validate</span>
<span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span> <span class="o">=</span> <span class="n">additional_bq_parameters</span> <span class="ow">or</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_side_inputs</span> <span class="o">=</span> <span class="n">table_side_inputs</span> <span class="ow">or</span> <span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schema_side_inputs</span> <span class="o">=</span> <span class="n">schema_side_inputs</span> <span class="ow">or</span> <span class="p">()</span>
<div class="viewcode-block" id="WriteToBigQuery.get_table_schema_from_string"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.get_table_schema_from_string">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">get_table_schema_from_string</span><span class="p">(</span><span class="n">schema</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Transform the string table schema into a</span>
<span class="sd"> :class:`~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema` instance.</span>
<span class="sd"> Args:</span>
<span class="sd"> schema (str): The sting schema to be used if the BigQuery table to write</span>
<span class="sd"> has to be created.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema:</span>
<span class="sd"> The schema to be used if the BigQuery table to write has to be created</span>
<span class="sd"> but in the :class:`~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema` format.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">table_schema</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">()</span>
<span class="n">schema_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">s</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span> <span class="k">for</span> <span class="n">s</span> <span class="ow">in</span> <span class="n">schema</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;,&#39;</span><span class="p">)]</span>
<span class="k">for</span> <span class="n">field_and_type</span> <span class="ow">in</span> <span class="n">schema_list</span><span class="p">:</span>
<span class="n">field_name</span><span class="p">,</span> <span class="n">field_type</span> <span class="o">=</span> <span class="n">field_and_type</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;:&#39;</span><span class="p">)</span>
<span class="n">field_schema</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableFieldSchema</span><span class="p">()</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">field_name</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">type</span> <span class="o">=</span> <span class="n">field_type</span>
<span class="n">field_schema</span><span class="o">.</span><span class="n">mode</span> <span class="o">=</span> <span class="s1">&#39;NULLABLE&#39;</span>
<span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">field_schema</span><span class="p">)</span>
<span class="k">return</span> <span class="n">table_schema</span></div>
<div class="viewcode-block" id="WriteToBigQuery.table_schema_to_dict"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.table_schema_to_dict">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">table_schema_to_dict</span><span class="p">(</span><span class="n">table_schema</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Create a dictionary representation of table schema for serialization</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">get_table_field</span><span class="p">(</span><span class="n">field</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Create a dictionary representation of a table field</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">result</span> <span class="o">=</span> <span class="p">{}</span>
<span class="n">result</span><span class="p">[</span><span class="s1">&#39;name&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">name</span>
<span class="n">result</span><span class="p">[</span><span class="s1">&#39;type&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span>
<span class="n">result</span><span class="p">[</span><span class="s1">&#39;mode&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">field</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">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="s1">&#39;description&#39;</span><span class="p">)</span> <span class="ow">and</span> <span class="n">field</span><span class="o">.</span><span class="n">description</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">result</span><span class="p">[</span><span class="s1">&#39;description&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">description</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="s1">&#39;fields&#39;</span><span class="p">)</span> <span class="ow">and</span> <span class="n">field</span><span class="o">.</span><span class="n">fields</span><span class="p">:</span>
<span class="n">result</span><span class="p">[</span><span class="s1">&#39;fields&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="p">[</span><span class="n">get_table_field</span><span class="p">(</span><span class="n">f</span><span class="p">)</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">field</span><span class="o">.</span><span class="n">fields</span><span class="p">]</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">table_schema</span><span class="p">,</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Table schema must be of the type bigquery.TableSchema&quot;</span><span class="p">)</span>
<span class="n">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="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">table_schema</span><span class="o">.</span><span class="n">fields</span><span class="p">:</span>
<span class="n">schema</span><span class="p">[</span><span class="s1">&#39;fields&#39;</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">get_table_field</span><span class="p">(</span><span class="n">field</span><span class="p">))</span>
<span class="k">return</span> <span class="n">schema</span></div>
<div class="viewcode-block" id="WriteToBigQuery.get_dict_table_schema"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">get_dict_table_schema</span><span class="p">(</span><span class="n">schema</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Transform the table schema into a dictionary instance.</span>
<span class="sd"> Args:</span>
<span class="sd"> schema (~apache_beam.io.gcp.internal.clients.bigquery.\</span>
<span class="sd">bigquery_v2_messages.TableSchema):</span>
<span class="sd"> The schema to be used if the BigQuery table to write has to be created.</span>
<span class="sd"> This can either be a dict or string or in the TableSchema format.</span>
<span class="sd"> Returns:</span>
<span class="sd"> Dict[str, Any]: The schema to be used if the BigQuery table to write has</span>
<span class="sd"> to be created but in the dictionary format.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="p">(</span><span class="nb">dict</span><span class="p">,</span> <span class="n">vp</span><span class="o">.</span><span class="n">ValueProvider</span><span class="p">))</span> <span class="ow">or</span>
<span class="n">callable</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span> <span class="ow">or</span>
<span class="n">schema</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">schema</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">unicode</span><span class="p">)):</span>
<span class="n">table_schema</span> <span class="o">=</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">get_table_schema_from_string</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span>
<span class="k">return</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">table_schema_to_dict</span><span class="p">(</span><span class="n">table_schema</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">schema</span><span class="p">,</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">):</span>
<span class="k">return</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">table_schema_to_dict</span><span class="p">(</span><span class="n">schema</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">schema</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_compute_method</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">experiments</span><span class="p">,</span> <span class="n">is_streaming_pipeline</span><span class="p">):</span>
<span class="c1"># If the new BQ sink is not activated for experiment flags, then we use</span>
<span class="c1"># streaming inserts by default (it gets overridden in dataflow_runner.py).</span>
<span class="k">if</span> <span class="s1">&#39;use_beam_bq_sink&#39;</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">experiments</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">STREAMING_INSERTS</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">method</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">DEFAULT</span> <span class="ow">and</span> <span class="n">is_streaming_pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">STREAMING_INSERTS</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">method</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">DEFAULT</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">is_streaming_pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">FILE_LOADS</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">method</span>
<div class="viewcode-block" id="WriteToBigQuery.expand"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.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="n">p</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span>
<span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="p">,</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableReference</span><span class="p">)</span>
<span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span>
<span class="n">GoogleCloudOptions</span><span class="p">)</span><span class="o">.</span><span class="n">project</span>
<span class="n">experiments</span> <span class="o">=</span> <span class="n">p</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">DebugOptions</span><span class="p">)</span><span class="o">.</span><span class="n">experiments</span> <span class="ow">or</span> <span class="p">[]</span>
<span class="c1"># TODO(pabloem): Use a different method to determine if streaming or batch.</span>
<span class="n">is_streaming_pipeline</span> <span class="o">=</span> <span class="n">p</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span><span class="o">.</span><span class="n">streaming</span>
<span class="n">method_to_use</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compute_method</span><span class="p">(</span><span class="n">experiments</span><span class="p">,</span> <span class="n">is_streaming_pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">method_to_use</span> <span class="o">==</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">STREAMING_INSERTS</span>
<span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</span> <span class="o">==</span> <span class="n">SCHEMA_AUTODETECT</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Schema auto-detection is not supported for streaming &#39;</span>
<span class="s1">&#39;inserts into BigQuery. Only for File Loads.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">method_to_use</span> <span class="o">==</span> <span class="n">WriteToBigQuery</span><span class="o">.</span><span class="n">Method</span><span class="o">.</span><span class="n">STREAMING_INSERTS</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggering_frequency</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;triggering_frequency can only be used with &#39;</span>
<span class="s1">&#39;FILE_LOADS method of writing to BigQuery.&#39;</span><span class="p">)</span>
<span class="n">bigquery_write_fn</span> <span class="o">=</span> <span class="n">BigQueryWriteFn</span><span class="p">(</span>
<span class="n">schema</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span>
<span class="n">batch_size</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span><span class="p">,</span>
<span class="n">create_disposition</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">,</span>
<span class="n">write_disposition</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">,</span>
<span class="n">kms_key</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">kms_key</span><span class="p">,</span>
<span class="n">retry_strategy</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">insert_retry_strategy</span><span class="p">,</span>
<span class="n">test_client</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">test_client</span><span class="p">,</span>
<span class="n">additional_bq_parameters</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span><span class="p">)</span>
<span class="n">outputs</span> <span class="o">=</span> <span class="p">(</span><span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;AppendDestination&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span>
<span class="n">bigquery_tools</span><span class="o">.</span><span class="n">AppendDestinationsFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="p">),</span>
<span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">table_side_inputs</span><span class="p">)</span>
<span class="o">|</span> <span class="s1">&#39;StreamInsertRows&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">ParDo</span><span class="p">(</span>
<span class="n">bigquery_write_fn</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">schema_side_inputs</span><span class="p">)</span><span class="o">.</span><span class="n">with_outputs</span><span class="p">(</span>
<span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">FAILED_ROWS</span><span class="p">,</span> <span class="n">main</span><span class="o">=</span><span class="s1">&#39;main&#39;</span><span class="p">))</span>
<span class="k">return</span> <span class="p">{</span><span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">FAILED_ROWS</span><span class="p">:</span> <span class="n">outputs</span><span class="p">[</span><span class="n">BigQueryWriteFn</span><span class="o">.</span><span class="n">FAILED_ROWS</span><span class="p">]}</span>
<span class="k">else</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.gcp</span> <span class="k">import</span> <span class="n">bigquery_file_loads</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">bigquery_file_loads</span><span class="o">.</span><span class="n">BigQueryBatchFileLoads</span><span class="p">(</span>
<span class="n">destination</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="p">,</span>
<span class="n">schema</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">,</span>
<span class="n">create_disposition</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">,</span>
<span class="n">write_disposition</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">,</span>
<span class="n">triggering_frequency</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">triggering_frequency</span><span class="p">,</span>
<span class="n">max_file_size</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">max_file_size</span><span class="p">,</span>
<span class="n">max_files_per_bundle</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">max_files_per_bundle</span><span class="p">,</span>
<span class="n">custom_gcs_temp_location</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">custom_gcs_temp_location</span><span class="p">,</span>
<span class="n">test_client</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">test_client</span><span class="p">,</span>
<span class="n">table_side_inputs</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">table_side_inputs</span><span class="p">,</span>
<span class="n">schema_side_inputs</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">schema_side_inputs</span><span class="p">,</span>
<span class="n">additional_bq_parameters</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">additional_bq_parameters</span><span class="p">,</span>
<span class="n">validate</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_validate</span><span class="p">,</span>
<span class="n">is_streaming_pipeline</span><span class="o">=</span><span class="n">is_streaming_pipeline</span><span class="p">)</span></div>
<div class="viewcode-block" id="WriteToBigQuery.display_data"><a class="viewcode-back" href="../../../../apache_beam.io.gcp.bigquery.html#apache_beam.io.gcp.bigquery.WriteToBigQuery.display_data">[docs]</a> <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="n">res</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tableSpec</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span>
<span class="n">tableSpec</span><span class="p">)</span>
<span class="n">res</span><span class="p">[</span><span class="s1">&#39;table&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="n">tableSpec</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Table&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">res</span></div></div>
</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>