You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ns...@apache.org on 2011/10/11 19:42:35 UTC

svn commit: r1181928 [1/3] - in /hbase/branches/0.89: ./ src/main/java/org/apache/hadoop/hbase/thrift/ src/main/java/org/apache/hadoop/hbase/thrift/generated/ src/test/java/org/apache/hadoop/hbase/thrift/

Author: nspiegelberg
Date: Tue Oct 11 17:42:34 2011
New Revision: 1181928

URL: http://svn.apache.org/viewvc?rev=1181928&view=rev
Log:
Upgrading Hbase-89 branch's Thrift version from 0.2.0 to 0.5.0

Summary:
The most notable change made is to introduce ByteBuffers everywhere.

Test Plan: TestThriftServer.java
Reviewed By: kannan
Reviewers: kannan, pkhemani, jgray, dgoode
CC: hbase@lists, kannan
Differential Revision: 302673

Modified:
    hbase/branches/0.89/pom.xml
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
    hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
    hbase/branches/0.89/src/test/java/org/apache/hadoop/hbase/thrift/TestThriftServer.java

Modified: hbase/branches/0.89/pom.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/pom.xml?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/pom.xml (original)
+++ hbase/branches/0.89/pom.xml Tue Oct 11 17:42:34 2011
@@ -458,7 +458,7 @@
     <protobuf.version>2.3.0</protobuf.version>
     <slf4j.version>1.5.8</slf4j.version>
     <stax-api>1.0.1</stax-api>
-    <thrift.version>0.2.0</thrift.version>
+    <thrift.version>0.5.0</thrift.version>
     <guava.version>r05</guava.version>
   </properties>
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Tue Oct 11 17:42:34 2011
@@ -84,6 +84,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
+import java.nio.ByteBuffer;
+
 /**
  * ThriftServer - this class starts up a Thrift server which implements the
  * Hbase API specified in the Hbase.thrift IDL file.
@@ -108,7 +110,6 @@ public class ThriftServer {
       protected Map<String, HTable> initialValue() {
         return new TreeMap<String, HTable>();
       }
-
     };
 
     /**
@@ -123,7 +124,7 @@ public class ThriftServer {
       byte[][] columns = new byte[cds.length][];
       for (int i = 0; i < cds.length; i++) {
         columns[i] = Bytes.add(cds[i].getName(),
-            KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
+                               KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
       }
       return columns;
     }
@@ -138,7 +139,7 @@ public class ThriftServer {
      * @throws IOError
      */
     protected HTable getTable(final byte[] tableName) throws IOError,
