You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/06/01 21:50:13 UTC

svn commit: r1130275 [1/3] - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/thrift/ src/main/java/org/apache/hadoop/hbase/thrift/generated/ src/main/resources/org/apache/hadoop/hbase/thrift/

Author: stack
Date: Wed Jun  1 19:50:12 2011
New Revision: 1130275

URL: http://svn.apache.org/viewvc?rev=1130275&view=rev
Log:
HBASE-3942 The thrift scannerOpen functions should support row caching

Added:
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/generated/TScan.java
Modified:
    hbase/trunk/CHANGES.txt
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
    hbase/trunk/src/main/resources/org/apache/hadoop/hbase/thrift/Hbase.thrift

Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1130275&r1=1130274&r2=1130275&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Jun  1 19:50:12 2011
@@ -234,6 +234,8 @@ Release 0.91.0 - Unreleased
    HBASE-2937  Facilitate Timeouts In HBase Client (Karthick Sankarachary)
    HBASE-3921  Allow adding arbitrary blobs to Put (dhruba borthakur)
    HBASE-3931  Allow adding attributes to Get
+   HBASE-3942  The thrift scannerOpen functions should support row caching
+               (Adam Worthington)
 
   TASKS
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=1130275&r1=1130274&r2=1130275&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Wed Jun  1 19:50:12 2011
@@ -68,6 +68,7 @@ import org.apache.hadoop.hbase.thrift.ge
 import org.apache.hadoop.hbase.thrift.generated.TCell;
 import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
 import org.apache.hadoop.hbase.thrift.generated.TRowResult;
+import org.apache.hadoop.hbase.thrift.generated.TScan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -720,6 +721,39 @@ public class ThriftServer {
     public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
         return scannerGetList(id,1);
     }
+
+    public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan) throws IOError {
+        try {
+          HTable table = getTable(tableName);
+          Scan scan = new Scan();
+          if (tScan.isSetStartRow()) {
+              scan.setStartRow(tScan.getStartRow());
+          }
+          if (tScan.isSetStopRow()) {
+              scan.setStopRow(tScan.getStopRow());
+          }
+          if (tScan.isSetTimestamp()) {
+              scan.setTimeRange(Long.MIN_VALUE, tScan.getTimestamp());              
+          }
+          if (tScan.isSetCaching()) {
+              scan.setCaching(tScan.getCaching());
+          }
+          if(tScan.isSetColumns() && tScan.getColumns().size() != 0) {
+            for(ByteBuffer column : tScan.getColumns()) {
+              byte [][] famQf = KeyValue.parseColumn(getBytes(column));
+              if(famQf.length == 1) {
+                scan.addFamily(famQf[0]);
+              } else {
+                scan.addColumn(famQf[0], famQf[1]);
+              }
+            }
+          }
+          return addScanner(table.getScanner(scan));
+        } catch (IOException e) {
+          throw new IOError(e.getMessage());
+        }
+    }
+
     @Override
     public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
             List<ByteBuffer> columns) throws IOError {