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 2013/04/24 06:44:10 UTC

svn commit: r1471246 [1/7] - in /hbase/trunk: bin/ dev-support/ hbase-examples/thrift2/ hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ hbase-server/src/main/java/org/apache/hado...

Author: stack
Date: Wed Apr 24 04:44:09 2013
New Revision: 1471246

URL: http://svn.apache.org/r1471246
Log:
HBASE-5472 LoadIncrementalHFiles loops forever if the target table misses a CF -- REAPPLY

Modified:
    hbase/trunk/bin/hbase
    hbase/trunk/bin/hbase.cmd
    hbase/trunk/dev-support/findbugs-exclude.xml
    hbase/trunk/hbase-examples/thrift2/DemoClient.java
    hbase/trunk/hbase-examples/thrift2/DemoClient.py
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnIncrement.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumnValue.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDelete.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TDeleteType.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TGet.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/THBaseService.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIOError.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIllegalArgument.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TIncrement.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TPut.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TResult.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TScan.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TTimeRange.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/package.html
    hbase/trunk/hbase-server/src/main/javadoc/org/apache/hadoop/hbase/thrift/package.html
    hbase/trunk/hbase-server/src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestLoadIncrementalHFiles.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/thrift2/TestThriftHBaseServiceHandler.java

Modified: hbase/trunk/bin/hbase
URL: http://svn.apache.org/viewvc/hbase/trunk/bin/hbase?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/bin/hbase (original)
+++ hbase/trunk/bin/hbase Wed Apr 24 04:44:09 2013
@@ -84,7 +84,6 @@ if [ $# = 0 ]; then
   echo "  zookeeper        run a Zookeeper server"
   echo "  rest             run an HBase REST server" 
   echo "  thrift           run the HBase Thrift server" 
-  echo "  thrift2          run the HBase Thrift2 server"
   echo ""
   echo "PACKAGE MANAGEMENT"
   echo "  classpath        dump hbase CLASSPATH"
@@ -237,7 +236,7 @@ fi
 unset IFS
 
 #Set the right GC options based on the what we are running
-declare -a server_cmds=("master" "regionserver" "thrift" "thrift2" "rest" "avro" "zookeeper")
+declare -a server_cmds=("master" "regionserver" "thrift" "rest" "avro" "zookeeper")
 for cmd in ${server_cmds[@]}; do
 	if [[ $cmd == $COMMAND ]]; then
 		server=true
@@ -291,11 +290,6 @@ elif [ "$COMMAND" = "thrift" ] ; then
   if [ "$1" != "stop" ] ; then
     HBASE_OPTS="$HBASE_OPTS $HBASE_THRIFT_OPTS"
   fi
-elif [ "$COMMAND" = "thrift2" ] ; then
-  CLASS='org.apache.hadoop.hbase.thrift2.ThriftServer'
-  if [ "$1" != "stop" ] ; then
-    HBASE_OPTS="$HBASE_OPTS $HBASE_THRIFT_OPTS"
-  fi
 elif [ "$COMMAND" = "rest" ] ; then
   CLASS='org.apache.hadoop.hbase.rest.RESTServer'
   if [ "$1" != "stop" ] ; then

Modified: hbase/trunk/bin/hbase.cmd
URL: http://svn.apache.org/viewvc/hbase/trunk/bin/hbase.cmd?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/bin/hbase.cmd (original)
+++ hbase/trunk/bin/hbase.cmd Wed Apr 24 04:44:09 2013
@@ -197,7 +197,7 @@ goto :MakeCmdArgsLoop 
 set hbase-command-arguments=%_hbasearguments%
 
 @rem figure out which class to run
-set corecommands=shell master regionserver thrift thrift2 rest avro hlog hbck hfile zookeeper zkcli
+set corecommands=shell master regionserver thrift rest avro hlog hbck hfile zookeeper zkcli
 for %%i in ( %corecommands% ) do (
   if "%hbase-command%"=="%%i" set corecommand=true
 )
@@ -296,13 +296,6 @@ goto :eof
   )
   goto :eof
 
