Class HBaseIO
- java.lang.Object
-
- org.apache.beam.sdk.io.hbase.HBaseIO
-
public class HBaseIO extends java.lang.ObjectA bounded source and sink for HBase.For more information, see the online documentation at HBase.
Reading from HBase
The HBase source returns a set of rows from a single table, returning a
PCollection<Result>.To configure a HBase source, you must supply a table id and a
Configurationto identify the HBase instance. By default,HBaseIO.Readwill read all rows in the table. The row range to be read can optionally be restricted using with aScanobject or using theHBaseIO.Read.withKeyRange(org.apache.beam.sdk.io.range.ByteKeyRange), and aFilterusingHBaseIO.Read.withFilter(org.apache.hadoop.hbase.filter.Filter), for example:// Scan the entire table. p.apply("read", HBaseIO.read() .withConfiguration(configuration) .withTableId("table")); // Filter data using a HBaseIO Scan Scan scan = ... p.apply("read", HBaseIO.read() .withConfiguration(configuration) .withTableId("table")) .withScan(scan)); // Scan a prefix of the table. ByteKeyRange keyRange = ...; p.apply("read", HBaseIO.read() .withConfiguration(configuration) .withTableId("table") .withKeyRange(keyRange)); // Scan a subset of rows that match the specified row filter. p.apply("filtered read", HBaseIO.read() .withConfiguration(configuration) .withTableId("table") .withFilter(filter));readAll()allows to execute multipleScans to multipleTables. These queries are encapsulated via an initialPCollectionofHBaseIO.Reads and can be used to create advanced compositional patterns like reading from a Source and then based on the data create new HBase scans.Note:
HBaseIO.ReadAllonly works with runners that support Splittable DoFn.PCollection<Read> queries = ...; queries.apply("readAll", HBaseIO.readAll().withConfiguration(configuration));Writing to HBase
Writing
MutationThe HBase sink executes a set of row mutations on a single table. It takes as input a
PCollection<Mutation>, where eachMutationrepresents an idempotent transformation on a row.To configure a HBase sink, you must supply a table id and a
Configurationto identify the HBase instance, for example:Configuration configuration = ...; PCollection<Mutation> data = ...; data.apply("write", HBaseIO.write() .withConfiguration(configuration) .withTableId("table"));Writing
RowMutationsAn alternative way to write to HBase is with
writeRowMutations(), which takes as input aPCollection, representing KVs of bytes row keys andRowMutations.This implementation is useful for preserving mutation order if the upstream is ordered by row key, as RowMutations will only be applied after previous RowMutations are successful.
To configure the sink, you must supply a table id string and a
Configurationto identify the HBase instance, for example:Configuration configuration = ...; PCollection<KV<byte[], RowMutations>> data = ...; data.apply("write", HBaseIO.writeRowMutations() .withConfiguration(configuration) .withTableId("table"));Note that the transformation emits the number of RowMutations written as an integer after successfully writing to HBase.
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static classHBaseIO.ReadAPTransformthat reads from HBase.static classHBaseIO.ReadAllImplementation ofreadAll().static classHBaseIO.WriteAPTransformthat writes to HBase.static classHBaseIO.WriteRowMutationsTransformation that writes RowMutation objects to a Hbase table.
-
Method Summary
All Methods Static Methods Concrete Methods Modifier and Type Method Description static HBaseIO.Readread()Creates an uninitializedHBaseIO.Read.static HBaseIO.ReadAllreadAll()APTransformthat works likeread(), but executes read operations coming from aPCollectionofHBaseIO.Read.static HBaseIO.Writewrite()Creates an uninitializedHBaseIO.Write.static HBaseIO.WriteRowMutationswriteRowMutations()
-
-
-
Method Detail
-
read
public static HBaseIO.Read read()
Creates an uninitializedHBaseIO.Read. Before use, theReadmust be initialized with aHBaseIO.Read.withConfiguration(Configuration)that specifies the HBase instance, and atableIdthat specifies which table to read. AFiltermay also optionally be specified usingHBaseIO.Read.withFilter(org.apache.hadoop.hbase.filter.Filter).
-
readAll
public static HBaseIO.ReadAll readAll()
APTransformthat works likeread(), but executes read operations coming from aPCollectionofHBaseIO.Read.
-
write
public static HBaseIO.Write write()
Creates an uninitializedHBaseIO.Write. Before use, theWritemust be initialized with aHBaseIO.Write.withConfiguration(Configuration)that specifies the destination HBase instance, and atableIdthat specifies which table to write.
-
writeRowMutations
public static HBaseIO.WriteRowMutations writeRowMutations()
-
-