You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by li...@apache.org on 2014/04/24 20:18:44 UTC

svn commit: r1589822 - in /hbase/branches/0.89-fb/src/main: java/org/apache/hadoop/hbase/coprocessor/endpoints/ java/org/apache/hadoop/hbase/ipc/ java/org/apache/hadoop/hbase/ipc/thrift/ java/org/apache/hadoop/hbase/regionserver/ resources/org/apache/h...

Author: liyin
Date: Thu Apr 24 18:18:44 2014
New Revision: 1589822

URL: http://svn.apache.org/r1589822
Log:
[master] Generate thrift files, some refactoring on thrift related things

Author: daviddeng

Summary:
`gen_thrift_from_swift.sh` is optimized, adding new classes, adding namespaces
`ArrayList` in endpoint service is replaced with `List`
Methods in parents of `ThriftHRegionInterface` are copied to `ThriftHRegionInterface` and annotated there because thrift doesn't support extending more than one parent.

Test Plan: `TestEndpoint`

Reviewers: manukranthk, adela, gauravm

Reviewed By: gauravm

CC: hbase-eng@, andrewcox

Differential Revision: https://phabricator.fb.com/D1287779

Task ID: 4147004

Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointManager.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/IEndpointService.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/IRegionScanService.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ScannerResult.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java
    hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift
    hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointManager.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointManager.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointManager.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointManager.java Thu Apr 24 18:18:44 2014
@@ -22,9 +22,9 @@ package org.apache.hadoop.hbase.coproces
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -44,7 +44,7 @@ public class EndpointManager implements 
     private Map<String, Method> methods = new HashMap<>();
     private Map<String, IBytesDecoder[]> mthToDecs = new ConcurrentHashMap<>();
 
