[ 
https://issues.apache.org/jira/browse/BEAM-4020?focusedWorklogId=122255&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-122255
 ]

ASF GitHub Bot logged work on BEAM-4020:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Jul/18 09:28
            Start Date: 12/Jul/18 09:28
    Worklog Time Spent: 10m 
      Work Description: iemejia closed pull request #5212: [BEAM-4020] Add 
HBaseIO.readAll() based on SDF
URL: https://github.com/apache/beam/pull/5212
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java
 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java
index f836ebe3551..4127224a75c 100644
--- 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java
+++ 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java
@@ -33,6 +33,7 @@
   public List<CoderProvider> getCoderProviders() {
     return ImmutableList.of(
         HBaseMutationCoder.getCoderProvider(),
-        CoderProviders.forCoder(TypeDescriptor.of(Result.class), 
HBaseResultCoder.of()));
+        CoderProviders.forCoder(TypeDescriptor.of(Result.class), 
HBaseResultCoder.of()),
+        CoderProviders.forCoder(TypeDescriptor.of(HBaseQuery.class), 
HBaseQueryCoder.of()));
   }
 }
diff --git 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
index 5f04ba18124..9283ed040bb 100644
--- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
+++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
@@ -107,6 +107,20 @@
  *         .withFilter(filter));
  * }</pre>
  *
