You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2009/05/05 09:26:51 UTC

svn commit: r771604 [1/3] - in /hadoop/hbase/trunk: ./ lib/ src/java/org/apache/hadoop/hbase/thrift/ src/java/org/apache/hadoop/hbase/thrift/generated/ src/test/org/apache/hadoop/hbase/thrift/

Author: apurtell
Date: Tue May  5 07:26:49 2009
New Revision: 771604

URL: http://svn.apache.org/viewvc?rev=771604&view=rev
Log:
HBASE-1367 Get rid of Thrift exception 'NotFound'

Added:
    hadoop/hbase/trunk/lib/libthrift-r771587.jar   (with props)
Removed:
    hadoop/hbase/trunk/lib/libthrift-20080411p1.jar
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Constants.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/NotFound.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/RegionDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ScanEntry.java
Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/package.html
    hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/thrift/DisabledTestThriftServer.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Tue May  5 07:26:49 2009
@@ -16,6 +16,7 @@
                (Ryan Rawson and Stack)
    HBASE-1342  Add to filesystem info needed to rebuild .META.
    HBASE-1361  Disable bloom filters
+   HBASE-1367  Get rid of Thrift exception 'NotFound'
 
   BUG FIXES
    HBASE-1140  "ant clean test" fails (Nitay Joffe via Stack)

Added: hadoop/hbase/trunk/lib/libthrift-r771587.jar
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/lib/libthrift-r771587.jar?rev=771604&view=auto
==============================================================================
Binary file - no diff available.

Propchange: hadoop/hbase/trunk/lib/libthrift-r771587.jar
------------------------------------------------------------------------------
    svn:mime-type = application/octet-stream

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/Hbase.thrift Tue May  5 07:26:49 2009
@@ -141,14 +141,6 @@
 }
 
 /**
- * A NotFound exception is used to indicate that no value was found
- * for a query, or that a scanner has reached it's end.
- */
-exception NotFound {
-  1:string message
-}
-
-/**
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
@@ -230,22 +222,23 @@
   /**
    * Deletes a table
    * @param tableName name of table to delete
-   * @throws NotFound if table doesn't exist on server
+   * @throws IOError if table doesn't exist on server or there was some other
+   * problem
    */
   void deleteTable(1:Text tableName)
-    throws (1:IOError io, 2:NotFound nf)
+    throws (1:IOError io)
 
   /** 
    * Get a single TCell for the specified table, row, and column at the
-   * latest timestamp.
+   * latest timestamp. Returns an empty list if no such value exists.
    *
    * @param tableName name of table
    * @param row row key
    * @param column column name
    * @return value for specified row/column
    */
-  TCell get(1:Text tableName, 2:Text row, 3:Text column) 
-    throws (1:IOError io, 2:NotFound nf)
+  list<TCell> get(1:Text tableName, 2:Text row, 3:Text column) 
+    throws (1:IOError io)
 
   /** 
    * Get the specified number of versions for the specified table,
@@ -257,8 +250,9 @@
    * @param numVersions number of versions to retrieve
    * @return list of cells for specified row/column
    */
-  list<TCell> getVer(1:Text tableName, 2:Text row, 3:Text column, 4:i32 numVersions) 
-    throws (1:IOError io, 2:NotFound nf)
+  list<TCell> getVer(1:Text tableName, 2:Text row, 3:Text column,
+      4:i32 numVersions) 
+    throws (1:IOError io)
 
   /** 
    * Get the specified number of versions for the specified table,
@@ -272,59 +266,58 @@
    * @param numVersions number of versions to retrieve
    * @return list of cells for specified row/column
    */
-  list<TCell> getVerTs(1:Text tableName, 2:Text row, 3:Text column, 4:i64 timestamp,  5:i32 numVersions)
-    throws (1:IOError io, 2:NotFound nf)
+  list<TCell> getVerTs(1:Text tableName, 2:Text row, 3:Text column, 
+      4:i64 timestamp,  5:i32 numVersions)
+    throws (1:IOError io)
 
   /** 
    * Get all the data for the specified table and row at the latest
-   * timestamp.
+   * timestamp. Returns an empty list if the row does not exist.
    * 
    * @param tableName name of table
    * @param row row key
    * @return TRowResult containing the row and map of columns to TCells
-   * @throws NotFound if the row does not exist
    */
-  TRowResult getRow(1:Text tableName, 2:Text row)
-    throws (1:IOError io, 2:NotFound nf)
+  list<TRowResult> getRow(1:Text tableName, 2:Text row)
+    throws (1:IOError io)
 
   /** 
    * Get the specified columns for the specified table and row at the latest
-   * timestamp.
+   * timestamp. Returns an empty list if the row does not exist.
    * 
    * @param tableName name of table
    * @param row row key
    * @param columns List of columns to return, null for all columns
    * @return TRowResult containing the row and map of columns to TCells
-   * @throws NotFound if the row does not exist
    */
-  TRowResult getRowWithColumns(1:Text tableName, 2:Text row, 3:list<Text> columns)
-    throws (1:IOError io, 2:NotFound nf)
+  list<TRowResult> getRowWithColumns(1:Text tableName, 2:Text row,
+      3:list<Text> columns)
+    throws (1:IOError io)
 
   /** 
    * Get all the data for the specified table and row at the specified
-   * timestamp.
+   * timestamp. Returns an empty list if the row does not exist.
    * 
    * @param tableName of table
    * @param row row key
    * @param timestamp timestamp
    * @return TRowResult containing the row and map of columns to TCells
-   * @throws NotFound if the row does not exist
    */
-  TRowResult getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
-    throws (1:IOError io, 2:NotFound nf)
+  list<TRowResult> getRowTs(1:Text tableName, 2:Text row, 3:i64 timestamp)
+    throws (1:IOError io)
     
   /** 
    * Get the specified columns for the specified table and row at the specified
-   * timestamp.
+   * timestamp. Returns an empty list if the row does not exist.
    * 
    * @param tableName name of table
    * @param row row key
    * @param columns List of columns to return, null for all columns
    * @return TRowResult containing the row and map of columns to TCells
-   * @throws NotFound if the row does not exist
    */
-  TRowResult getRowWithColumnsTs(1:Text tableName, 2:Text row, 3:list<Text> columns, 4:i64 timestamp)
-    throws (1:IOError io, 2:NotFound nf)
+  list<TRowResult> getRowWithColumnsTs(1:Text tableName, 2:Text row,
+      3:list<Text> columns, 4:i64 timestamp)
+    throws (1:IOError io)
 
   /** 
    * Apply a series of mutations (updates/deletes) to a row in a
@@ -520,15 +513,15 @@
    * Returns the scanner's current row value and advances to the next
    * row in the table.  When there are no more rows in the table, or a key
    * greater-than-or-equal-to the scanner's specified stopRow is reached,
-   * a NotFound exception is returned.
+   * an empty list is returned.
    *
    * @param id id of a scanner returned by scannerOpen
    * @return a TRowResult containing the current row and a map of the columns to TCells.
    * @throws IllegalArgument if ScannerID is invalid
    * @throws NotFound when the scanner reaches the end
    */
