| |
| |
| <!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 — 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> »</li> |
| |
| <li><a href="../../../index.html">Module code</a> »</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 "License"); 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 "AS IS" 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">"""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. 'month:STRING,event_count:INTEGER').</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 | 'VeryBig' >> beam.io.Read(beam.io.BigQuerySource()</span> |
| <span class="sd"> side_table = pipeline | 'NotBig' >> beam.io.Read(beam.io.BigQuerySource()</span> |
| <span class="sd"> results = (</span> |
| <span class="sd"> main_table</span> |
| <span class="sd"> | 'ProcessData' >> 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 'side input' 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='SELECT year, mean_temp FROM samples.weather_stations'))</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">**Time partitioned tables**</span> |
| |
| <span class="sd">BigQuery sink currently does not fully support writing to BigQuery</span> |
| <span class="sd">time partitioned tables. But writing to a *single* partition may work if</span> |
| <span class="sd">that does not involve creating a new table (for example, when writing to an</span> |
| <span class="sd">existing table with `create_disposition=CREATE_NEVER` and</span> |
| <span class="sd">`write_disposition=WRITE_APPEND`).</span> |
| <span class="sd">BigQuery source supports reading from a single time partition with the partition</span> |
| <span class="sd">decorator specified as a part of the table identifier.</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, 'field', 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 'name' and 'type'. Common values for</span> |
| <span class="sd"> the type attribute are: 'STRING', 'INTEGER', 'FLOAT', 'BOOLEAN', 'NUMERIC'.</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, 'f', 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"> 'v', 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">"""</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">datetime</span> |
| <span class="kn">import</span> <span class="nn">decimal</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">re</span> |
| <span class="kn">import</span> <span class="nn">time</span> |
| <span class="kn">import</span> <span class="nn">uuid</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">iteritems</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">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.internal.gcp</span> <span class="k">import</span> <span class="n">auth</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.internal.clients</span> <span class="k">import</span> <span class="n">bigquery</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.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.utils</span> <span class="k">import</span> <span class="n">retry</span> |
| |
| <span class="c1"># Protect against environments where bigquery library is not available.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">apitools.base.py.exceptions</span> <span class="k">import</span> <span class="n">HttpError</span> |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| <span class="k">pass</span> |
| <span class="c1"># pylint: enable=wrong-import-order, wrong-import-position</span> |
| |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'TableRowJsonCoder'</span><span class="p">,</span> |
| <span class="s1">'BigQueryDisposition'</span><span class="p">,</span> |
| <span class="s1">'BigQuerySource'</span><span class="p">,</span> |
| <span class="s1">'BigQuerySink'</span><span class="p">,</span> |
| <span class="s1">'WriteToBigQuery'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="n">JSON_COMPLIANCE_ERROR</span> <span class="o">=</span> <span class="s1">'NAN, INF and -INF values are not JSON compliant.'</span> |
| <span class="n">MAX_RETRIES</span> <span class="o">=</span> <span class="mi">3</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">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">obj</span><span class="p">,</span> <span class="n">decimal</span><span class="o">.</span><span class="n">Decimal</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">str</span><span class="p">(</span><span class="n">obj</span><span class="p">)</span> |
| <span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span> |
| <span class="s2">"Object of type '</span><span class="si">%s</span><span class="s2">' is not JSON serializable"</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">obj</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">RowAsDictJsonCoder</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">"""A coder for a table row (represented as a dict) to/from a JSON string.</span> |
| |
| <span class="sd"> This is the default coder for sources and sinks if the coder argument is not</span> |
| <span class="sd"> specified.</span> |
| <span class="sd"> """</span> |
| |
| <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="c1"># The normal error when dumping NAN/INF values is:</span> |
| <span class="c1"># ValueError: Out of range float values are not JSON compliant</span> |
| <span class="c1"># This code will catch this error to emit an error that explains</span> |
| <span class="c1"># to the programmer that they have used NAN/INF values.</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">table_row</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">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">'</span><span class="si">%s</span><span class="s1">. </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">e</span><span class="p">,</span> <span class="n">JSON_COMPLIANCE_ERROR</span><span class="p">))</span> |
| |
| <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="k">return</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> |
| |
| |
| <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">"""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"> """</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">'The TableRowJsonCoder requires a table schema for '</span> |
| <span class="s1">'encoding operations. Please specify a table_schema argument.'</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">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">'</span><span class="si">%s</span><span class="s1">. </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">e</span><span class="p">,</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> |
| |
| |
| <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="sd">"""Parse the Table Schema provided as string.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> schema_string: String serialized table schema, should be a valid JSON.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A TableSchema of the BigQuery export from either the Query or the Table.</span> |
| <span class="sd"> """</span> |
| <span class="n">json_schema</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">schema_string</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_parse_schema_field</span><span class="p">(</span><span class="n">field</span><span class="p">):</span> |
| <span class="sd">"""Parse a single schema field from dictionary.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> field: Dictionary object containing serialized schema.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A TableFieldSchema for a single column in BigQuery.</span> |
| <span class="sd"> """</span> |
| <span class="n">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">schema</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">field</span><span class="p">[</span><span class="s1">'name'</span><span class="p">]</span> |
| <span class="n">schema</span><span class="o">.</span><span class="n">type</span> <span class="o">=</span> <span class="n">field</span><span class="p">[</span><span class="s1">'type'</span><span class="p">]</span> |
| <span class="k">if</span> <span class="s1">'mode'</span> <span class="ow">in</span> <span class="n">field</span><span class="p">:</span> |
| <span class="n">schema</span><span class="o">.</span><span class="n">mode</span> <span class="o">=</span> <span class="n">field</span><span class="p">[</span><span class="s1">'mode'</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="n">mode</span> <span class="o">=</span> <span class="s1">'NULLABLE'</span> |
| <span class="k">if</span> <span class="s1">'description'</span> <span class="ow">in</span> <span class="n">field</span><span class="p">:</span> |
| <span class="n">schema</span><span class="o">.</span><span class="n">description</span> <span class="o">=</span> <span class="n">field</span><span class="p">[</span><span class="s1">'description'</span><span class="p">]</span> |
| <span class="k">if</span> <span class="s1">'fields'</span> <span class="ow">in</span> <span class="n">field</span><span class="p">:</span> |
| <span class="n">schema</span><span class="o">.</span><span class="n">fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">_parse_schema_field</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">field</span><span class="p">[</span><span class="s1">'fields'</span><span class="p">]]</span> |
| <span class="k">return</span> <span class="n">schema</span> |
| |
| <span class="n">fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">_parse_schema_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">json_schema</span><span class="p">[</span><span class="s1">'fields'</span><span class="p">]]</span> |
| <span class="k">return</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">(</span><span class="n">fields</span><span class="o">=</span><span class="n">fields</span><span class="p">)</span> |
| |
| |
| <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">"""Class holding standard strings used for create and write dispositions."""</span> |
| |
| <span class="n">CREATE_NEVER</span> <span class="o">=</span> <span class="s1">'CREATE_NEVER'</span> |
| <span class="n">CREATE_IF_NEEDED</span> <span class="o">=</span> <span class="s1">'CREATE_IF_NEEDED'</span> |
| <span class="n">WRITE_TRUNCATE</span> <span class="o">=</span> <span class="s1">'WRITE_TRUNCATE'</span> |
| <span class="n">WRITE_APPEND</span> <span class="o">=</span> <span class="s1">'WRITE_APPEND'</span> |
| <span class="n">WRITE_EMPTY</span> <span class="o">=</span> <span class="s1">'WRITE_EMPTY'</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">'Invalid create disposition </span><span class="si">%s</span><span class="s1">. Expecting </span><span class="si">%s</span><span class="s1">'</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">'Invalid write disposition </span><span class="si">%s</span><span class="s1">. Expecting </span><span class="si">%s</span><span class="s1">'</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="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="sd">"""Parses a table reference into a (project, dataset, table) tuple.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> table: 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 dataset argument is None</span> |
| <span class="sd"> then the table argument must contain the entire table reference:</span> |
| <span class="sd"> 'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'. This argument can be a</span> |
| <span class="sd"> bigquery.TableReference instance in which case dataset and project are</span> |
| <span class="sd"> ignored and the reference is returned as a result. Additionally, for date</span> |
| <span class="sd"> partitioned tables, appending '$YYYYmmdd' to the table name is supported,</span> |
| <span class="sd"> e.g. 'DATASET.TABLE$YYYYmmdd'.</span> |
| <span class="sd"> dataset: The ID of the dataset containing this table or null if the table</span> |
| <span class="sd"> reference is specified entirely by the table argument.</span> |
| <span class="sd"> project: The ID of the project containing this table or null if the table</span> |
| <span class="sd"> reference is specified entirely by the table (and possibly dataset)</span> |
| <span class="sd"> argument.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A bigquery.TableReference object. The object has the following attributes:</span> |
| <span class="sd"> projectId, datasetId, and tableId.</span> |
| |
| <span class="sd"> Raises:</span> |
| <span class="sd"> ValueError: if the table reference as a string does not match the expected</span> |
| <span class="sd"> format.</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">table</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="k">return</span> <span class="n">table</span> |
| |
| <span class="n">table_reference</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">TableReference</span><span class="p">()</span> |
| <span class="c1"># If dataset argument is not specified, the expectation is that the</span> |
| <span class="c1"># table argument will contain a full table reference instead of just a</span> |
| <span class="c1"># table name.</span> |
| <span class="k">if</span> <span class="n">dataset</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">match</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span> |
| <span class="sa">r</span><span class="s1">'^((?P<project>.+):)?(?P<dataset>\w+)\.(?P<table>[\w\$]+)$'</span><span class="p">,</span> <span class="n">table</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">match</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Expected a table reference (PROJECT:DATASET.TABLE or '</span> |
| <span class="s1">'DATASET.TABLE) instead of </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="n">table</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">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s1">'project'</span><span class="p">)</span> |
| <span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s1">'dataset'</span><span class="p">)</span> |
| <span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s1">'table'</span><span class="p">)</span> |
| <span class="k">else</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">project</span> |
| <span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span> <span class="o">=</span> <span class="n">dataset</span> |
| <span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span> <span class="o">=</span> <span class="n">table</span> |
| <span class="k">return</span> <span class="n">table_reference</span> |
| |
| |
| <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">"""A source based on a BigQuery table."""</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="sd">"""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"> ``'DATASET.TABLE'`` or ``'PROJECT:DATASET.TABLE'``.</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.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'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'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"> 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"> """</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">'Google Cloud IO not available, '</span> |
| <span class="s1">'please install apache_beam[gcp]'</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">'Both a BigQuery table and a query were specified.'</span> |
| <span class="s1">' Please specify only one of these.'</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">'A BigQuery table or a query must be specified'</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">_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">RowAsDictJsonCoder</span><span class="p">()</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">'query'</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">'Query'</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">'</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">'</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">'</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">'</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">'table'</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">'Table'</span><span class="p">)}</span> |
| |
| <span class="n">res</span><span class="p">[</span><span class="s1">'validation'</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">'Validation Enabled'</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">"""Source format name required for remote execution."""</span> |
| <span class="k">return</span> <span class="s1">'bigquery'</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">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></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="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">"""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"> """</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="sd">"""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: ``'DATASET.TABLE'`` or</span> |
| <span class="sd"> ``'PROJECT:DATASET.TABLE'``.</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"> ``'field1:type1,field2:type2,field3:type3'`` that defines a comma</span> |
| <span class="sd"> separated list of fields. Here ``'type'`` 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 ``'NULLABLE'``).</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.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"> 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"> """</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">'Google Cloud IO not available, '</span> |
| <span class="s1">'please install apache_beam[gcp]'</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">_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">' '</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">','</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">':'</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">'NULLABLE'</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">'Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.'</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">RowAsDictJsonCoder</span><span class="p">()</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">'</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">'</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">'</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">'</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">'table'</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">'Table'</span><span class="p">)</span> |
| |
| <span class="n">res</span><span class="p">[</span><span class="s1">'validation'</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">"Validation Enabled"</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">"""Returns the TableSchema associated with the sink as a JSON string."""</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">"""Returns a list of TableFieldSchema objects as a list of dicts."""</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">'name'</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">'type'</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">'description'</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">'mode'</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">'record'</span><span class="p">:</span> |
| <span class="n">fs</span><span class="p">[</span><span class="s1">'fields'</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">'fields'</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">"""Sink format name required for remote execution."""</span> |
| <span class="k">return</span> <span class="s1">'bigquery'</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">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="c1"># -----------------------------------------------------------------------------</span> |
| <span class="c1"># BigQueryReader, BigQueryWriter.</span> |
| |
| |
| <span class="k">class</span> <span class="nc">BigQueryReader</span><span class="p">(</span><span class="n">dataflow_io</span><span class="o">.</span><span class="n">NativeSourceReader</span><span class="p">):</span> |
| <span class="sd">"""A reader for a BigQuery source."""</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">source</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">use_legacy_sql</span><span class="o">=</span><span class="kc">True</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="bp">self</span><span class="o">.</span><span class="n">source</span> <span class="o">=</span> <span class="n">source</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">test_bigquery_client</span> <span class="o">=</span> <span class="n">test_bigquery_client</span> |
| <span class="k">if</span> <span class="n">auth</span><span class="o">.</span><span class="n">is_running_in_gce</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span> <span class="o">=</span> <span class="n">auth</span><span class="o">.</span><span class="n">executing_project</span> |
| <span class="k">elif</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">source</span><span class="p">,</span> <span class="s1">'pipeline_options'</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">source</span><span class="o">.</span><span class="n">pipeline_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="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="c1"># TODO(silviuc): Try to automatically get it from gcloud config info.</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span> <span class="ow">and</span> <span class="n">test_bigquery_client</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Missing executing project information. Please use the --project '</span> |
| <span class="s1">'command line option to specify it.'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">row_as_dict</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">coder</span><span class="p">,</span> <span class="n">RowAsDictJsonCoder</span><span class="p">)</span> |
| <span class="c1"># Schema for the rows being read by the reader. It is initialized the</span> |
| <span class="c1"># first time something gets read from the table. It is not required</span> |
| <span class="c1"># for reading the field values in each row but could be useful for</span> |
| <span class="c1"># getting additional details.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">schema</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="n">use_legacy_sql</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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</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="c1"># If table schema did not define a project we default to executing</span> |
| <span class="c1"># project.</span> |
| <span class="n">project_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">project_id</span><span class="p">:</span> |
| <span class="n">project_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">query</span> <span class="o">=</span> <span class="s1">'SELECT * FROM [</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">];'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">project_id</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">source</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">source</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">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</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="bp">self</span><span class="o">.</span><span class="n">query</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">query</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Enforce the "modes" enforced by BigQuerySource.__init__.</span> |
| <span class="c1"># If this exception has been raised, the BigQuerySource "modes" have</span> |
| <span class="c1"># changed and this method will need to be updated as well.</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">"BigQuerySource must have either a table or query"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_get_source_location</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Get the source location (e.g. ``"EU"`` or ``"US"``) from either</span> |
| |
| <span class="sd"> - :data:`source.table_reference`</span> |
| <span class="sd"> or</span> |
| <span class="sd"> - The first referenced table in :data:`source.query`</span> |
| |
| <span class="sd"> See Also:</span> |
| <span class="sd"> - :meth:`BigQueryWrapper.get_query_location`</span> |
| <span class="sd"> - :meth:`BigQueryWrapper.get_table_location`</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> Optional[str]: The source location, if any.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</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">tr</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">get_table_location</span><span class="p">(</span> |
| <span class="n">tr</span><span class="o">.</span><span class="n">projectId</span> <span class="k">if</span> <span class="n">tr</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="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span><span class="p">,</span> |
| <span class="n">tr</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span> <span class="n">tr</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="c1"># It's a query source</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">get_query_location</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">query</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">use_legacy_sql</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">__enter__</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">client</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_bigquery_client</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">create_temporary_dataset</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span><span class="p">,</span> <span class="n">location</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_get_source_location</span><span class="p">())</span> |
| <span class="k">return</span> <span class="bp">self</span> |
| |
| <span class="k">def</span> <span class="nf">__exit__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">exception_type</span><span class="p">,</span> <span class="n">exception_value</span><span class="p">,</span> <span class="n">traceback</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">clean_up_temporary_dataset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">__iter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">rows</span><span class="p">,</span> <span class="n">schema</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">run_query</span><span class="p">(</span> |
| <span class="n">project_id</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">executing_project</span><span class="p">,</span> <span class="n">query</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">query</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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">schema</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">schema</span> <span class="o">=</span> <span class="n">schema</span> |
| <span class="k">for</span> <span class="n">row</span> <span class="ow">in</span> <span class="n">rows</span><span class="p">:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">row_as_dict</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">convert_row_to_dict</span><span class="p">(</span><span class="n">row</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">yield</span> <span class="n">row</span> |
| |
| |
| <span class="k">class</span> <span class="nc">BigQueryWriter</span><span class="p">(</span><span class="n">dataflow_io</span><span class="o">.</span><span class="n">NativeSinkWriter</span><span class="p">):</span> |
| <span class="sd">"""The sink writer for a BigQuerySink."""</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">sink</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="bp">self</span><span class="o">.</span><span class="n">sink</span> <span class="o">=</span> <span class="n">sink</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">test_bigquery_client</span> <span class="o">=</span> <span class="n">test_bigquery_client</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">row_as_dict</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">coder</span><span class="p">,</span> <span class="n">RowAsDictJsonCoder</span><span class="p">)</span> |
| <span class="c1"># Buffer used to batch written rows so we reduce communication with the</span> |
| <span class="c1"># BigQuery service.</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">rows_buffer_flush_threshold</span> <span class="o">=</span> <span class="n">buffer_size</span> <span class="ow">or</span> <span class="mi">1000</span> |
| <span class="c1"># Figure out the project, dataset, and table used for the sink.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> |
| |
| <span class="c1"># If table schema did not define a project we default to executing project.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">sink</span><span class="p">,</span> <span class="s1">'pipeline_options'</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">sink</span><span class="o">.</span><span class="n">pipeline_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="p">)</span> |
| |
| <span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span> |
| |
| <span class="k">def</span> <span class="nf">_flush_rows_buffer</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">rows_buffer</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">'Writing </span><span class="si">%d</span><span class="s1"> rows to </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"> table.'</span><span class="p">,</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="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">)</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">client</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="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="n">table_id</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</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="bp">self</span><span class="o">.</span><span class="n">rows_buffer</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">passed</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">'Could not successfully insert rows to BigQuery'</span> |
| <span class="s1">' table [</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">]. Errors: </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</span> <span class="n">errors</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">__enter__</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">client</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_bigquery_client</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">get_or_create_table</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_schema</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span> |
| |
| <span class="k">def</span> <span class="nf">__exit__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">exception_type</span><span class="p">,</span> <span class="n">exception_value</span><span class="p">,</span> <span class="n">traceback</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_flush_rows_buffer</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">Write</span><span class="p">(</span><span class="bp">self</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">rows_buffer</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="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="o">></span> <span class="bp">self</span><span class="o">.</span><span class="n">rows_buffer_flush_threshold</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_flush_rows_buffer</span><span class="p">()</span> |
| |
| |
| <span class="c1"># -----------------------------------------------------------------------------</span> |
| <span class="c1"># BigQueryWrapper.</span> |
| |
| |
| <span class="k">class</span> <span class="nc">BigQueryWrapper</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="sd">"""BigQuery client wrapper with utilities for querying.</span> |
| |
| <span class="sd"> The wrapper is used to organize all the BigQuery integration points and</span> |
| <span class="sd"> offer a common place where retry logic for failures can be controlled.</span> |
| <span class="sd"> In addition it offers various functions used both in sources and sinks</span> |
| <span class="sd"> (e.g., find and create tables, query a table, etc.).</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">TEMP_TABLE</span> <span class="o">=</span> <span class="s1">'temp_table_'</span> |
| <span class="n">TEMP_DATASET</span> <span class="o">=</span> <span class="s1">'temp_dataset_'</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">client</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span> <span class="o">=</span> <span class="n">client</span> <span class="ow">or</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryV2</span><span class="p">(</span> |
| <span class="n">credentials</span><span class="o">=</span><span class="n">auth</span><span class="o">.</span><span class="n">get_service_credentials</span><span class="p">())</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_unique_row_id</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="c1"># For testing scenarios where we pass in a client we do not want a</span> |
| <span class="c1"># randomized prefix for row IDs.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_row_id_prefix</span> <span class="o">=</span> <span class="s1">''</span> <span class="k">if</span> <span class="n">client</span> <span class="k">else</span> <span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_temporary_table_suffix</span> <span class="o">=</span> <span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span><span class="o">.</span><span class="n">hex</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">unique_row_id</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Returns a unique row ID (str) used to avoid multiple insertions.</span> |
| |
| <span class="sd"> If the row ID is provided, BigQuery will make a best effort to not insert</span> |
| <span class="sd"> the same row multiple times for fail and retry scenarios in which the insert</span> |
| <span class="sd"> request may be issued several times. This comes into play for sinks executed</span> |
| <span class="sd"> in a local runner.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> a unique row ID string</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_unique_row_id</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">_</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_row_id_prefix</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_unique_row_id</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_get_temp_table</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_parse_table_reference</span><span class="p">(</span> |
| <span class="n">table</span><span class="o">=</span><span class="n">BigQueryWrapper</span><span class="o">.</span><span class="n">TEMP_TABLE</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_temporary_table_suffix</span><span class="p">,</span> |
| <span class="n">dataset</span><span class="o">=</span><span class="n">BigQueryWrapper</span><span class="o">.</span><span class="n">TEMP_DATASET</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_temporary_table_suffix</span><span class="p">,</span> |
| <span class="n">project</span><span class="o">=</span><span class="n">project_id</span><span class="p">)</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">get_query_location</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">query</span><span class="p">,</span> <span class="n">use_legacy_sql</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Get the location of tables referenced in a query.</span> |
| |
| <span class="sd"> This method returns the location of the first referenced table in the query</span> |
| <span class="sd"> and depends on the BigQuery service to provide error handling for</span> |
| <span class="sd"> queries that reference tables in multiple locations.</span> |
| <span class="sd"> """</span> |
| <span class="n">reference</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">JobReference</span><span class="p">(</span><span class="n">jobId</span><span class="o">=</span><span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span><span class="o">.</span><span class="n">hex</span><span class="p">,</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">)</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryJobsInsertRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> |
| <span class="n">job</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">Job</span><span class="p">(</span> |
| <span class="n">configuration</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">JobConfiguration</span><span class="p">(</span> |
| <span class="n">dryRun</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">query</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">JobConfigurationQuery</span><span class="p">(</span> |
| <span class="n">query</span><span class="o">=</span><span class="n">query</span><span class="p">,</span> |
| <span class="n">useLegacySql</span><span class="o">=</span><span class="n">use_legacy_sql</span><span class="p">,</span> |
| <span class="p">)),</span> |
| <span class="n">jobReference</span><span class="o">=</span><span class="n">reference</span><span class="p">))</span> |
| |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">jobs</span><span class="o">.</span><span class="n">Insert</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">response</span><span class="o">.</span><span class="n">statistics</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># This behavior is only expected in tests</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span> |
| <span class="s2">"Unable to get location, missing response.statistics. Query: </span><span class="si">%s</span><span class="s2">"</span><span class="p">,</span> |
| <span class="n">query</span><span class="p">)</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| |
| <span class="n">referenced_tables</span> <span class="o">=</span> <span class="n">response</span><span class="o">.</span><span class="n">statistics</span><span class="o">.</span><span class="n">query</span><span class="o">.</span><span class="n">referencedTables</span> |
| <span class="k">if</span> <span class="n">referenced_tables</span><span class="p">:</span> <span class="c1"># Guards against both non-empty and non-None</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="n">referenced_tables</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="n">location</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_table_location</span><span class="p">(</span> |
| <span class="n">table</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span> |
| <span class="n">table</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span> |
| <span class="n">table</span><span class="o">.</span><span class="n">tableId</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="s2">"Using location </span><span class="si">%r</span><span class="s2"> from table </span><span class="si">%r</span><span class="s2"> referenced by query </span><span class="si">%s</span><span class="s2">"</span><span class="p">,</span> |
| <span class="n">location</span><span class="p">,</span> <span class="n">table</span><span class="p">,</span> <span class="n">query</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">location</span> |
| |
| <span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="s2">"Query </span><span class="si">%s</span><span class="s2"> does not reference any tables."</span><span class="p">,</span> <span class="n">query</span><span class="p">)</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_start_query_job</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">query</span><span class="p">,</span> <span class="n">use_legacy_sql</span><span class="p">,</span> <span class="n">flatten_results</span><span class="p">,</span> |
| <span class="n">job_id</span><span class="p">,</span> <span class="n">dry_run</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="n">reference</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">JobReference</span><span class="p">(</span><span class="n">jobId</span><span class="o">=</span><span class="n">job_id</span><span class="p">,</span> <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">)</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryJobsInsertRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> |
| <span class="n">job</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">Job</span><span class="p">(</span> |
| <span class="n">configuration</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">JobConfiguration</span><span class="p">(</span> |
| <span class="n">dryRun</span><span class="o">=</span><span class="n">dry_run</span><span class="p">,</span> |
| <span class="n">query</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">JobConfigurationQuery</span><span class="p">(</span> |
| <span class="n">query</span><span class="o">=</span><span class="n">query</span><span class="p">,</span> |
| <span class="n">useLegacySql</span><span class="o">=</span><span class="n">use_legacy_sql</span><span class="p">,</span> |
| <span class="n">allowLargeResults</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">destinationTable</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_get_temp_table</span><span class="p">(</span><span class="n">project_id</span><span class="p">),</span> |
| <span class="n">flattenResults</span><span class="o">=</span><span class="n">flatten_results</span><span class="p">)),</span> |
| <span class="n">jobReference</span><span class="o">=</span><span class="n">reference</span><span class="p">))</span> |
| |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">jobs</span><span class="o">.</span><span class="n">Insert</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">response</span><span class="o">.</span><span class="n">jobReference</span><span class="o">.</span><span class="n">jobId</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_get_query_results</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">job_id</span><span class="p">,</span> |
| <span class="n">page_token</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">max_results</span><span class="o">=</span><span class="mi">10000</span><span class="p">):</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryJobsGetQueryResultsRequest</span><span class="p">(</span> |
| <span class="n">jobId</span><span class="o">=</span><span class="n">job_id</span><span class="p">,</span> <span class="n">pageToken</span><span class="o">=</span><span class="n">page_token</span><span class="p">,</span> <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> |
| <span class="n">maxResults</span><span class="o">=</span><span class="n">max_results</span><span class="p">)</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">jobs</span><span class="o">.</span><span class="n">GetQueryResults</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">response</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_insert_all_rows</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">,</span> <span class="n">rows</span><span class="p">):</span> |
| <span class="c1"># The rows argument is a list of</span> |
| <span class="c1"># bigquery.TableDataInsertAllRequest.RowsValueListEntry instances as</span> |
| <span class="c1"># required by the InsertAll() method.</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryTabledataInsertAllRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">tableId</span><span class="o">=</span><span class="n">table_id</span><span class="p">,</span> |
| <span class="n">tableDataInsertAllRequest</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">TableDataInsertAllRequest</span><span class="p">(</span> |
| <span class="c1"># TODO(silviuc): Should have an option for skipInvalidRows?</span> |
| <span class="c1"># TODO(silviuc): Should have an option for ignoreUnknownValues?</span> |
| <span class="n">rows</span><span class="o">=</span><span class="n">rows</span><span class="p">))</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">tabledata</span><span class="o">.</span><span class="n">InsertAll</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="c1"># response.insertErrors is not [] if errors encountered.</span> |
| <span class="k">return</span> <span class="ow">not</span> <span class="n">response</span><span class="o">.</span><span class="n">insertErrors</span><span class="p">,</span> <span class="n">response</span><span class="o">.</span><span class="n">insertErrors</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_get_table</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">):</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryTablesGetRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">tableId</span><span class="o">=</span><span class="n">table_id</span><span class="p">)</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">tables</span><span class="o">.</span><span class="n">Get</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="c1"># The response is a bigquery.Table instance.</span> |
| <span class="k">return</span> <span class="n">response</span> |
| |
| <span class="k">def</span> <span class="nf">_create_table</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">,</span> <span class="n">schema</span><span class="p">):</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">Table</span><span class="p">(</span> |
| <span class="n">tableReference</span><span class="o">=</span><span class="n">bigquery</span><span class="o">.</span><span class="n">TableReference</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">tableId</span><span class="o">=</span><span class="n">table_id</span><span class="p">),</span> |
| <span class="n">schema</span><span class="o">=</span><span class="n">schema</span><span class="p">)</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryTablesInsertRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">table</span><span class="o">=</span><span class="n">table</span><span class="p">)</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">tables</span><span class="o">.</span><span class="n">Insert</span><span class="p">(</span><span class="n">request</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">"Created the table with id </span><span class="si">%s</span><span class="s2">"</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| <span class="c1"># The response is a bigquery.Table instance.</span> |
| <span class="k">return</span> <span class="n">response</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">get_or_create_dataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">location</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="c1"># Check if dataset already exists otherwise create it</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">dataset</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">datasets</span><span class="o">.</span><span class="n">Get</span><span class="p">(</span><span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryDatasetsGetRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">dataset</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="n">dataset_reference</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">DatasetReference</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">)</span> |
| <span class="n">dataset</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">Dataset</span><span class="p">(</span><span class="n">datasetReference</span><span class="o">=</span><span class="n">dataset_reference</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">location</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">dataset</span><span class="o">.</span><span class="n">location</span> <span class="o">=</span> <span class="n">location</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryDatasetsInsertRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset</span><span class="o">=</span><span class="n">dataset</span><span class="p">)</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">datasets</span><span class="o">.</span><span class="n">Insert</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="c1"># The response is a bigquery.Dataset instance.</span> |
| <span class="k">return</span> <span class="n">response</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_is_table_empty</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">):</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryTabledataListRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">tableId</span><span class="o">=</span><span class="n">table_id</span><span class="p">,</span> |
| <span class="n">maxResults</span><span class="o">=</span><span class="mi">1</span><span class="p">)</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">tabledata</span><span class="o">.</span><span class="n">List</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="c1"># The response is a bigquery.TableDataList instance.</span> |
| <span class="k">return</span> <span class="n">response</span><span class="o">.</span><span class="n">totalRows</span> <span class="o">==</span> <span class="mi">0</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_delete_table</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">):</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryTablesDeleteRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> <span class="n">tableId</span><span class="o">=</span><span class="n">table_id</span><span class="p">)</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">tables</span><span class="o">.</span><span class="n">Delete</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">'Table </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"> does not exist'</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> |
| <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| <span class="k">return</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">_delete_dataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">delete_contents</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="n">request</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryDatasetsDeleteRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="n">deleteContents</span><span class="o">=</span><span class="n">delete_contents</span><span class="p">)</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">datasets</span><span class="o">.</span><span class="n">Delete</span><span class="p">(</span><span class="n">request</span><span class="p">)</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">'Dataset </span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1"> does not exist'</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> |
| <span class="n">dataset_id</span><span class="p">)</span> |
| <span class="k">return</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">get_table_location</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">):</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_table</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">table</span><span class="o">.</span><span class="n">location</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">create_temporary_dataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">location</span><span class="p">):</span> |
| <span class="n">dataset_id</span> <span class="o">=</span> <span class="n">BigQueryWrapper</span><span class="o">.</span><span class="n">TEMP_DATASET</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_temporary_table_suffix</span> |
| <span class="c1"># Check if dataset exists to make sure that the temporary id is unique</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">datasets</span><span class="o">.</span><span class="n">Get</span><span class="p">(</span><span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryDatasetsGetRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">project_id</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Unittests don't pass projectIds so they can be run without error</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Dataset </span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1"> already exists so cannot be used as temporary.'</span> |
| <span class="o">%</span> <span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">))</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span> |
| <span class="s1">'Dataset </span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1"> does not exist so we will create it as temporary '</span> |
| <span class="s1">'with location=</span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> |
| <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">location</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">get_or_create_dataset</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">location</span><span class="o">=</span><span class="n">location</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">clean_up_temporary_dataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">):</span> |
| <span class="n">temp_table</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_temp_table</span><span class="p">(</span><span class="n">project_id</span><span class="p">)</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">client</span><span class="o">.</span><span class="n">datasets</span><span class="o">.</span><span class="n">Get</span><span class="p">(</span><span class="n">bigquery</span><span class="o">.</span><span class="n">BigqueryDatasetsGetRequest</span><span class="p">(</span> |
| <span class="n">projectId</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">datasetId</span><span class="o">=</span><span class="n">temp_table</span><span class="o">.</span><span class="n">datasetId</span><span class="p">))</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">'Dataset </span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1"> does not exist'</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> |
| <span class="n">temp_table</span><span class="o">.</span><span class="n">datasetId</span><span class="p">)</span> |
| <span class="k">return</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_delete_dataset</span><span class="p">(</span><span class="n">temp_table</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span> <span class="n">temp_table</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span> |
| |
| <span class="nd">@retry</span><span class="o">.</span><span class="n">with_exponential_backoff</span><span class="p">(</span> |
| <span class="n">num_retries</span><span class="o">=</span><span class="n">MAX_RETRIES</span><span class="p">,</span> |
| <span class="n">retry_filter</span><span class="o">=</span><span class="n">retry</span><span class="o">.</span><span class="n">retry_on_server_errors_and_timeout_filter</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">get_or_create_table</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">,</span> <span class="n">schema</span><span class="p">,</span> |
| <span class="n">create_disposition</span><span class="p">,</span> <span class="n">write_disposition</span><span class="p">):</span> |
| <span class="sd">"""Gets or creates a table based on create and write dispositions.</span> |
| |
| <span class="sd"> The function mimics the behavior of BigQuery import jobs when using the</span> |
| <span class="sd"> same create and write dispositions.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> project_id: The project id owning the table.</span> |
| <span class="sd"> dataset_id: The dataset id owning the table.</span> |
| <span class="sd"> table_id: The table id.</span> |
| <span class="sd"> schema: A bigquery.TableSchema instance or None.</span> |
| <span class="sd"> create_disposition: CREATE_NEVER or CREATE_IF_NEEDED.</span> |
| <span class="sd"> write_disposition: WRITE_APPEND, WRITE_EMPTY or WRITE_TRUNCATE.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A bigquery.Table instance if table was found or created.</span> |
| |
| <span class="sd"> Raises:</span> |
| <span class="sd"> RuntimeError: For various mismatches between the state of the table and</span> |
| <span class="sd"> the create/write dispositions passed in. For example if the table is not</span> |
| <span class="sd"> empty and WRITE_EMPTY was specified then an error will be raised since</span> |
| <span class="sd"> the table was expected to be empty.</span> |
| <span class="sd"> """</span> |
| <span class="n">found_table</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">found_table</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_table</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| <span class="k">except</span> <span class="n">HttpError</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">exn</span><span class="o">.</span><span class="n">status_code</span> <span class="o">==</span> <span class="mi">404</span><span class="p">:</span> |
| <span class="k">if</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="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Table </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"> not found but create disposition is CREATE_NEVER.'</span> |
| <span class="o">%</span> <span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="c1"># If table exists already then handle the semantics for WRITE_EMPTY and</span> |
| <span class="c1"># WRITE_TRUNCATE write dispositions.</span> |
| <span class="k">if</span> <span class="n">found_table</span><span class="p">:</span> |
| <span class="n">table_empty</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_table_empty</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| <span class="k">if</span> <span class="p">(</span><span class="ow">not</span> <span class="n">table_empty</span> <span class="ow">and</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="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Table </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"> is not empty but write disposition is WRITE_EMPTY.'</span> |
| <span class="o">%</span> <span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">))</span> |
| <span class="c1"># Delete the table and recreate it (later) if WRITE_TRUNCATE was</span> |
| <span class="c1"># specified.</span> |
| <span class="k">if</span> <span class="n">write_disposition</span> <span class="o">==</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_TRUNCATE</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_delete_table</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">)</span> |
| |
| <span class="c1"># Create a new table potentially reusing the schema from a previously</span> |
| <span class="c1"># found table in case the schema was not specified.</span> |
| <span class="k">if</span> <span class="n">schema</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">found_table</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Table </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"> requires a schema. None can be inferred because the '</span> |
| <span class="s1">'table does not exist.'</span> |
| <span class="o">%</span> <span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">found_table</span> <span class="ow">and</span> <span class="n">write_disposition</span> <span class="o">!=</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_TRUNCATE</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">found_table</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">created_table</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_create_table</span><span class="p">(</span><span class="n">project_id</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> |
| <span class="n">dataset_id</span><span class="o">=</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="n">table_id</span><span class="o">=</span><span class="n">table_id</span><span class="p">,</span> |
| <span class="n">schema</span><span class="o">=</span><span class="n">schema</span> <span class="ow">or</span> <span class="n">found_table</span><span class="o">.</span><span class="n">schema</span><span class="p">)</span> |
| <span class="c1"># if write_disposition == BigQueryDisposition.WRITE_TRUNCATE we delete</span> |
| <span class="c1"># the table before this point.</span> |
| <span class="k">if</span> <span class="n">write_disposition</span> <span class="o">==</span> <span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_TRUNCATE</span><span class="p">:</span> |
| <span class="c1"># BigQuery can route data to the old table for 2 mins max so wait</span> |
| <span class="c1"># that much time before creating the table and writing it</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">'Sleeping for 150 seconds before the write as '</span> <span class="o">+</span> |
| <span class="s1">'BigQuery inserts can be routed to deleted table '</span> <span class="o">+</span> |
| <span class="s1">'for 2 mins after the delete and create.'</span><span class="p">)</span> |
| <span class="c1"># TODO(BEAM-2673): Remove this sleep by migrating to load api</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="mi">150</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">created_table</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">created_table</span> |
| |
| <span class="k">def</span> <span class="nf">run_query</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">query</span><span class="p">,</span> <span class="n">use_legacy_sql</span><span class="p">,</span> <span class="n">flatten_results</span><span class="p">,</span> |
| <span class="n">dry_run</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="n">job_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_start_query_job</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">query</span><span class="p">,</span> <span class="n">use_legacy_sql</span><span class="p">,</span> |
| <span class="n">flatten_results</span><span class="p">,</span> <span class="n">job_id</span><span class="o">=</span><span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span><span class="o">.</span><span class="n">hex</span><span class="p">,</span> |
| <span class="n">dry_run</span><span class="o">=</span><span class="n">dry_run</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">dry_run</span><span class="p">:</span> |
| <span class="c1"># If this was a dry run then the fact that we get here means the</span> |
| <span class="c1"># query has no errors. The start_query_job would raise an error otherwise.</span> |
| <span class="k">return</span> |
| <span class="n">page_token</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_query_results</span><span class="p">(</span><span class="n">project_id</span><span class="p">,</span> <span class="n">job_id</span><span class="p">,</span> <span class="n">page_token</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">response</span><span class="o">.</span><span class="n">jobComplete</span><span class="p">:</span> |
| <span class="c1"># The jobComplete field can be False if the query request times out</span> |
| <span class="c1"># (default is 10 seconds). Note that this is a timeout for the query</span> |
| <span class="c1"># request not for the actual execution of the query in the service. If</span> |
| <span class="c1"># the request times out we keep trying. This situation is quite possible</span> |
| <span class="c1"># if the query will return a large number of rows.</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">'Waiting on response from query: </span><span class="si">%s</span><span class="s1"> ...'</span><span class="p">,</span> <span class="n">query</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="mf">1.0</span><span class="p">)</span> |
| <span class="k">continue</span> |
| <span class="c1"># We got some results. The last page is signalled by a missing pageToken.</span> |
| <span class="k">yield</span> <span class="n">response</span><span class="o">.</span><span class="n">rows</span><span class="p">,</span> <span class="n">response</span><span class="o">.</span><span class="n">schema</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">response</span><span class="o">.</span><span class="n">pageToken</span><span class="p">:</span> |
| <span class="k">break</span> |
| <span class="n">page_token</span> <span class="o">=</span> <span class="n">response</span><span class="o">.</span><span class="n">pageToken</span> |
| |
| <span class="k">def</span> <span class="nf">insert_rows</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">,</span> <span class="n">rows</span><span class="p">):</span> |
| <span class="sd">"""Inserts rows into the specified table.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> project_id: The project id owning the table.</span> |
| <span class="sd"> dataset_id: The dataset id owning the table.</span> |
| <span class="sd"> table_id: The table id.</span> |
| <span class="sd"> rows: A list of plain Python dictionaries. Each dictionary is a row and</span> |
| <span class="sd"> each key in it is the name of a field.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A tuple (bool, errors). If first element is False then the second element</span> |
| <span class="sd"> will be a bigquery.InserttErrorsValueListEntry instance containing</span> |
| <span class="sd"> specific errors.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># Prepare rows for insertion. Of special note is the row ID that we add to</span> |
| <span class="c1"># each row in order to help BigQuery avoid inserting a row multiple times.</span> |
| <span class="c1"># BigQuery will do a best-effort if unique IDs are provided. This situation</span> |
| <span class="c1"># can happen during retries on failures.</span> |
| <span class="c1"># TODO(silviuc): Must add support to writing TableRow's instead of dicts.</span> |
| <span class="n">final_rows</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">row</span> <span class="ow">in</span> <span class="n">rows</span><span class="p">:</span> |
| <span class="n">json_object</span> <span class="o">=</span> <span class="n">bigquery</span><span class="o">.</span><span class="n">JsonObject</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">iteritems</span><span class="p">(</span><span class="n">row</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">decimal</span><span class="o">.</span><span class="n">Decimal</span><span class="p">):</span> |
| <span class="c1"># decimal values are converted into string because JSON does not</span> |
| <span class="c1"># support the precision that decimal supports. BQ is able to handle</span> |
| <span class="c1"># inserts into NUMERIC columns by receiving JSON with string attrs.</span> |
| <span class="n">v</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> |
| <span class="n">json_object</span><span class="o">.</span><span class="n">additionalProperties</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="n">bigquery</span><span class="o">.</span><span class="n">JsonObject</span><span class="o">.</span><span class="n">AdditionalProperty</span><span class="p">(</span> |
| <span class="n">key</span><span class="o">=</span><span class="n">k</span><span class="p">,</span> <span class="n">value</span><span class="o">=</span><span class="n">to_json_value</span><span class="p">(</span><span class="n">v</span><span class="p">)))</span> |
| <span class="n">final_rows</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="n">bigquery</span><span class="o">.</span><span class="n">TableDataInsertAllRequest</span><span class="o">.</span><span class="n">RowsValueListEntry</span><span class="p">(</span> |
| <span class="n">insertId</span><span class="o">=</span><span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">unique_row_id</span><span class="p">),</span> |
| <span class="n">json</span><span class="o">=</span><span class="n">json_object</span><span class="p">))</span> |
| <span class="n">result</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">_insert_all_rows</span><span class="p">(</span> |
| <span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">table_id</span><span class="p">,</span> <span class="n">final_rows</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">result</span><span class="p">,</span> <span class="n">errors</span> |
| |
| <span class="k">def</span> <span class="nf">_convert_cell_value_to_dict</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">,</span> <span class="n">field</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'STRING'</span><span class="p">:</span> |
| <span class="c1"># Input: "XYZ" --> Output: "XYZ"</span> |
| <span class="k">return</span> <span class="n">value</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'BOOLEAN'</span><span class="p">:</span> |
| <span class="c1"># Input: "true" --> Output: True</span> |
| <span class="k">return</span> <span class="n">value</span> <span class="o">==</span> <span class="s1">'true'</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'INTEGER'</span><span class="p">:</span> |
| <span class="c1"># Input: "123" --> Output: 123</span> |
| <span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'FLOAT'</span><span class="p">:</span> |
| <span class="c1"># Input: "1.23" --> Output: 1.23</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'TIMESTAMP'</span><span class="p">:</span> |
| <span class="c1"># The UTC should come from the timezone library but this is a known</span> |
| <span class="c1"># issue in python 2.7 so we'll just hardcode it as we're reading using</span> |
| <span class="c1"># utcfromtimestamp.</span> |
| <span class="c1"># Input: 1478134176.985864 --> Output: "2016-11-03 00:49:36.985864 UTC"</span> |
| <span class="n">dt</span> <span class="o">=</span> <span class="n">datetime</span><span class="o">.</span><span class="n">datetime</span><span class="o">.</span><span class="n">utcfromtimestamp</span><span class="p">(</span><span class="nb">float</span><span class="p">(</span><span class="n">value</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">dt</span><span class="o">.</span><span class="n">strftime</span><span class="p">(</span><span class="s1">'%Y-%m-</span><span class="si">%d</span><span class="s1"> %H:%M:%S.</span><span class="si">%f</span><span class="s1"> UTC'</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'BYTES'</span><span class="p">:</span> |
| <span class="c1"># Input: "YmJi" --> Output: "YmJi"</span> |
| <span class="k">return</span> <span class="n">value</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'DATE'</span><span class="p">:</span> |
| <span class="c1"># Input: "2016-11-03" --> Output: "2016-11-03"</span> |
| <span class="k">return</span> <span class="n">value</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'DATETIME'</span><span class="p">:</span> |
| <span class="c1"># Input: "2016-11-03T00:49:36" --> Output: "2016-11-03T00:49:36"</span> |
| <span class="k">return</span> <span class="n">value</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'TIME'</span><span class="p">:</span> |
| <span class="c1"># Input: "00:49:36" --> Output: "00:49:36"</span> |
| <span class="k">return</span> <span class="n">value</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'RECORD'</span><span class="p">:</span> |
| <span class="c1"># Note that a schema field object supports also a RECORD type. However</span> |
| <span class="c1"># when querying, the repeated and/or record fields are flattened</span> |
| <span class="c1"># unless we pass the flatten_results flag as False to the source</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">convert_row_to_dict</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">field</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span> <span class="o">==</span> <span class="s1">'NUMERIC'</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">decimal</span><span class="o">.</span><span class="n">Decimal</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">'Unexpected field type: </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="n">field</span><span class="o">.</span><span class="n">type</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">convert_row_to_dict</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">row</span><span class="p">,</span> <span class="n">schema</span><span class="p">):</span> |
| <span class="sd">"""Converts a TableRow instance using the schema to a Python dict."""</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="k">for</span> <span class="n">index</span><span class="p">,</span> <span class="n">field</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">schema</span><span class="o">.</span><span class="n">fields</span><span class="p">):</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="kc">None</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="n">bigquery</span><span class="o">.</span><span class="n">TableSchema</span><span class="p">):</span> |
| <span class="n">cell</span> <span class="o">=</span> <span class="n">row</span><span class="o">.</span><span class="n">f</span><span class="p">[</span><span class="n">index</span><span class="p">]</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="n">from_json_value</span><span class="p">(</span><span class="n">cell</span><span class="o">.</span><span class="n">v</span><span class="p">)</span> <span class="k">if</span> <span class="n">cell</span><span class="o">.</span><span class="n">v</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="kc">None</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">TableFieldSchema</span><span class="p">):</span> |
| <span class="n">cell</span> <span class="o">=</span> <span class="n">row</span><span class="p">[</span><span class="s1">'f'</span><span class="p">][</span><span class="n">index</span><span class="p">]</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="n">cell</span><span class="p">[</span><span class="s1">'v'</span><span class="p">]</span> <span class="k">if</span> <span class="s1">'v'</span> <span class="ow">in</span> <span class="n">cell</span> <span class="k">else</span> <span class="kc">None</span> |
| <span class="k">if</span> <span class="n">field</span><span class="o">.</span><span class="n">mode</span> <span class="o">==</span> <span class="s1">'REPEATED'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">value</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Ideally this should never happen as repeated fields default to</span> |
| <span class="c1"># returning an empty list</span> |
| <span class="n">result</span><span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result</span><span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">name</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">_convert_cell_value_to_dict</span><span class="p">(</span><span class="n">x</span><span class="p">[</span><span class="s1">'v'</span><span class="p">],</span> <span class="n">field</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">value</span><span class="p">]</span> |
| <span class="k">elif</span> <span class="n">value</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">field</span><span class="o">.</span><span class="n">mode</span> <span class="o">==</span> <span class="s1">'NULLABLE'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'Received </span><span class="se">\'</span><span class="s1">None</span><span class="se">\'</span><span class="s1"> as the value for the field </span><span class="si">%s</span><span class="s1"> '</span> |
| <span class="s1">'but the field is not NULLABLE.'</span> <span class="o">%</span> <span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">)</span> |
| <span class="n">result</span><span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result</span><span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_convert_cell_value_to_dict</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">field</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| |
| <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">"""A ``DoFn`` that streams writes to BigQuery once the table is created.</span> |
| <span class="sd"> """</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_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="p">,</span> <span class="n">project_id</span><span class="p">,</span> <span class="n">batch_size</span><span class="p">,</span> <span class="n">schema</span><span class="p">,</span> |
| <span class="n">create_disposition</span><span class="p">,</span> <span class="n">write_disposition</span><span class="p">,</span> <span class="n">client</span><span class="p">):</span> |
| <span class="sd">"""Initialize a WriteToBigQuery transform.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> table_id: 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 dataset argument is</span> |
| <span class="sd"> None then the table argument must contain the entire table reference</span> |
| <span class="sd"> specified as: 'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'.</span> |
| <span class="sd"> dataset_id: The ID of the dataset containing this table or null if the</span> |
| <span class="sd"> table reference is specified entirely by the table argument.</span> |
| <span class="sd"> project_id: The ID of the project containing this table or null if the</span> |
| <span class="sd"> table reference is specified entirely by the table argument.</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 'bigquery.TableSchema' object</span> |
| <span class="sd"> or a single string of the form 'field1:type1,field2:type2,field3:type3'</span> |
| <span class="sd"> that defines a comma separated list of fields. Here 'type' 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 'NULLABLE').</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"> test_client: Override the default bigquery client used for testing.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span> <span class="o">=</span> <span class="n">table_id</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span> <span class="o">=</span> <span class="n">dataset_id</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span> <span class="o">=</span> <span class="n">project_id</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">client</span> <span class="o">=</span> <span class="n">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="c1"># The default batch size is 500</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="mi">500</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">'table_id'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</span> |
| <span class="s1">'dataset_id'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="s1">'project_id'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> |
| <span class="s1">'schema'</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">schema</span><span class="p">),</span> |
| <span class="s1">'max_batch_size'</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="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">"""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"> """</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="nb">dict</span><span class="p">):</span> |
| <span class="k">return</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">'Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.'</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">_rows_buffer</span> <span class="o">=</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="bp">self</span><span class="o">.</span><span class="n">get_table_schema</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">bigquery_wrapper</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">client</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="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</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">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="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="n">unused_create_fn_output</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">element</span><span class="p">)</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="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</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="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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</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="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span> <span class="o">=</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="c1"># Flush the current batch of rows to BigQuery.</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="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="n">dataset_id</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="n">table_id</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</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="k">if</span> <span class="ow">not</span> <span class="n">passed</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">'Could not successfully insert rows to BigQuery'</span> |
| <span class="s1">' table [</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">]. Errors: </span><span class="si">%s</span><span class="s1">'</span><span class="o">%</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">project_id</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">dataset_id</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">table_id</span><span class="p">,</span> <span class="n">errors</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">"Successfully wrote </span><span class="si">%d</span><span class="s2"> rows."</span><span class="p">,</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="bp">self</span><span class="o">.</span><span class="n">_rows_buffer</span> <span class="o">=</span> <span class="p">[]</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="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">batch_size</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="sd">"""Initialize a WriteToBigQuery transform.</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 dataset</span> |
| <span class="sd"> argument is :data:`None` then the table argument must contain the</span> |
| <span class="sd"> entire table reference specified as: ``'DATASET.TABLE'`` or</span> |
| <span class="sd"> ``'PROJECT:DATASET.TABLE'``.</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 to</span> |
| <span class="sd"> 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`</span> |
| <span class="sd"> object or a single string of the form</span> |
| <span class="sd"> ``'field1:type1,field2:type2,field3:type3'`` that defines a comma</span> |
| <span class="sd"> separated list of fields. Here ``'type'`` 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 ``'NULLABLE'``).</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"> batch_size (int): Number of rows to be written to BQ per streaming API</span> |
| <span class="sd"> insert.</span> |
| <span class="sd"> test_client: Override the default bigquery client used for testing.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">table_reference</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="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">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">test_client</span> <span class="o">=</span> <span class="n">test_client</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">"""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"> """</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">','</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">':'</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">'NULLABLE'</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">"""Create a dictionary representation of table schema for serialization</span> |
| <span class="sd"> """</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">"""Create a dictionary representation of a table field</span> |
| <span class="sd"> """</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">'name'</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">'type'</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">'mode'</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">'mode'</span><span class="p">,</span> <span class="s1">'NULLABLE'</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">'description'</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">'description'</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">'fields'</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">'fields'</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">"Table schema must be of the type bigquery.TableSchema"</span><span class="p">)</span> |
| <span class="n">schema</span> <span class="o">=</span> <span class="p">{</span><span class="s1">'fields'</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">'fields'</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">"""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"> """</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="nb">dict</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">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="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">'Unexpected schema argument: </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="n">schema</span><span class="p">)</span></div> |
| |
| <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="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="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">bigquery_write_fn</span> <span class="o">=</span> <span class="n">BigQueryWriteFn</span><span class="p">(</span> |
| <span class="n">table_id</span><span class="o">=</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">dataset_id</span><span class="o">=</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="n">project_id</span><span class="o">=</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">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">schema</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">get_dict_table_schema</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">schema</span><span class="p">),</span> |
| <span class="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">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="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">'WriteToBigQuery'</span> <span class="o">>></span> <span class="n">ParDo</span><span class="p">(</span><span class="n">bigquery_write_fn</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">'</span><span class="si">{}</span><span class="s1">.</span><span class="si">{}</span><span class="s1">'</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">'</span><span class="si">{}</span><span class="s1">:</span><span class="si">{}</span><span class="s1">'</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">'table'</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">'Table'</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> |
| © 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> |