+ * <p>{@link HBaseIO#readAll()} allows to execute multiple {@link Scan}s to 
multiple {@link Table}s.
+ * These queries are encapsulated via an initial {@link PCollection} of {@link 
HBaseQuery}s and can
+ * be used to create advanced compositional patterns like reading from a 
Source and then based on
+ * the data create new HBase scans.
+ *
+ * <p><b>Note:</b> {@link HBaseIO.ReadAll} only works with <a
+ * 
href="https://beam.apache.org/documentation/runners/capability-matrix/";>runners 
that support
+ * Splittable DoFn</a>.
+ *
+ * <pre>{@code
+ * PCollection<HBaseQuery> queries = ...;
+ * queries.apply("readAll", 
HBaseIO.readAll().withConfiguration(configuration));
+ * }</pre>
+ *
  * <h3>Writing to HBase</h3>
  *
  * <p>The HBase sink executes a set of row mutations on a single table. It 
takes as input a {@link
@@ -145,7 +159,6 @@ private HBaseIO() {}
    * and a {@link HBaseIO.Read#withTableId tableId} that specifies which table 
to read. A {@link
    * Filter} may also optionally be specified using {@link 
HBaseIO.Read#withFilter}.
    */
-  @Experimental
   public static Read read() {
     return new Read(null, "", new SerializableScan(new Scan()));
   }
@@ -219,11 +232,9 @@ private Read(
       } catch (IOException e) {
         LOG.warn("Error checking whether table {} exists; proceeding.", 
tableId, e);
       }
-      return input
-          .getPipeline()
-          .apply(
-              org.apache.beam.sdk.io.Read.from(
-                  new HBaseSource(this, null /* estimatedSizeBytes */)));
+
+      return input.apply(
+          org.apache.beam.sdk.io.Read.from(new HBaseSource(this, null /* 
estimatedSizeBytes */)));
     }
 
     @Override
@@ -234,12 +245,16 @@ public void populateDisplayData(DisplayData.Builder 
builder) {
       builder.addIfNotNull(DisplayData.item("scan", 
serializableScan.get().toString()));
     }
 
+    public Configuration getConfiguration() {
+      return serializableConfiguration.get();
+    }
+
     public String getTableId() {
       return tableId;
     }
 
-    public Configuration getConfiguration() {
-      return serializableConfiguration.get();
+    public Scan getScan() {
+      return serializableScan.get();
     }
 
     /** Returns the range of keys that will be read from the table. */
@@ -254,6 +269,36 @@ public ByteKeyRange getKeyRange() {
     private final SerializableScan serializableScan;
   }
 
+  /**
+   * A {@link PTransform} that works like {@link #read}, but executes read 
operations coming from a
+   * {@link PCollection} of {@link HBaseQuery}.
+   */
+  public static ReadAll readAll() {
+    return new ReadAll(null);
+  }
+
+  /** Implementation of {@link #readAll}. */
+  public static class ReadAll extends PTransform<PCollection<HBaseQuery>, 
PCollection<Result>> {
+
+    private ReadAll(SerializableConfiguration serializableConfiguration) {
+      this.serializableConfiguration = serializableConfiguration;
+    }
+
+    /** Reads from the HBase instance indicated by the* given configuration. */
+    public ReadAll withConfiguration(Configuration configuration) {
+      checkArgument(configuration != null, "configuration can not be null");
+      return new ReadAll(new SerializableConfiguration(configuration));
+    }
+
+    @Override
+    public PCollection<Result> expand(PCollection<HBaseQuery> input) {
+      checkArgument(serializableConfiguration != null, "withConfiguration() is 
required");
+      return input.apply(ParDo.of(new 
HBaseReadSplittableDoFn(serializableConfiguration)));
+    }
+
+    private SerializableConfiguration serializableConfiguration;
+  }
+
   static class HBaseSource extends BoundedSource<Result> {
     private final Read read;
     @Nullable private Long estimatedSizeBytes;
diff --git 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQuery.java 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQuery.java
new file mode 100644
index 00000000000..b6ac2e95e99
--- /dev/null
+++ 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQuery.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.hbase;
+
+import org.apache.hadoop.hbase.client.Scan;
+
+/**
+ * A class to encapsulate a query to HBase. It is composed by the id of the 
table and the {@link
+ * Scan} object.
+ */
+public final class HBaseQuery {
+
+  public static HBaseQuery of(String tableId, Scan scan) {
+    return new HBaseQuery(tableId, scan);
+  }
+
+  private HBaseQuery(String tableId, Scan scan) {
+    this.tableId = tableId;
+    this.scan = scan;
+  }
+
+  public String getTableId() {
+    return tableId;
+  }
+
+  public Scan getScan() {
+    return scan;
+  }
+
+  @Override
+  public String toString() {
+    return "HBaseQuery{" + "tableId='" + tableId + '\'' + ", scan=" + scan + 
'}';
+  }
+
+  private final String tableId;
+  private final Scan scan;
+}
diff --git 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQueryCoder.java
 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQueryCoder.java
new file mode 100644
index 00000000000..5556e91e39d
--- /dev/null
+++ 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseQueryCoder.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.hbase;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+
+/**
+ * A {@link Coder} that serializes and deserializes the {@link HBaseQuery} 
objects using an {@link
+ * StringUtf8Coder} to represent the tableId and {@link ProtobufUtil} to 
represent the HBase {@link
+ * Scan} object.
+ */
+class HBaseQueryCoder extends AtomicCoder<HBaseQuery> implements Serializable {
+  private static final HBaseQueryCoder INSTANCE = new HBaseQueryCoder();
+
+  private HBaseQueryCoder() {}
+
+  static HBaseQueryCoder of() {
+    return INSTANCE;
+  }
+
+  @Override
+  public void encode(HBaseQuery query, OutputStream outputStream) throws 
IOException {
+    StringUtf8Coder.of().encode(query.getTableId(), outputStream);
+    ProtobufUtil.toScan(query.getScan()).writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public HBaseQuery decode(InputStream inputStream) throws IOException {
+    final String tableId = StringUtf8Coder.of().decode(inputStream);
+    final Scan scan = 
ProtobufUtil.toScan(ClientProtos.Scan.parseDelimitedFrom(inputStream));
+    return HBaseQuery.of(tableId, scan);
+  }
+}
diff --git 
a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseReadSplittableDoFn.java
 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseReadSplittableDoFn.java
new file mode 100644
index 00000000000..5df8c037ea0
--- /dev/null
+++ 
b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseReadSplittableDoFn.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.hbase;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.beam.sdk.io.hadoop.SerializableConfiguration;
+import org.apache.beam.sdk.io.range.ByteKey;
+import org.apache.beam.sdk.io.range.ByteKeyRange;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement;
+import org.apache.beam.sdk.transforms.splittabledofn.ByteKeyRangeTracker;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+
+/** A SplittableDoFn to read from HBase. */
+@BoundedPerElement
+class HBaseReadSplittableDoFn extends DoFn<HBaseQuery, Result> {
+  private final SerializableConfiguration serializableConfiguration;
+
+  private transient Connection connection;
+
+  HBaseReadSplittableDoFn(SerializableConfiguration serializableConfiguration) 
{
+    this.serializableConfiguration = serializableConfiguration;
+  }
+
+  @Setup
+  public void setup() throws Exception {
+    connection = 
ConnectionFactory.createConnection(serializableConfiguration.get());
+  }
+
+  private static Scan newScanInRange(Scan scan, ByteKeyRange range) throws 
IOException {
+    return new Scan(scan)
+        .setStartRow(range.getStartKey().getBytes())
+        .setStopRow(range.getEndKey().getBytes());
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c, ByteKeyRangeTracker tracker) 
throws Exception {
+    final HBaseQuery query = c.element();
+    TableName tableName = TableName.valueOf(query.getTableId());
+    Table table = connection.getTable(tableName);
+    final ByteKeyRange range = tracker.currentRestriction();
+    try (ResultScanner scanner = 
table.getScanner(newScanInRange(query.getScan(), range))) {
+      for (Result result : scanner) {
+        ByteKey key = ByteKey.copyFrom(result.getRow());
+        if (!tracker.tryClaim(key)) {
+          return;
+        }
+        c.output(result);
+      }
+      tracker.markDone();
+    }
+  }
+
+  @GetInitialRestriction
+  public ByteKeyRange getInitialRestriction(HBaseQuery query) {
+    return ByteKeyRange.of(
+        ByteKey.copyFrom(query.getScan().getStartRow()),
+        ByteKey.copyFrom(query.getScan().getStopRow()));
+  }
+
+  @SplitRestriction
+  public void splitRestriction(
+      HBaseQuery query, ByteKeyRange range, OutputReceiver<ByteKeyRange> 
receiver)
+      throws Exception {
+    List<HRegionLocation> regionLocations =
+        HBaseUtils.getRegionLocations(connection, query.getTableId(), 
query.getScan());
+    List<ByteKeyRange> splitRanges =
+        HBaseUtils.getRanges(regionLocations, query.getTableId(), 
query.getScan());
+    for (ByteKeyRange splitRange : splitRanges) {
+      receiver.output(ByteKeyRange.of(splitRange.getStartKey(), 
splitRange.getEndKey()));
+    }
+  }
+
+  @NewTracker
+  public ByteKeyRangeTracker newTracker(ByteKeyRange range) {
+    return ByteKeyRangeTracker.of(range);
+  }
+
+  @Teardown
+  public void tearDown() throws Exception {
+    if (connection != null) {
+      connection.close();
+      connection = null;
+    }
+  }
+}
diff --git 
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
 
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
index b100178a18e..5f291859e34 100644
--- 
a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
+++ 
b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
@@ -172,7 +172,9 @@ public void testReadingFailsTableDoesNotExist() {
     // Exception will be thrown by read.expand() when read is applied.
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(String.format("Table %s does not exist", table));
-    runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), new 
ArrayList<>());
+    runReadTest(
+        HBaseIO.read().withConfiguration(conf).withTableId(table), false, new 
ArrayList<>());
+    runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), 
true, new ArrayList<>());
   }
 
   /** Tests that when reading from an empty table, the read succeeds. */
@@ -180,16 +182,28 @@ public void testReadingFailsTableDoesNotExist() {
   public void testReadingEmptyTable() throws Exception {
     final String table = tmpTable.getName();
     createTable(table);
-    runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), new 
ArrayList<>());
+
+    runReadTest(
+        HBaseIO.read().withConfiguration(conf).withTableId(table), false, new 
ArrayList<>());
+    runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), 
true, new ArrayList<>());
   }
 
   @Test
   public void testReading() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 1001;
