Class BigtableIO
Transforms
for reading from and writing to Google Cloud Bigtable.
Please note the Cloud BigTable HBase connector available here. We recommend using that connector over this one if HBase APIinvalid input: '<'/> works for your needs.
For more information about Cloud Bigtable, see the online documentation at Google Cloud Bigtable.
Reading from Cloud Bigtable
The Bigtable source returns a set of rows from a single table, returning a
PCollection<Row>
.
To configure a Cloud Bigtable source, you must supply a table id, a project id, an instance id
and optionally a BigtableOptions
to provide more specific connection configuration. By
default, BigtableIO.Read
will read all rows in the table. The row ranges to be read can
optionally be restricted using BigtableIO.Read.withKeyRanges(org.apache.beam.sdk.options.ValueProvider<java.util.List<org.apache.beam.sdk.io.range.ByteKeyRange>>)
, and a RowFilter
can
be specified using BigtableIO.Read.withRowFilter(org.apache.beam.sdk.options.ValueProvider<com.google.bigtable.v2.RowFilter>)
. For example:
Pipeline p = ...;
// Scan the entire table.
p.apply("read",
BigtableIO.read()
.withProjectId(projectId)
.withInstanceId(instanceId)
.withTableId("table"));
// Scan a prefix of the table.
ByteKeyRange keyRange = ...;
p.apply("read",
BigtableIO.read()
.withProjectId(projectId)
.withInstanceId(instanceId)
.withTableId("table")
.withKeyRange(keyRange));
// Scan a subset of rows that match the specified row filter.
p.apply("filtered read",
BigtableIO.read()
.withProjectId(projectId)
.withInstanceId(instanceId)
.withTableId("table")
.withRowFilter(filter));
// Configure timeouts for reads.
// Let each attempt run for 1 second, retry if the attempt failed.
// Give up after the request is retried for 60 seconds.
Duration attemptTimeout = Duration.millis(1000);
Duration operationTimeout = Duration.millis(60 * 1000);
p.apply("read",
BigtableIO.read()
.withProjectId(projectId)
.withInstanceId(instanceId)
.withTableId("table")
.withKeyRange(keyRange)
.withAttemptTimeout(attemptTimeout)
.withOperationTimeout(operationTimeout);
Writing to Cloud Bigtable
The Bigtable sink executes a set of row mutations on a single table. It takes as input a
PCollection<KV<ByteString, Iterable<Mutation>>>
, where the
ByteString
is the key of the row being mutated, and each Mutation
represents an
idempotent transformation to that row.
To configure a Cloud Bigtable sink, you must supply a table id, a project id, an instance id
and optionally a configuration function for BigtableOptions
to provide more specific
connection configuration, for example:
PCollection<KV<ByteString, Iterable<Mutation>>> data = ...;
data.apply("write",
BigtableIO.write()
.withProjectId("project")
.withInstanceId("instance")
.withTableId("table"));
// Configure batch size for writes
data.apply("write",
BigtableIO.write()
.withProjectId("project")
.withInstanceId("instance")
.withTableId("table")
.withBatchElements(100)); // every batch will have 100 elements
}
Configure timeout for writes:
// Let each attempt run for 1 second, retry if the attempt failed.
// Give up after the request is retried for 60 seconds.
Duration attemptTimeout = Duration.millis(1000);
Duration operationTimeout = Duration.millis(60 * 1000);
data.apply("write",
BigtableIO.write()
.withProjectId("project")
.withInstanceId("instance")
.withTableId("table")
.withAttemptTimeout(attemptTimeout)
.withOperationTimeout(operationTimeout));
You can also limit the wait time in the finish bundle step by setting the bigtable_writer_wait_timeout_ms experimental flag when you run the pipeline. For example, --experiments=bigtable_writer_wait_timeout_ms=60000 will limit the wait time in finish bundle to be 10 minutes.
Optionally, BigtableIO.write() may be configured to emit BigtableWriteResult
elements
after each group of inputs is written to Bigtable. These can be used to then trigger user code
after writes have completed. See Wait
for details on the
windowing requirements of the signal and input PCollections.
// See Wait.on
PCollection<KV<ByteString, Iterable<Mutation>>> data = ...;
PCollection<BigtableWriteResult> writeResults =
data.apply("write",
BigtableIO.write()
.withProjectId("project")
.withInstanceId("instance")
.withTableId("table"))
.withWriteResults();
// The windowing of `moreData` must be compatible with `data`, see {@link org.apache.beam.sdk.transforms.Wait#on}
// for details.
PCollection<...> moreData = ...;
moreData
.apply("wait for writes", Wait.on(writeResults))
.apply("do something", ParDo.of(...))
Streaming Changes from Cloud Bigtable
Cloud Bigtable change streams enable users to capture and stream out mutations from their Cloud Bigtable tables in real-time. Cloud Bigtable change streams enable many use cases including integrating with a user's data analytics pipelines, support audit and archival requirements as well as triggering downstream application logic on specific database changes.
Change stream connector creates and manages a metadata table to manage the state of the
connector. By default, the table is created in the same instance as the table being streamed.
However, it can be overridden with BigtableIO.ReadChangeStream.withMetadataTableProjectId(java.lang.String)
, BigtableIO.ReadChangeStream.withMetadataTableInstanceId(java.lang.String)
, BigtableIO.ReadChangeStream.withMetadataTableTableId(java.lang.String)
, and BigtableIO.ReadChangeStream.withMetadataTableAppProfileId(java.lang.String)
. The app profile for the metadata
table must be a single cluster app profile with single row transaction enabled.
Note - To prevent unforeseen stream stalls, the BigtableIO connector outputs all data with an output timestamp of zero, making all data late, which will ensure that the stream will not stall. However, it means that you may have to deal with all data as late data, and features that depend on watermarks will not function. This means that Windowing functions and States and Timers are no longer effectively usable. Example use cases that are not possible because of this include:
- Completeness in a replicated cluster with writes to a row on multiple clusters.
- Ordering at the row level in a replicated cluster where the row is being written through multiple-clusters.
Pipeline pipeline = ...;
pipeline
.apply(
BigtableIO.readChangeStream()
.withProjectId(projectId)
.withInstanceId(instanceId)
.withTableId(tableId)
.withAppProfileId(appProfileId)
.withStartTime(startTime));
Enable client side metrics
Client side metrics can be enabled with an experiments flag when you run the pipeline: --experiments=bigtable_enable_client_side_metrics. These metrics can provide additional insights to your job. You can read more about client side metrics in this documentation: https://cloud.google.com/bigtable/docs/client-side-metrics.
Permissions
Permission requirements depend on the PipelineRunner
that is used to execute the
pipeline. Please refer to the documentation of corresponding PipelineRunners
for more details.
Updates to the I/O connector code
For any significant updates to this I/O connector, please consider involving corresponding code reviewers mentioned here.-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic enum
Overwrite options to determine what to do if change stream name is being reused and there exists metadata of the same change stream name.static class
APTransform
that reads from Google Cloud Bigtable.static class
static class
APTransform
that writes to Google Cloud Bigtable.static class
APTransform
that writes to Google Cloud Bigtable and emits aBigtableWriteResult
for each batch written. -
Method Summary
Modifier and TypeMethodDescriptionstatic boolean
createOrUpdateReadChangeStreamMetadataTable
(String projectId, String instanceId, @Nullable String tableId) Utility method to create or update Read Change Stream metadata table.static BigtableIO.Read
read()
Creates an uninitializedBigtableIO.Read
.static BigtableIO.ReadChangeStream
Creates an uninitializedBigtableIO.ReadChangeStream
.static BigtableIO.Write
write()
Creates an uninitializedBigtableIO.Write
.
-
Method Details
-
read
Creates an uninitializedBigtableIO.Read
. Before use, theRead
must be initialized with aBigtableIO.Read.withInstanceId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
andBigtableIO.Read.withProjectId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
that specifies the source Cloud Bigtable instance, and aBigtableIO.Read.withTableId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
that specifies which table to read. ARowFilter
may also optionally be specified usingBigtableIO.Read.withRowFilter(RowFilter)
. -
write
Creates an uninitializedBigtableIO.Write
. Before use, theWrite
must be initialized with aBigtableIO.Write.withProjectId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
andBigtableIO.Write.withInstanceId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
that specifies the destination Cloud Bigtable instance, and aBigtableIO.Write.withTableId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>)
that specifies which table to write. -
readChangeStream
Creates an uninitializedBigtableIO.ReadChangeStream
. Before use, theReadChangeStream
must be initialized withBigtableIO.ReadChangeStream.withProjectId(java.lang.String)
BigtableIO.ReadChangeStream.withInstanceId(java.lang.String)
BigtableIO.ReadChangeStream.withTableId(java.lang.String)
BigtableIO.ReadChangeStream.withAppProfileId(java.lang.String)
And optionally with
BigtableIO.ReadChangeStream.withStartTime(org.joda.time.Instant)
which defaults to now.BigtableIO.ReadChangeStream.withMetadataTableProjectId(java.lang.String)
which defaults to value fromBigtableIO.ReadChangeStream.withProjectId(java.lang.String)
BigtableIO.ReadChangeStream.withMetadataTableInstanceId(java.lang.String)
which defaults to value fromBigtableIO.ReadChangeStream.withInstanceId(java.lang.String)
BigtableIO.ReadChangeStream.withMetadataTableTableId(java.lang.String)
which defaults toMetadataTableAdminDao.DEFAULT_METADATA_TABLE_NAME
BigtableIO.ReadChangeStream.withMetadataTableAppProfileId(java.lang.String)
which defaults to value fromBigtableIO.ReadChangeStream.withAppProfileId(java.lang.String)
BigtableIO.ReadChangeStream.withChangeStreamName(java.lang.String)
which defaults to randomly generated string.
-
createOrUpdateReadChangeStreamMetadataTable
public static boolean createOrUpdateReadChangeStreamMetadataTable(String projectId, String instanceId, @Nullable String tableId) throws IOException Utility method to create or update Read Change Stream metadata table. This requires Bigtable table create permissions. This method is useful if the pipeline isn't granted permissions to create Bigtable tables. Run this method with correct permissions to create the metadata table, which is required to read Bigtable change streams. This method only needs to be run once, and the metadata table can be reused for all pipelines.- Parameters:
projectId
- project id of the metadata table, usually the same as the project of the table being streamedinstanceId
- instance id of the metadata table, usually the same as the instance of the table being streamedtableId
- name of the metadata table, leave it null or empty to use default.- Returns:
- true if the table was successfully created. Otherwise, false.
- Throws:
IOException
-