-        IOException {
+      IOException {
       String table = new String(tableName);
       Map<String, HTable> tables = threadLocalTables.get();
       if (!tables.containsKey(table)) {
@@ -192,52 +193,52 @@ public class ThriftServer {
       scannerMap = new HashMap<Integer, ResultScanner>();
     }
 
-    public void enableTable(final byte[] tableName) throws IOError {
+    public void enableTable(ByteBuffer tableName) throws IOError {
       try{
-        admin.enableTable(tableName);
+        admin.enableTable(tableName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public void disableTable(final byte[] tableName) throws IOError{
+    public void disableTable(ByteBuffer tableName) throws IOError{
       try{
-        admin.disableTable(tableName);
+        admin.disableTable(tableName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public boolean isTableEnabled(final byte[] tableName) throws IOError {
+    public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
       try {
-        return HTable.isTableEnabled(tableName);
+        return HTable.isTableEnabled(tableName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public void compact(byte[] tableNameOrRegionName) throws IOError {
+    public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
       try{
-        admin.compact(tableNameOrRegionName);
+        admin.compact(tableNameOrRegionName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public void majorCompact(byte[] tableNameOrRegionName) throws IOError {
+    public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
       try{
-        admin.majorCompact(tableNameOrRegionName);
+        admin.majorCompact(tableNameOrRegionName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public List<byte[]> getTableNames() throws IOError {
+    public List<ByteBuffer> getTableNames() throws IOError {
       try {
         HTableDescriptor[] tables = this.admin.listTables();
-        ArrayList<byte[]> list = new ArrayList<byte[]>(tables.length);
+        List<ByteBuffer> list = new ArrayList<ByteBuffer>(tables.length);
         for (int i = 0; i < tables.length; i++) {
-          list.add(tables[i].getName());
+          list.add(ByteBuffer.wrap(tables[i].getName()));
         }
         return list;
       } catch (IOException e) {
@@ -245,19 +246,19 @@ public class ThriftServer {
       }
     }
 
-    public List<TRegionInfo> getTableRegions(byte[] tableName)
+    public List<TRegionInfo> getTableRegions(ByteBuffer tableName)
     throws IOError {
       try{
-        HTable table = getTable(tableName);
+        HTable table = getTable(tableName.array());
         Map<HRegionInfo, HServerAddress> regionsInfo = table.getRegionsInfo();
         List<TRegionInfo> regions = new ArrayList<TRegionInfo>();
 
         for (HRegionInfo regionInfo : regionsInfo.keySet()){
           TRegionInfo region = new TRegionInfo();
-          region.startKey = regionInfo.getStartKey();
-          region.endKey = regionInfo.getEndKey();
+          region.setStartKey(regionInfo.getStartKey());
+          region.setEndKey(regionInfo.getEndKey());
           region.id = regionInfo.getRegionId();
-          region.name = regionInfo.getRegionName();
+          region.setName(regionInfo.getRegionName());
           region.version = regionInfo.getVersion();
           regions.add(region);
         }
@@ -268,13 +269,13 @@ public class ThriftServer {
     }
 
     @Deprecated
-    public List<TCell> get(byte[] tableName, byte[] row, byte[] column)
+    public List<TCell> get(ByteBuffer tableName, ByteBuffer row, ByteBuffer column)
         throws IOError {
-      byte [][] famAndQf = KeyValue.parseColumn(column);
+      byte [][] famAndQf = KeyValue.parseColumn(column.array());
       if(famAndQf.length == 1) {
-        return get(tableName, row, famAndQf[0], new byte[0]);
+        return get(tableName.array(), row.array(), famAndQf[0], new byte[0]);
       }
-      return get(tableName, row, famAndQf[0], famAndQf[1]);
+      return get(tableName.array(), row.array(), famAndQf[0], famAndQf[1]);
     }
 
     public List<TCell> get(byte [] tableName, byte [] row, byte [] family,
@@ -295,20 +296,20 @@ public class ThriftServer {
     }
 
     @Deprecated
-    public List<TCell> getVer(byte[] tableName, byte[] row,
-        byte[] column, int numVersions) throws IOError {
-      byte [][] famAndQf = KeyValue.parseColumn(column);
+    public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row,
+                              ByteBuffer column, int numVersions) throws IOError {
+      byte [][] famAndQf = KeyValue.parseColumn(column.array());
       if(famAndQf.length == 1) {
         return getVer(tableName, row, famAndQf[0], new byte[0], numVersions);
       }
       return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions);
     }
 
-    public List<TCell> getVer(byte [] tableName, byte [] row, byte [] family,
-        byte [] qualifier, int numVersions) throws IOError {
+    public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, byte [] family,
+                              byte [] qualifier, int numVersions) throws IOError {
       try {
-        HTable table = getTable(tableName);
-        Get get = new Get(row);
+        HTable table = getTable(tableName.array());
+        Get get = new Get(row.array());
         get.addColumn(family, qualifier);
         get.setMaxVersions(numVersions);
         Result result = table.get(get);
@@ -319,9 +320,9 @@ public class ThriftServer {
     }
 
     @Deprecated
-    public List<TCell> getVerTs(byte[] tableName, byte[] row,
-        byte[] column, long timestamp, int numVersions) throws IOError {
-      byte [][] famAndQf = KeyValue.parseColumn(column);
+    public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row,
+                                ByteBuffer column, long timestamp, int numVersions) throws IOError {
+      byte [][] famAndQf = KeyValue.parseColumn(column.array());
       if(famAndQf.length == 1) {
         return getVerTs(tableName, row, famAndQf[0], new byte[0], timestamp,
             numVersions);
@@ -330,11 +331,11 @@ public class ThriftServer {
           numVersions);
     }
 
-    public List<TCell> getVerTs(byte [] tableName, byte [] row, byte [] family,
+    public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, byte [] family,
         byte [] qualifier, long timestamp, int numVersions) throws IOError {
       try {
-        HTable table = getTable(tableName);
-        Get get = new Get(row);
+        HTable table = getTable(tableName.array());
+        Get get = new Get(row.array());
         get.addColumn(family, qualifier);
         get.setTimeRange(Long.MIN_VALUE, timestamp);
         get.setMaxVersions(numVersions);
@@ -345,42 +346,42 @@ public class ThriftServer {
       }
     }
 
-    public List<TRowResult> getRow(byte[] tableName, byte[] row)
-        throws IOError {
+    public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row)
+      throws IOError {
       return getRowWithColumnsTs(tableName, row, null,
                                  HConstants.LATEST_TIMESTAMP);
     }
 
-    public List<TRowResult> getRowWithColumns(byte[] tableName, byte[] row,
-        List<byte[]> columns) throws IOError {
+    public List<TRowResult> getRowWithColumns(ByteBuffer tableName, ByteBuffer row,
+                                              List<ByteBuffer> columns) throws IOError {
       return getRowWithColumnsTs(tableName, row, columns,
                                  HConstants.LATEST_TIMESTAMP);
     }
 
-    public List<TRowResult> getRowTs(byte[] tableName, byte[] row,
-        long timestamp) throws IOError {
-      return getRowWithColumnsTs(tableName, row, null,
-                                 timestamp);
+    public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
+                                     long timestamp) throws IOError {
+      return getRowWithColumnsTs(tableName, row, null, timestamp);
     }
 
-    public List<TRowResult> getRowWithColumnsTs(byte[] tableName, byte[] row,
-        List<byte[]> columns, long timestamp) throws IOError {
+    public List<TRowResult> getRowWithColumnsTs(ByteBuffer tableName, ByteBuffer row,
+                                                List<ByteBuffer> columns, long timestamp)
+      throws IOError {
       try {
-        HTable table = getTable(tableName);
+        HTable table = getTable(tableName.array());
         if (columns == null) {
-          Get get = new Get(row);
+          Get get = new Get(row.array());
           get.setTimeRange(Long.MIN_VALUE, timestamp);
           Result result = table.get(get);
           return ThriftUtilities.rowResultFromHBase(result);
         }
-        byte[][] columnArr = columns.toArray(new byte[columns.size()][]);
-        Get get = new Get(row);
-        for(byte [] column : columnArr) {
-          byte [][] famAndQf = KeyValue.parseColumn(column);
+        ByteBuffer[] columnArr = columns.toArray(new ByteBuffer[columns.size()]);
+        Get get = new Get(row.array());
+        for(ByteBuffer column : columnArr) {
+          byte [][] famAndQf = KeyValue.parseColumn(column.array());
           if (famAndQf.length == 1) {
               get.addFamily(famAndQf[0]);
           } else {
-              get.addColumn(famAndQf[0], famAndQf[1]);
+            get.addColumn(famAndQf[0], famAndQf[1]);
           }
         }
         get.setTimeRange(Long.MIN_VALUE, timestamp);
@@ -391,17 +392,17 @@ public class ThriftServer {
       }
     }
 
-    public void deleteAll(byte[] tableName, byte[] row, byte[] column)
-        throws IOError {
+    public void deleteAll(ByteBuffer tableName, ByteBuffer row, ByteBuffer column)
+      throws IOError {
       deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP);
     }
 
-    public void deleteAllTs(byte[] tableName, byte[] row, byte[] column,
-        long timestamp) throws IOError {
+    public void deleteAllTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+                            long timestamp) throws IOError {
       try {
-        HTable table = getTable(tableName);
-        Delete delete  = new Delete(row);
-        byte [][] famAndQf = KeyValue.parseColumn(column);
+        HTable table = getTable(tableName.array());
+        Delete delete  = new Delete(row.array());
+        byte [][] famAndQf = KeyValue.parseColumn(column.array());
         if (famAndQf.length == 1) {
           delete.deleteFamily(famAndQf[0], timestamp);
         } else {
@@ -414,29 +415,29 @@ public class ThriftServer {
       }
     }
 
-    public void deleteAllRow(byte[] tableName, byte[] row) throws IOError {
+    public void deleteAllRow(ByteBuffer tableName, ByteBuffer row) throws IOError {
       deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
     }
 
-    public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp)
-        throws IOError {
+    public void deleteAllRowTs(ByteBuffer tableName, ByteBuffer row, long timestamp)
+      throws IOError {
       try {
-        HTable table = getTable(tableName);
-        Delete delete  = new Delete(row, timestamp, null);
+        HTable table = getTable(tableName.array());
+        Delete delete  = new Delete(row.array(), timestamp, null);
         table.delete(delete);
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public void createTable(byte[] tableName,
-        List<ColumnDescriptor> columnFamilies) throws IOError,
-        IllegalArgument, AlreadyExists {
+    public void createTable(ByteBuffer tableName,
+                            List<ColumnDescriptor> columnFamilies) throws IOError,
+      IllegalArgument, AlreadyExists {
       try {
-        if (admin.tableExists(tableName)) {
+        if (admin.tableExists(tableName.array())) {
           throw new AlreadyExists("table name already in use");
         }
-        HTableDescriptor desc = new HTableDescriptor(tableName);
+        HTableDescriptor desc = new HTableDescriptor(tableName.array());
         for (ColumnDescriptor col : columnFamilies) {
           HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
           desc.addFamily(colDesc);
@@ -449,37 +450,38 @@ public class ThriftServer {
       }
     }
 
-    public void deleteTable(byte[] tableName) throws IOError {
+    public void deleteTable(ByteBuffer tableName) throws IOError {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("deleteTable: table=" + new String(tableName));
+        LOG.debug("deleteTable: table=" + new String(tableName.array()));
       }
       try {
-        if (!admin.tableExists(tableName)) {
+        if (!admin.tableExists(tableName.array())) {
           throw new IOError("table does not exist");
         }
-        admin.deleteTable(tableName);
+        admin.deleteTable(tableName.array());
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
 
-    public void mutateRow(byte[] tableName, byte[] row,
-        List<Mutation> mutations) throws IOError, IllegalArgument {
+    public void mutateRow(ByteBuffer tableName, ByteBuffer row,
+                          List<Mutation> mutations) throws IOError, IllegalArgument {
       mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP);
     }
 
-    public void mutateRowTs(byte[] tableName, byte[] row,
-        List<Mutation> mutations, long timestamp) throws IOError, IllegalArgument {
+    public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
+                            List<Mutation> mutations, long timestamp)
+      throws IOError, IllegalArgument {
       HTable table = null;
       try {
-        table = getTable(tableName);
-        Put put = new Put(row, timestamp, null);
+        table = getTable(tableName.array());
+        Put put = new Put(row.array(), timestamp, null);
 
-        Delete delete = new Delete(row);
+        Delete delete = new Delete(row.array());
 
         // I apologize for all this mess :)
         for (Mutation m : mutations) {
-          byte[][] famAndQf = KeyValue.parseColumn(m.column);
+          byte[][] famAndQf = KeyValue.parseColumn(m.column.array());
           if (m.isDelete) {
             if (famAndQf.length == 1) {
               delete.deleteFamily(famAndQf[0], timestamp);
@@ -488,9 +490,9 @@ public class ThriftServer {
             }
           } else {
             if(famAndQf.length == 1) {
-              put.add(famAndQf[0], new byte[0], m.value);
+              put.add(famAndQf[0], new byte[0], m.value.array());
             } else {
-              put.add(famAndQf[0], famAndQf[1], m.value);
+              put.add(famAndQf[0], famAndQf[1], m.value.array());
             }
           }
         }
@@ -505,23 +507,25 @@ public class ThriftServer {
       }
     }
 
-    public void mutateRows(byte[] tableName, List<BatchMutation> rowBatches)
-        throws IOError, IllegalArgument, TException {
+    public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches)
+      throws IOError, IllegalArgument, TException {
       mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP);
     }
 
-    public void mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp)
-        throws IOError, IllegalArgument, TException {
+    public void mutateRowsTs(ByteBuffer tableName, List<BatchMutation> rowBatches,
+                             long timestamp)
+      throws IOError, IllegalArgument, TException {
       List<Put> puts = new ArrayList<Put>();
       List<Delete> deletes = new ArrayList<Delete>();
 
       for (BatchMutation batch : rowBatches) {
-        byte[] row = batch.row;
+        ByteBuffer row = batch.row;
         List<Mutation> mutations = batch.mutations;
-        Delete delete = new Delete(row);
-        Put put = new Put(row, timestamp, null);
+
+        Delete delete = new Delete(row.array());
+        Put put = new Put(row.array(), timestamp, null);
         for (Mutation m : mutations) {
-          byte[][] famAndQf = KeyValue.parseColumn(m.column);
+          byte[][] famAndQf = KeyValue.parseColumn(m.column.array());
           if (m.isDelete) {
             // no qualifier, family only.
             if (famAndQf.length == 1) {
@@ -531,9 +535,9 @@ public class ThriftServer {
             }
           } else {
             if(famAndQf.length == 1) {
-              put.add(famAndQf[0], new byte[0], m.value);
+              put.add(famAndQf[0], new byte[0], m.value.array());
             } else {
-              put.add(famAndQf[0], famAndQf[1], m.value);
+              put.add(famAndQf[0], famAndQf[1], m.value.array());
             }
           }
         }
@@ -545,7 +549,7 @@ public class ThriftServer {
 
       HTable table = null;
       try {
-        table = getTable(tableName);
+        table = getTable(tableName.array());
         if (!puts.isEmpty())
           table.put(puts);
         for (Delete del : deletes) {
@@ -559,18 +563,19 @@ public class ThriftServer {
     }
 
     @Deprecated
-    public long atomicIncrement(byte[] tableName, byte[] row, byte[] column,
-        long amount) throws IOError, IllegalArgument, TException {
-      byte [][] famAndQf = KeyValue.parseColumn(column);
+    public long atomicIncrement(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
+                                long amount) throws IOError, IllegalArgument, TException {
+      byte [][] famAndQf = KeyValue.parseColumn(column.array());
       if(famAndQf.length == 1) {
-        return atomicIncrement(tableName, row, famAndQf[0], new byte[0],
-            amount);
+        return atomicIncrement(tableName.array(), row.array(), famAndQf[0], new byte[0],
+                               amount);
       }
-      return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
+      return atomicIncrement(tableName.array(), row.array(), famAndQf[0], famAndQf[1],
+                             amount);
     }
 
     public long atomicIncrement(byte [] tableName, byte [] row, byte [] family,
-        byte [] qualifier, long amount)
+                                byte [] qualifier, long amount)
     throws IOError, IllegalArgument, TException {
       HTable table;
       try {
@@ -609,38 +614,41 @@ public class ThriftServer {
         }
         return ThriftUtilities.rowResultFromHBase(results);
     }
+
     public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
         return scannerGetList(id,1);
     }
-    public int scannerOpen(byte[] tableName, byte[] startRow,
-            List<byte[]> columns) throws IOError {
-        try {
-          HTable table = getTable(tableName);
-          Scan scan = new Scan(startRow);
-          if(columns != null && columns.size() != 0) {
-            for(byte [] column : columns) {
-              byte [][] famQf = KeyValue.parseColumn(column);
-              if(famQf.length == 1) {
-                scan.addFamily(famQf[0]);
-              } else {
-                scan.addColumn(famQf[0], famQf[1]);
-              }
+
+    public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
+                           List<ByteBuffer> columns) throws IOError {
+      try {
+        HTable table = getTable(tableName.array());
+        Scan scan = new Scan(startRow.array());
+        if(columns != null && columns.size() != 0) {
+          for(ByteBuffer column : columns) {
+            byte [][] famQf = KeyValue.parseColumn(column.array());
+            if(famQf.length == 1) {
+              scan.addFamily(famQf[0]);
+            } else {
+              scan.addColumn(famQf[0], famQf[1]);
             }
           }
-          return addScanner(table.getScanner(scan));
-        } catch (IOException e) {
-          throw new IOError(e.getMessage());
         }
+        return addScanner(table.getScanner(scan));
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
     }
 
-    public int scannerOpenWithStop(byte[] tableName, byte[] startRow,
-        byte[] stopRow, List<byte[]> columns) throws IOError, TException {
+    public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
+                                   ByteBuffer stopRow, List<ByteBuffer> columns)
+      throws IOError, TException {
       try {
-        HTable table = getTable(tableName);
-        Scan scan = new Scan(startRow, stopRow);
+        HTable table = getTable(tableName.array());
+        Scan scan = new Scan(startRow.array(), stopRow.array());
         if(columns != null && columns.size() != 0) {
-          for(byte [] column : columns) {
-            byte [][] famQf = KeyValue.parseColumn(column);
+          for(ByteBuffer column : columns) {
+            byte [][] famQf = KeyValue.parseColumn(column.array());
             if(famQf.length == 1) {
               scan.addFamily(famQf[0]);
             } else {
@@ -655,16 +663,17 @@ public class ThriftServer {
     }
 
     @Override
-    public int scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws IOError, TException {
+    public int scannerOpenWithPrefix(ByteBuffer tableName, ByteBuffer startAndPrefix,
+                                     List<ByteBuffer> columns) throws IOError, TException {
       try {
-        HTable table = getTable(tableName);
-        Scan scan = new Scan(startAndPrefix);
+        HTable table = getTable(tableName.array());
+        Scan scan = new Scan(startAndPrefix.array());
         Filter f = new WhileMatchFilter(
-            new PrefixFilter(startAndPrefix));
+          new PrefixFilter(startAndPrefix.array()));
         scan.setFilter(f);
         if(columns != null && columns.size() != 0) {
-          for(byte [] column : columns) {
-            byte [][] famQf = KeyValue.parseColumn(column);
+          for(ByteBuffer column : columns) {
+            byte [][] famQf = KeyValue.parseColumn(column.array());
             if(famQf.length == 1) {
               scan.addFamily(famQf[0]);
             } else {
@@ -678,15 +687,16 @@ public class ThriftServer {
       }
     }
 
-    public int scannerOpenTs(byte[] tableName, byte[] startRow,
-        List<byte[]> columns, long timestamp) throws IOError, TException {
+    public int scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow,
+                             List<ByteBuffer> columns, long timestamp)
+      throws IOError, TException {
       try {
-        HTable table = getTable(tableName);
-        Scan scan = new Scan(startRow);
+        HTable table = getTable(tableName.array());
+        Scan scan = new Scan(startRow.array());
         scan.setTimeRange(Long.MIN_VALUE, timestamp);
-        if(columns != null && columns.size() != 0) {
-          for(byte [] column : columns) {
-            byte [][] famQf = KeyValue.parseColumn(column);
+        if(columns != null && !columns.isEmpty()) {
+          for(ByteBuffer column : columns) {
+            byte [][] famQf = KeyValue.parseColumn(column.array());
             if(famQf.length == 1) {
               scan.addFamily(famQf[0]);
             } else {
@@ -700,16 +710,17 @@ public class ThriftServer {
       }
     }
 
-    public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow,
-        byte[] stopRow, List<byte[]> columns, long timestamp)
-        throws IOError, TException {
+    public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
+                                     ByteBuffer stopRow, List<ByteBuffer> columns,
+                                     long timestamp)
+      throws IOError, TException {
       try {
-        HTable table = getTable(tableName);
-        Scan scan = new Scan(startRow, stopRow);
+        HTable table = getTable(tableName.array());
+        Scan scan = new Scan(startRow.array(), stopRow.array());
         scan.setTimeRange(Long.MIN_VALUE, timestamp);
-        if(columns != null && columns.size() != 0) {
-          for(byte [] column : columns) {
-            byte [][] famQf = KeyValue.parseColumn(column);
+        if(columns != null && !columns.isEmpty()) {
+          for(ByteBuffer column : columns) {
+            byte [][] famQf = KeyValue.parseColumn(column.array());
             if(famQf.length == 1) {
               scan.addFamily(famQf[0]);
             } else {
@@ -724,13 +735,13 @@ public class ThriftServer {
       }
     }
 
-    public Map<byte[], ColumnDescriptor> getColumnDescriptors(
-        byte[] tableName) throws IOError, TException {
+    public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(ByteBuffer tableName)
+      throws IOError, TException {
       try {
-        TreeMap<byte[], ColumnDescriptor> columns =
-          new TreeMap<byte[], ColumnDescriptor>(Bytes.BYTES_COMPARATOR);
+        Map<ByteBuffer, ColumnDescriptor> columns =
+          new TreeMap<ByteBuffer, ColumnDescriptor>();
 
-        HTable table = getTable(tableName);
+        HTable table = getTable(tableName.array());
         HTableDescriptor desc = table.getTableDescriptor();
 
         for (HColumnDescriptor e : desc.getFamilies()) {
@@ -758,6 +769,7 @@ public class ThriftServer {
       System.exit(exitCode);
   }
 
+
   private static final String DEFAULT_LISTEN_PORT = "9090";
 
   /*

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java Tue Oct 11 17:42:34 2011
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hbase.thrift;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.TreeMap;
@@ -52,10 +53,10 @@ public class ThriftUtilities {
     StoreFile.BloomType bt =
       BloomType.valueOf(in.bloomFilterType);
 
-    if (in.name == null || in.name.length <= 0) {
+    if (in.name == null || in.name.array().length <= 0) {
       throw new IllegalArgument("column name is empty");
     }
-    byte [] parsedName = KeyValue.parseColumn(in.name)[0];
+    byte [] parsedName = KeyValue.parseColumn(in.name.array())[0];
     HColumnDescriptor col = new HColumnDescriptor(parsedName,
         in.maxVersions, comp.getName(), in.inMemory, in.blockCacheEnabled,
         in.timeToLive, bt.toString());
@@ -72,7 +73,7 @@ public class ThriftUtilities {
    */
   static public ColumnDescriptor colDescFromHbase(HColumnDescriptor in) {
     ColumnDescriptor col = new ColumnDescriptor();
-    col.name = Bytes.add(in.getName(), KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
+    col.name = ByteBuffer.wrap(Bytes.add(in.getName(), KeyValue.COLUMN_FAMILY_DELIM_ARRAY));
     col.maxVersions = in.getMaxVersions();
     col.compression = in.getCompression().toString();
     col.inMemory = in.isInMemory();
@@ -92,7 +93,7 @@ public class ThriftUtilities {
   static public List<TCell> cellFromHBase(KeyValue in) {
     List<TCell> list = new ArrayList<TCell>(1);
     if (in != null) {
-      list.add(new TCell(in.getValue(), in.getTimestamp()));
+      list.add(new TCell(ByteBuffer.wrap(in.getValue()), in.getTimestamp()));
     }
     return list;
   }
@@ -108,7 +109,7 @@ public class ThriftUtilities {
     if (in != null) {
       list = new ArrayList<TCell>(in.length);
       for (int i = 0; i < in.length; i++) {
-        list.add(new TCell(in[i].getValue(), in[i].getTimestamp()));
+        list.add(new TCell(ByteBuffer.wrap(in[i].getValue()), in[i].getTimestamp()));
       }
     } else {
       list = new ArrayList<TCell>(0);
@@ -132,11 +133,12 @@ public class ThriftUtilities {
             continue;
         }
         TRowResult result = new TRowResult();
-        result.row = result_.getRow();
-        result.columns = new TreeMap<byte[], TCell>(Bytes.BYTES_COMPARATOR);
+        result.row = ByteBuffer.wrap(result_.getRow());
+        result.columns = new TreeMap<ByteBuffer, TCell>();
         for(KeyValue kv : result_.sorted()) {
-          result.columns.put(KeyValue.makeColumn(kv.getFamily(),
-              kv.getQualifier()), new TCell(kv.getValue(), kv.getTimestamp()));
+          result.columns.put(ByteBuffer.wrap(KeyValue.makeColumn(kv.getFamily(),
+                                                                 kv.getQualifier())),
+                             new TCell(ByteBuffer.wrap(kv.getValue()), kv.getTimestamp()));
         }
         results.add(result);
     }

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java Tue Oct 11 17:42:34 2011
@@ -1,35 +1,36 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
+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;
+
 import org.apache.thrift.*;
-import org.apache.thrift.meta_data.FieldMetaData;
-import org.apache.thrift.meta_data.FieldValueMetaData;
+import org.apache.thrift.async.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
 import org.apache.thrift.protocol.*;
 
-import java.util.*;
-
 /**
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-public class AlreadyExists extends Exception implements TBase<AlreadyExists._Fields>, java.io.Serializable, Cloneable, Comparable<AlreadyExists> {
+public class AlreadyExists extends Exception implements TBase<AlreadyExists, AlreadyExists._Fields>, java.io.Serializable, Cloneable {
   private static final TStruct STRUCT_DESC = new TStruct("AlreadyExists");
 
   private static final TField MESSAGE_FIELD_DESC = new TField("message", TType.STRING, (short)1);
@@ -40,12 +41,10 @@ public class AlreadyExists extends Excep
   public enum _Fields implements TFieldIdEnum {
     MESSAGE((short)1, "message");
 
-    private static final Map<Integer, _Fields> byId = new HashMap<Integer, _Fields>();
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
     static {
       for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byId.put((int)field._thriftId, field);
         byName.put(field.getFieldName(), field);
       }
     }
@@ -54,7 +53,12 @@ public class AlreadyExists extends Excep
      * Find the _Fields constant that matches fieldId, or null if its not found.
      */
     public static _Fields findByThriftId(int fieldId) {
-      return byId.get(fieldId);
+      switch(fieldId) {
+        case 1: // MESSAGE
+          return MESSAGE;
+        default:
+          return null;
+      }
     }
 
     /**
@@ -93,12 +97,12 @@ public class AlreadyExists extends Excep
 
   // isset id assignments
 
-  public static final Map<_Fields, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new EnumMap<_Fields, FieldMetaData>(_Fields.class) {{
-    put(_Fields.MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT,
-        new FieldValueMetaData(TType.STRING)));
-  }});
-
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
   static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT,
+        new FieldValueMetaData(TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
     FieldMetaData.addStructMetaDataMap(AlreadyExists.class, metaDataMap);
   }
 
@@ -125,9 +129,9 @@ public class AlreadyExists extends Excep
     return new AlreadyExists(this);
   }
 
-  @Deprecated
-  public AlreadyExists clone() {
-    return new AlreadyExists(this);
+  @Override
+  public void clear() {
+    this.message = null;
   }
 
   public String getMessage() {
@@ -167,10 +171,6 @@ public class AlreadyExists extends Excep
     }
   }
 
-  public void setFieldValue(int fieldID, Object value) {
-    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
-  }
-
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case MESSAGE:
@@ -180,12 +180,12 @@ public class AlreadyExists extends Excep
     throw new IllegalStateException();
   }
 
-  public Object getFieldValue(int fieldId) {
-    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
-  }
-
   /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
   public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
     switch (field) {
     case MESSAGE:
       return isSetMessage();
@@ -193,10 +193,6 @@ public class AlreadyExists extends Excep
     throw new IllegalStateException();
   }
 
-  public boolean isSet(int fieldID) {
-    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
-  }
-
   @Override
   public boolean equals(Object that) {
     if (that == null)
@@ -224,14 +220,7 @@ public class AlreadyExists extends Excep
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
-
-    boolean present_message = true && (isSetMessage());
-    builder.append(present_message);
-    if (present_message)
-      builder.append(message);
-
-    return builder.toHashCode();
+    return 0;
   }
 
   public int compareTo(AlreadyExists other) {
@@ -242,17 +231,23 @@ public class AlreadyExists extends Excep
     int lastComparison = 0;
     AlreadyExists typedOther = (AlreadyExists)other;
 
-    lastComparison = Boolean.valueOf(isSetMessage()).compareTo(isSetMessage());
+    lastComparison = Boolean.valueOf(isSetMessage()).compareTo(typedOther.isSetMessage());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(message, typedOther.message);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetMessage()) {
+      lastComparison = TBaseHelper.compareTo(this.message, typedOther.message);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
     return 0;
   }
 
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
   public void read(TProtocol iprot) throws TException {
     TField field;
     iprot.readStructBegin();
@@ -262,21 +257,18 @@ public class AlreadyExists extends Excep
       if (field.type == TType.STOP) {
         break;
       }
-      _Fields fieldId = _Fields.findByThriftId(field.id);
-      if (fieldId == null) {
-        TProtocolUtil.skip(iprot, field.type);
-      } else {
-        switch (fieldId) {
-          case MESSAGE:
-            if (field.type == TType.STRING) {
-              this.message = iprot.readString();
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-        }
-        iprot.readFieldEnd();
+      switch (field.id) {
+        case 1: // MESSAGE
+          if (field.type == TType.STRING) {
+            this.message = iprot.readString();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
       }
+      iprot.readFieldEnd();
     }
     iprot.readStructEnd();
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java Tue Oct 11 17:42:34 2011
@@ -1,23 +1,10 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Map;
@@ -28,24 +15,27 @@ 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;
 
 import org.apache.thrift.*;
+import org.apache.thrift.async.*;
 import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
 import org.apache.thrift.protocol.*;
 
 /**
  * A BatchMutation object is used to apply a number of Mutations to a single row.
  */
-public class BatchMutation implements TBase<BatchMutation._Fields>, java.io.Serializable, Cloneable, Comparable<BatchMutation> {
+public class BatchMutation implements TBase<BatchMutation, BatchMutation._Fields>, java.io.Serializable, Cloneable {
   private static final TStruct STRUCT_DESC = new TStruct("BatchMutation");
 
   private static final TField ROW_FIELD_DESC = new TField("row", TType.STRING, (short)1);
   private static final TField MUTATIONS_FIELD_DESC = new TField("mutations", TType.LIST, (short)2);
 
-  public byte[] row;
+  public ByteBuffer row;
   public List<Mutation> mutations;
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
@@ -53,12 +43,10 @@ public class BatchMutation implements TB
     ROW((short)1, "row"),
     MUTATIONS((short)2, "mutations");
 
-    private static final Map<Integer, _Fields> byId = new HashMap<Integer, _Fields>();
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
     static {
       for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byId.put((int)field._thriftId, field);
         byName.put(field.getFieldName(), field);
       }
     }
@@ -67,7 +55,14 @@ public class BatchMutation implements TB
      * Find the _Fields constant that matches fieldId, or null if its not found.
      */
     public static _Fields findByThriftId(int fieldId) {
-      return byId.get(fieldId);
+      switch(fieldId) {
+        case 1: // ROW
+          return ROW;
+        case 2: // MUTATIONS
+          return MUTATIONS;
+        default:
+          return null;
+      }
     }
 
     /**
@@ -106,15 +101,15 @@ public class BatchMutation implements TB
 
   // isset id assignments
 
-  public static final Map<_Fields, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new EnumMap<_Fields, FieldMetaData>(_Fields.class) {{
-    put(_Fields.ROW, new FieldMetaData("row", TFieldRequirementType.DEFAULT,
-        new FieldValueMetaData(TType.STRING)));
-    put(_Fields.MUTATIONS, new FieldMetaData("mutations", TFieldRequirementType.DEFAULT,
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.ROW, new FieldMetaData("row", TFieldRequirementType.DEFAULT,
+        new FieldValueMetaData(TType.STRING        , "Text")));
+    tmpMap.put(_Fields.MUTATIONS, new FieldMetaData("mutations", TFieldRequirementType.DEFAULT,
         new ListMetaData(TType.LIST,
             new StructMetaData(TType.STRUCT, Mutation.class))));
-  }});
-
-  static {
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
     FieldMetaData.addStructMetaDataMap(BatchMutation.class, metaDataMap);
   }
 
@@ -122,7 +117,7 @@ public class BatchMutation implements TB
   }
 
   public BatchMutation(
-    byte[] row,
+    ByteBuffer row,
     List<Mutation> mutations)
   {
     this();
@@ -150,16 +145,27 @@ public class BatchMutation implements TB
     return new BatchMutation(this);
   }
 
-  @Deprecated
-  public BatchMutation clone() {
-    return new BatchMutation(this);
+  @Override
+  public void clear() {
+    this.row = null;
+    this.mutations = null;
   }
 
   public byte[] getRow() {
-    return this.row;
+    setRow(TBaseHelper.rightSize(row));
+    return row.array();
+  }
+
+  public ByteBuffer BufferForRow() {
+    return row;
   }
 
   public BatchMutation setRow(byte[] row) {
+    setRow(ByteBuffer.wrap(row));
+    return this;
+  }
+
+  public BatchMutation setRow(ByteBuffer row) {
     this.row = row;
     return this;
   }
@@ -224,7 +230,7 @@ public class BatchMutation implements TB
       if (value == null) {
         unsetRow();
       } else {
-        setRow((byte[])value);
+        setRow((ByteBuffer)value);
       }
       break;
 
@@ -239,10 +245,6 @@ public class BatchMutation implements TB
     }
   }
 
-  public void setFieldValue(int fieldID, Object value) {
-    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
-  }
-
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case ROW:
@@ -255,12 +257,12 @@ public class BatchMutation implements TB
     throw new IllegalStateException();
   }
 
-  public Object getFieldValue(int fieldId) {
-    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
-  }
-
   /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
   public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
     switch (field) {
     case ROW:
       return isSetRow();
@@ -270,10 +272,6 @@ public class BatchMutation implements TB
     throw new IllegalStateException();
   }
 
-  public boolean isSet(int fieldID) {
-    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
-  }
-
   @Override
   public boolean equals(Object that) {
     if (that == null)
@@ -292,7 +290,7 @@ public class BatchMutation implements TB
     if (this_present_row || that_present_row) {
       if (!(this_present_row && that_present_row))
         return false;
-      if (!java.util.Arrays.equals(this.row, that.row))
+      if (!this.row.equals(that.row))
         return false;
     }
 
@@ -310,19 +308,7 @@ public class BatchMutation implements TB
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
-
-    boolean present_row = true && (isSetRow());
-    builder.append(present_row);
-    if (present_row)
-      builder.append(row);
-
-    boolean present_mutations = true && (isSetMutations());
-    builder.append(present_mutations);
-    if (present_mutations)
-      builder.append(mutations);
-
-    return builder.toHashCode();
+    return 0;
   }
 
   public int compareTo(BatchMutation other) {
@@ -333,25 +319,33 @@ public class BatchMutation implements TB
     int lastComparison = 0;
     BatchMutation typedOther = (BatchMutation)other;
 
-    lastComparison = Boolean.valueOf(isSetRow()).compareTo(isSetRow());
+    lastComparison = Boolean.valueOf(isSetRow()).compareTo(typedOther.isSetRow());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(row, typedOther.row);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetRow()) {
+      lastComparison = TBaseHelper.compareTo(this.row, typedOther.row);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetMutations()).compareTo(isSetMutations());
+    lastComparison = Boolean.valueOf(isSetMutations()).compareTo(typedOther.isSetMutations());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(mutations, typedOther.mutations);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetMutations()) {
+      lastComparison = TBaseHelper.compareTo(this.mutations, typedOther.mutations);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
     return 0;
   }
 
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
   public void read(TProtocol iprot) throws TException {
     TField field;
     iprot.readStructBegin();
@@ -361,39 +355,36 @@ public class BatchMutation implements TB
       if (field.type == TType.STOP) {
         break;
       }
-      _Fields fieldId = _Fields.findByThriftId(field.id);
-      if (fieldId == null) {
-        TProtocolUtil.skip(iprot, field.type);
-      } else {
-        switch (fieldId) {
-          case ROW:
-            if (field.type == TType.STRING) {
-              this.row = iprot.readBinary();
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case MUTATIONS:
-            if (field.type == TType.LIST) {
+      switch (field.id) {
+        case 1: // ROW
+          if (field.type == TType.STRING) {
+            this.row = iprot.readBinary();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // MUTATIONS
+          if (field.type == TType.LIST) {
+            {
+              TList _list0 = iprot.readListBegin();
+              this.mutations = new ArrayList<Mutation>(_list0.size);
+              for (int _i1 = 0; _i1 < _list0.size; ++_i1)
               {
-                TList _list0 = iprot.readListBegin();
-                this.mutations = new ArrayList<Mutation>(_list0.size);
-                for (int _i1 = 0; _i1 < _list0.size; ++_i1)
-                {
-                  Mutation _elem2;
-                  _elem2 = new Mutation();
-                  _elem2.read(iprot);
-                  this.mutations.add(_elem2);
-                }
-                iprot.readListEnd();
+                Mutation _elem2;
+                _elem2 = new Mutation();
+                _elem2.read(iprot);
+                this.mutations.add(_elem2);
               }
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
+              iprot.readListEnd();
             }
-            break;
-        }
-        iprot.readFieldEnd();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
       }
+      iprot.readFieldEnd();
     }
     iprot.readStructEnd();
 

Modified: hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java?rev=1181928&r1=1181927&r2=1181928&view=diff
==============================================================================
--- hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java (original)
+++ hbase/branches/0.89/src/main/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java Tue Oct 11 17:42:34 2011
@@ -1,23 +1,10 @@
 /**
- * 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
+ * Autogenerated by Thrift
  *
- *     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.
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
  */
 package org.apache.hadoop.hbase.thrift.generated;
 
-import org.apache.commons.lang.builder.HashCodeBuilder;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Map;
@@ -28,12 +15,15 @@ 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;
 
 import org.apache.thrift.*;
+import org.apache.thrift.async.*;
 import org.apache.thrift.meta_data.*;
+import org.apache.thrift.transport.*;
 import org.apache.thrift.protocol.*;
 
 /**
@@ -41,7 +31,7 @@ import org.apache.thrift.protocol.*;
  * such as the number of versions, compression settings, etc. It is
  * used as input when creating a table or adding a column.
  */
-public class ColumnDescriptor implements TBase<ColumnDescriptor._Fields>, java.io.Serializable, Cloneable, Comparable<ColumnDescriptor> {
+public class ColumnDescriptor implements TBase<ColumnDescriptor, ColumnDescriptor._Fields>, java.io.Serializable, Cloneable {
   private static final TStruct STRUCT_DESC = new TStruct("ColumnDescriptor");
 
   private static final TField NAME_FIELD_DESC = new TField("name", TType.STRING, (short)1);
@@ -54,7 +44,7 @@ public class ColumnDescriptor implements
   private static final TField BLOCK_CACHE_ENABLED_FIELD_DESC = new TField("blockCacheEnabled", TType.BOOL, (short)8);
   private static final TField TIME_TO_LIVE_FIELD_DESC = new TField("timeToLive", TType.I32, (short)9);
 
-  public byte[] name;
+  public ByteBuffer name;
   public int maxVersions;
   public String compression;
   public boolean inMemory;
@@ -76,12 +66,10 @@ public class ColumnDescriptor implements
     BLOCK_CACHE_ENABLED((short)8, "blockCacheEnabled"),
     TIME_TO_LIVE((short)9, "timeToLive");
 
-    private static final Map<Integer, _Fields> byId = new HashMap<Integer, _Fields>();
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
     static {
       for (_Fields field : EnumSet.allOf(_Fields.class)) {
-        byId.put((int)field._thriftId, field);
         byName.put(field.getFieldName(), field);
       }
     }
@@ -90,7 +78,28 @@ public class ColumnDescriptor implements
      * Find the _Fields constant that matches fieldId, or null if its not found.
      */
     public static _Fields findByThriftId(int fieldId) {
-      return byId.get(fieldId);
+      switch(fieldId) {
+        case 1: // NAME
+          return NAME;
+        case 2: // MAX_VERSIONS
+          return MAX_VERSIONS;
+        case 3: // COMPRESSION
+          return COMPRESSION;
+        case 4: // IN_MEMORY
+          return IN_MEMORY;
+        case 5: // BLOOM_FILTER_TYPE
+          return BLOOM_FILTER_TYPE;
+        case 6: // BLOOM_FILTER_VECTOR_SIZE
+          return BLOOM_FILTER_VECTOR_SIZE;
+        case 7: // BLOOM_FILTER_NB_HASHES
+          return BLOOM_FILTER_NB_HASHES;
+        case 8: // BLOCK_CACHE_ENABLED
+          return BLOCK_CACHE_ENABLED;
+        case 9: // TIME_TO_LIVE
+          return TIME_TO_LIVE;
+        default:
+          return null;
+      }
     }
 
     /**
@@ -136,28 +145,28 @@ public class ColumnDescriptor implements
   private static final int __TIMETOLIVE_ISSET_ID = 5;
   private BitSet __isset_bit_vector = new BitSet(6);
 
-  public static final Map<_Fields, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new EnumMap<_Fields, FieldMetaData>(_Fields.class) {{
-    put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT,
-        new FieldValueMetaData(TType.STRING)));
-    put(_Fields.MAX_VERSIONS, new FieldMetaData("maxVersions", TFieldRequirementType.DEFAULT,
+  public static final Map<_Fields, FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, FieldMetaData> tmpMap = new EnumMap<_Fields, FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT,
+        new FieldValueMetaData(TType.STRING        , "Text")));
+    tmpMap.put(_Fields.MAX_VERSIONS, new FieldMetaData("maxVersions", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.I32)));
-    put(_Fields.COMPRESSION, new FieldMetaData("compression", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.COMPRESSION, new FieldMetaData("compression", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.STRING)));
-    put(_Fields.IN_MEMORY, new FieldMetaData("inMemory", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.IN_MEMORY, new FieldMetaData("inMemory", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.BOOL)));
-    put(_Fields.BLOOM_FILTER_TYPE, new FieldMetaData("bloomFilterType", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.BLOOM_FILTER_TYPE, new FieldMetaData("bloomFilterType", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.STRING)));
-    put(_Fields.BLOOM_FILTER_VECTOR_SIZE, new FieldMetaData("bloomFilterVectorSize", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.BLOOM_FILTER_VECTOR_SIZE, new FieldMetaData("bloomFilterVectorSize", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.I32)));
-    put(_Fields.BLOOM_FILTER_NB_HASHES, new FieldMetaData("bloomFilterNbHashes", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.BLOOM_FILTER_NB_HASHES, new FieldMetaData("bloomFilterNbHashes", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.I32)));
-    put(_Fields.BLOCK_CACHE_ENABLED, new FieldMetaData("blockCacheEnabled", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.BLOCK_CACHE_ENABLED, new FieldMetaData("blockCacheEnabled", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.BOOL)));
-    put(_Fields.TIME_TO_LIVE, new FieldMetaData("timeToLive", TFieldRequirementType.DEFAULT,
+    tmpMap.put(_Fields.TIME_TO_LIVE, new FieldMetaData("timeToLive", TFieldRequirementType.DEFAULT,
         new FieldValueMetaData(TType.I32)));
-  }});
-
-  static {
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
     FieldMetaData.addStructMetaDataMap(ColumnDescriptor.class, metaDataMap);
   }
 
@@ -181,7 +190,7 @@ public class ColumnDescriptor implements
   }
 
   public ColumnDescriptor(
-    byte[] name,
+    ByteBuffer name,
     int maxVersions,
     String compression,
     boolean inMemory,
@@ -236,16 +245,42 @@ public class ColumnDescriptor implements
     return new ColumnDescriptor(this);
   }
 
-  @Deprecated
-  public ColumnDescriptor clone() {
-    return new ColumnDescriptor(this);
+  @Override
+  public void clear() {
+    this.name = null;
+    this.maxVersions = 3;
+
+    this.compression = "NONE";
+
+    this.inMemory = false;
+
+    this.bloomFilterType = "NONE";
+
+    this.bloomFilterVectorSize = 0;
+
+    this.bloomFilterNbHashes = 0;
+
+    this.blockCacheEnabled = false;
+
+    this.timeToLive = -1;
+
   }
 
   public byte[] getName() {
-    return this.name;
+    setName(TBaseHelper.rightSize(name));
+    return name.array();
+  }
+
+  public ByteBuffer BufferForName() {
+    return name;
   }
 
   public ColumnDescriptor setName(byte[] name) {
+    setName(ByteBuffer.wrap(name));
+    return this;
+  }
+
+  public ColumnDescriptor setName(ByteBuffer name) {
     this.name = name;
     return this;
   }
@@ -457,7 +492,7 @@ public class ColumnDescriptor implements
       if (value == null) {
         unsetName();
       } else {
-        setName((byte[])value);
+        setName((ByteBuffer)value);
       }
       break;
 
@@ -528,10 +563,6 @@ public class ColumnDescriptor implements
     }
   }
 
-  public void setFieldValue(int fieldID, Object value) {
-    setFieldValue(_Fields.findByThriftIdOrThrow(fieldID), value);
-  }
-
   public Object getFieldValue(_Fields field) {
     switch (field) {
     case NAME:
@@ -565,12 +596,12 @@ public class ColumnDescriptor implements
     throw new IllegalStateException();
   }
 
-  public Object getFieldValue(int fieldId) {
-    return getFieldValue(_Fields.findByThriftIdOrThrow(fieldId));
-  }
-
   /** Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise */
   public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
     switch (field) {
     case NAME:
       return isSetName();
@@ -594,10 +625,6 @@ public class ColumnDescriptor implements
     throw new IllegalStateException();
   }
 
-  public boolean isSet(int fieldID) {
-    return isSet(_Fields.findByThriftIdOrThrow(fieldID));
-  }
-
   @Override
   public boolean equals(Object that) {
     if (that == null)
@@ -616,7 +643,7 @@ public class ColumnDescriptor implements
     if (this_present_name || that_present_name) {
       if (!(this_present_name && that_present_name))
         return false;
-      if (!java.util.Arrays.equals(this.name, that.name))
+      if (!this.name.equals(that.name))
         return false;
     }
 
@@ -697,54 +724,7 @@ public class ColumnDescriptor implements
 
   @Override
   public int hashCode() {
-    HashCodeBuilder builder = new HashCodeBuilder();
-
-    boolean present_name = true && (isSetName());
-    builder.append(present_name);
-    if (present_name)
-      builder.append(name);
-
-    boolean present_maxVersions = true;
-    builder.append(present_maxVersions);
-    if (present_maxVersions)
-      builder.append(maxVersions);
-
-    boolean present_compression = true && (isSetCompression());
-    builder.append(present_compression);
-    if (present_compression)
-      builder.append(compression);
-
-    boolean present_inMemory = true;
-    builder.append(present_inMemory);
-    if (present_inMemory)
-      builder.append(inMemory);
-
-    boolean present_bloomFilterType = true && (isSetBloomFilterType());
-    builder.append(present_bloomFilterType);
-    if (present_bloomFilterType)
-      builder.append(bloomFilterType);
-
-    boolean present_bloomFilterVectorSize = true;
-    builder.append(present_bloomFilterVectorSize);
-    if (present_bloomFilterVectorSize)
-      builder.append(bloomFilterVectorSize);
-
-    boolean present_bloomFilterNbHashes = true;
-    builder.append(present_bloomFilterNbHashes);
-    if (present_bloomFilterNbHashes)
-      builder.append(bloomFilterNbHashes);
-
-    boolean present_blockCacheEnabled = true;
-    builder.append(present_blockCacheEnabled);
-    if (present_blockCacheEnabled)
-      builder.append(blockCacheEnabled);
-
-    boolean present_timeToLive = true;
-    builder.append(present_timeToLive);
-    if (present_timeToLive)
-      builder.append(timeToLive);
-
-    return builder.toHashCode();
+    return 0;
   }
 
   public int compareTo(ColumnDescriptor other) {
@@ -755,81 +735,103 @@ public class ColumnDescriptor implements
     int lastComparison = 0;
     ColumnDescriptor typedOther = (ColumnDescriptor)other;
 
-    lastComparison = Boolean.valueOf(isSetName()).compareTo(isSetName());
+    lastComparison = Boolean.valueOf(isSetName()).compareTo(typedOther.isSetName());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(name, typedOther.name);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetName()) {
+      lastComparison = TBaseHelper.compareTo(this.name, typedOther.name);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(isSetMaxVersions());
+    lastComparison = Boolean.valueOf(isSetMaxVersions()).compareTo(typedOther.isSetMaxVersions());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(maxVersions, typedOther.maxVersions);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetMaxVersions()) {
+      lastComparison = TBaseHelper.compareTo(this.maxVersions, typedOther.maxVersions);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetCompression()).compareTo(isSetCompression());
+    lastComparison = Boolean.valueOf(isSetCompression()).compareTo(typedOther.isSetCompression());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(compression, typedOther.compression);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetCompression()) {
+      lastComparison = TBaseHelper.compareTo(this.compression, typedOther.compression);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetInMemory()).compareTo(isSetInMemory());
+    lastComparison = Boolean.valueOf(isSetInMemory()).compareTo(typedOther.isSetInMemory());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(inMemory, typedOther.inMemory);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetInMemory()) {
+      lastComparison = TBaseHelper.compareTo(this.inMemory, typedOther.inMemory);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetBloomFilterType()).compareTo(isSetBloomFilterType());
+    lastComparison = Boolean.valueOf(isSetBloomFilterType()).compareTo(typedOther.isSetBloomFilterType());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(bloomFilterType, typedOther.bloomFilterType);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetBloomFilterType()) {
+      lastComparison = TBaseHelper.compareTo(this.bloomFilterType, typedOther.bloomFilterType);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetBloomFilterVectorSize()).compareTo(isSetBloomFilterVectorSize());
+    lastComparison = Boolean.valueOf(isSetBloomFilterVectorSize()).compareTo(typedOther.isSetBloomFilterVectorSize());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(bloomFilterVectorSize, typedOther.bloomFilterVectorSize);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetBloomFilterVectorSize()) {
+      lastComparison = TBaseHelper.compareTo(this.bloomFilterVectorSize, typedOther.bloomFilterVectorSize);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetBloomFilterNbHashes()).compareTo(isSetBloomFilterNbHashes());
+    lastComparison = Boolean.valueOf(isSetBloomFilterNbHashes()).compareTo(typedOther.isSetBloomFilterNbHashes());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(bloomFilterNbHashes, typedOther.bloomFilterNbHashes);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetBloomFilterNbHashes()) {
+      lastComparison = TBaseHelper.compareTo(this.bloomFilterNbHashes, typedOther.bloomFilterNbHashes);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetBlockCacheEnabled()).compareTo(isSetBlockCacheEnabled());
+    lastComparison = Boolean.valueOf(isSetBlockCacheEnabled()).compareTo(typedOther.isSetBlockCacheEnabled());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(blockCacheEnabled, typedOther.blockCacheEnabled);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetBlockCacheEnabled()) {
+      lastComparison = TBaseHelper.compareTo(this.blockCacheEnabled, typedOther.blockCacheEnabled);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
-    lastComparison = Boolean.valueOf(isSetTimeToLive()).compareTo(isSetTimeToLive());
+    lastComparison = Boolean.valueOf(isSetTimeToLive()).compareTo(typedOther.isSetTimeToLive());
     if (lastComparison != 0) {
       return lastComparison;
     }
-    lastComparison = TBaseHelper.compareTo(timeToLive, typedOther.timeToLive);
-    if (lastComparison != 0) {
-      return lastComparison;
+    if (isSetTimeToLive()) {
+      lastComparison = TBaseHelper.compareTo(this.timeToLive, typedOther.timeToLive);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
     }
     return 0;
   }
 
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
   public void read(TProtocol iprot) throws TException {
     TField field;
     iprot.readStructBegin();
@@ -839,83 +841,80 @@ public class ColumnDescriptor implements
       if (field.type == TType.STOP) {
         break;
       }
-      _Fields fieldId = _Fields.findByThriftId(field.id);
-      if (fieldId == null) {
-        TProtocolUtil.skip(iprot, field.type);
-      } else {
-        switch (fieldId) {
-          case NAME:
-            if (field.type == TType.STRING) {
-              this.name = iprot.readBinary();
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case MAX_VERSIONS:
-            if (field.type == TType.I32) {
-              this.maxVersions = iprot.readI32();
-              setMaxVersionsIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case COMPRESSION:
-            if (field.type == TType.STRING) {
-              this.compression = iprot.readString();
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case IN_MEMORY:
-            if (field.type == TType.BOOL) {
-              this.inMemory = iprot.readBool();
-              setInMemoryIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case BLOOM_FILTER_TYPE:
-            if (field.type == TType.STRING) {
-              this.bloomFilterType = iprot.readString();
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case BLOOM_FILTER_VECTOR_SIZE:
-            if (field.type == TType.I32) {
-              this.bloomFilterVectorSize = iprot.readI32();
-              setBloomFilterVectorSizeIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case BLOOM_FILTER_NB_HASHES:
-            if (field.type == TType.I32) {
-              this.bloomFilterNbHashes = iprot.readI32();
-              setBloomFilterNbHashesIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case BLOCK_CACHE_ENABLED:
-            if (field.type == TType.BOOL) {
-              this.blockCacheEnabled = iprot.readBool();
-              setBlockCacheEnabledIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-          case TIME_TO_LIVE:
-            if (field.type == TType.I32) {
-              this.timeToLive = iprot.readI32();
-              setTimeToLiveIsSet(true);
-            } else {
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
-        }
-        iprot.readFieldEnd();
+      switch (field.id) {
+        case 1: // NAME
+          if (field.type == TType.STRING) {
+            this.name = iprot.readBinary();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 2: // MAX_VERSIONS
+          if (field.type == TType.I32) {
+            this.maxVersions = iprot.readI32();
+            setMaxVersionsIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 3: // COMPRESSION
+          if (field.type == TType.STRING) {
+            this.compression = iprot.readString();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 4: // IN_MEMORY
+          if (field.type == TType.BOOL) {
+            this.inMemory = iprot.readBool();
+            setInMemoryIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 5: // BLOOM_FILTER_TYPE
+          if (field.type == TType.STRING) {
+            this.bloomFilterType = iprot.readString();
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 6: // BLOOM_FILTER_VECTOR_SIZE
+          if (field.type == TType.I32) {
+            this.bloomFilterVectorSize = iprot.readI32();
+            setBloomFilterVectorSizeIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 7: // BLOOM_FILTER_NB_HASHES
+          if (field.type == TType.I32) {
+            this.bloomFilterNbHashes = iprot.readI32();
+            setBloomFilterNbHashesIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 8: // BLOCK_CACHE_ENABLED
+          if (field.type == TType.BOOL) {
+            this.blockCacheEnabled = iprot.readBool();
+            setBlockCacheEnabledIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        case 9: // TIME_TO_LIVE
+          if (field.type == TType.I32) {
+            this.timeToLive = iprot.readI32();
+            setTimeToLiveIsSet(true);
+          } else {
+            TProtocolUtil.skip(iprot, field.type);
+          }
+          break;
+        default:
+          TProtocolUtil.skip(iprot, field.type);
       }
+      iprot.readFieldEnd();
     }
     iprot.readStructEnd();