-    createTable(table);
-    writeData(table, numRows);
-    
runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table), 
1001);
+    createAndWriteData(table, numRows);
+
+    
runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table), 
false, numRows);
+  }
+
+  @Test
+  public void testReadingSDF() throws Exception {
+    final String table = tmpTable.getName();
+    final int numRows = 1001;
+    createAndWriteData(table, numRows);
+
+    
runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table), 
true, numRows);
   }
 
   /** Tests reading all rows from a split table. */
@@ -199,10 +213,7 @@ public void testReadingWithSplits() throws Exception {
     final int numRows = 1500;
     final int numRegions = 4;
     final long bytesPerRow = 100L;
-
-    // Set up test table data and sample row keys for size estimation and 
splitting.
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     HBaseIO.Read read = 
HBaseIO.read().withConfiguration(conf).withTableId(table);
     HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */);
@@ -219,10 +230,7 @@ public void testReadingWithSplits() throws Exception {
   public void testReadingSourceTwice() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 10;
-
-    // Set up test table data and sample row keys for size estimation and 
splitting.
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     HBaseIO.Read read = 
HBaseIO.read().withConfiguration(conf).withTableId(table);
     HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */);
@@ -236,15 +244,24 @@ public void testReadingSourceTwice() throws Exception {
   public void testReadingWithFilter() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 1001;
+    createAndWriteData(table, numRows);
 
-    createTable(table);
-    writeData(table, numRows);
+    String regex = ".*17.*";
+    Filter filter = new RowFilter(CompareFilter.CompareOp.EQUAL, new 
RegexStringComparator(regex));
+    runReadTestLength(
+        
HBaseIO.read().withConfiguration(conf).withTableId(table).withFilter(filter), 
false, 20);
+  }
+
+  @Test
+  public void testReadingWithFilterSDF() throws Exception {
+    final String table = tmpTable.getName();
+    final int numRows = 1001;
+    createAndWriteData(table, numRows);
 
     String regex = ".*17.*";
     Filter filter = new RowFilter(CompareFilter.CompareOp.EQUAL, new 
RegexStringComparator(regex));
-    HBaseIO.Read read =
-        
HBaseIO.read().withConfiguration(conf).withTableId(table).withFilter(filter);
-    runReadTestLength(read, 20);
+    runReadTestLength(
+        
HBaseIO.read().withConfiguration(conf).withTableId(table).withFilter(filter), 
true, 20);
   }
 
   /**
@@ -256,14 +273,14 @@ public void testReadingKeyRangePrefix() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 1001;
     final ByteKey startKey = 
ByteKey.copyFrom("2".getBytes(StandardCharsets.UTF_8));
-
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     // Test prefix: [beginning, startKey).
     final ByteKeyRange prefixRange = 
ByteKeyRange.ALL_KEYS.withEndKey(startKey);
     runReadTestLength(
-        
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(prefixRange),
 126);
+        
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(prefixRange),
+        false,
+        126);
   }
 
   /**
@@ -275,14 +292,14 @@ public void testReadingKeyRangeSuffix() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 1001;
     final ByteKey startKey = 
ByteKey.copyFrom("2".getBytes(StandardCharsets.UTF_8));
-
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     // Test suffix: [startKey, end).
     final ByteKeyRange suffixRange = 
ByteKeyRange.ALL_KEYS.withStartKey(startKey);
     runReadTestLength(
-        
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(suffixRange),
 875);
+        
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(suffixRange),
+        false,
+        875);
   }
 
   /**
@@ -295,14 +312,29 @@ public void testReadingKeyRangeMiddle() throws Exception {
     final int numRows = 1001;
     final byte[] startRow = "2".getBytes(StandardCharsets.UTF_8);
     final byte[] stopRow = "9".getBytes(StandardCharsets.UTF_8);
+    createAndWriteData(table, numRows);
 
-    createTable(table);
-    writeData(table, numRows);
+    // Test restricted range: [startKey, endKey).
+    // This one tests the second signature of .withKeyRange
+    runReadTestLength(
+        
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(startRow,
 stopRow),
+        false,
+        441);
+  }
+
+  @Test
+  public void testReadingKeyRangeMiddleSDF() throws Exception {
+    final String table = tmpTable.getName();
+    final int numRows = 1001;
+    final byte[] startRow = "2".getBytes(StandardCharsets.UTF_8);
+    final byte[] stopRow = "9".getBytes(StandardCharsets.UTF_8);
+    createAndWriteData(table, numRows);
 
     // Test restricted range: [startKey, endKey).
     // This one tests the second signature of .withKeyRange
     runReadTestLength(
         
HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(startRow,
 stopRow),
+        true,
         441);
   }
 
@@ -311,9 +343,7 @@ public void testReadingKeyRangeMiddle() throws Exception {
   public void testReadingSplitAtFractionExhaustive() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 7;
-
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     HBaseIO.Read read = 
HBaseIO.read().withConfiguration(conf).withTableId(table);
     HBaseSource source =
@@ -329,9 +359,7 @@ public void testReadingSplitAtFractionExhaustive() throws 
Exception {
   public void testReadingSplitAtFraction() throws Exception {
     final String table = tmpTable.getName();
     final int numRows = 10;
-
-    createTable(table);
-    writeData(table, numRows);
+    createAndWriteData(table, numRows);
 
     HBaseIO.Read read = 
HBaseIO.read().withConfiguration(conf).withTableId(table);
     HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */);
