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

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

                Author: ASF GitHub Bot
            Created on: 10/Feb/20 08:55
            Start Date: 10/Feb/20 08:55
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on pull request #10815: [BEAM-9279] 
Make HBase.ReadAll based on Reads instead of HBaseQuery
URL: https://github.com/apache/beam/pull/10815#discussion_r376930619
 
 

 ##########
 File path: 
sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseReadSplittableDoFn.java
 ##########
 @@ -32,65 +31,50 @@
 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());
-  }
+class HBaseReadSplittableDoFn extends DoFn<Read, Result> {
+  HBaseReadSplittableDoFn() {}
 
   @ProcessElement
-  public void processElement(ProcessContext c, 
RestrictionTracker<ByteKeyRange, ByteKey> tracker)
+  public void processElement(
+      @Element Read read,
+      OutputReceiver<Result> out,
+      RestrictionTracker<ByteKeyRange, ByteKey> tracker)
       throws Exception {
-    final HBaseQuery query = c.element();
-    TableName tableName = TableName.valueOf(query.getTableId());
+    Connection connection = 
ConnectionFactory.createConnection(read.getConfiguration());
 
 Review comment:
   This creates a connection for each Read element, this is arguable costly 
because its nature is 1 to many, but it is an acceptable trade-off to support 
multiple Configurations and have the multi cluster flexibility.
   
   Connection initialization could be improved in the future via some sort of 
class based Pool mechanism (as we do for JdbcIO).
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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: 384264)
    Time Spent: 0.5h  (was: 20m)

> Make HBase.ReadAll based on Reads instead of HBaseQuery
> -------------------------------------------------------
>
>                 Key: BEAM-9279
>                 URL: https://issues.apache.org/jira/browse/BEAM-9279
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-hbase
>            Reporter: Ismaël Mejía
>            Assignee: Ismaël Mejía
>            Priority: Minor
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> HBaseIO support for SplittableDoFn introduced a new request type HBaseQuery, 
> however the attributes defined in that class are covered by common 
> HBase.Read, and additionally if we rely on Read we allow users to create 
> pipelines who can read from multiple clusters because the Configuration now 
> is part of the request.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to