blob: 986e7c010a5d2bd568dde5db0a9f37f8352e1ec0 [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<!-- NewPage -->
<html lang="en">
<head>
<!-- Generated by javadoc -->
<title>SpannerIO (Apache Beam 2.38.0-SNAPSHOT)</title>
<link rel="stylesheet" type="text/css" href="../../../../../../../stylesheet.css" title="Style">
<script type="text/javascript" src="../../../../../../../script.js"></script>
</head>
<body>
<script type="text/javascript"><!--
try {
if (location.href.indexOf('is-external=true') == -1) {
parent.document.title="SpannerIO (Apache Beam 2.38.0-SNAPSHOT)";
}
}
catch(err) {
}
//-->
var methods = {"i0":9,"i1":9,"i2":9,"i3":9,"i4":9};
var tabs = {65535:["t0","All Methods"],1:["t1","Static Methods"],8:["t4","Concrete Methods"]};
var altColor = "altColor";
var rowColor = "rowColor";
var tableTab = "tableTab";
var activeTableTab = "activeTableTab";
</script>
<noscript>
<div>JavaScript is disabled on your browser.</div>
</noscript>
<!-- ========= START OF TOP NAVBAR ======= -->
<div class="topNav"><a name="navbar.top">
<!-- -->
</a>
<div class="skipNav"><a href="#skip.navbar.top" title="Skip navigation links">Skip navigation links</a></div>
<a name="navbar.top.firstrow">
<!-- -->
</a>
<ul class="navList" title="Navigation">
<li><a href="../../../../../../../overview-summary.html">Overview</a></li>
<li><a href="package-summary.html">Package</a></li>
<li class="navBarCell1Rev">Class</li>
<li><a href="package-tree.html">Tree</a></li>
<li><a href="../../../../../../../deprecated-list.html">Deprecated</a></li>
<li><a href="../../../../../../../index-all.html">Index</a></li>
<li><a href="../../../../../../../help-doc.html">Help</a></li>
</ul>
</div>
<div class="subNav">
<ul class="navList">
<li><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.Builder.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><span class="typeNameLink">Prev&nbsp;Class</span></a></li>
<li><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><span class="typeNameLink">Next&nbsp;Class</span></a></li>
</ul>
<ul class="navList">
<li><a href="../../../../../../../index.html?org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html" target="_top">Frames</a></li>
<li><a href="SpannerIO.html" target="_top">No&nbsp;Frames</a></li>
</ul>
<ul class="navList" id="allclasses_navbar_top">
<li><a href="../../../../../../../allclasses-noframe.html">All&nbsp;Classes</a></li>
</ul>
<div>
<script type="text/javascript"><!--
allClassesLink = document.getElementById("allclasses_navbar_top");
if(window==top) {
allClassesLink.style.display = "block";
}
else {
allClassesLink.style.display = "none";
}
//-->
</script>
</div>
<div>
<ul class="subNavList">
<li>Summary:&nbsp;</li>
<li><a href="#nested.class.summary">Nested</a>&nbsp;|&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li>Constr&nbsp;|&nbsp;</li>
<li><a href="#method.summary">Method</a></li>
</ul>
<ul class="subNavList">
<li>Detail:&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li>Constr&nbsp;|&nbsp;</li>
<li><a href="#method.detail">Method</a></li>
</ul>
</div>
<a name="skip.navbar.top">
<!-- -->
</a></div>
<!-- ========= END OF TOP NAVBAR ========= -->
<!-- ======== START OF CLASS DATA ======== -->
<div class="header">
<div class="subTitle">org.apache.beam.sdk.io.gcp.spanner</div>
<h2 title="Class SpannerIO" class="title">Class SpannerIO</h2>
</div>
<div class="contentContainer">
<ul class="inheritance">
<li>java.lang.Object</li>
<li>
<ul class="inheritance">
<li>org.apache.beam.sdk.io.gcp.spanner.SpannerIO</li>
</ul>
</li>
</ul>
<div class="description">
<ul class="blockList">
<li class="blockList">
<hr>
<br>
<pre><a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.html" title="annotation in org.apache.beam.sdk.annotations">@Experimental</a>(<a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.html#value--">value</a>=<a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.Kind.html#SOURCE_SINK">SOURCE_SINK</a>)
public class <span class="typeNameLabel">SpannerIO</span>
extends java.lang.Object</pre>
<div class="block">Experimental <a href="../../../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>Transforms</code></a> for reading from and writing to <a
href="https://cloud.google.com/spanner">Google Cloud Spanner</a>.
<h3>Reading from Cloud Spanner</h3>
<p>To read from Cloud Spanner, apply <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Read</code></a> transformation. It will return a <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> of <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Struct.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Structs</code></a>, where each element represents an individual row returned
from the read operation. Both Query and Read APIs are supported. See more information about <a
href="https://cloud.google.com/spanner/docs/reads">reading from Cloud Spanner</a>
<p>To execute a <strong>query</strong>, specify a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withQuery-com.google.cloud.spanner.Statement-"><code>SpannerIO.Read.withQuery(Statement)</code></a> or <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withQuery-java.lang.String-"><code>SpannerIO.Read.withQuery(String)</code></a> during the construction of the transform.
<pre><code>
PCollection&lt;Struct&gt; rows = p.apply(
SpannerIO.read()
.withInstanceId(instanceId)
.withDatabaseId(dbId)
.withQuery("SELECT id, name, email FROM users"));
</code></pre>
<p>To use the Read API, specify a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withTable-java.lang.String-"><code>table name</code></a> and a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withColumns-java.util.List-"><code>list of columns</code></a>.
<pre><code>
PCollection&lt;Struct&gt; rows = p.apply(
SpannerIO.read()
.withInstanceId(instanceId)
.withDatabaseId(dbId)
.withTable("users")
.withColumns("id", "name", "email"));
</code></pre>
<p>To optimally read using index, specify the index name using <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withIndex-java.lang.String-"><code>SpannerIO.Read.withIndex(java.lang.String)</code></a>.
<p>The transform is guaranteed to be executed on a consistent snapshot of data, utilizing the
power of read only transactions. Staleness of data can be controlled using <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withTimestampBound-com.google.cloud.spanner.TimestampBound-"><code>SpannerIO.Read.withTimestampBound(com.google.cloud.spanner.TimestampBound)</code></a> or <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withTimestamp-com.google.cloud.Timestamp-"><code>SpannerIO.Read.withTimestamp(Timestamp)</code></a> methods. <a
href="https://cloud.google.com/spanner/docs/transactions">Read more</a> about transactions in
Cloud Spanner.
<p>It is possible to read several <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a> within a single transaction.
Apply <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#createTransaction--"><code>createTransaction()</code></a> transform, that lazily creates a transaction. The
result of this transformation can be passed to read operation using <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withTransaction-org.apache.beam.sdk.values.PCollectionView-"><code>SpannerIO.Read.withTransaction(PCollectionView)</code></a>.
<pre><code>
SpannerConfig spannerConfig = ...
PCollectionView&lt;Transaction&gt; tx = p.apply(
SpannerIO.createTransaction()
.withSpannerConfig(spannerConfig)
.withTimestampBound(TimestampBound.strong()));
PCollection&lt;Struct&gt; users = p.apply(
SpannerIO.read()
.withSpannerConfig(spannerConfig)
.withQuery("SELECT name, email FROM users")
.withTransaction(tx));
PCollection&lt;Struct&gt; tweets = p.apply(
SpannerIO.read()
.withSpannerConfig(spannerConfig)
.withQuery("SELECT user, tweet, date FROM tweets")
.withTransaction(tx));
</code></pre>
<h3>Writing to Cloud Spanner</h3>
<p>The Cloud Spanner <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a> transform writes to Cloud Spanner by executing a collection of
input row <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutations</code></a>. The mutations are grouped into batches for efficiency.
<p>To configure the write transform, create an instance using <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#write--"><code>write()</code></a> and then specify
the destination Cloud Spanner instance (<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withInstanceId-java.lang.String-"><code>SpannerIO.Write.withInstanceId(String)</code></a> and destination
database (<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withDatabaseId-java.lang.String-"><code>SpannerIO.Write.withDatabaseId(String)</code></a>). For example:
<pre><code>
// Earlier in the pipeline, create a PCollection of Mutations to be written to Cloud Spanner.
PCollection&lt;Mutation&gt; mutations = ...;
// Write mutations.
SpannerWriteResult result = mutations.apply(
"Write", SpannerIO.write().withInstanceId("instance").withDatabaseId("database"));
</code></pre>
<h3>SpannerWriteResult</h3>
<p>The <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerWriteResult.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerWriteResult</code></a> object contains the results of the
transform, including a <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> of MutationGroups that failed to write, and a <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> that can be used in batch pipelines as a completion signal to <a href="../../../../../../../org/apache/beam/sdk/transforms/Wait.html" title="class in org.apache.beam.sdk.transforms"><code>Wait.OnSignal</code></a> to indicate when all input has been written. Note that in streaming pipelines,
this signal will never be triggered as the input is unbounded and this <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> is
using the <a href="../../../../../../../org/apache/beam/sdk/transforms/windowing/GlobalWindow.html" title="class in org.apache.beam.sdk.transforms.windowing"><code>GlobalWindow</code></a>.
<h3>Batching and Grouping</h3>
<p>To reduce the number of transactions sent to Spanner, the <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutations</code></a> are
grouped into batches. The default maximum size of the batch is set to 1MB or 5000 mutated cells,
or 500 rows (whichever is reached first). To override this use <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withBatchSizeBytes-long-"><code>withBatchSizeBytes()</code></a>, <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withMaxNumMutations-long-"><code>withMaxNumMutations()</code></a> or <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withMaxNumMutations-long-"><code>withMaxNumRows()</code></a>. Setting
either to a small value or zero disables batching.
<p>Note that the <a
href="https://cloud.google.com/spanner/quotas#limits_for_creating_reading_updating_and_deleting_data">maximum
size of a single transaction</a> is 20,000 mutated cells - including cells in indexes. If you
have a large number of indexes and are getting exceptions with message: <tt>INVALID_ARGUMENT: The
transaction contains too many mutations</tt> you will need to specify a smaller number of <code>MaxNumMutations</code>.
<p>The batches written are obtained from by grouping enough <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutations</code></a> from the
Bundle provided by Beam to form several batches. This group of <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutations</code></a> is then
sorted by table and primary key, and the batches are created from the sorted group. Each batch
will then have rows for the same table, with keys that are 'close' to each other, thus optimising
write efficiency by each batch affecting as few table splits as possible performance.
<p>This grouping factor (number of batches) is controlled by the parameter <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withGroupingFactor-int-"><code>withGroupingFactor()</code></a>.
<p>Note that each worker will need enough memory to hold <code>GroupingFactor x
MaxBatchSizeBytes</code> Mutations, so if you have a large <code>MaxBatchSize</code> you may need to reduce
<code>GroupingFactor</code>
<p>While Grouping and Batching increases write efficiency, it dramatically increases the latency
between when a Mutation is received by the transform, and when it is actually written to the
database. This is because enough Mutations need to be received to fill the grouped batches. In
Batch pipelines (bounded sources), this is not normally an issue, but in Streaming (unbounded)
pipelines, this latency is often seen as unacceptable.
<p>There are therefore 3 different ways that this transform can be configured:
<ul>
<li>With Grouping and Batching. <br>
This is the default for Batch pipelines, where sorted batches of Mutations are created and
written. This is the most efficient way to ingest large amounts of data, but the highest
latency before writing
<li>With Batching but no Grouping <br>
If <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withGroupingFactor-int-"><code>.withGroupingFactor(1)</code></a>, is set, grouping is
disabled. This is the default for Streaming pipelines. Unsorted batches are created and
written as soon as enough mutations to fill a batch are received. This reflects a
compromise where a small amount of additional latency enables more efficient writes
<li>Without any Batching <br>
If <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withBatchSizeBytes-long-"><code>.withBatchSizeBytes(0)</code></a> is set, no batching is
performed and the Mutations are written to the database as soon as they are received.
ensuring the lowest latency before Mutations are written.
</ul>
<h3>Monitoring</h3>
<p>Several counters are provided for monitoring purpooses:
<ul>
<li><tt>batchable_mutation_groups</tt><br>
Counts the mutations that are batched for writing to Spanner.
<li><tt>unbatchable_mutation_groups</tt><br>
Counts the mutations that can not be batched and are applied individually - either because
they are too large to fit into a batch, or they are ranged deletes.
<li><tt>mutation_group_batches_received, mutation_group_batches_write_success,
mutation_group_batches_write_failed</tt><br>
Count the number of batches that are processed. If Failure Mode is set to <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.FailureMode.html#REPORT_FAILURES"><code>REPORT_FAILURES</code></a>, then failed batches will be split up and the
individual mutation groups retried separately.
<li><tt>mutation_groups_received, mutation_groups_write_success,
mutation_groups_write_fail</tt><br>
Count the number of individual MutationGroups that are processed.
<li><tt>spanner_write_success, spanner_write_fail</tt><br>
The number of writes to Spanner that have occurred.
<li><tt>spanner_write_retries</tt><br>
The number of times a write is retried after a failure - either due to a timeout, or when
batches fail and <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.FailureMode.html#REPORT_FAILURES"><code>REPORT_FAILURES</code></a> is set so that
individual Mutation Groups are retried.
<li><tt>spanner_write_timeouts</tt><br>
The number of timeouts that occur when writing to Spanner. Writes that timed out are
retried after a backoff. Large numbers of timeouts suggest an overloaded Spanner instance.
<li><tt>spanner_write_total_latency_ms</tt><br>
The total amount of time spent writing to Spanner, in milliseconds.
</ul>
<h3>Database Schema Preparation</h3>
<p>The Write transform reads the database schema on pipeline start to know which columns are used
as primary keys of the tables and indexes. This is so that the transform knows how to sort the
grouped Mutations by table name and primary key as described above.
<p>If the database schema, any additional tables or indexes are created in the same pipeline then
there will be a race condition, leading to a situation where the schema is read before the table
is created its primary key will not be known. This will mean that the sorting/batching will not
be optimal and performance will be reduced (warnings will be logged for rows using unknown
tables)
<p>To prevent this race condition, use <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withSchemaReadySignal-org.apache.beam.sdk.values.PCollection-"><code>SpannerIO.Write.withSchemaReadySignal(PCollection)</code></a> to pass a
signal <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> (for example the output of the transform that creates the table(s))
which will be used with <code>Wait.OnSignal</code> to prevent the schema from being read until it is
ready. The Write transform will be paused until this signal <a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> is closed.
<h3>Transactions</h3>
<p>The transform does not provide same transactional guarantees as Cloud Spanner. In particular,
<ul>
<li>Individual Mutations are submitted atomically, but all Mutations are not submitted in the
same transaction.
<li>A Mutation is applied at least once;
<li>If the pipeline was unexpectedly stopped, mutations that were already applied will not get
rolled back.
</ul>
<p>Use <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/MutationGroup.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>MutationGroups</code></a> with the <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.WriteGrouped.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.WriteGrouped</code></a> transform to ensure
that a small set mutations is bundled together. It is guaranteed that mutations in a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/MutationGroup.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>MutationGroup</code></a> are submitted in the same transaction. Note that a MutationGroup must not exceed
the Spanner transaction limits.
<pre><code>
// Earlier in the pipeline, create a PCollection of MutationGroups to be written to Cloud Spanner.
PCollection&lt;MutationGroup&gt; mutationGroups = ...;
// Write mutation groups.
SpannerWriteResult result = mutationGroups.apply(
"Write",
SpannerIO.write().withInstanceId("instance").withDatabaseId("database").grouped());
</code></pre>
<h3>Streaming Support</h3>
<p><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a> can be used as a streaming sink, however as with batch mode note that the write
order of individual <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutation</code></a>/<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/MutationGroup.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>MutationGroup</code></a> objects is not guaranteed.</div>
</li>
</ul>
</div>
<div class="summary">
<ul class="blockList">
<li class="blockList">
<!-- ======== NESTED CLASS SUMMARY ======== -->
<ul class="blockList">
<li class="blockList"><a name="nested.class.summary">
<!-- -->
</a>
<h3>Nested Class Summary</h3>
<table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Nested Class Summary table, listing nested classes, and an explanation">
<caption><span>Nested Classes</span><span class="tabEnd">&nbsp;</span></caption>
<tr>
<th class="colFirst" scope="col">Modifier and Type</th>
<th class="colLast" scope="col">Class and Description</th>
</tr>
<tr class="altColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.CreateTransaction</a></span></code>
<div class="block">A <a href="../../../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that create a transaction.</div>
</td>
</tr>
<tr class="rowColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.FailureMode.html" title="enum in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.FailureMode</a></span></code>
<div class="block">A failure handling strategy.</div>
</td>
</tr>
<tr class="altColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Read</a></span></code>
<div class="block">Implementation of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#read--"><code>read()</code></a>.</div>
</td>
</tr>
<tr class="rowColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadAll.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadAll</a></span></code>
<div class="block">Implementation of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#readAll--"><code>readAll()</code></a>.</div>
</td>
</tr>
<tr class="altColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadChangeStream</a></span></code>&nbsp;</td>
</tr>
<tr class="rowColor">
<td class="colFirst"><code>static interface&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.SpannerChangeStreamOptions.html" title="interface in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.SpannerChangeStreamOptions</a></span></code>
<div class="block">Interface to display the name of the metadata table on Dataflow UI.</div>
</td>
</tr>
<tr class="altColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Write</a></span></code>
<div class="block">A <a href="../../../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that writes <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/Mutation.html?is-external=true" title="class or interface in com.google.cloud.spanner"><code>Mutation</code></a> objects to Google Cloud Spanner.</div>
</td>
</tr>
<tr class="rowColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.WriteGrouped.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.WriteGrouped</a></span></code>
<div class="block">Same as <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a> but supports grouped mutations.</div>
</td>
</tr>
</table>
</li>
</ul>
<!-- ========== METHOD SUMMARY =========== -->
<ul class="blockList">
<li class="blockList"><a name="method.summary">
<!-- -->
</a>
<h3>Method Summary</h3>
<table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Method Summary table, listing methods, and an explanation">
<caption><span id="t0" class="activeTableTab"><span>All Methods</span><span class="tabEnd">&nbsp;</span></span><span id="t1" class="tableTab"><span><a href="javascript:show(1);">Static Methods</a></span><span class="tabEnd">&nbsp;</span></span><span id="t4" class="tableTab"><span><a href="javascript:show(8);">Concrete Methods</a></span><span class="tabEnd">&nbsp;</span></span></caption>
<tr>
<th class="colFirst" scope="col">Modifier and Type</th>
<th class="colLast" scope="col">Method and Description</th>
</tr>
<tr id="i0" class="altColor">
<td class="colFirst"><code>static <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.CreateTransaction</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#createTransaction--">createTransaction</a></span>()</code>
<div class="block">Returns a transform that creates a batch transaction.</div>
</td>
</tr>
<tr id="i1" class="rowColor">
<td class="colFirst"><code>static <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Read</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#read--">read</a></span>()</code>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Read</code></a>.</div>
</td>
</tr>
<tr id="i2" class="altColor">
<td class="colFirst"><code>static <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadAll.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadAll</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#readAll--">readAll</a></span>()</code>
<div class="block">A <a href="../../../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that works like <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#read--"><code>read()</code></a>, but executes read operations coming from a
<a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>.</div>
</td>
</tr>
<tr id="i3" class="rowColor">
<td class="colFirst"><code>static <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadChangeStream</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#readChangeStream--">readChangeStream</a></span>()</code>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.ReadChangeStream</code></a>.</div>
</td>
</tr>
<tr id="i4" class="altColor">
<td class="colFirst"><code>static <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Write</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#write--">write</a></span>()</code>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a>.</div>
</td>
</tr>
</table>
<ul class="blockList">
<li class="blockList"><a name="methods.inherited.from.class.java.lang.Object">
<!-- -->
</a>
<h3>Methods inherited from class&nbsp;java.lang.Object</h3>
<code>clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait</code></li>
</ul>
</li>
</ul>
</li>
</ul>
</div>
<div class="details">
<ul class="blockList">
<li class="blockList">
<!-- ============ METHOD DETAIL ========== -->
<ul class="blockList">
<li class="blockList"><a name="method.detail">
<!-- -->
</a>
<h3>Method Detail</h3>
<a name="read--">
<!-- -->
</a>
<ul class="blockList">
<li class="blockList">
<h4>read</h4>
<pre>public static&nbsp;<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Read</a>&nbsp;read()</pre>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Read</code></a>. Before use, the <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Read</code></a> must be
configured with a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withInstanceId-java.lang.String-"><code>SpannerIO.Read.withInstanceId(java.lang.String)</code></a> and <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Read.html#withDatabaseId-java.lang.String-"><code>SpannerIO.Read.withDatabaseId(java.lang.String)</code></a> that identify the
Cloud Spanner database.</div>
</li>
</ul>
<a name="readAll--">
<!-- -->
</a>
<ul class="blockList">
<li class="blockList">
<h4>readAll</h4>
<pre>public static&nbsp;<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadAll.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadAll</a>&nbsp;readAll()</pre>
<div class="block">A <a href="../../../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that works like <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html#read--"><code>read()</code></a>, but executes read operations coming from a
<a href="../../../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>.</div>
</li>
</ul>
<a name="createTransaction--">
<!-- -->
</a>
<ul class="blockList">
<li class="blockList">
<h4>createTransaction</h4>
<pre><a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.html" title="annotation in org.apache.beam.sdk.annotations">@Experimental</a>
public static&nbsp;<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.CreateTransaction</a>&nbsp;createTransaction()</pre>
<div class="block">Returns a transform that creates a batch transaction. By default, <a href="https://static.javadoc.io/com.google.cloud/google-cloud-spanner/6.20.0/com/google/cloud/spanner/TimestampBound.html?is-external=true#strong--" title="class or interface in com.google.cloud.spanner"><code>TimestampBound.strong()</code></a> transaction is created, to override this use <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html#withTimestampBound-com.google.cloud.spanner.TimestampBound-"><code>SpannerIO.CreateTransaction.withTimestampBound(TimestampBound)</code></a>.</div>
</li>
</ul>
<a name="write--">
<!-- -->
</a>
<ul class="blockList">
<li class="blockList">
<h4>write</h4>
<pre><a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.html" title="annotation in org.apache.beam.sdk.annotations">@Experimental</a>
public static&nbsp;<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.Write</a>&nbsp;write()</pre>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a>. Before use, the <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.Write</code></a> must be
configured with a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withInstanceId-java.lang.String-"><code>SpannerIO.Write.withInstanceId(java.lang.String)</code></a> and <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.Write.html#withDatabaseId-java.lang.String-"><code>SpannerIO.Write.withDatabaseId(java.lang.String)</code></a> that identify
the Cloud Spanner database being written.</div>
</li>
</ul>
<a name="readChangeStream--">
<!-- -->
</a>
<ul class="blockListLast">
<li class="blockList">
<h4>readChangeStream</h4>
<pre><a href="../../../../../../../org/apache/beam/sdk/annotations/Experimental.html" title="annotation in org.apache.beam.sdk.annotations">@Experimental</a>
public static&nbsp;<a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner">SpannerIO.ReadChangeStream</a>&nbsp;readChangeStream()</pre>
<div class="block">Creates an uninitialized instance of <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.ReadChangeStream</code></a>. Before use, the <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><code>SpannerIO.ReadChangeStream</code></a> must be configured with a <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html#withProjectId-java.lang.String-"><code>SpannerIO.ReadChangeStream.withProjectId(java.lang.String)</code></a>, <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html#withInstanceId-java.lang.String-"><code>SpannerIO.ReadChangeStream.withInstanceId(java.lang.String)</code></a>, and <a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.ReadChangeStream.html#withDatabaseId-java.lang.String-"><code>SpannerIO.ReadChangeStream.withDatabaseId(java.lang.String)</code></a> that identify the
Cloud Spanner database being written. It must also be configured with the start time and the
change stream name.</div>
</li>
</ul>
</li>
</ul>
</li>
</ul>
</div>
</div>
<!-- ========= END OF CLASS DATA ========= -->
<!-- ======= START OF BOTTOM NAVBAR ====== -->
<div class="bottomNav"><a name="navbar.bottom">
<!-- -->
</a>
<div class="skipNav"><a href="#skip.navbar.bottom" title="Skip navigation links">Skip navigation links</a></div>
<a name="navbar.bottom.firstrow">
<!-- -->
</a>
<ul class="navList" title="Navigation">
<li><a href="../../../../../../../overview-summary.html">Overview</a></li>
<li><a href="package-summary.html">Package</a></li>
<li class="navBarCell1Rev">Class</li>
<li><a href="package-tree.html">Tree</a></li>
<li><a href="../../../../../../../deprecated-list.html">Deprecated</a></li>
<li><a href="../../../../../../../index-all.html">Index</a></li>
<li><a href="../../../../../../../help-doc.html">Help</a></li>
</ul>
</div>
<div class="subNav">
<ul class="navList">
<li><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.Builder.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><span class="typeNameLink">Prev&nbsp;Class</span></a></li>
<li><a href="../../../../../../../org/apache/beam/sdk/io/gcp/spanner/SpannerIO.CreateTransaction.html" title="class in org.apache.beam.sdk.io.gcp.spanner"><span class="typeNameLink">Next&nbsp;Class</span></a></li>
</ul>
<ul class="navList">
<li><a href="../../../../../../../index.html?org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html" target="_top">Frames</a></li>
<li><a href="SpannerIO.html" target="_top">No&nbsp;Frames</a></li>
</ul>
<ul class="navList" id="allclasses_navbar_bottom">
<li><a href="../../../../../../../allclasses-noframe.html">All&nbsp;Classes</a></li>
</ul>
<div>
<script type="text/javascript"><!--
allClassesLink = document.getElementById("allclasses_navbar_bottom");
if(window==top) {
allClassesLink.style.display = "block";
}
else {
allClassesLink.style.display = "none";
}
//-->
</script>
</div>
<div>
<ul class="subNavList">
<li>Summary:&nbsp;</li>
<li><a href="#nested.class.summary">Nested</a>&nbsp;|&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li>Constr&nbsp;|&nbsp;</li>
<li><a href="#method.summary">Method</a></li>
</ul>
<ul class="subNavList">
<li>Detail:&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li>Constr&nbsp;|&nbsp;</li>
<li><a href="#method.detail">Method</a></li>
</ul>
</div>
<a name="skip.navbar.bottom">
<!-- -->
</a></div>
<!-- ======== END OF BOTTOM NAVBAR ======= -->
</body>
</html>