-:thrift2
-  set CLASS=org.apache.hadoop.hbase.thrift2.ThriftServer
-  if NOT "%1" == "stop" (
-    set HBASE_OPTS=%HBASE_OPTS% %HBASE_THRIFT_OPTS%
-  )
-  goto :eof
-
 :rest
   set CLASS=org.apache.hadoop.hbase.rest.Main
   if NOT "%1"=="stop" (
@@ -369,7 +362,6 @@ goto :eof
   echo   zookeeper        run a Zookeeper server
   echo   rest             run an HBase REST server 
   echo   thrift           run the HBase Thrift server 
-  echo   thrift2          run the HBase Thrift2 server 
   echo   avro             run an HBase Avro server 
   echo. 
   echo PACKAGE MANAGEMENT

Modified: hbase/trunk/dev-support/findbugs-exclude.xml
URL: http://svn.apache.org/viewvc/hbase/trunk/dev-support/findbugs-exclude.xml?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/dev-support/findbugs-exclude.xml (original)
+++ hbase/trunk/dev-support/findbugs-exclude.xml Wed Apr 24 04:44:09 2013
@@ -17,10 +17,6 @@
 
 <FindBugsFilter>
   <Match>
-    <Package name="org.apache.hadoop.hbase.thrift2.generated"/>
-  </Match>
-
-  <Match>
     <Package name="org.apache.hadoop.hbase.thrift.generated"/>
   </Match>
 

Modified: hbase/trunk/hbase-examples/thrift2/DemoClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-examples/thrift2/DemoClient.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-examples/thrift2/DemoClient.java (original)
+++ hbase/trunk/hbase-examples/thrift2/DemoClient.java Wed Apr 24 04:44:09 2013
@@ -1,89 +0,0 @@
-/**
- *
- * 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.hadoop.hbase.thrift2;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.hbase.thrift2.generated.TColumnValue;
-import org.apache.hadoop.hbase.thrift2.generated.TGet;
-import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.thrift2.generated.TIOError;
-import org.apache.hadoop.hbase.thrift2.generated.TPut;
-import org.apache.hadoop.hbase.thrift2.generated.TResult;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-public class DemoClient {
-  public static void main(String[] args) throws TIOError, TException {
-    System.out.println("Thrift2 Demo");
-    System.out.println("This demo assumes you have a table called \"example\" with a column family called \"family1\"");
-    
-    String host = "localhost";
-    int port = 9090;
-    int timeout = 10000;
-    boolean framed = false;
-
-    TTransport transport = new TSocket(host, port, timeout);
-    if (framed) {
-      transport = new TFramedTransport(transport);
-    }
-    TProtocol protocol = new TBinaryProtocol(transport);
-    // This is our thrift client.
-    THBaseService.Iface client = new THBaseService.Client(protocol);
-
-    // open the transport
-    transport.open();
-    
-    ByteBuffer table = ByteBuffer.wrap("example".getBytes());
-
-    TPut put = new TPut();
-    put.setRow("row1".getBytes());
-
-    TColumnValue columnValue = new TColumnValue();
-    columnValue.setFamily("family1".getBytes());
-    columnValue.setQualifier("qualifier1".getBytes());
-    columnValue.setValue("value1".getBytes());
-    List<TColumnValue> columnValues = new ArrayList<TColumnValue>();
-    columnValues.add(columnValue);
-    put.setColumnValues(columnValues);
-
-    client.put(table, put);
-
-    TGet get = new TGet();
-    get.setRow("row1".getBytes());
-
-    TResult result = client.get(table, get);
-
-    System.out.print("row = " + new String(result.getRow()));
-    for (TColumnValue resultColumnValue : result.getColumnValues()) {
-      System.out.print("family = " + new String(resultColumnValue.getFamily()));
-      System.out.print("qualifier = " + new String(resultColumnValue.getFamily()));
-      System.out.print("value = " + new String(resultColumnValue.getValue()));
-      System.out.print("timestamp = " + resultColumnValue.getTimestamp());
-    }
-    
-    transport.close();
-  }
-}

Modified: hbase/trunk/hbase-examples/thrift2/DemoClient.py
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-examples/thrift2/DemoClient.py?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-examples/thrift2/DemoClient.py (original)
+++ hbase/trunk/hbase-examples/thrift2/DemoClient.py Wed Apr 24 04:44:09 2013
@@ -1,68 +0,0 @@
-"""
-  
-  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.
-"""
-# Instructions:
-# 1. Run Thrift to generate the python module hbase
-#    thrift --gen py ../../../src/main/resources/org/apache/hadoop/hbase/thrift2/hbase.thrift
-# 2. Create a directory of your choosing that contains:
-#     a. This file (DemoClient.py).
-#     b. The directory gen-py/hbase (generated by instruction step 1).
-# 3. pip install thrift==0.7.0
-# 4. Create a table call "example", with a family called "family1" using the hbase shell.
-# 5. Start the hbase thrift2 server
-#    bin/hbase thrift2 start
-# 6. Execute {python DemoClient.py}.
-
-from thrift.transport import TTransport
-from thrift.transport import TSocket
-from thrift.transport import THttpClient
-from thrift.protocol import TBinaryProtocol
-
-from hbase import THBaseService
-from hbase.ttypes import *
-
-print "Thrift2 Demo"
-print "This demo assumes you have a table called \"example\" with a column family called \"family1\""
-
-host = "localhost"
-port = 9090
-framed = False
-
-socket = TSocket.TSocket(host, port)
-if framed:
-  transport = TTransport.TFramedTransport(socket)
-else:
-  transport = TTransport.TBufferedTransport(socket)
-protocol = TBinaryProtocol.TBinaryProtocol(transport)
-client = THBaseService.Client(protocol)
-
-transport.open()
-
-table = "example"
-
-put = TPut(row="row1", columnValues=[TColumnValue(family="family1",qualifier="qualifier1",value="value1")])
-print "Putting:", put
-client.put(table, put)
-
-get = TGet(row="row1")
-print "Getting:", get
-result = client.get(table, get)
-
-print "Result:", result
-  
-transport.close()

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/LoadIncrementalHFiles.java Wed Apr 24 04:44:09 2013
@@ -187,7 +187,7 @@ public class LoadIncrementalHFiles exten
   /**
    * Perform a bulk load of the given directory into the given
    * pre-existing table.  This method is not threadsafe.
-   * 
+   *
    * @param hfofDir the directory that was provided as the output path
    * of a job using HFileOutputFormat
    * @param table the table to load into
@@ -220,6 +220,27 @@ public class LoadIncrementalHFiles exten
     Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
     try {
       discoverLoadQueue(queue, hfofDir);
+      // check whether there is invalid family name in HFiles to be bulkloaded
+      Collection<HColumnDescriptor> families = table.getTableDescriptor().getFamilies();
+      ArrayList<String> familyNames = new ArrayList<String>();
+      for (HColumnDescriptor family : families) {
+        familyNames.add(family.getNameAsString());
+      }
+      ArrayList<String> unmatchedFamilies = new ArrayList<String>();
+      for (LoadQueueItem lqi : queue) {
+        String familyNameInHFile = Bytes.toString(lqi.family);
+        if (!familyNames.contains(familyNameInHFile)) {
+          unmatchedFamilies.add(familyNameInHFile);
+        }
+      }
+      if (unmatchedFamilies.size() > 0) {
+        String msg =
+            "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "
+                + unmatchedFamilies + "; valid family names of table "
+                + Bytes.toString(table.getTableName()) + " are: " + familyNames;
+        LOG.error(msg);
+        throw new IOException(msg);
+      }
       int count = 0;
 
       if (queue.isEmpty()) {
@@ -358,7 +379,7 @@ public class LoadIncrementalHFiles exten
     Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
     while (!queue.isEmpty()) {
       final LoadQueueItem item = queue.remove();
-      
+
       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
         public List<LoadQueueItem> call() throws Exception {
           List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
@@ -492,12 +513,12 @@ public class LoadIncrementalHFiles exten
    * Attempts to do an atomic load of many hfiles into a region.  If it fails,
    * it returns a list of hfiles that need to be retried.  If it is successful
    * it will return an empty list.
-   * 
+   *
    * NOTE: To maintain row atomicity guarantees, region server callable should
    * succeed atomically and fails atomically.
-   * 
+   *
    * Protected for testing.
-   * 
+   *
    * @return empty list if success, list of items to retry on recoverable
    * failure
    */
@@ -650,7 +671,7 @@ public class LoadIncrementalHFiles exten
   private boolean doesTableExist(String tableName) throws Exception {
     return hbAdmin.tableExists(tableName);
   }
-  
+
   /*
    * Infers region boundaries for a new table.
    * Parameter:
@@ -658,29 +679,29 @@ public class LoadIncrementalHFiles exten
    *     If a key is a start key of a file, then it maps to +1
    *     If a key is an end key of a file, then it maps to -1
    * Algo:
-   * 1) Poll on the keys in order: 
-   *    a) Keep adding the mapped values to these keys (runningSum) 
+   * 1) Poll on the keys in order:
+   *    a) Keep adding the mapped values to these keys (runningSum)
    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
-   * 2) Return the boundary list. 
+   * 2) Return the boundary list.
    */
   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
     ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
     int runningValue = 0;
     byte[] currStartKey = null;
     boolean firstBoundary = true;
-    
+
     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
       if (runningValue == 0) currStartKey = item.getKey();
       runningValue += item.getValue();
       if (runningValue == 0) {
         if (!firstBoundary) keysArray.add(currStartKey);
         firstBoundary = false;
-      } 
+      }
     }
-    
+
     return keysArray.toArray(new byte[0][0]);
   }