-    private Object[] encodeParams(String methodKey, ArrayList<byte[]> params) {
+    private Object[] encodeParams(String methodKey, List<byte[]> params) {
       IBytesDecoder[] decoders = mthToDecs.get(methodKey);
       Object[] res = new Object[params.size()];
       for (int i = 0; i < res.length; i++) {
@@ -112,7 +112,7 @@ public class EndpointManager implements 
      * @param params the encoded parameters.
      * @return the encoded return results.
      */
-    public byte[] invoke(IEndpoint ep, String methodName, ArrayList<byte[]> params)
+    public byte[] invoke(IEndpoint ep, String methodName, List<byte[]> params)
         throws IllegalAccessException, IllegalArgumentException,
         InvocationTargetException, IOException {
       String methodKey = EndpointInfo.makeMethodKey(methodName, params.size());

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointServer.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/EndpointServer.java Thu Apr 24 18:18:44 2014
@@ -19,7 +19,7 @@
  */
 package org.apache.hadoop.hbase.coprocessor.endpoints;
 
-import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.NotServingRegionException;
@@ -48,7 +48,7 @@ public class EndpointServer implements I
 
   @Override
   public byte[] callEndpoint(String epName, String methodName,
-      ArrayList<byte[]> params, final byte[] regionName, final byte[] startRow,
+      List<byte[]> params, final byte[] regionName, final byte[] startRow,
       final byte[] stopRow) throws ThriftHBaseException {
     try {
       EndpointInfo ent = manager.getEndpointEntry(epName);

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/IEndpointService.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/IEndpointService.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/IEndpointService.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/coprocessor/endpoints/IEndpointService.java Thu Apr 24 18:18:44 2014
@@ -19,19 +19,16 @@
  */
 package org.apache.hadoop.hbase.coprocessor.endpoints;
 
-import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
 
-import com.facebook.swift.codec.ThriftField;
-import com.facebook.swift.service.ThriftException;
-import com.facebook.swift.service.ThriftMethod;
-import com.facebook.swift.service.ThriftService;
-
 /**
  * The interface of a server executing endpoints.
+ *
+ * Methods defined here are redefined in ThriftHRegionInterface and annotated.
+ * The reason is Thrift doesn't support multi inheritance even for interface.
  */
-@ThriftService
 public interface IEndpointService {
   /**
    * Calls an endpoint on an region server.
@@ -45,13 +42,7 @@ public interface IEndpointService {
    * @param stopRow the stop row, exclusive
    * @return the computed value.
    */
-  @ThriftMethod(value = "callEndpoint", exception = {
-      @ThriftException(type = ThriftHBaseException.class, id = 1) })
-  public byte[] callEndpoint(@ThriftField(name = "epName") String epName,
-      @ThriftField(name = "methodName") String methodName,
-      @ThriftField(name = "params") ArrayList<byte[]> params,
-      @ThriftField(name = "regionName") byte[] regionName,
-      @ThriftField(name = "startRow") byte[] startRow,
-      @ThriftField(name = "stopRow") byte[] stopRow)
-      throws ThriftHBaseException;
+  public byte[] callEndpoint(String epName, String methodName,
+      List<byte[]> params, byte[] regionName, byte[] startRow,
+      byte[] stopRow) throws ThriftHBaseException;
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/IRegionScanService.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/IRegionScanService.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/IRegionScanService.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/IRegionScanService.java Thu Apr 24 18:18:44 2014
@@ -22,15 +22,12 @@ package org.apache.hadoop.hbase.ipc;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException;
 
-import com.facebook.swift.codec.ThriftField;
-import com.facebook.swift.service.ThriftException;
-import com.facebook.swift.service.ThriftMethod;
-import com.facebook.swift.service.ThriftService;
-
 /**
  * Interface of scanner service in a region.
+ *
+ * Methods defined here are redefined in ThriftHRegionInterface and annotated.
+ * The reason is Thrift doesn't support multi inheritance even for interface.
  */
-@ThriftService
 public interface IRegionScanService {
   /**
    * Opens a scanner, optionally returns some data if numberOfRows > 0.
@@ -43,12 +40,7 @@ public interface IRegionScanService {
    *         The length of the Result list of the return value could be empty
    *         and EOR is set to true for sure in this case.
    */
-  @ThriftMethod(value = "scanOpen", exception = {
-      @ThriftException(type = ThriftHBaseException.class, id = 1) })
-  ScannerResult scanOpen(
-      @ThriftField(name="regionName") byte[] regionName,
-      @ThriftField(name = "scan") Scan scan,
-      @ThriftField(name = "numberOfRows") int numberOfRows)
+  ScannerResult scanOpen(byte[] regionName, Scan scan, int numberOfRows)
       throws ThriftHBaseException;
 
   /**
@@ -60,12 +52,7 @@ public interface IRegionScanService {
    *         The length of the Result list of the return value could be empty
    *         and EOR is set to true for sure in this case.
    */
-  @ThriftMethod(value = "scanNext", exception = {
-      @ThriftException(type = ThriftHBaseException.class, id = 1) })
-  ScannerResult scanNext(
-      @ThriftField(name="id") long id,
-      @ThriftField(name = "numberOfRows") int numberOfRows)
-      throws ThriftHBaseException;
+  ScannerResult scanNext(long id, int numberOfRows) throws ThriftHBaseException;
 
   /**
    * Closes the scanner on the server side.
@@ -73,8 +60,5 @@ public interface IRegionScanService {
    * @param id  the ID of the scanner to close
    * @return true  if a scanner is closed. false if the scanner doesn't exist.
    */
-  @ThriftMethod(value = "scanClose", exception = {
-      @ThriftException(type = ThriftHBaseException.class, id = 1) })
-  boolean scanClose(@ThriftField(name = "id") long id)
-      throws ThriftHBaseException;
+  boolean scanClose(long id) throws ThriftHBaseException;
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ScannerResult.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ScannerResult.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ScannerResult.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ScannerResult.java Thu Apr 24 18:18:44 2014
@@ -94,13 +94,13 @@ public class ScannerResult {
     private List<Result> results = EMPTY_RESULTS;
     private long id = -1L;
 
-    @ThriftField(1)
+    @ThriftField(value = 1, name = "EOS")
     public Builder setEOS(boolean vl) {
       this.eos = vl;
       return this;
     }
 
-    @ThriftField(2)
+    @ThriftField(value = 2, name = "EOR")
     public Builder setEOR(boolean vl) {
       this.eor = vl;
       return this;
@@ -121,7 +121,7 @@ public class ScannerResult {
       return this;
     }
 
-    @ThriftField(4)
+    @ThriftField(value = 4, name = "ID")
     public Builder setID(long vl) {
       this.id = vl;
       return this;

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/ThriftHRegionInterface.java Thu Apr 24 18:18:44 2014
@@ -61,6 +61,64 @@ public interface ThriftHRegionInterface 
     IEndpointService, IRegionScanService {
 
   /**
+   * Opens a scanner, optionally returns some data if numberOfRows > 0.
+   *
+   * @param regionName the name of the region to scan
+   * @param scan the Scan instance defining scan query.
+   * @param numberOfRows maximum number of rows to return after successfully
+   *          open the scanner.
+   * @return the result as a ScannerResult.
+   *         The length of the Result list of the return value could be empty
+   *         and EOR is set to true for sure in this case.
+   */
+  @Override
+  @ThriftMethod(value = "scanOpen", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  ScannerResult scanOpen(@ThriftField(name = "regionName") byte[] regionName,
+      @ThriftField(name = "scan") Scan scan,
+      @ThriftField(name = "numberOfRows") int numberOfRows)
+      throws ThriftHBaseException;
+
+  /**
+   * Returns next scanning results.
+   *
+   * @param ID the ID of the scanner
+   * @param numberOfRows maximum number of rows to return,
+   * @return the result as a ScannerResult.
+   *         The length of the Result list of the return value could be empty
+   *         and EOR is set to true for sure in this case.
+   */
+  @Override
+  @ThriftMethod(value = "scanNext", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  ScannerResult scanNext(@ThriftField(name = "id") long id,
+      @ThriftField(name = "numberOfRows") int numberOfRows)
+      throws ThriftHBaseException;
+
+  /**
+   * Closes the scanner on the server side.
+   *
+   * @param id the ID of the scanner to close
+   * @return true if a scanner is closed. false if the scanner doesn't exist.
+   */
+  @Override
+  @ThriftMethod(value = "scanClose", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  boolean scanClose(@ThriftField(name = "id") long id)
+      throws ThriftHBaseException;
+
+  @Override
+  @ThriftMethod(value = "callEndpoint", exception = {
+      @ThriftException(type = ThriftHBaseException.class, id = 1) })
+  public byte[] callEndpoint(@ThriftField(name = "epName") String epName,
+      @ThriftField(name = "methodName") String methodName,
+      @ThriftField(name = "params") List<byte[]> params,
+      @ThriftField(name = "regionName") byte[] regionName,
+      @ThriftField(name = "startRow") byte[] startRow,
+      @ThriftField(name = "stopRow") byte[] stopRow)
+      throws ThriftHBaseException;
+
+  /**
    * Get metainfo about an HRegion
    *
    * @param regionName name of the region

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/thrift/HBaseToThriftAdapter.java Thu Apr 24 18:18:44 2014
@@ -1200,7 +1200,7 @@ public class HBaseToThriftAdapter implem
 
   @Override
   public byte[] callEndpoint(String epName, String methodName,
-      ArrayList<byte[]> params, byte[] regionName, byte[] startRow,
+      List<byte[]> params, byte[] regionName, byte[] startRow,
       byte[] stopRow) throws ThriftHBaseException {
     preProcess();
     try {

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Thu Apr 24 18:18:44 2014
@@ -4174,7 +4174,7 @@ public class HRegionServer implements HR
 
   @Override
   public byte[] callEndpoint(String epName, String methodName,
-      ArrayList<byte[]> params, final byte[] regionName, final byte[] startRow,
+      List<byte[]> params, final byte[] regionName, final byte[] startRow,
       final byte[] stopRow) throws ThriftHBaseException {
     return endpointServer.callEndpoint(epName, methodName, params, regionName,
         startRow, stopRow);

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/ThriftHRegionServer.java Thu Apr 24 18:18:44 2014
@@ -635,7 +635,7 @@ public class ThriftHRegionServer impleme
 
   @Override
   public byte[] callEndpoint(String epName, String methodName,
-      ArrayList<byte[]> params, final byte[] regionName, final byte[] startRow,
+      List<byte[]> params, final byte[] regionName, final byte[] startRow,
       final byte[] stopRow) throws ThriftHBaseException {
     return server.callEndpoint(epName, methodName, params, regionName,
         startRow, stopRow);

Modified: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift (original)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/HBase.thrift Thu Apr 24 18:18:44 2014
@@ -1,5 +1,6 @@
 namespace java.swift org.apache.hadoop.hbase
 namespace cpp facebook.hbase.hbcpp
+namespace php hbase
 
 
 enum Type {
@@ -189,10 +190,23 @@ struct Bucket {
   4: map<HFileStat, double> hfileStats;
 }
 
+struct HRegionLocation {
+  1: HRegionInfo regionInfo;
+  2: HServerAddress serverAddress;
+  3: i64 serverStartCode;
+}
+
+struct ScannerResult {
+  1: bool EOS;
+  2: bool EOR;
+  3: list<Result> result;
+  4: i64 ID;
+}
+
 service ThriftHRegionInterface {
   void bulkLoadHFile(1: string hfilePath, 2: binary regionName, 3: binary familyName) throws (1: ThriftHBaseException ex1);
   void bulkLoadHFileSeqNum(1: string hfilePath, 2: binary regionName, 3: binary familyName, 4: bool assignSeqNum) throws (1: ThriftHBaseException ex1);
-  binary callEndpoint(1: string epName, 2: string methodName, 3: binary regionName, 4: binary startRow, 5: binary stopRow) throws (1: ThriftHBaseException ex1);
+  binary callEndpoint(1: string epName, 2: string methodName, 3: list<binary> params, 4: binary regionName, 5: binary startRow, 6: binary stopRow) throws (1: ThriftHBaseException ex1);
   bool checkAndDelete(1: binary regionName, 2: binary row, 3: binary family, 4: binary qualifier, 5: binary value, 6: Delete deleteArg) throws (1: ThriftHBaseException ex1);
   bool checkAndPut(1: binary regionName, 2: binary row, 3: binary family, 4: binary qualifier, 5: binary value, 6: Put put) throws (1: ThriftHBaseException ex1);
   void close(1: i64 scannerId) throws (1: ThriftHBaseException ex1);
@@ -210,8 +224,10 @@ service ThriftHRegionInterface {
   HServerInfo getHServerInfo() throws (1: ThriftHBaseException ex1);
   list<Bucket> getHistogram(1: binary arg0) throws (1: ThriftHBaseException ex1);
   list<Bucket> getHistogramForStore(1: binary arg0, 2: binary arg1) throws (1: ThriftHBaseException ex1);
+  list<list<Bucket>> getHistograms(1: list<binary> arg0) throws (1: ThriftHBaseException ex1);
   i64 getLastFlushTime(1: binary regionName);
   map<binary, i64> getLastFlushTimes();
+  HRegionLocation getLocation(1: binary tableName, 2: binary row, 3: bool reload) throws (1: ThriftHBaseException ex1);
   HRegionInfo getRegionInfo(1: binary regionName) throws (1: ThriftHBaseException ex1);
   list<HRegionInfo> getRegionsAssignment() throws (1: ThriftHBaseException ex1);
   list<Result> getRows(1: binary regionName, 2: list<Get> gets) throws (1: ThriftHBaseException ex1);
@@ -237,6 +253,9 @@ service ThriftHRegionInterface {
   i32 processListOfDeletes(1: binary regionName, 2: list<Delete> deletes) throws (1: ThriftHBaseException ex1);
   void processPut(1: binary regionName, 2: Put put) throws (1: ThriftHBaseException ex1);
   i32 putRows(1: binary regionName, 2: list<Put> puts) throws (1: ThriftHBaseException ex1);
+  bool scanClose(1: i64 id) throws (1: ThriftHBaseException ex1);
+  ScannerResult scanNext(1: i64 id, 2: i32 numberOfRows) throws (1: ThriftHBaseException ex1);
+  ScannerResult scanOpen(1: binary regionName, 2: Scan scan, 3: i32 numberOfRows) throws (1: ThriftHBaseException ex1);
   void setHDFSQuorumReadTimeoutMillis(1: i64 timeoutMillis);
   void setNumHDFSQuorumReadThreads(1: i32 maxThreads);
   void stop(1: string why);

Modified: hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh?rev=1589822&r1=1589821&r2=1589822&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh (original)
+++ hbase/branches/0.89-fb/src/main/resources/org/apache/hadoop/hbase/thrift/gen_thrift_from_swift.sh Thu Apr 24 18:18:44 2014
@@ -11,4 +11,4 @@ export CLASSPATH=$CLASSPATH:$1:$2:$3
 
 dirpath=`dirname $0`
 
-$JAVA_HOME/bin/java com.facebook.swift.generator.swift2thrift.Main -allow_multiple_packages org.apache.hadoop.hbase org.apache.hadoop.hbase.KeyValue org.apache.hadoop.hbase.client.Put org.apache.hadoop.hbase.io.TimeRange org.apache.hadoop.hbase.filter.TFilter org.apache.hadoop.hbase.client.Get org.apache.hadoop.hbase.client.MultiPut org.apache.hadoop.hbase.client.Delete org.apache.hadoop.hbase.client.Scan org.apache.hadoop.hbase.HColumnDescriptor org.apache.hadoop.hbase.HTableDescriptor org.apache.hadoop.hbase.HRegionInfo org.apache.hadoop.hbase.client.MultiPutResponse org.apache.hadoop.hbase.client.Result org.apache.hadoop.hbase.HServerAddress 'org.apache.hadoop.hbase.HServerLoad$RegionLoad' org.apache.hadoop.hbase.HServerLoad org.apache.hadoop.hbase.HServerInfo org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException org.apache.hadoop.hbase.client.MultiAction 'org.apache.hadoop.hbase.client.IntegerOrResultOrException$Type' org.apache.hadoop.hbase.client.IntegerOrResultOr
 Exception org.apache.hadoop.hbase.client.TMultiResponse org.apache.hadoop.hbase.client.TRowMutations org.apache.hadoop.hbase.master.AssignmentPlan org.apache.hadoop.hbase.client.RowLock 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$HFileStat' 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$Bucket' org.apache.hadoop.hbase.ipc.ThriftHRegionInterface -out "$dirpath/HBase.thrift"
+$JAVA_HOME/bin/java com.facebook.swift.generator.swift2thrift.Main -allow_multiple_packages org.apache.hadoop.hbase org.apache.hadoop.hbase.KeyValue org.apache.hadoop.hbase.client.Put org.apache.hadoop.hbase.io.TimeRange org.apache.hadoop.hbase.filter.TFilter org.apache.hadoop.hbase.client.Get org.apache.hadoop.hbase.client.MultiPut org.apache.hadoop.hbase.client.Delete org.apache.hadoop.hbase.client.Scan org.apache.hadoop.hbase.HColumnDescriptor org.apache.hadoop.hbase.HTableDescriptor org.apache.hadoop.hbase.HRegionInfo org.apache.hadoop.hbase.client.MultiPutResponse org.apache.hadoop.hbase.client.Result org.apache.hadoop.hbase.HServerAddress 'org.apache.hadoop.hbase.HServerLoad$RegionLoad' org.apache.hadoop.hbase.HServerLoad org.apache.hadoop.hbase.HServerInfo org.apache.hadoop.hbase.ipc.thrift.exceptions.ThriftHBaseException org.apache.hadoop.hbase.client.MultiAction 'org.apache.hadoop.hbase.client.IntegerOrResultOrException$Type' org.apache.hadoop.hbase.client.IntegerOrResultOr
 Exception org.apache.hadoop.hbase.client.TMultiResponse org.apache.hadoop.hbase.client.TRowMutations org.apache.hadoop.hbase.master.AssignmentPlan org.apache.hadoop.hbase.client.RowLock 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$HFileStat' 'org.apache.hadoop.hbase.io.hfile.histogram.HFileHistogram$Bucket' org.apache.hadoop.hbase.HRegionLocation org.apache.hadoop.hbase.ipc.ScannerResult org.apache.hadoop.hbase.ipc.ThriftHRegionInterface -out "$dirpath/HBase.thrift" -namespace cpp facebook.hbase.hbcpp -namespace php hbase