@@ -447,6 +475,11 @@ private static void writeData(String tableId, int numRows) 
throws Exception {
     mutator.close();
   }
 
+  private static void createAndWriteData(final String tableId, final int 
numRows) throws Exception {
+    createTable(tableId);
+    writeData(tableId, numRows);
+  }
+
   private static List<Mutation> makeTableData(int numRows) {
     List<Mutation> mutations = new ArrayList<>(numRows);
     for (int i = 0; i < numRows; ++i) {
@@ -499,21 +532,31 @@ private static Mutation makeBadMutation(String key) {
     return new Put(key.getBytes(StandardCharsets.UTF_8));
   }
 
-  private void runReadTest(HBaseIO.Read read, List<Result> expected) {
-    final String transformId = read.getTableId() + "_" + read.getKeyRange();
-    PCollection<Result> rows = p.apply("Read" + transformId, read);
+  private void runReadTest(HBaseIO.Read read, boolean useSdf, List<Result> 
expected) {
+    PCollection<Result> rows = applyRead(read, useSdf);
     PAssert.that(rows).containsInAnyOrder(expected);
     p.run().waitUntilFinish();
   }
 
-  private void runReadTestLength(HBaseIO.Read read, long numElements) {
+  private void runReadTestLength(HBaseIO.Read read, boolean useSdf, long 
numElements) {
+    PCollection<Result> rows = applyRead(read, useSdf);
     final String transformId = read.getTableId() + "_" + read.getKeyRange();
-    PCollection<Result> rows = p.apply("Read" + transformId, read);
     PAssert.thatSingleton(rows.apply("Count" + transformId, Count.globally()))
         .isEqualTo(numElements);
     p.run().waitUntilFinish();
   }
 
+  private PCollection<Result> applyRead(HBaseIO.Read read, boolean useSdf) {
+    final String transformId = read.getTableId() + "_" + read.getKeyRange();
+    return useSdf
+        ? p.apply(
+                "Create" + transformId, 
Create.of(HBaseQuery.of(read.getTableId(), read.getScan())))
+            .apply(
+                "ReadAll" + transformId,
+                HBaseIO.readAll().withConfiguration(read.getConfiguration()))
+        : p.apply("Read" + transformId, read);
+  }
+
   private static class TemporaryHBaseTable extends ExternalResource {
     private String name;
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 122255)
    Time Spent: 4.5h  (was: 4h 20m)

> Add HBaseIO.readAll() based on SDF
> ----------------------------------
>
>                 Key: BEAM-4020
>                 URL: https://issues.apache.org/jira/browse/BEAM-4020
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-hbase
>            Reporter: Ismaël Mejía
>            Assignee: Ismaël Mejía
>            Priority: Minor
>          Time Spent: 4.5h
>  Remaining Estimate: 0h
>
> Since the support from runners is still limited, it is probably wise to 
> create a first IO based on the current SDF batch implementation in Java to 
> validate/test it with a real data-store. Since HBase partitioning model is 
> quite straightforward it is a perfect candidate.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to