- 
+
   /*
    * If the table is created for the first time, then "completebulkload" reads the files twice.
    * More modifications necessary if we want to avoid doing it.
@@ -706,7 +727,7 @@ public class LoadIncrementalHFiles exten
     // Build a set of keys
     byte[][] keys;
     TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
-    
+
     for (FileStatus stat : familyDirStatuses) {
       if (!stat.isDir()) {
         LOG.warn("Skipping non-directory " + stat.getPath());
@@ -716,10 +737,10 @@ public class LoadIncrementalHFiles exten
       // Skip _logs, etc
       if (familyDir.getName().startsWith("_")) continue;
       byte[] family = familyDir.getName().getBytes();
-     
+
       hcd = new HColumnDescriptor(family);
       htd.addFamily(hcd);
-      
+
       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
       for (Path hfile : hfiles) {
         if (hfile.getName().startsWith("_")) continue;
@@ -739,7 +760,7 @@ public class LoadIncrementalHFiles exten
           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
             " first=" + Bytes.toStringBinary(first) +
             " last="  + Bytes.toStringBinary(last));
-          
+
           // To eventually infer start key-end key boundaries
           Integer value = map.containsKey(first)? map.get(first):0;
           map.put(first, value+1);
@@ -751,7 +772,7 @@ public class LoadIncrementalHFiles exten
         }
       }
     }
-    
+
     keys = LoadIncrementalHFiles.inferBoundaries(map);
     this.hbAdmin.createTable(htd,keys);
 

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftHBaseServiceHandler.java Wed Apr 24 04:44:09 2013
@@ -1,343 +0,0 @@
-/**
- *
- * 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.hadoop.hbase.thrift2;
-
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deleteFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.deletesFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.getsFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.incrementFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.putsFromThrift;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultFromHBase;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.resultsFromHBase;
-import static org.apache.hadoop.hbase.thrift2.ThriftUtilities.scanFromThrift;
-
-import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.HTablePool;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
-import org.apache.hadoop.hbase.thrift2.generated.TDelete;
-import org.apache.hadoop.hbase.thrift2.generated.TGet;
-import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.thrift2.generated.TIOError;
-import org.apache.hadoop.hbase.thrift2.generated.TIllegalArgument;
-import org.apache.hadoop.hbase.thrift2.generated.TIncrement;
-import org.apache.hadoop.hbase.thrift2.generated.TPut;
-import org.apache.hadoop.hbase.thrift2.generated.TResult;
-import org.apache.hadoop.hbase.thrift2.generated.TScan;
-import org.apache.thrift.TException;
-
-/**
- * This class is a glue object that connects Thrift RPC calls to the HBase client API primarily defined in the
- * HTableInterface.
- */
-@InterfaceAudience.Private
-public class ThriftHBaseServiceHandler implements THBaseService.Iface {
-
-  // TODO: Size of pool configuraple
-  private final HTablePool htablePool;
-  private static final Log LOG = LogFactory.getLog(ThriftHBaseServiceHandler.class);
-
-  // nextScannerId and scannerMap are used to manage scanner state
-  // TODO: Cleanup thread for Scanners, Scanner id wrap
-  private final AtomicInteger nextScannerId = new AtomicInteger(0);
-  private final Map<Integer, ResultScanner> scannerMap = new ConcurrentHashMap<Integer, ResultScanner>();
-
-  public static THBaseService.Iface newInstance(
-      Configuration conf, ThriftMetrics metrics) {
-    THBaseService.Iface handler = new ThriftHBaseServiceHandler(conf);
-    return (THBaseService.Iface) Proxy.newProxyInstance(
-        handler.getClass().getClassLoader(),
-        new Class[]{THBaseService.Iface.class},
-        new THBaseServiceMetricsProxy(handler, metrics));
-  }
-
-  private static class THBaseServiceMetricsProxy implements InvocationHandler {
-    private final THBaseService.Iface handler;
-    private final ThriftMetrics metrics;
-
-    private THBaseServiceMetricsProxy(
-        THBaseService.Iface handler, ThriftMetrics metrics) {
-      this.handler = handler;
-      this.metrics = metrics;
-    }
-
-    @Override
-    public Object invoke(Object proxy, Method m, Object[] args)
-        throws Throwable {
-      Object result;
-      try {
-        long start = now();
-        result = m.invoke(handler, args);
-        int processTime = (int)(now() - start);
-        metrics.incMethodTime(m.getName(), processTime);
-      } catch (InvocationTargetException e) {
-        throw e.getTargetException();
-      } catch (Exception e) {
-        throw new RuntimeException(
-            "unexpected invocation exception: " + e.getMessage());
-      }
-      return result;
-    }
-  }
-    
-  private static long now() {
-    return System.nanoTime();
-  }
-
-  ThriftHBaseServiceHandler(Configuration conf) {
-    htablePool = new HTablePool(conf, Integer.MAX_VALUE);
-  }
-
-  private HTableInterface getTable(byte[] tableName) {
-    return htablePool.getTable(tableName);
-  }
-
-  private void closeTable(HTableInterface table) throws TIOError {
-    try {
-      table.close();
-    } catch (IOException e) {
-      throw getTIOError(e);
-    }
-  }
-
-  private TIOError getTIOError(IOException e) {
-    TIOError err = new TIOError();
-    err.setMessage(e.getMessage());
-    return err;
-  }
-
-  /**
-   * Assigns a unique ID to the scanner and adds the mapping to an internal HashMap.
-   * 
-   * @param scanner to add
-   * @return Id for this Scanner
-   */
-  private int addScanner(ResultScanner scanner) {
-    int id = nextScannerId.getAndIncrement();
-    scannerMap.put(id, scanner);
-    return id;
-  }
-
-  /**
-   * Returns the Scanner associated with the specified Id.
-   * 
-   * @param id of the Scanner to get
-   * @return a Scanner, or null if the Id is invalid
-   */
-  private ResultScanner getScanner(int id) {
-    return scannerMap.get(id);
-  }
-
-  /**
-   * Removes the scanner associated with the specified ID from the internal HashMap.
-   * 
-   * @param id of the Scanner to remove
-   * @return the removed Scanner, or <code>null</code> if the Id is invalid
-   */
-  protected ResultScanner removeScanner(int id) {
-    return scannerMap.remove(id);
-  }
-
-  @Override
-  public boolean exists(ByteBuffer table, TGet get) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      return htable.exists(getFromThrift(get));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public TResult get(ByteBuffer table, TGet get) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      return resultFromHBase(htable.get(getFromThrift(get)));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public List<TResult> getMultiple(ByteBuffer table, List<TGet> gets) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      return resultsFromHBase(htable.get(getsFromThrift(gets)));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public void put(ByteBuffer table, TPut put) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      htable.put(putFromThrift(put));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public boolean checkAndPut(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value, TPut put)
-    throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      return htable.checkAndPut(row.array(), family.array(), qualifier.array(), (value == null) ? null : value.array(), putFromThrift(put));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public void putMultiple(ByteBuffer table, List<TPut> puts) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      htable.put(putsFromThrift(puts));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public void deleteSingle(ByteBuffer table, TDelete deleteSingle) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      htable.delete(deleteFromThrift(deleteSingle));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public List<TDelete> deleteMultiple(ByteBuffer table, List<TDelete> deletes) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      htable.delete(deletesFromThrift(deletes));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-    return Collections.emptyList();
-  }
-
-  @Override
-  public boolean checkAndDelete(ByteBuffer table, ByteBuffer row, ByteBuffer family, ByteBuffer qualifier, ByteBuffer value,
-      TDelete deleteSingle) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-
-    try {
-      if (value == null) {
-        return htable.checkAndDelete(row.array(), family.array(), qualifier.array(), null, deleteFromThrift(deleteSingle));
-      } else {
-        return htable.checkAndDelete(row.array(), family.array(), qualifier.array(), value.array(), deleteFromThrift(deleteSingle));
-      }
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public TResult increment(ByteBuffer table, TIncrement increment) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    try {
-      return resultFromHBase(htable.increment(incrementFromThrift(increment)));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-  }
-
-  @Override
-  public int openScanner(ByteBuffer table, TScan scan) throws TIOError, TException {
-    HTableInterface htable = getTable(table.array());
-    ResultScanner resultScanner = null;
-    try {
-      resultScanner = htable.getScanner(scanFromThrift(scan));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    } finally {
-      closeTable(htable);
-    }
-    return addScanner(resultScanner);
-  }
-
-  @Override
-  public List<TResult> getScannerRows(int scannerId, int numRows) throws TIOError, TIllegalArgument, TException {
-    ResultScanner scanner = getScanner(scannerId);
-    if (scanner == null) {
-      TIllegalArgument ex = new TIllegalArgument();
-      ex.setMessage("Invalid scanner Id");
-      throw ex;
-    }
-
-    try {
-      return resultsFromHBase(scanner.next(numRows));
-    } catch (IOException e) {
-      throw getTIOError(e);
-    }
-  }
-
-  @Override
-  public void closeScanner(int scannerId) throws TIOError, TIllegalArgument, TException {
-    if (removeScanner(scannerId) == null) {
-      TIllegalArgument ex = new TIllegalArgument();
-      ex.setMessage("Invalid scanner Id");
-      throw ex;
-    }
-  }
-
-}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftServer.java Wed Apr 24 04:44:09 2013
@@ -1,291 +0,0 @@
-/**
- *
- * 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.hadoop.hbase.thrift2;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.CommandLineParser;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Option;
-import org.apache.commons.cli.OptionGroup;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.cli.PosixParser;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.thrift.CallQueue;
-import org.apache.hadoop.hbase.thrift.CallQueue.Call;
-import org.apache.hadoop.hbase.thrift.ThriftMetrics;
-import org.apache.hadoop.hbase.thrift2.generated.THBaseService;
-import org.apache.hadoop.hbase.util.InfoServer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TCompactProtocol;
-import org.apache.thrift.protocol.TProtocolFactory;
-import org.apache.thrift.server.THsHaServer;
-import org.apache.thrift.server.TNonblockingServer;
-import org.apache.thrift.server.TServer;
-import org.apache.thrift.server.TThreadPoolServer;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TNonblockingServerSocket;
-import org.apache.thrift.transport.TNonblockingServerTransport;
-import org.apache.thrift.transport.TServerSocket;
-import org.apache.thrift.transport.TServerTransport;
-import org.apache.thrift.transport.TTransportException;
-import org.apache.thrift.transport.TTransportFactory;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * ThriftServer - this class starts up a Thrift server which implements the HBase API specified in the
- * HbaseClient.thrift IDL file.
- */
-@InterfaceAudience.Private
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class ThriftServer {
-  private static final Log log = LogFactory.getLog(ThriftServer.class);
-
-  public static final String DEFAULT_LISTEN_PORT = "9090";
-
-  public ThriftServer() {
-  }
-
-  private static void printUsage() {
-    HelpFormatter formatter = new HelpFormatter();
-    formatter.printHelp("Thrift", null, getOptions(),
-        "To start the Thrift server run 'bin/hbase-daemon.sh start thrift2'\n" +
-            "To shutdown the thrift server run 'bin/hbase-daemon.sh stop thrift2' or" +
-            " send a kill signal to the thrift server pid",
-        true);
-  }
-
-  private static Options getOptions() {
-    Options options = new Options();
-    options.addOption("b", "bind", true,
-        "Address to bind the Thrift server to. [default: 0.0.0.0]");
-    options.addOption("p", "port", true, "Port to bind to [default: " + DEFAULT_LISTEN_PORT + "]");
-    options.addOption("f", "framed", false, "Use framed transport");
-    options.addOption("c", "compact", false, "Use the compact protocol");
-    options.addOption("h", "help", false, "Print help information");
-    options.addOption(null, "infoport", true, "Port for web UI");
-
-    OptionGroup servers = new OptionGroup();
-    servers.addOption(
-        new Option("nonblocking", false, "Use the TNonblockingServer. This implies the framed transport."));
-    servers.addOption(new Option("hsha", false, "Use the THsHaServer. This implies the framed transport."));
-    servers.addOption(new Option("threadpool", false, "Use the TThreadPoolServer. This is the default."));
-    options.addOptionGroup(servers);
-    return options;
-  }
-
-  private static CommandLine parseArguments(Options options, String[] args) throws ParseException {
-    CommandLineParser parser = new PosixParser();
-    return parser.parse(options, args);
-  }
-
-  private static TProtocolFactory getTProtocolFactory(boolean isCompact) {
-    if (isCompact) {
-      log.debug("Using compact protocol");
-      return new TCompactProtocol.Factory();
-    } else {
-      log.debug("Using binary protocol");
-      return new TBinaryProtocol.Factory();
-    }
-  }
-
-  private static TTransportFactory getTTransportFactory(boolean framed) {
-    if (framed) {
-      log.debug("Using framed transport");
-      return new TFramedTransport.Factory();
-    } else {
-      return new TTransportFactory();
-    }
-  }
-
-  /*
-   * If bindValue is null, we don't bind. 
-   */
-  private static InetSocketAddress bindToPort(String bindValue, int listenPort)
-      throws UnknownHostException {
-    try {
-      if (bindValue == null) {
-        return new InetSocketAddress(listenPort);
-      } else {
-        return new InetSocketAddress(InetAddress.getByName(bindValue), listenPort);
-      }
-    } catch (UnknownHostException e) {
-      throw new RuntimeException("Could not bind to provided ip address", e);
-    }
-  }
-
-  private static TServer getTNonBlockingServer(TProtocolFactory protocolFactory, THBaseService.Processor processor,
-      TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase Nonblocking Thrift server on " + inetSocketAddress.toString());
-    TNonblockingServer.Args serverArgs = new TNonblockingServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TNonblockingServer(serverArgs);
-  }
-
-  private static TServer getTHsHaServer(TProtocolFactory protocolFactory,
-      THBaseService.Processor processor, TTransportFactory transportFactory,
-      InetSocketAddress inetSocketAddress, ThriftMetrics metrics)
-      throws TTransportException {
-    TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(inetSocketAddress);
-    log.info("starting HBase HsHA Thrift server on " + inetSocketAddress.toString());
-    THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
-    ExecutorService executorService = createExecutor(
-        serverArgs.getWorkerThreads(), metrics);
-    serverArgs.executorService(executorService);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new THsHaServer(serverArgs);
-  }
-
-  private static ExecutorService createExecutor(
-      int workerThreads, ThriftMetrics metrics) {
-    CallQueue callQueue = new CallQueue(
-        new LinkedBlockingQueue<Call>(), metrics);
-    ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
-    tfb.setDaemon(true);
-    tfb.setNameFormat("thrift2-worker-%d");
-    return new ThreadPoolExecutor(workerThreads, workerThreads,
-            Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build());
-  }
-
-  private static TServer getTThreadPoolServer(TProtocolFactory protocolFactory, THBaseService.Processor processor,
-      TTransportFactory transportFactory, InetSocketAddress inetSocketAddress) throws TTransportException {
-    TServerTransport serverTransport = new TServerSocket(inetSocketAddress);
-    log.info("starting HBase ThreadPool Thrift server on " + inetSocketAddress.toString());
-    TThreadPoolServer.Args serverArgs = new TThreadPoolServer.Args(serverTransport);
-    serverArgs.processor(processor);
-    serverArgs.transportFactory(transportFactory);
-    serverArgs.protocolFactory(protocolFactory);
-    return new TThreadPoolServer(serverArgs);
-  }
-
-  /**
-   * Start up the Thrift2 server.
-   * 
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    TServer server = null;
-    Options options = getOptions();
-    try {
-      CommandLine cmd = parseArguments(options, args);
-
-      /**
-       * This is to please both bin/hbase and bin/hbase-daemon. hbase-daemon provides "start" and "stop" arguments hbase
-       * should print the help if no argument is provided
-       */
-      List<?> argList = cmd.getArgList();
-      if (cmd.hasOption("help") || !argList.contains("start") || argList.contains("stop")) {
-        printUsage();
-        System.exit(1);
-      }
-
-      // Get port to bind to
-      int listenPort = 0;
-      try {
-        listenPort = Integer.parseInt(cmd.getOptionValue("port", DEFAULT_LISTEN_PORT));
-      } catch (NumberFormatException e) {
-        throw new RuntimeException("Could not parse the value provided for the port option", e);
-      }
-
-      boolean nonblocking = cmd.hasOption("nonblocking");
-      boolean hsha = cmd.hasOption("hsha");
-
-      Configuration conf = HBaseConfiguration.create();
-      ThriftMetrics metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.TWO);
-
-      String implType = "threadpool";
-      if (nonblocking) {
-        implType = "nonblocking";
-      } else if (hsha) {
-        implType = "hsha";
-      }
-
-      conf.set("hbase.regionserver.thrift.server.type", implType);
-      conf.setInt("hbase.regionserver.thrift.port", listenPort);
-
-      // Construct correct ProtocolFactory
-      boolean compact = cmd.hasOption("compact");
-      TProtocolFactory protocolFactory = getTProtocolFactory(compact);
-      THBaseService.Iface handler =
-          ThriftHBaseServiceHandler.newInstance(conf, metrics);
-      THBaseService.Processor processor = new THBaseService.Processor(handler);
-      conf.setBoolean("hbase.regionserver.thrift.compact", compact);
-
-      boolean framed = cmd.hasOption("framed") || nonblocking || hsha;
-      TTransportFactory transportFactory = getTTransportFactory(framed);
-      InetSocketAddress inetSocketAddress = bindToPort(cmd.getOptionValue("bind"), listenPort);
-      conf.setBoolean("hbase.regionserver.thrift.framed", framed);
-
-      // check for user-defined info server port setting, if so override the conf
-      try {
-        if (cmd.hasOption("infoport")) {
-          String val = cmd.getOptionValue("infoport");
-          conf.setInt("hbase.thrift.info.port", Integer.valueOf(val));
-          log.debug("Web UI port set to " + val);
-        }
-      } catch (NumberFormatException e) {
-        log.error("Could not parse the value provided for the infoport option", e);
-        printUsage();
-        System.exit(1);
-      }
-
-      // Put up info server.
-      int port = conf.getInt("hbase.thrift.info.port", 9095);
-      if (port >= 0) {
-        conf.setLong("startcode", System.currentTimeMillis());
-        String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
-        InfoServer infoServer = new InfoServer("thrift", a, port, false, conf);
-        infoServer.setAttribute("hbase.conf", conf);
-        infoServer.start();
-      }
-
-      if (nonblocking) {
-        server = getTNonBlockingServer(protocolFactory, processor, transportFactory, inetSocketAddress);
-      } else if (hsha) {
-        server = getTHsHaServer(protocolFactory, processor, transportFactory, inetSocketAddress, metrics);
-      } else {
-        server = getTThreadPoolServer(protocolFactory, processor, transportFactory, inetSocketAddress);
-      }
-    } catch (Exception e) {
-      log.error(e.getMessage(), e);
-      printUsage();
-      System.exit(1);
-    }
-    server.serve();
-  }
-}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/ThriftUtilities.java Wed Apr 24 04:44:09 2013
@@ -1,328 +0,0 @@
-/**
- *
- * 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.hadoop.hbase.thrift2;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValueUtil;
-import org.apache.hadoop.hbase.client.*;
-import org.apache.hadoop.hbase.thrift2.generated.*;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-@InterfaceAudience.Private
-public class ThriftUtilities {
-
-  private ThriftUtilities() {
-    throw new UnsupportedOperationException("Can't initialize class");
-  }
-
-  /**
-   * Creates a {@link Get} (HBase) from a {@link TGet} (Thrift).
-   *
-   * This ignores any timestamps set on {@link TColumn} objects.
-   *
-   * @param in the <code>TGet</code> to convert
-   *
-   * @return <code>Get</code> object
-   *
-   * @throws IOException if an invalid time range or max version parameter is given
-   */
-  public static Get getFromThrift(TGet in) throws IOException {
-    Get out = new Get(in.getRow());
-
-    // Timestamp overwrites time range if both are set
-    if (in.isSetTimestamp()) {
-      out.setTimeStamp(in.getTimestamp());
-    } else if (in.isSetTimeRange()) {
-      out.setTimeRange(in.getTimeRange().getMinStamp(), in.getTimeRange().getMaxStamp());
-    }
-
-    if (in.isSetMaxVersions()) {
-      out.setMaxVersions(in.getMaxVersions());
-    }
-
-    if (!in.isSetColumns()) {
-      return out;
-    }
-
-    for (TColumn column : in.getColumns()) {
-      if (column.isSetQualifier()) {
-        out.addColumn(column.getFamily(), column.getQualifier());
-      } else {
-        out.addFamily(column.getFamily());
-      }
-    }
-
-    return out;
-  }
-
-  /**
-   * Converts multiple {@link TGet}s (Thrift) into a list of {@link Get}s (HBase).
-   *
-   * @param in list of <code>TGet</code>s to convert
-   *
-   * @return list of <code>Get</code> objects
-   *
-   * @throws IOException if an invalid time range or max version parameter is given
-   * @see #getFromThrift(TGet)
-   */
-  public static List<Get> getsFromThrift(List<TGet> in) throws IOException {
-    List<Get> out = new ArrayList<Get>(in.size());
-    for (TGet get : in) {
-      out.add(getFromThrift(get));
-    }
-    return out;
-  }
-
-  /**
-   * Creates a {@link TResult} (Thrift) from a {@link Result} (HBase).
-   *
-   * @param in the <code>Result</code> to convert
-   *
-   * @return converted result, returns an empty result if the input is <code>null</code>
-   */
-  public static TResult resultFromHBase(Result in) {
-    KeyValue[] raw = in.raw();
-    TResult out = new TResult();
-    byte[] row = in.getRow();
-    if (row != null) {
-      out.setRow(in.getRow());
-    }
-    List<TColumnValue> columnValues = new ArrayList<TColumnValue>();
-    for (KeyValue kv : raw) {
-      TColumnValue col = new TColumnValue();
-      col.setFamily(kv.getFamily());
-      col.setQualifier(kv.getQualifier());
-      col.setTimestamp(kv.getTimestamp());
-      col.setValue(kv.getValue());
-      columnValues.add(col);
-    }
-    out.setColumnValues(columnValues);
-    return out;
-  }
-
-  /**
-   * Converts multiple {@link Result}s (HBase) into a list of {@link TResult}s (Thrift).
-   *
-   * @param in array of <code>Result</code>s to convert
-   *
-   * @return list of converted <code>TResult</code>s
-   *
-   * @see #resultFromHBase(Result)
-   */
-  public static List<TResult> resultsFromHBase(Result[] in) {
-    List<TResult> out = new ArrayList<TResult>(in.length);
-    for (Result result : in) {
-      out.add(resultFromHBase(result));
-    }
-    return out;
-  }
-
-  /**
-   * Creates a {@link Put} (HBase) from a {@link TPut} (Thrift)
-   *
-   * @param in the <code>TPut</code> to convert
-   *
-   * @return converted <code>Put</code>
-   */
-  public static Put putFromThrift(TPut in) {
-    Put out;
-
-    if (in.isSetTimestamp()) {
-      out = new Put(in.getRow(), in.getTimestamp());
-    } else {
-      out = new Put(in.getRow());
-    }
-
-    out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-
-    for (TColumnValue columnValue : in.getColumnValues()) {
-      if (columnValue.isSetTimestamp()) {
-        out.add(columnValue.getFamily(), columnValue.getQualifier(), columnValue.getTimestamp(),
-            columnValue.getValue());
-      } else {
-        out.add(columnValue.getFamily(), columnValue.getQualifier(), columnValue.getValue());
-      }
-    }
-
-    return out;
-  }
-
-  /**
-   * Converts multiple {@link TPut}s (Thrift) into a list of {@link Put}s (HBase).
-   *
-   * @param in list of <code>TPut</code>s to convert
-   *
-   * @return list of converted <code>Put</code>s
-   *
-   * @see #putFromThrift(TPut)
-   */
-  public static List<Put> putsFromThrift(List<TPut> in) {
-    List<Put> out = new ArrayList<Put>(in.size());
-    for (TPut put : in) {
-      out.add(putFromThrift(put));
-    }
-    return out;
-  }
-
-  /**
-   * Creates a {@link Delete} (HBase) from a {@link TDelete} (Thrift).
-   *
-   * @param in the <code>TDelete</code> to convert
-   *
-   * @return converted <code>Delete</code>
-   */
-  public static Delete deleteFromThrift(TDelete in) {
-    Delete out;
-
-    if (in.isSetColumns()) {
-      out = new Delete(in.getRow());
-      for (TColumn column : in.getColumns()) {
-        if (column.isSetQualifier()) {
-          if (column.isSetTimestamp()) {
-            if (in.isSetDeleteType() &&
-                in.getDeleteType().equals(TDeleteType.DELETE_COLUMNS))
-              out.deleteColumns(column.getFamily(), column.getQualifier(), column.getTimestamp());
-            else
-              out.deleteColumn(column.getFamily(), column.getQualifier(), column.getTimestamp());
-          } else {
-            if (in.isSetDeleteType() &&
-                in.getDeleteType().equals(TDeleteType.DELETE_COLUMNS))
-              out.deleteColumns(column.getFamily(), column.getQualifier());
-            else
-              out.deleteColumn(column.getFamily(), column.getQualifier());
-          }
-
-        } else {
-          if (column.isSetTimestamp()) {
-            out.deleteFamily(column.getFamily(), column.getTimestamp());
-          } else {
-            out.deleteFamily(column.getFamily());
-          }
-        }
-      }
-    } else {
-      if (in.isSetTimestamp()) {
-        out = new Delete(in.getRow(), in.getTimestamp());
-      } else {
-        out = new Delete(in.getRow());
-      }
-    }
-    out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-    return out;
-  }
-
-  /**
-   * Converts multiple {@link TDelete}s (Thrift) into a list of {@link Delete}s (HBase).
-   *
-   * @param in list of <code>TDelete</code>s to convert
-   *
-   * @return list of converted <code>Delete</code>s
-   *
-   * @see #deleteFromThrift(TDelete)
-   */
-
-  public static List<Delete> deletesFromThrift(List<TDelete> in) {
-    List<Delete> out = new ArrayList<Delete>(in.size());
-    for (TDelete delete : in) {
-      out.add(deleteFromThrift(delete));
-    }
-    return out;
-  }
-
-  public static TDelete deleteFromHBase(Delete in) {
-    TDelete out = new TDelete(ByteBuffer.wrap(in.getRow()));
-
-    List<TColumn> columns = new ArrayList<TColumn>();
-    long rowTimestamp = in.getTimeStamp();
-    if (rowTimestamp != HConstants.LATEST_TIMESTAMP) {
-      out.setTimestamp(rowTimestamp);
-    }
-
-    // Map<family, List<KeyValue>>
-    for (Map.Entry<byte[], List<? extends org.apache.hadoop.hbase.Cell>> familyEntry:
-        in.getFamilyMap().entrySet()) {
-      TColumn column = new TColumn(ByteBuffer.wrap(familyEntry.getKey()));
-      for (org.apache.hadoop.hbase.Cell cell: familyEntry.getValue()) {
-        KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
-        byte[] family = kv.getFamily();
-        byte[] qualifier = kv.getQualifier();
-        long timestamp = kv.getTimestamp();
-        if (family != null) {
-          column.setFamily(family);
-        }
-        if (qualifier != null) {
-          column.setQualifier(qualifier);
-        }
-        if (timestamp != HConstants.LATEST_TIMESTAMP) {
-          column.setTimestamp(kv.getTimestamp());
-        }
-      }
-      columns.add(column);
-    }
-    out.setColumns(columns);
-
-    return out;
-  }
-
-  public static Scan scanFromThrift(TScan in) throws IOException {
-    Scan out = new Scan();
-
-    if (in.isSetStartRow())
-      out.setStartRow(in.getStartRow());
-    if (in.isSetStopRow())
-      out.setStopRow(in.getStopRow());
-    if (in.isSetCaching())
-      out.setCaching(in.getCaching());
-    if (in.isSetMaxVersions()) {
-      out.setMaxVersions(in.getMaxVersions());
-    }
-
-    if (in.isSetColumns()) {
-      for (TColumn column : in.getColumns()) {
-        if (column.isSetQualifier()) {
-          out.addColumn(column.getFamily(), column.getQualifier());
-        } else {
-          out.addFamily(column.getFamily());
-        }
-      }
-    }
-
-    TTimeRange timeRange = in.getTimeRange();
-    if (timeRange != null &&
-        timeRange.isSetMinStamp() && timeRange.isSetMaxStamp()) {
-      out.setTimeRange(timeRange.getMinStamp(), timeRange.getMaxStamp());
-    }
-
-    return out;
-  }
-
-  public static Increment incrementFromThrift(TIncrement in) throws IOException {
-    Increment out = new Increment(in.getRow());
-    for (TColumnIncrement column : in.getColumns()) {
-      out.addColumn(column.getFamily(), column.getQualifier(), column.getAmount());
-    }
-    out.setDurability(in.isWriteToWal() ? Durability.SYNC_WAL : Durability.SKIP_WAL);
-    return out;
-  }
-}

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java?rev=1471246&r1=1471245&r2=1471246&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/thrift2/generated/TColumn.java Wed Apr 24 04:44:09 2013
@@ -1,611 +0,0 @@
-/**
- * Autogenerated by Thrift Compiler (0.9.0)
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- *  @generated
- */
-package org.apache.hadoop.hbase.thrift2.generated;
-
-import org.apache.thrift.scheme.IScheme;
-import org.apache.thrift.scheme.SchemeFactory;
-import org.apache.thrift.scheme.StandardScheme;
-
-import org.apache.thrift.scheme.TupleScheme;
-import org.apache.thrift.protocol.TTupleProtocol;
-import org.apache.thrift.protocol.TProtocolException;
-import org.apache.thrift.EncodingUtils;
-import org.apache.thrift.TException;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.EnumMap;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.EnumSet;
-import java.util.Collections;
-import java.util.BitSet;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Addresses a single cell or multiple cells
- * in a HBase table by column family and optionally
- * a column qualifier and timestamp
- */
-public class TColumn implements org.apache.thrift.TBase<TColumn, TColumn._Fields>, java.io.Serializable, Cloneable {
-  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn");
-
-  private static final org.apache.thrift.protocol.TField FAMILY_FIELD_DESC = new org.apache.thrift.protocol.TField("family", org.apache.thrift.protocol.TType.STRING, (short)1);
-  private static final org.apache.thrift.protocol.TField QUALIFIER_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifier", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField TIMESTAMP_FIELD_DESC = new org.apache.thrift.protocol.TField("timestamp", org.apache.thrift.protocol.TType.I64, (short)3);
-
-  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-  static {
-    schemes.put(StandardScheme.class, new TColumnStandardSchemeFactory());
-    schemes.put(TupleScheme.class, new TColumnTupleSchemeFactory());
-  }
-
-  public ByteBuffer family; // required
-  public ByteBuffer qualifier; // optional
-  public long timestamp; // optional
-
-  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-    FAMILY((short)1, "family"),
-    QUALIFIER((short)2, "qualifier"),
-    TIMESTAMP((short)3, "timestamp");
-
-    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-    static {
-      for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byName.put(field.getFieldName(), field);
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, or null if its not found.
-     */
-    public static _Fields findByThriftId(int fieldId) {
-      switch(fieldId) {
-        case 1: // FAMILY
-          return FAMILY;
-        case 2: // QUALIFIER
-          return QUALIFIER;
-        case 3: // TIMESTAMP
-          return TIMESTAMP;
-        default:
-          return null;
-      }
-    }
-
-    /**
-     * Find the _Fields constant that matches fieldId, throwing an exception
-     * if it is not found.
-     */
-    public static _Fields findByThriftIdOrThrow(int fieldId) {
-      _Fields fields = findByThriftId(fieldId);
-      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-      return fields;
-    }
-
-    /**
-     * Find the _Fields constant that matches name, or null if its not found.
-     */
-    public static _Fields findByName(String name) {
-      return byName.get(name);
-    }
-
-    private final short _thriftId;
-    private final String _fieldName;
-
-    _Fields(short thriftId, String fieldName) {
-      _thriftId = thriftId;
-      _fieldName = fieldName;
-    }
-
-    public short getThriftFieldId() {
-      return _thriftId;
-    }
-
-    public String getFieldName() {
-      return _fieldName;
-    }
-  }
-
-  // isset id assignments
-  private static final int __TIMESTAMP_ISSET_ID = 0;
-  private byte __isset_bitfield = 0;
-  private _Fields optionals[] = {_Fields.QUALIFIER,_Fields.TIMESTAMP};
-  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-  static {
-    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-    tmpMap.put(_Fields.FAMILY, new org.apache.thrift.meta_data.FieldMetaData("family", org.apache.thrift.TFieldRequirementType.REQUIRED, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
-    tmpMap.put(_Fields.QUALIFIER, new org.apache.thrift.meta_data.FieldMetaData("qualifier", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING        , true)));
-    tmpMap.put(_Fields.TIMESTAMP, new org.apache.thrift.meta_data.FieldMetaData("timestamp", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
-        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)));
-    metaDataMap = Collections.unmodifiableMap(tmpMap);
-    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap);
-  }
-
-  public TColumn() {
-  }
-
-  public TColumn(
-    ByteBuffer family)
-  {
-    this();
-    this.family = family;
-  }
-
-  /**
-   * Performs a deep copy on <i>other</i>.
-   */
-  public TColumn(TColumn other) {
-    __isset_bitfield = other.__isset_bitfield;
-    if (other.isSetFamily()) {
-      this.family = org.apache.thrift.TBaseHelper.copyBinary(other.family);
-;
-    }
-    if (other.isSetQualifier()) {
-      this.qualifier = org.apache.thrift.TBaseHelper.copyBinary(other.qualifier);
-;
-    }
-    this.timestamp = other.timestamp;
-  }
-
-  public TColumn deepCopy() {
-    return new TColumn(this);
-  }
-
-  @Override
-  public void clear() {
-    this.family = null;
-    this.qualifier = null;
-    setTimestampIsSet(false);
-    this.timestamp = 0;
-  }
-
-  public byte[] getFamily() {
-    setFamily(org.apache.thrift.TBaseHelper.rightSize(family));
-    return family == null ? null : family.array();
-  }
-
-  public ByteBuffer bufferForFamily() {
-    return family;
-  }
-
-  public TColumn setFamily(byte[] family) {
-    setFamily(family == null ? (ByteBuffer)null : ByteBuffer.wrap(family));
-    return this;
-  }
-
-  public TColumn setFamily(ByteBuffer family) {
-    this.family = family;
-    return this;
-  }
-
-  public void unsetFamily() {
-    this.family = null;
-  }
-
-  /** Returns true if field family is set (has been assigned a value) and false otherwise */
-  public boolean isSetFamily() {
-    return this.family != null;
-  }
-
-  public void setFamilyIsSet(boolean value) {
-    if (!value) {
-      this.family = null;
-    }
-  }
-
-  public byte[] getQualifier() {
-    setQualifier(org.apache.thrift.TBaseHelper.rightSize(qualifier));
-    return qualifier == null ? null : qualifier.array();
-  }
-
-  public ByteBuffer bufferForQualifier() {
-    return qualifier;
-  }
-
-  public TColumn setQualifier(byte[] qualifier) {
-    setQualifier(qualifier == null ? (ByteBuffer)null : ByteBuffer.wrap(qualifier));
-    return this;
-  }
-
-  public TColumn setQualifier(ByteBuffer qualifier) {
-    this.qualifier = qualifier;
-    return this;
-  }
-
-  public void unsetQualifier() {
-    this.qualifier = null;
-  }
-
-  /** Returns true if field qualifier is set (has been assigned a value) and false otherwise */
-  public boolean isSetQualifier() {
-    return this.qualifier != null;
-  }
-
-  public void setQualifierIsSet(boolean value) {
-    if (!value) {
-      this.qualifier = null;
-    }
-  }
-
-  public long getTimestamp() {
-    return this.timestamp;
-  }
-
-  public TColumn setTimestamp(long timestamp) {
-    this.timestamp = timestamp;
-    setTimestampIsSet(true);
-    return this;
-  }
-
-  public void unsetTimestamp() {
-    __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
-  }
-
-  /** Returns true if field timestamp is set (has been assigned a value) and false otherwise */
-  public boolean isSetTimestamp() {
-    return EncodingUtils.testBit(__isset_bitfield, __TIMESTAMP_ISSET_ID);
-  }
-
-  public void setTimestampIsSet(boolean value) {
-    __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __TIMESTAMP_ISSET_ID, value);
-  }
-
-  public void setFieldValue(_Fields field, Object value) {
-    switch (field) {
-    case FAMILY:
-      if (value == null) {
-        unsetFamily();
-      } else {
-        setFamily((ByteBuffer)value);
-      }
-      break;
-
-    case QUALIFIER:
-      if (value == null) {
-        unsetQualifier();
-      } else {
-        setQualifier((ByteBuffer)value);
-      }
-      break;
-
-    case TIMESTAMP:
-      if (value == null) {
-        unsetTimestamp();
-      } else {
-        setTimestamp((Long)value);
-      }
-      break;
-
-    }
-  }
-
-  public Object getFieldValue(_Fields field) {
-    switch (field) {
-    case FAMILY:
-      return getFamily();
-
-    case QUALIFIER:
-      return getQualifier();
-
-    case TIMESTAMP:
-      return Long.valueOf(getTimestamp());
-
-    }
-    throw new IllegalStateException();
-  }
-
-  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-  public boolean isSet(_Fields field) {
-    if (field == null) {
-      throw new IllegalArgumentException();
-    }
-
-    switch (field) {
-    case FAMILY:
-      return isSetFamily();
-    case QUALIFIER:
-      return isSetQualifier();
-    case TIMESTAMP:
-      return isSetTimestamp();
-    }
-    throw new IllegalStateException();
-  }
-
-  @Override
-  public boolean equals(Object that) {
-    if (that == null)
-      return false;
-    if (that instanceof TColumn)
-      return this.equals((TColumn)that);
-    return false;
-  }
-
-  public boolean equals(TColumn that) {
-    if (that == null)
-      return false;
-
-    boolean this_present_family = true && this.isSetFamily();
-    boolean that_present_family = true && that.isSetFamily();
-    if (this_present_family || that_present_family) {
-      if (!(this_present_family && that_present_family))
-        return false;
-      if (!this.family.equals(that.family))
-        return false;
-    }
-
-    boolean this_present_qualifier = true && this.isSetQualifier();
-    boolean that_present_qualifier = true && that.isSetQualifier();
-    if (this_present_qualifier || that_present_qualifier) {
-      if (!(this_present_qualifier && that_present_qualifier))
-        return false;
-      if (!this.qualifier.equals(that.qualifier))
-        return false;
-    }
-
-    boolean this_present_timestamp = true && this.isSetTimestamp();
-    boolean that_present_timestamp = true && that.isSetTimestamp();
-    if (this_present_timestamp || that_present_timestamp) {
-      if (!(this_present_timestamp && that_present_timestamp))
-        return false;
-      if (this.timestamp != that.timestamp)
-        return false;
-    }
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    return 0;
-  }
-
-  public int compareTo(TColumn other) {
-    if (!getClass().equals(other.getClass())) {
-      return getClass().getName().compareTo(other.getClass().getName());
-    }
-
-    int lastComparison = 0;
-    TColumn typedOther = (TColumn)other;
-
-    lastComparison = Boolean.valueOf(isSetFamily()).compareTo(typedOther.isSetFamily());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetFamily()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.family, typedOther.family);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetQualifier()).compareTo(typedOther.isSetQualifier());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetQualifier()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifier, typedOther.qualifier);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    lastComparison = Boolean.valueOf(isSetTimestamp()).compareTo(typedOther.isSetTimestamp());
-    if (lastComparison != 0) {
-      return lastComparison;
-    }
-    if (isSetTimestamp()) {
-      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.timestamp, typedOther.timestamp);
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-    }
-    return 0;
-  }
-
-  public _Fields fieldForId(int fieldId) {
-    return _Fields.findByThriftId(fieldId);
-  }
-
-  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-  }
-
-  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder("TColumn(");
-    boolean first = true;
-
-    sb.append("family:");
-    if (this.family == null) {
-      sb.append("null");
-    } else {
-      org.apache.thrift.TBaseHelper.toString(this.family, sb);
-    }
-    first = false;
-    if (isSetQualifier()) {
-      if (!first) sb.append(", ");
-      sb.append("qualifier:");
-      if (this.qualifier == null) {
-        sb.append("null");
-      } else {
-        org.apache.thrift.TBaseHelper.toString(this.qualifier, sb);
-      }
-      first = false;
-    }
-    if (isSetTimestamp()) {
-      if (!first) sb.append(", ");
-      sb.append("timestamp:");
-      sb.append(this.timestamp);
-      first = false;
-    }
-    sb.append(")");
-    return sb.toString();
-  }
-
-  public void validate() throws org.apache.thrift.TException {
-    // check for required fields
-    if (family == null) {
-      throw new org.apache.thrift.protocol.TProtocolException("Required field 'family' was not present! Struct: " + toString());
-    }
-    // check for sub-struct validity
-  }
-
-  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-    try {
-      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-    try {
-      // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor.
-      __isset_bitfield = 0;
-      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-    } catch (org.apache.thrift.TException te) {
-      throw new java.io.IOException(te);
-    }
-  }
-
-  private static class TColumnStandardSchemeFactory implements SchemeFactory {
-    public TColumnStandardScheme getScheme() {
-      return new TColumnStandardScheme();
-    }
-  }
-
-  private static class TColumnStandardScheme extends StandardScheme<TColumn> {
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot, TColumn struct) throws org.apache.thrift.TException {
-      org.apache.thrift.protocol.TField schemeField;
-      iprot.readStructBegin();
-      while (true)
-      {
-        schemeField = iprot.readFieldBegin();
-        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-          break;
-        }
-        switch (schemeField.id) {
-          case 1: // FAMILY
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.family = iprot.readBinary();
-              struct.setFamilyIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 2: // QUALIFIER
-            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
-              struct.qualifier = iprot.readBinary();
-              struct.setQualifierIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          case 3: // TIMESTAMP
-            if (schemeField.type == org.apache.thrift.protocol.TType.I64) {
-              struct.timestamp = iprot.readI64();
-              struct.setTimestampIsSet(true);
-            } else { 
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-            }
-            break;
-          default:
-            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-        }
-        iprot.readFieldEnd();
-      }
-      iprot.readStructEnd();
-
-      // check for required fields of primitive type, which can't be checked in the validate method
-      struct.validate();
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot, TColumn struct) throws org.apache.thrift.TException {
-      struct.validate();
-
-      oprot.writeStructBegin(STRUCT_DESC);
-      if (struct.family != null) {
-        oprot.writeFieldBegin(FAMILY_FIELD_DESC);
-        oprot.writeBinary(struct.family);
-        oprot.writeFieldEnd();
-      }
-      if (struct.qualifier != null) {
-        if (struct.isSetQualifier()) {
-          oprot.writeFieldBegin(QUALIFIER_FIELD_DESC);
-          oprot.writeBinary(struct.qualifier);
-          oprot.writeFieldEnd();
-        }
-      }
-      if (struct.isSetTimestamp()) {
-        oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
-        oprot.writeI64(struct.timestamp);
-        oprot.writeFieldEnd();
-      }
-      oprot.writeFieldStop();
-      oprot.writeStructEnd();
-    }
-
-  }
-
-  private static class TColumnTupleSchemeFactory implements SchemeFactory {
-    public TColumnTupleScheme getScheme() {
-      return new TColumnTupleScheme();
-    }
-  }
-
-  private static class TColumnTupleScheme extends TupleScheme<TColumn> {
-
-    @Override
-    public void write(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
-      TTupleProtocol oprot = (TTupleProtocol) prot;
-      oprot.writeBinary(struct.family);
-      BitSet optionals = new BitSet();
-      if (struct.isSetQualifier()) {
-        optionals.set(0);
-      }
-      if (struct.isSetTimestamp()) {
-        optionals.set(1);
-      }
-      oprot.writeBitSet(optionals, 2);
-      if (struct.isSetQualifier()) {
-        oprot.writeBinary(struct.qualifier);
-      }
-      if (struct.isSetTimestamp()) {
-        oprot.writeI64(struct.timestamp);
-      }
-    }
-
-    @Override
-    public void read(org.apache.thrift.protocol.TProtocol prot, TColumn struct) throws org.apache.thrift.TException {
-      TTupleProtocol iprot = (TTupleProtocol) prot;
-      struct.family = iprot.readBinary();
-      struct.setFamilyIsSet(true);
-      BitSet incoming = iprot.readBitSet(2);
-      if (incoming.get(0)) {
-        struct.qualifier = iprot.readBinary();
-        struct.setQualifierIsSet(true);
-      }
-      if (incoming.get(1)) {
-        struct.timestamp = iprot.readI64();
-        struct.setTimestampIsSet(true);
-      }
-    }
-  }
-
-}
-