-  TRowResult scannerGet(1:ScannerID id)
-    throws (1:IOError io, 2:IllegalArgument ia, 3:NotFound nf)
+  list<TRowResult> scannerGet(1:ScannerID id)
+    throws (1:IOError io, 2:IllegalArgument ia)
 
   /**
    * Closes the server-state associated with an open scanner.

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftServer.java Tue May  5 07:26:49 2009
@@ -47,19 +47,18 @@
 import org.apache.hadoop.hbase.thrift.generated.IOError;
 import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
 import org.apache.hadoop.hbase.thrift.generated.Mutation;
-import org.apache.hadoop.hbase.thrift.generated.NotFound;
 import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
 import org.apache.hadoop.hbase.thrift.generated.TCell;
 import org.apache.hadoop.hbase.thrift.generated.TRowResult;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import com.facebook.thrift.TException;
-import com.facebook.thrift.protocol.TBinaryProtocol;
-import com.facebook.thrift.protocol.TProtocolFactory;
-import com.facebook.thrift.server.TServer;
-import com.facebook.thrift.server.TThreadPoolServer;
-import com.facebook.thrift.transport.TServerSocket;
-import com.facebook.thrift.transport.TServerTransport;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.apache.thrift.protocol.TProtocolFactory;
+import org.apache.thrift.server.THsHaServer;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.transport.TNonblockingServerSocket;
+import org.apache.thrift.transport.TNonblockingServerTransport;
 
 /**
  * ThriftServer - this class starts up a Thrift server which implements the
@@ -179,7 +178,7 @@
       }
     }
     
-    public void compact(byte[] tableNameOrRegionName) throws IOError, TException {
+    public void compact(byte[] tableNameOrRegionName) throws IOError {
       try{
         admin.compact(tableNameOrRegionName);
       } catch (IOException e) {
@@ -187,7 +186,7 @@
       }
     }
 
-    public void majorCompact(byte[] tableNameOrRegionName) throws IOError, TException {
+    public void majorCompact(byte[] tableNameOrRegionName) throws IOError {
       try{
         admin.majorCompact(tableNameOrRegionName);
       } catch (IOException e) {
@@ -230,14 +229,11 @@
       }
     }
     
-    public TCell get(byte[] tableName, byte[] row, byte[] column)
-        throws NotFound, IOError {
+    public List<TCell> get(byte[] tableName, byte[] row, byte[] column)
+        throws IOError {
       try {
         HTable table = getTable(tableName);
         Cell cell = table.get(row, column);
-        if (cell == null) {
-          throw new NotFound();
-        }
         return ThriftUtilities.cellFromHBase(cell);
       } catch (IOException e) {
         throw new IOError(e.getMessage());
@@ -245,63 +241,48 @@
     }
     
     public List<TCell> getVer(byte[] tableName, byte[] row,
-        byte[] column, int numVersions) throws IOError, NotFound {
+        byte[] column, int numVersions) throws IOError {
       try {
         HTable table = getTable(tableName);
         Cell[] cells = 
           table.get(row, column, numVersions);
-        if (cells == null) {
-          throw new NotFound();
-        }
-        List<TCell> list = new ArrayList<TCell>();
-        for (int i = 0; i < cells.length; i++) {
-          list.add(ThriftUtilities.cellFromHBase(cells[i]));
-        }
-        return list;
+        return ThriftUtilities.cellFromHBase(cells);
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
     
     public List<TCell> getVerTs(byte[] tableName, byte[] row,
-        byte[] column, long timestamp, int numVersions) throws IOError,
-        NotFound {
+        byte[] column, long timestamp, int numVersions) throws IOError {
       try {
         HTable table = getTable(tableName);
         Cell[] cells = table.get(row, column, timestamp, numVersions);
-        if (cells == null) {
-          throw new NotFound();
-        }
-        List<TCell> list = new ArrayList<TCell>();
-        for (int i = 0; i < cells.length; i++) {
-          list.add(ThriftUtilities.cellFromHBase(cells[i]));
-        }
-        return list;
+        return ThriftUtilities.cellFromHBase(cells);
       } catch (IOException e) {
         throw new IOError(e.getMessage());
       }
     }
     
-    public TRowResult getRow(byte[] tableName, byte[] row)
-        throws IOError, NotFound {
+    public List<TRowResult> getRow(byte[] tableName, byte[] row)
+        throws IOError {
       return getRowWithColumnsTs(tableName, row, null,
                                  HConstants.LATEST_TIMESTAMP);
     }
     
-    public TRowResult getRowWithColumns(byte[] tableName, byte[] row,
-        List<byte[]> columns) throws IOError, NotFound {
+    public List<TRowResult> getRowWithColumns(byte[] tableName, byte[] row,
+        List<byte[]> columns) throws IOError {
       return getRowWithColumnsTs(tableName, row, columns,
                                  HConstants.LATEST_TIMESTAMP);
     }
     
-    public TRowResult getRowTs(byte[] tableName, byte[] row,
-        long timestamp) throws IOError, NotFound {
+    public List<TRowResult> getRowTs(byte[] tableName, byte[] row,
+        long timestamp) throws IOError {
       return getRowWithColumnsTs(tableName, row, null,
                                  timestamp);
     }
     
-    public TRowResult getRowWithColumnsTs(byte[] tableName, byte[] row,
-        List<byte[]> columns, long timestamp) throws IOError, NotFound {
+    public List<TRowResult> getRowWithColumnsTs(byte[] tableName, byte[] row,
+        List<byte[]> columns, long timestamp) throws IOError {
       try {
         HTable table = getTable(tableName);
         if (columns == null) {
@@ -365,13 +346,13 @@
       }
     }
     
-    public void deleteTable(byte[] tableName) throws IOError, NotFound {
+    public void deleteTable(byte[] tableName) throws IOError {
       if (LOG.isDebugEnabled()) {
         LOG.debug("deleteTable: table=" + new String(tableName));
       }
       try {
         if (!admin.tableExists(tableName)) {
-          throw new NotFound();
+          throw new IOError("table does not exist");
         }
         admin.deleteTable(tableName);
       } catch (IOException e) {
@@ -459,7 +440,7 @@
       removeScanner(id);
     }
     
-    public TRowResult scannerGet(int id) throws IllegalArgument, NotFound,
+    public List<TRowResult> scannerGet(int id) throws IllegalArgument,
         IOError {
       LOG.debug("scannerGet: id=" + id);
       Scanner scanner = getScanner(id);
@@ -472,7 +453,7 @@
       try {
         results = scanner.next();
         if (results == null) {
-          throw new NotFound("end of scanner reached");
+          return new ArrayList<TRowResult>();
         }
       } catch (IOException e) {
         throw new IOError(e.getMessage());
@@ -632,9 +613,10 @@
       Integer.toString(port));
     HBaseHandler handler = new HBaseHandler();
     Hbase.Processor processor = new Hbase.Processor(handler);
-    TServerTransport serverTransport = new TServerSocket(port);
-    TProtocolFactory protFactory = new TBinaryProtocol.Factory(true, true);
-    TServer server = new TThreadPoolServer(processor, serverTransport,
+    TNonblockingServerTransport serverTransport = 
+      new TNonblockingServerSocket(port);
+    TProtocolFactory protFactory = new TCompactProtocol.Factory();
+    TServer server = new THsHaServer(processor, serverTransport,
       protFactory);
     server.serve();
   }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/ThriftUtilities.java Tue May  5 07:26:49 2009
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.thrift;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
@@ -27,7 +29,6 @@
 import org.apache.hadoop.hbase.io.hfile.Compression;
 import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
 import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
-import org.apache.hadoop.hbase.thrift.generated.NotFound;
 import org.apache.hadoop.hbase.thrift.generated.TCell;
 import org.apache.hadoop.hbase.thrift.generated.TRowResult;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -82,38 +83,61 @@
   }
   
   /**
-   * This utility method creates a new Thrift TCell "struct" based on
-   * an Hbase Cell object.
+   * This utility method creates a list of Thrift TCell "struct" based on
+   * an Hbase Cell object. The empty list is returned if the input is null.
    * 
    * @param in
    *          Hbase Cell object
-   * @return Thrift TCell
+   * @return Thrift TCell array
    */
