You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/07/06 17:42:00 UTC

[jira] [Work logged] (BEAM-4020) Add HBaseIO.readAll() based on SDF

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

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

                Author: ASF GitHub Bot
            Created on: 06/Jul/18 17:41
            Start Date: 06/Jul/18 17:41
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on a change in pull request #5212: [BEAM-4020] Add HBaseIO.readAll() based on SDF
URL: https://github.com/apache/beam/pull/5212#discussion_r200723862
 
 

 ##########
 File path: sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseReadSplittableDoFn.java
 ##########
 @@ -0,0 +1,115 @@
+/*
+ * 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.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.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<Void, Result> {
+  private final SerializableConfiguration serializableConfiguration;
+  private final String tableId;
+  private final SerializableScan scan;
+
+  private Connection connection;
+
+  HBaseReadSplittableDoFn(
+      SerializableConfiguration serializableConfiguration, String tableId, SerializableScan scan) {
+    this.serializableConfiguration = serializableConfiguration;
+    this.tableId = tableId;
+    this.scan = scan;
+  }
+
+  @Setup
+  public void setup() throws Exception {
+    if (connection == null) {
+      connection = ConnectionFactory.createConnection(serializableConfiguration.get());
+    }
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c, ByteKeyRangeTracker tracker) throws Exception {
+    TableName tableName = TableName.valueOf(tableId);
+    Table table = connection.getTable(tableName);
+    final ByteKeyRange range = tracker.currentRestriction();
+    Scan newScan =
 
 Review comment:
   Actually I ended up creating a new entity `HBaseQuery` because Scan does not contain the tableId, but the DoFn is now consistent with what  you proposed.

----------------------------------------------------------------
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:
users@infra.apache.org


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

    Worklog Id:     (was: 119949)
    Time Spent: 3h  (was: 2h 50m)

> 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: 3h
>  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)