Class BigtableIO
- java.lang.Object
-
- org.apache.beam.sdk.io.gcp.bigtable.BigtableIO
-
public class BigtableIO extends java.lang.ObjectTransformsfor 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.
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
BigtableOptionsto provide more specific connection configuration. By default,BigtableIO.Readwill read all rows in the table. The row ranges to be read can optionally be restricted usingBigtableIO.Read.withKeyRanges(org.apache.beam.sdk.options.ValueProvider<java.util.List<org.apache.beam.sdk.io.range.ByteKeyRange>>), and aRowFiltercan be specified usingBigtableIO.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));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 theByteStringis the key of the row being mutated, and eachMutationrepresents 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
BigtableOptionsto provide more specific connection configuration, for example:PCollection<KV<ByteString, Iterable<Mutation>>> data = ...; data.apply("write", BigtableIO.write() .withProjectId("project") .withInstanceId("instance") .withTableId("table"));Optionally, BigtableIO.write() may be configured to emit
BigtableWriteResultelements after each group of inputs is written to Bigtable. These can be used to then trigger user code after writes have completed. SeeWaitfor 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(...))Permissions
Permission requirements depend on the
PipelineRunnerthat is used to execute the pipeline. Please refer to the documentation of correspondingPipelineRunnersfor 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 Classes Modifier and Type Class Description static classBigtableIO.ReadAPTransformthat reads from Google Cloud Bigtable.static classBigtableIO.WriteAPTransformthat writes to Google Cloud Bigtable.static classBigtableIO.WriteWithResultsAPTransformthat writes to Google Cloud Bigtable and emits aBigtableWriteResultfor each batch written.
-
Method Summary
All Methods Static Methods Concrete Methods Modifier and Type Method Description static BigtableIO.Readread()Creates an uninitializedBigtableIO.Read.static BigtableIO.Writewrite()Creates an uninitializedBigtableIO.Write.
-
-
-
Method Detail
-
read
public static BigtableIO.Read read()
Creates an uninitializedBigtableIO.Read. Before use, theReadmust 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. ARowFiltermay also optionally be specified usingBigtableIO.Read.withRowFilter(RowFilter).
-
write
public static BigtableIO.Write write()
Creates an uninitializedBigtableIO.Write. Before use, theWritemust 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.
-
-