-  static public TCell cellFromHBase(Cell in) {
-    return new TCell(in.getValue(), in.getTimestamp());
+  static public List<TCell> cellFromHBase(Cell in) {
+    List<TCell> list = new ArrayList<TCell>(1);
+    if (in != null) {
+      list.add(new TCell(in.getValue(), in.getTimestamp()));
+    }
+    return list;
   }
-  
+
+  /**
+   * This utility method creates a list of Thrift TCell "struct" based on
+   * an Hbase Cell array. The empty list is returned if the input is null.
+   * @param in Hbase Cell array
+   * @return Thrift TCell array
+   */
+  static public List<TCell> cellFromHBase(Cell[] in) {
+    List<TCell> list = new ArrayList<TCell>(in.length);
+    if (in != null) {
+      for (int i = 0; i < in.length; i++) {
+        list.add(new TCell(in[i].getValue(), in[i].getTimestamp()));
+      }
+    }
+    return list;
+  }
+
   /**
-   * This utility method creates a new Thrift TRowResult "struct" based on
-   * an Hbase RowResult object.
+   * This utility method creates a list of Thrift TRowResult "struct" based on
+   * an Hbase RowResult object. The empty list is returned if the input is
+   * null.
    * 
    * @param in
    *          Hbase RowResult object
-   * @return Thrift TRowResult
-   * @throws NotFound
+   * @return Thrift TRowResult array
    */
-  static public TRowResult rowResultFromHBase(RowResult in)
-      throws NotFound {
+  static public List<TRowResult> rowResultFromHBase(RowResult in) {
+    List<TRowResult> list = new ArrayList<TRowResult>();
     if(in == null) {
-      throw new NotFound();
+      return list;
     }
     TRowResult result = new TRowResult();
     result.row = in.getRow();
     result.columns = new TreeMap<byte[], TCell>(Bytes.BYTES_COMPARATOR);
     for (Map.Entry<byte[], Cell> entry : in.entrySet()){
-      result.columns.put(entry.getKey(), ThriftUtilities.cellFromHBase(entry.getValue()));
+      Cell cell = entry.getValue();
+      result.columns.put(entry.getKey(), 
+          new TCell(cell.getValue(), cell.getTimestamp()));
     }
-    return result;
+    list.add(result);
+    return list;
   }
 }
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/AlreadyExists.java Tue May  5 07:26:49 2009
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 /**
  * Autogenerated by Thrift
  *
@@ -29,21 +28,34 @@
 import java.util.HashMap;
 import java.util.Set;
 import java.util.HashSet;
-import com.facebook.thrift.*;
+import java.util.Collections;
 
-import com.facebook.thrift.protocol.*;
-import com.facebook.thrift.transport.*;
+import org.apache.thrift.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.protocol.*;
 
 /**
  * An AlreadyExists exceptions signals that a table with the specified
  * name already exists
  */
-public class AlreadyExists extends Exception implements TBase, java.io.Serializable {
+public class AlreadyExists extends Exception implements TBase, 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);
+
   public String message;
+  public static final int MESSAGE = 1;
+
+  private final Isset __isset = new Isset();
+  private static final class Isset implements java.io.Serializable {
+  }
+
+  public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
+    put(MESSAGE, new FieldMetaData("message", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+  }});
 
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean message = false;
+  static {
+    FieldMetaData.addStructMetaDataMap(AlreadyExists.class, metaDataMap);
   }
 
   public AlreadyExists() {
@@ -54,9 +66,81 @@
   {
     this();
     this.message = message;
-    this.__isset.message = true;
   }
 
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public AlreadyExists(AlreadyExists other) {
+    if (other.isSetMessage()) {
+      this.message = other.message;
+    }
+  }
+
+  @Override
+  public AlreadyExists clone() {
+    return new AlreadyExists(this);
+  }
+
+  public String getMessage() {
+    return this.message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  public void unsetMessage() {
+    this.message = null;
+  }
+
+  // Returns true if field message is set (has been asigned a value) and false otherwise
+  public boolean isSetMessage() {
+    return this.message != null;
+  }
+
+  public void setMessageIsSet(boolean value) {
+    if (!value) {
+      this.message = null;
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    switch (fieldID) {
+    case MESSAGE:
+      if (value == null) {
+        unsetMessage();
+      } else {
+        setMessage((String)value);
+      }
+      break;
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  public Object getFieldValue(int fieldID) {
+    switch (fieldID) {
+    case MESSAGE:
+      return getMessage();
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  // Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
+  public boolean isSet(int fieldID) {
+    switch (fieldID) {
+    case MESSAGE:
+      return isSetMessage();
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  @Override
   public boolean equals(Object that) {
     if (that == null)
       return false;
@@ -69,8 +153,8 @@
     if (that == null)
       return false;
 
-    boolean this_present_message = true && (this.message != null);
-    boolean that_present_message = true && (that.message != null);
+    boolean this_present_message = true && this.isSetMessage();
+    boolean that_present_message = true && that.isSetMessage();
     if (this_present_message || that_present_message) {
       if (!(this_present_message && that_present_message))
         return false;
@@ -81,6 +165,7 @@
     return true;
   }
 
+  @Override
   public int hashCode() {
     return 0;
   }
@@ -96,10 +181,9 @@
       }
       switch (field.id)
       {
-        case 1:
+        case MESSAGE:
           if (field.type == TType.STRING) {
             this.message = iprot.readString();
-            this.__isset.message = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
@@ -111,17 +195,18 @@
       iprot.readFieldEnd();
     }
     iprot.readStructEnd();
+
+
+    // check for required fields of primitive type, which can't be checked in the validate method
+    validate();
   }
 
   public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("AlreadyExists");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
     if (this.message != null) {
-      field.name = "message";
-      field.type = TType.STRING;
-      field.id = 1;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(MESSAGE_FIELD_DESC);
       oprot.writeString(this.message);
       oprot.writeFieldEnd();
     }
@@ -129,13 +214,26 @@
     oprot.writeStructEnd();
   }
 
+  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("AlreadyExists(");
+    boolean first = true;
+
     sb.append("message:");
-    sb.append(this.message);
+    if (this.message == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.message);
+    }
+    first = false;
     sb.append(")");
     return sb.toString();
   }
 
+  public void validate() throws TException {
+    // check for required fields
+    // check that fields of type enum have valid values
+  }
+
 }
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/BatchMutation.java Tue May  5 07:26:49 2009
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 /**
  * Autogenerated by Thrift
  *
@@ -29,22 +28,39 @@
 import java.util.HashMap;
 import java.util.Set;
 import java.util.HashSet;
-import com.facebook.thrift.*;
+import java.util.Collections;
 
-import com.facebook.thrift.protocol.*;
-import com.facebook.thrift.transport.*;
+import org.apache.thrift.*;
+import org.apache.thrift.meta_data.*;
+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, java.io.Serializable {
+public class BatchMutation implements TBase, 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 static final int ROW = 1;
   public List<Mutation> mutations;
+  public static final int MUTATIONS = 2;
+
+  private final Isset __isset = new Isset();
+  private static final class Isset implements java.io.Serializable {
+  }
+
+  public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
+    put(ROW, new FieldMetaData("row", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    put(MUTATIONS, new FieldMetaData("mutations", TFieldRequirementType.DEFAULT, 
+        new ListMetaData(TType.LIST, 
+            new StructMetaData(TType.STRUCT, Mutation.class))));
+  }});
 
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean row = false;
-    public boolean mutations = false;
+  static {
+    FieldMetaData.addStructMetaDataMap(BatchMutation.class, metaDataMap);
   }
 
   public BatchMutation() {
@@ -56,11 +72,140 @@
   {
     this();
     this.row = row;
-    this.__isset.row = true;
     this.mutations = mutations;
-    this.__isset.mutations = true;
   }
 
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public BatchMutation(BatchMutation other) {
+    if (other.isSetRow()) {
+      this.row = other.row;
+    }
+    if (other.isSetMutations()) {
+      List<Mutation> __this__mutations = new ArrayList<Mutation>();
+      for (Mutation other_element : other.mutations) {
+        __this__mutations.add(new Mutation(other_element));
+      }
+      this.mutations = __this__mutations;
+    }
+  }
+
+  @Override
+  public BatchMutation clone() {
+    return new BatchMutation(this);
+  }
+
+  public byte[] getRow() {
+    return this.row;
+  }
+
+  public void setRow(byte[] row) {
+    this.row = row;
+  }
+
+  public void unsetRow() {
+    this.row = null;
+  }
+
+  // Returns true if field row is set (has been asigned a value) and false otherwise
+  public boolean isSetRow() {
+    return this.row != null;
+  }
+
+  public void setRowIsSet(boolean value) {
+    if (!value) {
+      this.row = null;
+    }
+  }
+
+  public int getMutationsSize() {
+    return (this.mutations == null) ? 0 : this.mutations.size();
+  }
+
+  public java.util.Iterator<Mutation> getMutationsIterator() {
+    return (this.mutations == null) ? null : this.mutations.iterator();
+  }
+
+  public void addToMutations(Mutation elem) {
+    if (this.mutations == null) {
+      this.mutations = new ArrayList<Mutation>();
+    }
+    this.mutations.add(elem);
+  }
+
+  public List<Mutation> getMutations() {
+    return this.mutations;
+  }
+
+  public void setMutations(List<Mutation> mutations) {
+    this.mutations = mutations;
+  }
+
+  public void unsetMutations() {
+    this.mutations = null;
+  }
+
+  // Returns true if field mutations is set (has been asigned a value) and false otherwise
+  public boolean isSetMutations() {
+    return this.mutations != null;
+  }
+
+  public void setMutationsIsSet(boolean value) {
+    if (!value) {
+      this.mutations = null;
+    }
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    switch (fieldID) {
+    case ROW:
+      if (value == null) {
+        unsetRow();
+      } else {
+        setRow((byte[])value);
+      }
+      break;
+
+    case MUTATIONS:
+      if (value == null) {
+        unsetMutations();
+      } else {
+        setMutations((List<Mutation>)value);
+      }
+      break;
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  public Object getFieldValue(int fieldID) {
+    switch (fieldID) {
+    case ROW:
+      return getRow();
+
+    case MUTATIONS:
+      return getMutations();
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  // Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
+  public boolean isSet(int fieldID) {
+    switch (fieldID) {
+    case ROW:
+      return isSetRow();
+    case MUTATIONS:
+      return isSetMutations();
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  @Override
   public boolean equals(Object that) {
     if (that == null)
       return false;
@@ -73,8 +218,8 @@
     if (that == null)
       return false;
 
-    boolean this_present_row = true && (this.row != null);
-    boolean that_present_row = true && (that.row != null);
+    boolean this_present_row = true && this.isSetRow();
+    boolean that_present_row = true && that.isSetRow();
     if (this_present_row || that_present_row) {
       if (!(this_present_row && that_present_row))
         return false;
@@ -82,8 +227,8 @@
         return false;
     }
 
-    boolean this_present_mutations = true && (this.mutations != null);
-    boolean that_present_mutations = true && (that.mutations != null);
+    boolean this_present_mutations = true && this.isSetMutations();
+    boolean that_present_mutations = true && that.isSetMutations();
     if (this_present_mutations || that_present_mutations) {
       if (!(this_present_mutations && that_present_mutations))
         return false;
@@ -94,6 +239,7 @@
     return true;
   }
 
+  @Override
   public int hashCode() {
     return 0;
   }
@@ -109,29 +255,27 @@
       }
       switch (field.id)
       {
-        case 1:
+        case ROW:
           if (field.type == TType.STRING) {
             this.row = iprot.readBinary();
-            this.__isset.row = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 2:
+        case 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)
               {
-                Mutation _elem2 = new Mutation();
+                Mutation _elem2;
                 _elem2 = new Mutation();
                 _elem2.read(iprot);
                 this.mutations.add(_elem2);
               }
               iprot.readListEnd();
             }
-            this.__isset.mutations = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
@@ -143,25 +287,23 @@
       iprot.readFieldEnd();
     }
     iprot.readStructEnd();
+
+
+    // check for required fields of primitive type, which can't be checked in the validate method
+    validate();
   }
 
   public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("BatchMutation");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
     if (this.row != null) {
-      field.name = "row";
-      field.type = TType.STRING;
-      field.id = 1;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(ROW_FIELD_DESC);
       oprot.writeBinary(this.row);
       oprot.writeFieldEnd();
     }
     if (this.mutations != null) {
-      field.name = "mutations";
-      field.type = TType.LIST;
-      field.id = 2;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(MUTATIONS_FIELD_DESC);
       {
         oprot.writeListBegin(new TList(TType.STRUCT, this.mutations.size()));
         for (Mutation _iter3 : this.mutations)        {
@@ -175,15 +317,34 @@
     oprot.writeStructEnd();
   }
 
+  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("BatchMutation(");
+    boolean first = true;
+
     sb.append("row:");
-    sb.append(this.row);
-    sb.append(",mutations:");
-    sb.append(this.mutations);
+    if (this.row == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.row);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("mutations:");
+    if (this.mutations == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.mutations);
+    }
+    first = false;
     sb.append(")");
     return sb.toString();
   }
 
+  public void validate() throws TException {
+    // check for required fields
+    // check that fields of type enum have valid values
+  }
+
 }
 

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java?rev=771604&r1=771603&r2=771604&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/ColumnDescriptor.java Tue May  5 07:26:49 2009
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 /**
  * Autogenerated by Thrift
  *
@@ -29,42 +28,89 @@
 import java.util.HashMap;
 import java.util.Set;
 import java.util.HashSet;
-import com.facebook.thrift.*;
+import java.util.Collections;
 
-import com.facebook.thrift.protocol.*;
-import com.facebook.thrift.transport.*;
+import org.apache.thrift.*;
+import org.apache.thrift.meta_data.*;
+import org.apache.thrift.protocol.*;
 
 /**
  * An HColumnDescriptor contains information about a column family
  * 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, java.io.Serializable {
+public class ColumnDescriptor implements TBase, 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);
+  private static final TField MAX_VERSIONS_FIELD_DESC = new TField("maxVersions", TType.I32, (short)2);
+  private static final TField COMPRESSION_FIELD_DESC = new TField("compression", TType.STRING, (short)3);
+  private static final TField IN_MEMORY_FIELD_DESC = new TField("inMemory", TType.BOOL, (short)4);
+  private static final TField MAX_VALUE_LENGTH_FIELD_DESC = new TField("maxValueLength", TType.I32, (short)5);
+  private static final TField BLOOM_FILTER_TYPE_FIELD_DESC = new TField("bloomFilterType", TType.STRING, (short)6);
+  private static final TField BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC = new TField("bloomFilterVectorSize", TType.I32, (short)7);
+  private static final TField BLOOM_FILTER_NB_HASHES_FIELD_DESC = new TField("bloomFilterNbHashes", TType.I32, (short)8);
+  private static final TField BLOCK_CACHE_ENABLED_FIELD_DESC = new TField("blockCacheEnabled", TType.BOOL, (short)9);
+  private static final TField TIME_TO_LIVE_FIELD_DESC = new TField("timeToLive", TType.I32, (short)10);
+
   public byte[] name;
+  public static final int NAME = 1;
   public int maxVersions;
+  public static final int MAXVERSIONS = 2;
   public String compression;
+  public static final int COMPRESSION = 3;
   public boolean inMemory;
+  public static final int INMEMORY = 4;
   public int maxValueLength;
+  public static final int MAXVALUELENGTH = 5;
   public String bloomFilterType;
+  public static final int BLOOMFILTERTYPE = 6;
   public int bloomFilterVectorSize;
+  public static final int BLOOMFILTERVECTORSIZE = 7;
   public int bloomFilterNbHashes;
+  public static final int BLOOMFILTERNBHASHES = 8;
   public boolean blockCacheEnabled;
+  public static final int BLOCKCACHEENABLED = 9;
   public int timeToLive;
+  public static final int TIMETOLIVE = 10;
 
-  public final Isset __isset = new Isset();
-  public static final class Isset implements java.io.Serializable {
-    public boolean name = false;
+  private final Isset __isset = new Isset();
+  private static final class Isset implements java.io.Serializable {
     public boolean maxVersions = false;
-    public boolean compression = false;
     public boolean inMemory = false;
     public boolean maxValueLength = false;
-    public boolean bloomFilterType = false;
     public boolean bloomFilterVectorSize = false;
     public boolean bloomFilterNbHashes = false;
     public boolean blockCacheEnabled = false;
     public boolean timeToLive = false;
   }
 
+  public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
+    put(NAME, new FieldMetaData("name", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    put(MAXVERSIONS, new FieldMetaData("maxVersions", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.I32)));
+    put(COMPRESSION, new FieldMetaData("compression", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    put(INMEMORY, new FieldMetaData("inMemory", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.BOOL)));
+    put(MAXVALUELENGTH, new FieldMetaData("maxValueLength", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.I32)));
+    put(BLOOMFILTERTYPE, new FieldMetaData("bloomFilterType", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.STRING)));
+    put(BLOOMFILTERVECTORSIZE, new FieldMetaData("bloomFilterVectorSize", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.I32)));
+    put(BLOOMFILTERNBHASHES, new FieldMetaData("bloomFilterNbHashes", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.I32)));
+    put(BLOCKCACHEENABLED, new FieldMetaData("blockCacheEnabled", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.BOOL)));
+    put(TIMETOLIVE, new FieldMetaData("timeToLive", TFieldRequirementType.DEFAULT, 
+        new FieldValueMetaData(TType.I32)));
+  }});
+
+  static {
+    FieldMetaData.addStructMetaDataMap(ColumnDescriptor.class, metaDataMap);
+  }
+
   public ColumnDescriptor() {
     this.maxVersions = 3;
 
@@ -100,17 +146,14 @@
   {
     this();
     this.name = name;
-    this.__isset.name = true;
     this.maxVersions = maxVersions;
     this.__isset.maxVersions = true;
     this.compression = compression;
-    this.__isset.compression = true;
     this.inMemory = inMemory;
     this.__isset.inMemory = true;
     this.maxValueLength = maxValueLength;
     this.__isset.maxValueLength = true;
     this.bloomFilterType = bloomFilterType;
-    this.__isset.bloomFilterType = true;
     this.bloomFilterVectorSize = bloomFilterVectorSize;
     this.__isset.bloomFilterVectorSize = true;
     this.bloomFilterNbHashes = bloomFilterNbHashes;
@@ -121,6 +164,416 @@
     this.__isset.timeToLive = true;
   }
 
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public ColumnDescriptor(ColumnDescriptor other) {
+    if (other.isSetName()) {
+      this.name = other.name;
+    }
+    __isset.maxVersions = other.__isset.maxVersions;
+    this.maxVersions = other.maxVersions;
+    if (other.isSetCompression()) {
+      this.compression = other.compression;
+    }
+    __isset.inMemory = other.__isset.inMemory;
+    this.inMemory = other.inMemory;
+    __isset.maxValueLength = other.__isset.maxValueLength;
+    this.maxValueLength = other.maxValueLength;
+    if (other.isSetBloomFilterType()) {
+      this.bloomFilterType = other.bloomFilterType;
+    }
+    __isset.bloomFilterVectorSize = other.__isset.bloomFilterVectorSize;
+    this.bloomFilterVectorSize = other.bloomFilterVectorSize;
+    __isset.bloomFilterNbHashes = other.__isset.bloomFilterNbHashes;
+    this.bloomFilterNbHashes = other.bloomFilterNbHashes;
+    __isset.blockCacheEnabled = other.__isset.blockCacheEnabled;
+    this.blockCacheEnabled = other.blockCacheEnabled;
+    __isset.timeToLive = other.__isset.timeToLive;
+    this.timeToLive = other.timeToLive;
+  }
+
+  @Override
+  public ColumnDescriptor clone() {
+    return new ColumnDescriptor(this);
+  }
+
+  public byte[] getName() {
+    return this.name;
+  }
+
+  public void setName(byte[] name) {
+    this.name = name;
+  }
+
+  public void unsetName() {
+    this.name = null;
+  }
+
+  // Returns true if field name is set (has been asigned a value) and false otherwise
+  public boolean isSetName() {
+    return this.name != null;
+  }
+
+  public void setNameIsSet(boolean value) {
+    if (!value) {
+      this.name = null;
+    }
+  }
+
+  public int getMaxVersions() {
+    return this.maxVersions;
+  }
+
+  public void setMaxVersions(int maxVersions) {
+    this.maxVersions = maxVersions;
+    this.__isset.maxVersions = true;
+  }
+
+  public void unsetMaxVersions() {
+    this.__isset.maxVersions = false;
+  }
+
+  // Returns true if field maxVersions is set (has been asigned a value) and false otherwise
+  public boolean isSetMaxVersions() {
+    return this.__isset.maxVersions;
+  }
+
+  public void setMaxVersionsIsSet(boolean value) {
+    this.__isset.maxVersions = value;
+  }
+
+  public String getCompression() {
+    return this.compression;
+  }
+
+  public void setCompression(String compression) {
+    this.compression = compression;
+  }
+
+  public void unsetCompression() {
+    this.compression = null;
+  }
+
+  // Returns true if field compression is set (has been asigned a value) and false otherwise
+  public boolean isSetCompression() {
+    return this.compression != null;
+  }
+
+  public void setCompressionIsSet(boolean value) {
+    if (!value) {
+      this.compression = null;
+    }
+  }
+
+  public boolean isInMemory() {
+    return this.inMemory;
+  }
+
+  public void setInMemory(boolean inMemory) {
+    this.inMemory = inMemory;
+    this.__isset.inMemory = true;
+  }
+
+  public void unsetInMemory() {
+    this.__isset.inMemory = false;
+  }
+
+  // Returns true if field inMemory is set (has been asigned a value) and false otherwise
+  public boolean isSetInMemory() {
+    return this.__isset.inMemory;
+  }
+
+  public void setInMemoryIsSet(boolean value) {
+    this.__isset.inMemory = value;
+  }
+
+  public int getMaxValueLength() {
+    return this.maxValueLength;
+  }
+
+  public void setMaxValueLength(int maxValueLength) {
+    this.maxValueLength = maxValueLength;
+    this.__isset.maxValueLength = true;
+  }
+
+  public void unsetMaxValueLength() {
+    this.__isset.maxValueLength = false;
+  }
+
+  // Returns true if field maxValueLength is set (has been asigned a value) and false otherwise
+  public boolean isSetMaxValueLength() {
+    return this.__isset.maxValueLength;
+  }
+
+  public void setMaxValueLengthIsSet(boolean value) {
+    this.__isset.maxValueLength = value;
+  }
+
+  public String getBloomFilterType() {
+    return this.bloomFilterType;
+  }
+
+  public void setBloomFilterType(String bloomFilterType) {
+    this.bloomFilterType = bloomFilterType;
+  }
+
+  public void unsetBloomFilterType() {
+    this.bloomFilterType = null;
+  }
+
+  // Returns true if field bloomFilterType is set (has been asigned a value) and false otherwise
+  public boolean isSetBloomFilterType() {
+    return this.bloomFilterType != null;
+  }
+
+  public void setBloomFilterTypeIsSet(boolean value) {
+    if (!value) {
+      this.bloomFilterType = null;
+    }
+  }
+
+  public int getBloomFilterVectorSize() {
+    return this.bloomFilterVectorSize;
+  }
+
+  public void setBloomFilterVectorSize(int bloomFilterVectorSize) {
+    this.bloomFilterVectorSize = bloomFilterVectorSize;
+    this.__isset.bloomFilterVectorSize = true;
+  }
+
+  public void unsetBloomFilterVectorSize() {
+    this.__isset.bloomFilterVectorSize = false;
+  }
+
+  // Returns true if field bloomFilterVectorSize is set (has been asigned a value) and false otherwise
+  public boolean isSetBloomFilterVectorSize() {
+    return this.__isset.bloomFilterVectorSize;
+  }
+
+  public void setBloomFilterVectorSizeIsSet(boolean value) {
+    this.__isset.bloomFilterVectorSize = value;
+  }
+
+  public int getBloomFilterNbHashes() {
+    return this.bloomFilterNbHashes;
+  }
+
+  public void setBloomFilterNbHashes(int bloomFilterNbHashes) {
+    this.bloomFilterNbHashes = bloomFilterNbHashes;
+    this.__isset.bloomFilterNbHashes = true;
+  }
+
+  public void unsetBloomFilterNbHashes() {
+    this.__isset.bloomFilterNbHashes = false;
+  }
+
+  // Returns true if field bloomFilterNbHashes is set (has been asigned a value) and false otherwise
+  public boolean isSetBloomFilterNbHashes() {
+    return this.__isset.bloomFilterNbHashes;
+  }
+
+  public void setBloomFilterNbHashesIsSet(boolean value) {
+    this.__isset.bloomFilterNbHashes = value;
+  }
+
+  public boolean isBlockCacheEnabled() {
+    return this.blockCacheEnabled;
+  }
+
+  public void setBlockCacheEnabled(boolean blockCacheEnabled) {
+    this.blockCacheEnabled = blockCacheEnabled;
+    this.__isset.blockCacheEnabled = true;
+  }
+
+  public void unsetBlockCacheEnabled() {
+    this.__isset.blockCacheEnabled = false;
+  }
+
+  // Returns true if field blockCacheEnabled is set (has been asigned a value) and false otherwise
+  public boolean isSetBlockCacheEnabled() {
+    return this.__isset.blockCacheEnabled;
+  }
+
+  public void setBlockCacheEnabledIsSet(boolean value) {
+    this.__isset.blockCacheEnabled = value;
+  }
+
+  public int getTimeToLive() {
+    return this.timeToLive;
+  }
+
+  public void setTimeToLive(int timeToLive) {
+    this.timeToLive = timeToLive;
+    this.__isset.timeToLive = true;
+  }
+
+  public void unsetTimeToLive() {
+    this.__isset.timeToLive = false;
+  }
+
+  // Returns true if field timeToLive is set (has been asigned a value) and false otherwise
+  public boolean isSetTimeToLive() {
+    return this.__isset.timeToLive;
+  }
+
+  public void setTimeToLiveIsSet(boolean value) {
+    this.__isset.timeToLive = value;
+  }
+
+  public void setFieldValue(int fieldID, Object value) {
+    switch (fieldID) {
+    case NAME:
+      if (value == null) {
+        unsetName();
+      } else {
+        setName((byte[])value);
+      }
+      break;
+
+    case MAXVERSIONS:
+      if (value == null) {
+        unsetMaxVersions();
+      } else {
+        setMaxVersions((Integer)value);
+      }
+      break;
+
+    case COMPRESSION:
+      if (value == null) {
+        unsetCompression();
+      } else {
+        setCompression((String)value);
+      }
+      break;
+
+    case INMEMORY:
+      if (value == null) {
+        unsetInMemory();
+      } else {
+        setInMemory((Boolean)value);
+      }
+      break;
+
+    case MAXVALUELENGTH:
+      if (value == null) {
+        unsetMaxValueLength();
+      } else {
+        setMaxValueLength((Integer)value);
+      }
+      break;
+
+    case BLOOMFILTERTYPE:
+      if (value == null) {
+        unsetBloomFilterType();
+      } else {
+        setBloomFilterType((String)value);
+      }
+      break;
+
+    case BLOOMFILTERVECTORSIZE:
+      if (value == null) {
+        unsetBloomFilterVectorSize();
+      } else {
+        setBloomFilterVectorSize((Integer)value);
+      }
+      break;
+
+    case BLOOMFILTERNBHASHES:
+      if (value == null) {
+        unsetBloomFilterNbHashes();
+      } else {
+        setBloomFilterNbHashes((Integer)value);
+      }
+      break;
+
+    case BLOCKCACHEENABLED:
+      if (value == null) {
+        unsetBlockCacheEnabled();
+      } else {
+        setBlockCacheEnabled((Boolean)value);
+      }
+      break;
+
+    case TIMETOLIVE:
+      if (value == null) {
+        unsetTimeToLive();
+      } else {
+        setTimeToLive((Integer)value);
+      }
+      break;
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  public Object getFieldValue(int fieldID) {
+    switch (fieldID) {
+    case NAME:
+      return getName();
+
+    case MAXVERSIONS:
+      return new Integer(getMaxVersions());
+
+    case COMPRESSION:
+      return getCompression();
+
+    case INMEMORY:
+      return new Boolean(isInMemory());
+
+    case MAXVALUELENGTH:
+      return new Integer(getMaxValueLength());
+
+    case BLOOMFILTERTYPE:
+      return getBloomFilterType();
+
+    case BLOOMFILTERVECTORSIZE:
+      return new Integer(getBloomFilterVectorSize());
+
+    case BLOOMFILTERNBHASHES:
+      return new Integer(getBloomFilterNbHashes());
+
+    case BLOCKCACHEENABLED:
+      return new Boolean(isBlockCacheEnabled());
+
+    case TIMETOLIVE:
+      return new Integer(getTimeToLive());
+
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  // Returns true if field corresponding to fieldID is set (has been asigned a value) and false otherwise
+  public boolean isSet(int fieldID) {
+    switch (fieldID) {
+    case NAME:
+      return isSetName();
+    case MAXVERSIONS:
+      return isSetMaxVersions();
+    case COMPRESSION:
+      return isSetCompression();
+    case INMEMORY:
+      return isSetInMemory();
+    case MAXVALUELENGTH:
+      return isSetMaxValueLength();
+    case BLOOMFILTERTYPE:
+      return isSetBloomFilterType();
+    case BLOOMFILTERVECTORSIZE:
+      return isSetBloomFilterVectorSize();
+    case BLOOMFILTERNBHASHES:
+      return isSetBloomFilterNbHashes();
+    case BLOCKCACHEENABLED:
+      return isSetBlockCacheEnabled();
+    case TIMETOLIVE:
+      return isSetTimeToLive();
+    default:
+      throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+    }
+  }
+
+  @Override
   public boolean equals(Object that) {
     if (that == null)
       return false;
@@ -133,8 +586,8 @@
     if (that == null)
       return false;
 
-    boolean this_present_name = true && (this.name != null);
-    boolean that_present_name = true && (that.name != null);
+    boolean this_present_name = true && this.isSetName();
+    boolean that_present_name = true && that.isSetName();
     if (this_present_name || that_present_name) {
       if (!(this_present_name && that_present_name))
         return false;
@@ -151,8 +604,8 @@
         return false;
     }
 
-    boolean this_present_compression = true && (this.compression != null);
-    boolean that_present_compression = true && (that.compression != null);
+    boolean this_present_compression = true && this.isSetCompression();
+    boolean that_present_compression = true && that.isSetCompression();
     if (this_present_compression || that_present_compression) {
       if (!(this_present_compression && that_present_compression))
         return false;
@@ -178,8 +631,8 @@
         return false;
     }
 
-    boolean this_present_bloomFilterType = true && (this.bloomFilterType != null);
-    boolean that_present_bloomFilterType = true && (that.bloomFilterType != null);
+    boolean this_present_bloomFilterType = true && this.isSetBloomFilterType();
+    boolean that_present_bloomFilterType = true && that.isSetBloomFilterType();
     if (this_present_bloomFilterType || that_present_bloomFilterType) {
       if (!(this_present_bloomFilterType && that_present_bloomFilterType))
         return false;
@@ -226,6 +679,7 @@
     return true;
   }
 
+  @Override
   public int hashCode() {
     return 0;
   }
@@ -241,15 +695,14 @@
       }
       switch (field.id)
       {
-        case 1:
+        case NAME:
           if (field.type == TType.STRING) {
             this.name = iprot.readBinary();
-            this.__isset.name = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 2:
+        case MAXVERSIONS:
           if (field.type == TType.I32) {
             this.maxVersions = iprot.readI32();
             this.__isset.maxVersions = true;
@@ -257,15 +710,14 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 3:
+        case COMPRESSION:
           if (field.type == TType.STRING) {
             this.compression = iprot.readString();
-            this.__isset.compression = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 4:
+        case INMEMORY:
           if (field.type == TType.BOOL) {
             this.inMemory = iprot.readBool();
             this.__isset.inMemory = true;
@@ -273,7 +725,7 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 5:
+        case MAXVALUELENGTH:
           if (field.type == TType.I32) {
             this.maxValueLength = iprot.readI32();
             this.__isset.maxValueLength = true;
@@ -281,15 +733,14 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 6:
+        case BLOOMFILTERTYPE:
           if (field.type == TType.STRING) {
             this.bloomFilterType = iprot.readString();
-            this.__isset.bloomFilterType = true;
           } else { 
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 7:
+        case BLOOMFILTERVECTORSIZE:
           if (field.type == TType.I32) {
             this.bloomFilterVectorSize = iprot.readI32();
             this.__isset.bloomFilterVectorSize = true;
@@ -297,7 +748,7 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 8:
+        case BLOOMFILTERNBHASHES:
           if (field.type == TType.I32) {
             this.bloomFilterNbHashes = iprot.readI32();
             this.__isset.bloomFilterNbHashes = true;
@@ -305,7 +756,7 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 9:
+        case BLOCKCACHEENABLED:
           if (field.type == TType.BOOL) {
             this.blockCacheEnabled = iprot.readBool();
             this.__isset.blockCacheEnabled = true;
@@ -313,7 +764,7 @@
             TProtocolUtil.skip(iprot, field.type);
           }
           break;
-        case 10:
+        case TIMETOLIVE:
           if (field.type == TType.I32) {
             this.timeToLive = iprot.readI32();
             this.__isset.timeToLive = true;
@@ -328,107 +779,120 @@
       iprot.readFieldEnd();
     }
     iprot.readStructEnd();
+
+
+    // check for required fields of primitive type, which can't be checked in the validate method
+    validate();
   }
 
   public void write(TProtocol oprot) throws TException {
-    TStruct struct = new TStruct("ColumnDescriptor");
-    oprot.writeStructBegin(struct);
-    TField field = new TField();
+    validate();
+
+    oprot.writeStructBegin(STRUCT_DESC);
     if (this.name != null) {
-      field.name = "name";
-      field.type = TType.STRING;
-      field.id = 1;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(NAME_FIELD_DESC);
       oprot.writeBinary(this.name);
       oprot.writeFieldEnd();
     }
-    field.name = "maxVersions";
-    field.type = TType.I32;
-    field.id = 2;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(MAX_VERSIONS_FIELD_DESC);
     oprot.writeI32(this.maxVersions);
     oprot.writeFieldEnd();
     if (this.compression != null) {
-      field.name = "compression";
-      field.type = TType.STRING;
-      field.id = 3;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(COMPRESSION_FIELD_DESC);
       oprot.writeString(this.compression);
       oprot.writeFieldEnd();
     }
-    field.name = "inMemory";
-    field.type = TType.BOOL;
-    field.id = 4;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(IN_MEMORY_FIELD_DESC);
     oprot.writeBool(this.inMemory);
     oprot.writeFieldEnd();
-    field.name = "maxValueLength";
-    field.type = TType.I32;
-    field.id = 5;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(MAX_VALUE_LENGTH_FIELD_DESC);
     oprot.writeI32(this.maxValueLength);
     oprot.writeFieldEnd();
     if (this.bloomFilterType != null) {
-      field.name = "bloomFilterType";
-      field.type = TType.STRING;
-      field.id = 6;
-      oprot.writeFieldBegin(field);
+      oprot.writeFieldBegin(BLOOM_FILTER_TYPE_FIELD_DESC);
       oprot.writeString(this.bloomFilterType);
       oprot.writeFieldEnd();
     }
-    field.name = "bloomFilterVectorSize";
-    field.type = TType.I32;
-    field.id = 7;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(BLOOM_FILTER_VECTOR_SIZE_FIELD_DESC);
     oprot.writeI32(this.bloomFilterVectorSize);
     oprot.writeFieldEnd();
-    field.name = "bloomFilterNbHashes";
-    field.type = TType.I32;
-    field.id = 8;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(BLOOM_FILTER_NB_HASHES_FIELD_DESC);
     oprot.writeI32(this.bloomFilterNbHashes);
     oprot.writeFieldEnd();
-    field.name = "blockCacheEnabled";
-    field.type = TType.BOOL;
-    field.id = 9;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(BLOCK_CACHE_ENABLED_FIELD_DESC);
     oprot.writeBool(this.blockCacheEnabled);
     oprot.writeFieldEnd();
-    field.name = "timeToLive";
-    field.type = TType.I32;
-    field.id = 10;
-    oprot.writeFieldBegin(field);
+    oprot.writeFieldBegin(TIME_TO_LIVE_FIELD_DESC);
     oprot.writeI32(this.timeToLive);
     oprot.writeFieldEnd();
     oprot.writeFieldStop();
     oprot.writeStructEnd();
   }
 
+  @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("ColumnDescriptor(");
+    boolean first = true;
+
     sb.append("name:");
-    sb.append(this.name);
-    sb.append(",maxVersions:");
+    if (this.name == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.name);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("maxVersions:");
     sb.append(this.maxVersions);
-    sb.append(",compression:");
-    sb.append(this.compression);
-    sb.append(",inMemory:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("compression:");
+    if (this.compression == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.compression);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("inMemory:");
     sb.append(this.inMemory);
-    sb.append(",maxValueLength:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("maxValueLength:");
     sb.append(this.maxValueLength);
-    sb.append(",bloomFilterType:");
-    sb.append(this.bloomFilterType);
-    sb.append(",bloomFilterVectorSize:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bloomFilterType:");
+    if (this.bloomFilterType == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.bloomFilterType);
+    }
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bloomFilterVectorSize:");
     sb.append(this.bloomFilterVectorSize);
-    sb.append(",bloomFilterNbHashes:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("bloomFilterNbHashes:");
     sb.append(this.bloomFilterNbHashes);
-    sb.append(",blockCacheEnabled:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("blockCacheEnabled:");
     sb.append(this.blockCacheEnabled);
-    sb.append(",timeToLive:");
+    first = false;
+    if (!first) sb.append(", ");
+    sb.append("timeToLive:");
     sb.append(this.timeToLive);
+    first = false;
     sb.append(")");
     return sb.toString();
   }
 
+  public void validate() throws TException {
+    // check for required fields
+    // check that fields of type enum have valid values
+  }
+
 }