public class BigtableIO
extends java.lang.Object
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 API> works for your needs.
For more information about Cloud Bigtable, see the online documentation at Google 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);
 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
 }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(...))
 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:
 Pipeline pipeline = ...;
 pipeline
    .apply(
        BigtableIO.readChangeStream()
            .withProjectId(projectId)
            .withInstanceId(instanceId)
            .withTableId(tableId)
            .withAppProfileId(appProfileId)
            .withStartTime(startTime));
 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.
Permission requirements depend on the PipelineRunner that is used to execute the
 pipeline. Please refer to the documentation of corresponding PipelineRunners for more details.
 
| Modifier and Type | Class and Description | 
|---|---|
| static class  | BigtableIO.ExistingPipelineOptionsOverwrite 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  | BigtableIO.ReadA  PTransformthat reads from Google Cloud Bigtable. | 
| static class  | BigtableIO.ReadChangeStream | 
| static class  | BigtableIO.WriteA  PTransformthat writes to Google Cloud Bigtable. | 
| static class  | BigtableIO.WriteWithResultsA  PTransformthat writes to Google Cloud Bigtable and emits aBigtableWriteResultfor each batch written. | 
| Modifier and Type | Method and Description | 
|---|---|
| static boolean | createOrUpdateReadChangeStreamMetadataTable(java.lang.String projectId,
                                           java.lang.String instanceId,
                                           @Nullable java.lang.String tableId)Utility method to create or update Read Change Stream metadata table. | 
| static BigtableIO.Read | read()Creates an uninitialized  BigtableIO.Read. | 
| static BigtableIO.ReadChangeStream | readChangeStream()Creates an uninitialized  BigtableIO.ReadChangeStream. | 
| static BigtableIO.Write | write()Creates an uninitialized  BigtableIO.Write. | 
public static BigtableIO.Read read()
BigtableIO.Read. Before use, the Read must be
 initialized with a BigtableIO.Read.withInstanceId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) and BigtableIO.Read.withProjectId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) that specifies the source Cloud Bigtable instance, and a BigtableIO.Read.withTableId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) that specifies which table to read. A RowFilter may also
 optionally be specified using BigtableIO.Read.withRowFilter(RowFilter).public static BigtableIO.Write write()
BigtableIO.Write. Before use, the Write must be
 initialized with a BigtableIO.Write.withProjectId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) and BigtableIO.Write.withInstanceId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) that specifies the destination Cloud Bigtable instance, and a
 BigtableIO.Write.withTableId(org.apache.beam.sdk.options.ValueProvider<java.lang.String>) that specifies which table to write.public static BigtableIO.ReadChangeStream readChangeStream()
BigtableIO.ReadChangeStream. Before use, the ReadChangeStream must be initialized with
 BigtableIO.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
       from BigtableIO.ReadChangeStream.withProjectId(java.lang.String)
   BigtableIO.ReadChangeStream.withMetadataTableInstanceId(java.lang.String) which defaults to value
       from BigtableIO.ReadChangeStream.withInstanceId(java.lang.String)
   BigtableIO.ReadChangeStream.withMetadataTableTableId(java.lang.String) which defaults to MetadataTableAdminDao.DEFAULT_METADATA_TABLE_NAME
   BigtableIO.ReadChangeStream.withMetadataTableAppProfileId(java.lang.String) which defaults to value
       from BigtableIO.ReadChangeStream.withAppProfileId(java.lang.String)
   BigtableIO.ReadChangeStream.withChangeStreamName(java.lang.String) which defaults to randomly
       generated string.
 public static boolean createOrUpdateReadChangeStreamMetadataTable(java.lang.String projectId,
                                                                  java.lang.String instanceId,
                                                                  @Nullable java.lang.String tableId)
                                                           throws java.io.IOException
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.java.io.IOException