You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2009/07/28 01:48:23 UTC

svn commit: r798334 - in /hadoop/hbase/trunk: ./ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/mapred/ src/java/org/apache/hadoop/hbase/regionserver/ src/java/org/apache/hadoop/hbase/thrift/ src/java/org/apache/hadoop/hbase/...

Author: rawson
Date: Mon Jul 27 23:48:22 2009
New Revision: 798334

URL: http://svn.apache.org/viewvc?rev=798334&view=rev
Log:
HBASE-1714

Modified:
    hadoop/hbase/trunk/CHANGES.txt
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
    hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java
    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/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/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java

Modified: hadoop/hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/CHANGES.txt?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/CHANGES.txt (original)
+++ hadoop/hbase/trunk/CHANGES.txt Mon Jul 27 23:48:22 2009
@@ -521,6 +521,7 @@
                (Tim Sell via Stack)
    HBASE-1683  OOME on master splitting logs; stuck, won't go down
    HBASE-1704  Better zk error when failed connect
+   HBASE-1714  Thrift server: prefix scan API
 
   OPTIMIZATIONS
    HBASE-1412  Change values for delete column and column family in KeyValue

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/client/Delete.java Mon Jul 27 23:48:22 2009
@@ -228,6 +228,11 @@
     familyMap.put(family, list);
     return this;
   }
+
+  public void deleteColumns(byte [] column) {
+    byte [][] parts = KeyValue.parseColumn(column);
+    this.deleteColumns(parts[0], parts[1]);
+  }
   
   /**
    * Delete the latest version of the specified column, given in

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/mapred/TableInputFormatBase.java Mon Jul 27 23:48:22 2009
@@ -39,6 +39,7 @@
 import org.apache.hadoop.hbase.io.RowResult;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
@@ -106,21 +107,22 @@
     public void restart(byte[] firstRow) throws IOException {
       if ((endRow != null) && (endRow.length > 0)) {
         if (trrRowFilter != null) {
-          final Set<RowFilterInterface> rowFiltersSet =
-            new HashSet<RowFilterInterface>();
-          rowFiltersSet.add(new WhileMatchRowFilter(new StopRowFilter(endRow)));
-          rowFiltersSet.add(trrRowFilter);
-          Scan scan = new Scan(startRow);
+          Scan scan = new Scan(firstRow, endRow);
           scan.addColumns(trrInputColumns);
-//          scan.setFilter(new RowFilterSet(RowFilterSet.Operator.MUST_PASS_ALL,
-//              rowFiltersSet));
+          scan.setOldFilter(trrRowFilter);
           this.scanner = this.htable.getScanner(scan);
         } else {
+          LOG.debug("TIFB.restart, firstRow: " +
+              Bytes.toStringBinary(firstRow) + ", endRow: " +
+              Bytes.toStringBinary(endRow));
           Scan scan = new Scan(firstRow, endRow);
           scan.addColumns(trrInputColumns);
           this.scanner = this.htable.getScanner(scan);
         }
       } else {
+        LOG.debug("TIFB.restart, firstRow: " +
+            Bytes.toStringBinary(firstRow) + ", no endRow");
+
         Scan scan = new Scan(firstRow);
         scan.addColumns(trrInputColumns);
 //        scan.setFilter(trrRowFilter);

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/regionserver/DeleteCompare.java Mon Jul 27 23:48:22 2009
@@ -1,3 +1,23 @@
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.KeyValue;
@@ -69,7 +89,6 @@
     if(res > 0) {
       return DeleteCode.DONE;
     } else if(res < 0){
-      System.out.println("SKIPPING ROW");
       return DeleteCode.SKIP;
     }
 
@@ -113,7 +132,6 @@
       }
       return DeleteCode.DONE;
     } else {
-      System.out.println("SKIPPING TS");
       return DeleteCode.SKIP;
     }
   } 

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=798334&r1=798333&r2=798334&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 Mon Jul 27 23:48:22 2009
@@ -74,12 +74,11 @@
   2:i32 maxVersions = 3,
   3:string compression = "NONE",
   4:bool inMemory = 0,
-  5:i32 maxValueLength = 2147483647,
-  6:string bloomFilterType = "NONE",
-  7:i32 bloomFilterVectorSize = 0,
-  8:i32 bloomFilterNbHashes = 0,
-  9:bool blockCacheEnabled = 0,
-  10:i32 timeToLive = -1
+  5:string bloomFilterType = "NONE",
+  6:i32 bloomFilterVectorSize = 0,
+  7:i32 bloomFilterNbHashes = 0,
+  8:bool blockCacheEnabled = 0,
+  9:i32 timeToLive = -1
 }
 
 /**
@@ -463,6 +462,20 @@
                                 4:list<Text> columns)
     throws (1:IOError io)
 
+  /**
+   * Open a scanner for a given prefix.  That is all rows will have the specified
+   * prefix. No other rows will be returned.
+   *
+   * @param tableName name of table
+   * @param startAndPrefix the prefix (and thus start row) of the keys you want
+   * @param columns the columns you want returned
+   * @return scanner id to use with other scanner calls
+   */
+  ScannerID scannerOpenWithPrefix(1:Text tableName,
+                                  2:Text startAndPrefix,
+                                  3:list<Text> columns)
+    throws (1:IOError io)
+
   /** 
    * Get a scanner on the current table starting at the specified row and
    * ending at the last row in the table.  Return the specified columns.

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=798334&r1=798333&r2=798334&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 Mon Jul 27 23:48:22 2009
@@ -35,6 +35,9 @@
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.WhileMatchFilter;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
@@ -603,7 +606,24 @@
         throw new IOError(e.getMessage());
       }
     }
-    
+
+    @Override
+    public int scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws IOError, TException {
+      try {
+        HTable table = getTable(tableName);
+        byte [][] columnsArray = null;
+        columnsArray = columns.toArray(new byte[0][]);
+        Scan scan = new Scan(startAndPrefix);
+        scan.addColumns(columnsArray);
+        Filter f = new WhileMatchFilter(
+            new PrefixFilter(startAndPrefix));
+        scan.setFilter(f);
+        return addScanner(table.getScanner(scan));
+      } catch (IOException e) {
+        throw new IOError(e.getMessage());
+      }
+    }
+
     public int scannerOpenTs(byte[] tableName, byte[] startRow,
         List<byte[]> columns, long timestamp) throws IOError, TException {
       try {

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=798334&r1=798333&r2=798334&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 Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
+
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

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=798334&r1=798333&r2=798334&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 Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

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=798334&r1=798333&r2=798334&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 Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;
@@ -129,7 +126,6 @@
     int maxVersions,
     String compression,
     boolean inMemory,
-    int maxValueLength,
     String bloomFilterType,
     int bloomFilterVectorSize,
     int bloomFilterNbHashes,

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Hbase.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;
@@ -41,6 +38,8 @@
     /**
      * Brings a table on-line (enables it)
      * @param tableName name of the table
+     * 
+     * @param tableName
      */
     public void enableTable(byte[] tableName) throws IOError, TException;
 
@@ -48,12 +47,16 @@
      * Disables a table (takes it off-line) If it is being served, the master
      * will tell the servers to stop serving it.
      * @param tableName name of the table
+     * 
+     * @param tableName
      */
     public void disableTable(byte[] tableName) throws IOError, TException;
 
     /**
      * @param tableName name of table to check
      * @return true if table is on-line
+     * 
+     * @param tableName
      */
     public boolean isTableEnabled(byte[] tableName) throws IOError, TException;
 
@@ -71,6 +74,8 @@
      * List all the column families assoicated with a table.
      * @param tableName table name
      * @return list of column family descriptors
+     * 
+     * @param tableName
      */
     public Map<byte[],ColumnDescriptor> getColumnDescriptors(byte[] tableName) throws IOError, TException;
 
@@ -78,6 +83,8 @@
      * List the regions associated with a table.
      * @param tableName table name
      * @return list of region descriptors
+     * 
+     * @param tableName
      */
     public List<TRegionInfo> getTableRegions(byte[] tableName) throws IOError, TException;
 
@@ -92,6 +99,9 @@
      * 
      * @throws IllegalArgument if an input parameter is invalid
      * @throws AlreadyExists if the table name already exists
+     * 
+     * @param tableName
+     * @param columnFamilies
      */
     public void createTable(byte[] tableName, List<ColumnDescriptor> columnFamilies) throws IOError, IllegalArgument, AlreadyExists, TException;
 
@@ -100,6 +110,8 @@
      * @param tableName name of table to delete
      * @throws IOError if table doesn't exist on server or there was some other
      * problem
+     * 
+     * @param tableName
      */
     public void deleteTable(byte[] tableName) throws IOError, TException;
 
@@ -111,6 +123,10 @@
      * @param row row key
      * @param column column name
      * @return value for specified row/column
+     * 
+     * @param tableName
+     * @param row
+     * @param column
      */
     public List<TCell> get(byte[] tableName, byte[] row, byte[] column) throws IOError, TException;
 
@@ -123,6 +139,11 @@
      * @param column column name
      * @param numVersions number of versions to retrieve
      * @return list of cells for specified row/column
+     * 
+     * @param tableName
+     * @param row
+     * @param column
+     * @param numVersions
      */
     public List<TCell> getVer(byte[] tableName, byte[] row, byte[] column, int numVersions) throws IOError, TException;
 
@@ -137,6 +158,12 @@
      * @param timestamp timestamp
      * @param numVersions number of versions to retrieve
      * @return list of cells for specified row/column
+     * 
+     * @param tableName
+     * @param row
+     * @param column
+     * @param timestamp
+     * @param numVersions
      */
     public List<TCell> getVerTs(byte[] tableName, byte[] row, byte[] column, long timestamp, int numVersions) throws IOError, TException;
 
@@ -147,6 +174,9 @@
      * @param tableName name of table
      * @param row row key
      * @return TRowResult containing the row and map of columns to TCells
+     * 
+     * @param tableName
+     * @param row
      */
     public List<TRowResult> getRow(byte[] tableName, byte[] row) throws IOError, TException;
 
@@ -158,6 +188,10 @@
      * @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
+     * 
+     * @param tableName
+     * @param row
+     * @param columns
      */
     public List<TRowResult> getRowWithColumns(byte[] tableName, byte[] row, List<byte[]> columns) throws IOError, TException;
 
@@ -169,6 +203,10 @@
      * @param row row key
      * @param timestamp timestamp
      * @return TRowResult containing the row and map of columns to TCells
+     * 
+     * @param tableName
+     * @param row
+     * @param timestamp
      */
     public List<TRowResult> getRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException;
 
@@ -180,6 +218,11 @@
      * @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
+     * 
+     * @param tableName
+     * @param row
+     * @param columns
+     * @param timestamp
      */
     public List<TRowResult> getRowWithColumnsTs(byte[] tableName, byte[] row, List<byte[]> columns, long timestamp) throws IOError, TException;
 
@@ -192,6 +235,10 @@
      * @param tableName name of table
      * @param row row key
      * @param mutations list of mutation commands
+     * 
+     * @param tableName
+     * @param row
+     * @param mutations
      */
     public void mutateRow(byte[] tableName, byte[] row, List<Mutation> mutations) throws IOError, IllegalArgument, TException;
 
@@ -205,6 +252,11 @@
      * @param row row key
      * @param mutations list of mutation commands
      * @param timestamp timestamp
+     * 
+     * @param tableName
+     * @param row
+     * @param mutations
+     * @param timestamp
      */
     public void mutateRowTs(byte[] tableName, byte[] row, List<Mutation> mutations, long timestamp) throws IOError, IllegalArgument, TException;
 
@@ -216,6 +268,9 @@
      * 
      * @param tableName name of table
      * @param rowBatches list of row batches
+     * 
+     * @param tableName
+     * @param rowBatches
      */
     public void mutateRows(byte[] tableName, List<BatchMutation> rowBatches) throws IOError, IllegalArgument, TException;
 
@@ -228,6 +283,10 @@
      * @param tableName name of table
      * @param rowBatches list of row batches
      * @param timestamp timestamp
+     * 
+     * @param tableName
+     * @param rowBatches
+     * @param timestamp
      */
     public void mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp) throws IOError, IllegalArgument, TException;
 
@@ -237,6 +296,11 @@
      * @param row row to increment
      * @param column name of column
      * @param value amount to increment by
+     * 
+     * @param tableName
+     * @param row
+     * @param column
+     * @param value
      */
     public long atomicIncrement(byte[] tableName, byte[] row, byte[] column, long value) throws IOError, IllegalArgument, TException;
 
@@ -246,6 +310,10 @@
      * @param tableName name of table
      * @param row Row to update
      * @param column name of column whose value is to be deleted
+     * 
+     * @param tableName
+     * @param row
+     * @param column
      */
     public void deleteAll(byte[] tableName, byte[] row, byte[] column) throws IOError, TException;
 
@@ -257,6 +325,11 @@
      * @param row Row to update
      * @param column name of column whose value is to be deleted
      * @param timestamp timestamp
+     * 
+     * @param tableName
+     * @param row
+     * @param column
+     * @param timestamp
      */
     public void deleteAllTs(byte[] tableName, byte[] row, byte[] column, long timestamp) throws IOError, TException;
 
@@ -265,6 +338,9 @@
      * 
      * @param tableName name of table
      * @param row key of the row to be completely deleted.
+     * 
+     * @param tableName
+     * @param row
      */
     public void deleteAllRow(byte[] tableName, byte[] row) throws IOError, TException;
 
@@ -275,6 +351,10 @@
      * @param tableName name of table
      * @param row key of the row to be completely deleted.
      * @param timestamp timestamp
+     * 
+     * @param tableName
+     * @param row
+     * @param timestamp
      */
     public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp) throws IOError, TException;
 
@@ -290,6 +370,10 @@
      *                 start at the first row.
      * 
      * @return scanner id to be used with other scanner procedures
+     * 
+     * @param tableName
+     * @param startRow
+     * @param columns
      */
     public int scannerOpen(byte[] tableName, byte[] startRow, List<byte[]> columns) throws IOError, TException;
 
@@ -308,10 +392,30 @@
      *                scanner's results
      * 
      * @return scanner id to be used with other scanner procedures
+     * 
+     * @param tableName
+     * @param startRow
+     * @param stopRow
+     * @param columns
      */
     public int scannerOpenWithStop(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns) throws IOError, TException;
 
     /**
+     * Open a scanner for a given prefix.  That is all rows will have the specified
+     * prefix. No other rows will be returned.
+     * 
+     * @param tableName name of table
+     * @param startAndPrefix the prefix (and thus start row) of the keys you want
+     * @param columns the columns you want returned
+     * @return scanner id to use with other scanner calls
+     * 
+     * @param tableName
+     * @param startAndPrefix
+     * @param columns
+     */
+    public int scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws IOError, TException;
+
+    /**
      * Get a scanner on the current table starting at the specified row and
      * ending at the last row in the table.  Return the specified columns.
      * Only values with the specified timestamp are returned.
@@ -325,6 +429,11 @@
      * @param timestamp timestamp
      * 
      * @return scanner id to be used with other scanner procedures
+     * 
+     * @param tableName
+     * @param startRow
+     * @param columns
+     * @param timestamp
      */
     public int scannerOpenTs(byte[] tableName, byte[] startRow, List<byte[]> columns, long timestamp) throws IOError, TException;
 
@@ -345,6 +454,12 @@
      * @param timestamp timestamp
      * 
      * @return scanner id to be used with other scanner procedures
+     * 
+     * @param tableName
+     * @param startRow
+     * @param stopRow
+     * @param columns
+     * @param timestamp
      */
     public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow, byte[] stopRow, List<byte[]> columns, long timestamp) throws IOError, TException;
 
@@ -358,6 +473,8 @@
      * @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
+     * 
+     * @param id
      */
     public List<TRowResult> scannerGet(int id) throws IOError, IllegalArgument, TException;
 
@@ -372,6 +489,9 @@
      * @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
+     * 
+     * @param id
+     * @param nbRows
      */
     public List<TRowResult> scannerGetList(int id, int nbRows) throws IOError, IllegalArgument, TException;
 
@@ -380,6 +500,8 @@
      * 
      * @param id id of a scanner returned by scannerOpen
      * @throws IllegalArgument if ScannerID is invalid
+     * 
+     * @param id
      */
     public void scannerClose(int id) throws IOError, IllegalArgument, TException;
 
@@ -1440,6 +1562,44 @@
       throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpenWithStop failed: unknown result");
     }
 
+    public int scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws IOError, TException
+    {
+      send_scannerOpenWithPrefix(tableName, startAndPrefix, columns);
+      return recv_scannerOpenWithPrefix();
+    }
+
+    public void send_scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws TException
+    {
+      oprot_.writeMessageBegin(new TMessage("scannerOpenWithPrefix", TMessageType.CALL, seqid_));
+      scannerOpenWithPrefix_args args = new scannerOpenWithPrefix_args();
+      args.tableName = tableName;
+      args.startAndPrefix = startAndPrefix;
+      args.columns = columns;
+      args.write(oprot_);
+      oprot_.writeMessageEnd();
+      oprot_.getTransport().flush();
+    }
+
+    public int recv_scannerOpenWithPrefix() throws IOError, TException
+    {
+      TMessage msg = iprot_.readMessageBegin();
+      if (msg.type == TMessageType.EXCEPTION) {
+        TApplicationException x = TApplicationException.read(iprot_);
+        iprot_.readMessageEnd();
+        throw x;
+      }
+      scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result();
+      result.read(iprot_);
+      iprot_.readMessageEnd();
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      if (result.io != null) {
+        throw result.io;
+      }
+      throw new TApplicationException(TApplicationException.MISSING_RESULT, "scannerOpenWithPrefix failed: unknown result");
+    }
+
     public int scannerOpenTs(byte[] tableName, byte[] startRow, List<byte[]> columns, long timestamp) throws IOError, TException
     {
       send_scannerOpenTs(tableName, startRow, columns, timestamp);
@@ -1667,6 +1827,7 @@
       processMap_.put("deleteAllRowTs", new deleteAllRowTs());
       processMap_.put("scannerOpen", new scannerOpen());
       processMap_.put("scannerOpenWithStop", new scannerOpenWithStop());
+      processMap_.put("scannerOpenWithPrefix", new scannerOpenWithPrefix());
       processMap_.put("scannerOpenTs", new scannerOpenTs());
       processMap_.put("scannerOpenWithStopTs", new scannerOpenWithStopTs());
       processMap_.put("scannerGet", new scannerGet());
@@ -2277,6 +2438,27 @@
 
     }
 
+    private class scannerOpenWithPrefix implements ProcessFunction {
+      public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
+      {
+        scannerOpenWithPrefix_args args = new scannerOpenWithPrefix_args();
+        args.read(iprot);
+        iprot.readMessageEnd();
+        scannerOpenWithPrefix_result result = new scannerOpenWithPrefix_result();
+        try {
+          result.success = iface_.scannerOpenWithPrefix(args.tableName, args.startAndPrefix, args.columns);
+          result.__isset.success = true;
+        } catch (IOError io) {
+          result.io = io;
+        }
+        oprot.writeMessageBegin(new TMessage("scannerOpenWithPrefix", TMessageType.REPLY, seqid));
+        result.write(oprot);
+        oprot.writeMessageEnd();
+        oprot.getTransport().flush();
+      }
+
+    }
+
     private class scannerOpenTs implements ProcessFunction {
       public void process(int seqid, TProtocol iprot, TProtocol oprot) throws TException
       {
@@ -18802,69 +18984,60 @@
 
   }
 
-  public static class scannerOpenTs_args implements TBase, java.io.Serializable, Cloneable   {
-    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenTs_args");
+  public static class scannerOpenWithPrefix_args implements TBase, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenWithPrefix_args");
     private static final TField TABLE_NAME_FIELD_DESC = new TField("tableName", TType.STRING, (short)1);
-    private static final TField START_ROW_FIELD_DESC = new TField("startRow", TType.STRING, (short)2);
+    private static final TField START_AND_PREFIX_FIELD_DESC = new TField("startAndPrefix", TType.STRING, (short)2);
     private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.LIST, (short)3);
-    private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)4);
 
     public byte[] tableName;
     public static final int TABLENAME = 1;
-    public byte[] startRow;
-    public static final int STARTROW = 2;
+    public byte[] startAndPrefix;
+    public static final int STARTANDPREFIX = 2;
     public List<byte[]> columns;
     public static final int COLUMNS = 3;
-    public long timestamp;
-    public static final int TIMESTAMP = 4;
 
     private final Isset __isset = new Isset();
     private static final class Isset implements java.io.Serializable {
-      public boolean timestamp = false;
     }
 
     public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
       put(TABLENAME, new FieldMetaData("tableName", TFieldRequirementType.DEFAULT, 
           new FieldValueMetaData(TType.STRING)));
-      put(STARTROW, new FieldMetaData("startRow", TFieldRequirementType.DEFAULT, 
+      put(STARTANDPREFIX, new FieldMetaData("startAndPrefix", TFieldRequirementType.DEFAULT, 
           new FieldValueMetaData(TType.STRING)));
       put(COLUMNS, new FieldMetaData("columns", TFieldRequirementType.DEFAULT, 
           new ListMetaData(TType.LIST, 
               new FieldValueMetaData(TType.STRING))));
-      put(TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.DEFAULT, 
-          new FieldValueMetaData(TType.I64)));
     }});
 
     static {
-      FieldMetaData.addStructMetaDataMap(scannerOpenTs_args.class, metaDataMap);
+      FieldMetaData.addStructMetaDataMap(scannerOpenWithPrefix_args.class, metaDataMap);
     }
 
-    public scannerOpenTs_args() {
+    public scannerOpenWithPrefix_args() {
     }
 
-    public scannerOpenTs_args(
+    public scannerOpenWithPrefix_args(
       byte[] tableName,
-      byte[] startRow,
-      List<byte[]> columns,
-      long timestamp)
+      byte[] startAndPrefix,
+      List<byte[]> columns)
     {
       this();
       this.tableName = tableName;
-      this.startRow = startRow;
+      this.startAndPrefix = startAndPrefix;
       this.columns = columns;
-      this.timestamp = timestamp;
-      this.__isset.timestamp = true;
     }
 
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public scannerOpenTs_args(scannerOpenTs_args other) {
+    public scannerOpenWithPrefix_args(scannerOpenWithPrefix_args other) {
       if (other.isSetTableName()) {
         this.tableName = other.tableName;
       }
-      if (other.isSetStartRow()) {
-        this.startRow = other.startRow;
+      if (other.isSetStartAndPrefix()) {
+        this.startAndPrefix = other.startAndPrefix;
       }
       if (other.isSetColumns()) {
         List<byte[]> __this__columns = new ArrayList<byte[]>();
@@ -18873,13 +19046,11 @@
         }
         this.columns = __this__columns;
       }
-      __isset.timestamp = other.__isset.timestamp;
-      this.timestamp = other.timestamp;
     }
 
     @Override
-    public scannerOpenTs_args clone() {
-      return new scannerOpenTs_args(this);
+    public scannerOpenWithPrefix_args clone() {
+      return new scannerOpenWithPrefix_args(this);
     }
 
     public byte[] getTableName() {
@@ -18905,26 +19076,26 @@
       }
     }
 
-    public byte[] getStartRow() {
-      return this.startRow;
+    public byte[] getStartAndPrefix() {
+      return this.startAndPrefix;
     }
 
-    public void setStartRow(byte[] startRow) {
-      this.startRow = startRow;
+    public void setStartAndPrefix(byte[] startAndPrefix) {
+      this.startAndPrefix = startAndPrefix;
     }
 
-    public void unsetStartRow() {
-      this.startRow = null;
+    public void unsetStartAndPrefix() {
+      this.startAndPrefix = null;
     }
 
-    // Returns true if field startRow is set (has been asigned a value) and false otherwise
-    public boolean isSetStartRow() {
-      return this.startRow != null;
+    // Returns true if field startAndPrefix is set (has been asigned a value) and false otherwise
+    public boolean isSetStartAndPrefix() {
+      return this.startAndPrefix != null;
     }
 
-    public void setStartRowIsSet(boolean value) {
+    public void setStartAndPrefixIsSet(boolean value) {
       if (!value) {
-        this.startRow = null;
+        this.startAndPrefix = null;
       }
     }
 
@@ -18966,28 +19137,6 @@
       }
     }
 
-    public long getTimestamp() {
-      return this.timestamp;
-    }
-
-    public void setTimestamp(long timestamp) {
-      this.timestamp = timestamp;
-      this.__isset.timestamp = true;
-    }
-
-    public void unsetTimestamp() {
-      this.__isset.timestamp = false;
-    }
-
-    // Returns true if field timestamp is set (has been asigned a value) and false otherwise
-    public boolean isSetTimestamp() {
-      return this.__isset.timestamp;
-    }
-
-    public void setTimestampIsSet(boolean value) {
-      this.__isset.timestamp = value;
-    }
-
     public void setFieldValue(int fieldID, Object value) {
       switch (fieldID) {
       case TABLENAME:
@@ -18998,11 +19147,11 @@
         }
         break;
 
-      case STARTROW:
+      case STARTANDPREFIX:
         if (value == null) {
-          unsetStartRow();
+          unsetStartAndPrefix();
         } else {
-          setStartRow((byte[])value);
+          setStartAndPrefix((byte[])value);
         }
         break;
 
@@ -19014,14 +19163,6 @@
         }
         break;
 
-      case TIMESTAMP:
-        if (value == null) {
-          unsetTimestamp();
-        } else {
-          setTimestamp((Long)value);
-        }
-        break;
-
       default:
         throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
       }
@@ -19032,15 +19173,12 @@
       case TABLENAME:
         return getTableName();
 
-      case STARTROW:
-        return getStartRow();
+      case STARTANDPREFIX:
+        return getStartAndPrefix();
 
       case COLUMNS:
         return getColumns();
 
-      case TIMESTAMP:
-        return new Long(getTimestamp());
-
       default:
         throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
       }
@@ -19051,12 +19189,10 @@
       switch (fieldID) {
       case TABLENAME:
         return isSetTableName();
-      case STARTROW:
-        return isSetStartRow();
+      case STARTANDPREFIX:
+        return isSetStartAndPrefix();
       case COLUMNS:
         return isSetColumns();
-      case TIMESTAMP:
-        return isSetTimestamp();
       default:
         throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
       }
@@ -19066,12 +19202,12 @@
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof scannerOpenTs_args)
-        return this.equals((scannerOpenTs_args)that);
+      if (that instanceof scannerOpenWithPrefix_args)
+        return this.equals((scannerOpenWithPrefix_args)that);
       return false;
     }
 
-    public boolean equals(scannerOpenTs_args that) {
+    public boolean equals(scannerOpenWithPrefix_args that) {
       if (that == null)
         return false;
 
@@ -19084,12 +19220,12 @@
           return false;
       }
 
-      boolean this_present_startRow = true && this.isSetStartRow();
-      boolean that_present_startRow = true && that.isSetStartRow();
-      if (this_present_startRow || that_present_startRow) {
-        if (!(this_present_startRow && that_present_startRow))
+      boolean this_present_startAndPrefix = true && this.isSetStartAndPrefix();
+      boolean that_present_startAndPrefix = true && that.isSetStartAndPrefix();
+      if (this_present_startAndPrefix || that_present_startAndPrefix) {
+        if (!(this_present_startAndPrefix && that_present_startAndPrefix))
           return false;
-        if (!java.util.Arrays.equals(this.startRow, that.startRow))
+        if (!java.util.Arrays.equals(this.startAndPrefix, that.startAndPrefix))
           return false;
       }
 
@@ -19102,15 +19238,6 @@
           return false;
       }
 
-      boolean this_present_timestamp = true;
-      boolean that_present_timestamp = true;
-      if (this_present_timestamp || that_present_timestamp) {
-        if (!(this_present_timestamp && that_present_timestamp))
-          return false;
-        if (this.timestamp != that.timestamp)
-          return false;
-      }
-
       return true;
     }
 
@@ -19137,9 +19264,9 @@
               TProtocolUtil.skip(iprot, field.type);
             }
             break;
-          case STARTROW:
+          case STARTANDPREFIX:
             if (field.type == TType.STRING) {
-              this.startRow = iprot.readBinary();
+              this.startAndPrefix = iprot.readBinary();
             } else { 
               TProtocolUtil.skip(iprot, field.type);
             }
@@ -19161,14 +19288,6 @@
               TProtocolUtil.skip(iprot, field.type);
             }
             break;
-          case TIMESTAMP:
-            if (field.type == TType.I64) {
-              this.timestamp = iprot.readI64();
-              this.__isset.timestamp = true;
-            } else { 
-              TProtocolUtil.skip(iprot, field.type);
-            }
-            break;
           default:
             TProtocolUtil.skip(iprot, field.type);
             break;
@@ -19191,9 +19310,9 @@
         oprot.writeBinary(this.tableName);
         oprot.writeFieldEnd();
       }
-      if (this.startRow != null) {
-        oprot.writeFieldBegin(START_ROW_FIELD_DESC);
-        oprot.writeBinary(this.startRow);
+      if (this.startAndPrefix != null) {
+        oprot.writeFieldBegin(START_AND_PREFIX_FIELD_DESC);
+        oprot.writeBinary(this.startAndPrefix);
         oprot.writeFieldEnd();
       }
       if (this.columns != null) {
@@ -19207,16 +19326,13 @@
         }
         oprot.writeFieldEnd();
       }
-      oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
-      oprot.writeI64(this.timestamp);
-      oprot.writeFieldEnd();
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("scannerOpenTs_args(");
+      StringBuilder sb = new StringBuilder("scannerOpenWithPrefix_args(");
       boolean first = true;
 
       sb.append("tableName:");
@@ -19227,11 +19343,11 @@
       }
       first = false;
       if (!first) sb.append(", ");
-      sb.append("startRow:");
-      if (this.startRow == null) {
+      sb.append("startAndPrefix:");
+      if (this.startAndPrefix == null) {
         sb.append("null");
       } else {
-        sb.append(this.startRow);
+        sb.append(this.startAndPrefix);
       }
       first = false;
       if (!first) sb.append(", ");
@@ -19242,10 +19358,6 @@
         sb.append(this.columns);
       }
       first = false;
-      if (!first) sb.append(", ");
-      sb.append("timestamp:");
-      sb.append(this.timestamp);
-      first = false;
       sb.append(")");
       return sb.toString();
     }
@@ -19257,8 +19369,8 @@
 
   }
 
-  public static class scannerOpenTs_result implements TBase, java.io.Serializable, Cloneable   {
-    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenTs_result");
+  public static class scannerOpenWithPrefix_result implements TBase, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenWithPrefix_result");
     private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.I32, (short)0);
     private static final TField IO_FIELD_DESC = new TField("io", TType.STRUCT, (short)1);
 
@@ -19280,13 +19392,13 @@
     }});
 
     static {
-      FieldMetaData.addStructMetaDataMap(scannerOpenTs_result.class, metaDataMap);
+      FieldMetaData.addStructMetaDataMap(scannerOpenWithPrefix_result.class, metaDataMap);
     }
 
-    public scannerOpenTs_result() {
+    public scannerOpenWithPrefix_result() {
     }
 
-    public scannerOpenTs_result(
+    public scannerOpenWithPrefix_result(
       int success,
       IOError io)
     {
@@ -19299,7 +19411,7 @@
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public scannerOpenTs_result(scannerOpenTs_result other) {
+    public scannerOpenWithPrefix_result(scannerOpenWithPrefix_result other) {
       __isset.success = other.__isset.success;
       this.success = other.success;
       if (other.isSetIo()) {
@@ -19308,8 +19420,8 @@
     }
 
     @Override
-    public scannerOpenTs_result clone() {
-      return new scannerOpenTs_result(this);
+    public scannerOpenWithPrefix_result clone() {
+      return new scannerOpenWithPrefix_result(this);
     }
 
     public int getSuccess() {
@@ -19409,12 +19521,12 @@
     public boolean equals(Object that) {
       if (that == null)
         return false;
-      if (that instanceof scannerOpenTs_result)
-        return this.equals((scannerOpenTs_result)that);
+      if (that instanceof scannerOpenWithPrefix_result)
+        return this.equals((scannerOpenWithPrefix_result)that);
       return false;
     }
 
-    public boolean equals(scannerOpenTs_result that) {
+    public boolean equals(scannerOpenWithPrefix_result that) {
       if (that == null)
         return false;
 
@@ -19502,7 +19614,7 @@
 
     @Override
     public String toString() {
-      StringBuilder sb = new StringBuilder("scannerOpenTs_result(");
+      StringBuilder sb = new StringBuilder("scannerOpenWithPrefix_result(");
       boolean first = true;
 
       sb.append("success:");
@@ -19527,24 +19639,21 @@
 
   }
 
-  public static class scannerOpenWithStopTs_args implements TBase, java.io.Serializable, Cloneable   {
-    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenWithStopTs_args");
+  public static class scannerOpenTs_args implements TBase, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenTs_args");
     private static final TField TABLE_NAME_FIELD_DESC = new TField("tableName", TType.STRING, (short)1);
     private static final TField START_ROW_FIELD_DESC = new TField("startRow", TType.STRING, (short)2);
-    private static final TField STOP_ROW_FIELD_DESC = new TField("stopRow", TType.STRING, (short)3);
-    private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.LIST, (short)4);
-    private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)5);
+    private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.LIST, (short)3);
+    private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)4);
 
     public byte[] tableName;
     public static final int TABLENAME = 1;
     public byte[] startRow;
     public static final int STARTROW = 2;
-    public byte[] stopRow;
-    public static final int STOPROW = 3;
     public List<byte[]> columns;
-    public static final int COLUMNS = 4;
+    public static final int COLUMNS = 3;
     public long timestamp;
-    public static final int TIMESTAMP = 5;
+    public static final int TIMESTAMP = 4;
 
     private final Isset __isset = new Isset();
     private static final class Isset implements java.io.Serializable {
@@ -19556,8 +19665,6 @@
           new FieldValueMetaData(TType.STRING)));
       put(STARTROW, new FieldMetaData("startRow", TFieldRequirementType.DEFAULT, 
           new FieldValueMetaData(TType.STRING)));
-      put(STOPROW, new FieldMetaData("stopRow", TFieldRequirementType.DEFAULT, 
-          new FieldValueMetaData(TType.STRING)));
       put(COLUMNS, new FieldMetaData("columns", TFieldRequirementType.DEFAULT, 
           new ListMetaData(TType.LIST, 
               new FieldValueMetaData(TType.STRING))));
@@ -19566,23 +19673,21 @@
     }});
 
     static {
-      FieldMetaData.addStructMetaDataMap(scannerOpenWithStopTs_args.class, metaDataMap);
+      FieldMetaData.addStructMetaDataMap(scannerOpenTs_args.class, metaDataMap);
     }
 
-    public scannerOpenWithStopTs_args() {
+    public scannerOpenTs_args() {
     }
 
-    public scannerOpenWithStopTs_args(
+    public scannerOpenTs_args(
       byte[] tableName,
       byte[] startRow,
-      byte[] stopRow,
       List<byte[]> columns,
       long timestamp)
     {
       this();
       this.tableName = tableName;
       this.startRow = startRow;
-      this.stopRow = stopRow;
       this.columns = columns;
       this.timestamp = timestamp;
       this.__isset.timestamp = true;
@@ -19591,16 +19696,13 @@
     /**
      * Performs a deep copy on <i>other</i>.
      */
-    public scannerOpenWithStopTs_args(scannerOpenWithStopTs_args other) {
+    public scannerOpenTs_args(scannerOpenTs_args other) {
       if (other.isSetTableName()) {
         this.tableName = other.tableName;
       }
       if (other.isSetStartRow()) {
         this.startRow = other.startRow;
       }
-      if (other.isSetStopRow()) {
-        this.stopRow = other.stopRow;
-      }
       if (other.isSetColumns()) {
         List<byte[]> __this__columns = new ArrayList<byte[]>();
         for (byte[] other_element : other.columns) {
@@ -19613,8 +19715,8 @@
     }
 
     @Override
-    public scannerOpenWithStopTs_args clone() {
-      return new scannerOpenWithStopTs_args(this);
+    public scannerOpenTs_args clone() {
+      return new scannerOpenTs_args(this);
     }
 
     public byte[] getTableName() {
@@ -19663,8 +19765,743 @@
       }
     }
 
-    public byte[] getStopRow() {
-      return this.stopRow;
+    public int getColumnsSize() {
+      return (this.columns == null) ? 0 : this.columns.size();
+    }
+
+    public java.util.Iterator<byte[]> getColumnsIterator() {
+      return (this.columns == null) ? null : this.columns.iterator();
+    }
+
+    public void addToColumns(byte[] elem) {
+      if (this.columns == null) {
+        this.columns = new ArrayList<byte[]>();
+      }
+      this.columns.add(elem);
+    }
+
+    public List<byte[]> getColumns() {
+      return this.columns;
+    }
+
+    public void setColumns(List<byte[]> columns) {
+      this.columns = columns;
+    }
+
+    public void unsetColumns() {
+      this.columns = null;
+    }
+
+    // Returns true if field columns is set (has been asigned a value) and false otherwise
+    public boolean isSetColumns() {
+      return this.columns != null;
+    }
+
+    public void setColumnsIsSet(boolean value) {
+      if (!value) {
+        this.columns = null;
+      }
+    }
+
+    public long getTimestamp() {
+      return this.timestamp;
+    }
+
+    public void setTimestamp(long timestamp) {
+      this.timestamp = timestamp;
+      this.__isset.timestamp = true;
+    }
+
+    public void unsetTimestamp() {
+      this.__isset.timestamp = false;
+    }
+
+    // Returns true if field timestamp is set (has been asigned a value) and false otherwise
+    public boolean isSetTimestamp() {
+      return this.__isset.timestamp;
+    }
+
+    public void setTimestampIsSet(boolean value) {
+      this.__isset.timestamp = value;
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      switch (fieldID) {
+      case TABLENAME:
+        if (value == null) {
+          unsetTableName();
+        } else {
+          setTableName((byte[])value);
+        }
+        break;
+
+      case STARTROW:
+        if (value == null) {
+          unsetStartRow();
+        } else {
+          setStartRow((byte[])value);
+        }
+        break;
+
+      case COLUMNS:
+        if (value == null) {
+          unsetColumns();
+        } else {
+          setColumns((List<byte[]>)value);
+        }
+        break;
+
+      case TIMESTAMP:
+        if (value == null) {
+          unsetTimestamp();
+        } else {
+          setTimestamp((Long)value);
+        }
+        break;
+
+      default:
+        throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+      }
+    }
+
+    public Object getFieldValue(int fieldID) {
+      switch (fieldID) {
+      case TABLENAME:
+        return getTableName();
+
+      case STARTROW:
+        return getStartRow();
+
+      case COLUMNS:
+        return getColumns();
+
+      case TIMESTAMP:
+        return new Long(getTimestamp());
+
+      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 TABLENAME:
+        return isSetTableName();
+      case STARTROW:
+        return isSetStartRow();
+      case COLUMNS:
+        return isSetColumns();
+      case TIMESTAMP:
+        return isSetTimestamp();
+      default:
+        throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+      }
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof scannerOpenTs_args)
+        return this.equals((scannerOpenTs_args)that);
+      return false;
+    }
+
+    public boolean equals(scannerOpenTs_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tableName = true && this.isSetTableName();
+      boolean that_present_tableName = true && that.isSetTableName();
+      if (this_present_tableName || that_present_tableName) {
+        if (!(this_present_tableName && that_present_tableName))
+          return false;
+        if (!java.util.Arrays.equals(this.tableName, that.tableName))
+          return false;
+      }
+
+      boolean this_present_startRow = true && this.isSetStartRow();
+      boolean that_present_startRow = true && that.isSetStartRow();
+      if (this_present_startRow || that_present_startRow) {
+        if (!(this_present_startRow && that_present_startRow))
+          return false;
+        if (!java.util.Arrays.equals(this.startRow, that.startRow))
+          return false;
+      }
+
+      boolean this_present_columns = true && this.isSetColumns();
+      boolean that_present_columns = true && that.isSetColumns();
+      if (this_present_columns || that_present_columns) {
+        if (!(this_present_columns && that_present_columns))
+          return false;
+        if (!this.columns.equals(that.columns))
+          return false;
+      }
+
+      boolean this_present_timestamp = true;
+      boolean that_present_timestamp = true;
+      if (this_present_timestamp || that_present_timestamp) {
+        if (!(this_present_timestamp && that_present_timestamp))
+          return false;
+        if (this.timestamp != that.timestamp)
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case TABLENAME:
+            if (field.type == TType.STRING) {
+              this.tableName = iprot.readBinary();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case STARTROW:
+            if (field.type == TType.STRING) {
+              this.startRow = iprot.readBinary();
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case COLUMNS:
+            if (field.type == TType.LIST) {
+              {
+                TList _list90 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list90.size);
+                for (int _i91 = 0; _i91 < _list90.size; ++_i91)
+                {
+                  byte[] _elem92;
+                  _elem92 = iprot.readBinary();
+                  this.columns.add(_elem92);
+                }
+                iprot.readListEnd();
+              }
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case TIMESTAMP:
+            if (field.type == TType.I64) {
+              this.timestamp = iprot.readI64();
+              this.__isset.timestamp = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        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 {
+      validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (this.tableName != null) {
+        oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC);
+        oprot.writeBinary(this.tableName);
+        oprot.writeFieldEnd();
+      }
+      if (this.startRow != null) {
+        oprot.writeFieldBegin(START_ROW_FIELD_DESC);
+        oprot.writeBinary(this.startRow);
+        oprot.writeFieldEnd();
+      }
+      if (this.columns != null) {
+        oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
+        {
+          oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
+          for (byte[] _iter93 : this.columns)          {
+            oprot.writeBinary(_iter93);
+          }
+          oprot.writeListEnd();
+        }
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldBegin(TIMESTAMP_FIELD_DESC);
+      oprot.writeI64(this.timestamp);
+      oprot.writeFieldEnd();
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpenTs_args(");
+      boolean first = true;
+
+      sb.append("tableName:");
+      if (this.tableName == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tableName);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("startRow:");
+      if (this.startRow == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.startRow);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("columns:");
+      if (this.columns == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.columns);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("timestamp:");
+      sb.append(this.timestamp);
+      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
+    }
+
+  }
+
+  public static class scannerOpenTs_result implements TBase, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenTs_result");
+    private static final TField SUCCESS_FIELD_DESC = new TField("success", TType.I32, (short)0);
+    private static final TField IO_FIELD_DESC = new TField("io", TType.STRUCT, (short)1);
+
+    public int success;
+    public static final int SUCCESS = 0;
+    public IOError io;
+    public static final int IO = 1;
+
+    private final Isset __isset = new Isset();
+    private static final class Isset implements java.io.Serializable {
+      public boolean success = false;
+    }
+
+    public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
+      put(SUCCESS, new FieldMetaData("success", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.I32)));
+      put(IO, new FieldMetaData("io", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRUCT)));
+    }});
+
+    static {
+      FieldMetaData.addStructMetaDataMap(scannerOpenTs_result.class, metaDataMap);
+    }
+
+    public scannerOpenTs_result() {
+    }
+
+    public scannerOpenTs_result(
+      int success,
+      IOError io)
+    {
+      this();
+      this.success = success;
+      this.__isset.success = true;
+      this.io = io;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public scannerOpenTs_result(scannerOpenTs_result other) {
+      __isset.success = other.__isset.success;
+      this.success = other.success;
+      if (other.isSetIo()) {
+        this.io = new IOError(other.io);
+      }
+    }
+
+    @Override
+    public scannerOpenTs_result clone() {
+      return new scannerOpenTs_result(this);
+    }
+
+    public int getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(int success) {
+      this.success = success;
+      this.__isset.success = true;
+    }
+
+    public void unsetSuccess() {
+      this.__isset.success = false;
+    }
+
+    // Returns true if field success is set (has been asigned a value) and false otherwise
+    public boolean isSetSuccess() {
+      return this.__isset.success;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      this.__isset.success = value;
+    }
+
+    public IOError getIo() {
+      return this.io;
+    }
+
+    public void setIo(IOError io) {
+      this.io = io;
+    }
+
+    public void unsetIo() {
+      this.io = null;
+    }
+
+    // Returns true if field io is set (has been asigned a value) and false otherwise
+    public boolean isSetIo() {
+      return this.io != null;
+    }
+
+    public void setIoIsSet(boolean value) {
+      if (!value) {
+        this.io = null;
+      }
+    }
+
+    public void setFieldValue(int fieldID, Object value) {
+      switch (fieldID) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((Integer)value);
+        }
+        break;
+
+      case IO:
+        if (value == null) {
+          unsetIo();
+        } else {
+          setIo((IOError)value);
+        }
+        break;
+
+      default:
+        throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+      }
+    }
+
+    public Object getFieldValue(int fieldID) {
+      switch (fieldID) {
+      case SUCCESS:
+        return new Integer(getSuccess());
+
+      case IO:
+        return getIo();
+
+      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 SUCCESS:
+        return isSetSuccess();
+      case IO:
+        return isSetIo();
+      default:
+        throw new IllegalArgumentException("Field " + fieldID + " doesn't exist!");
+      }
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof scannerOpenTs_result)
+        return this.equals((scannerOpenTs_result)that);
+      return false;
+    }
+
+    public boolean equals(scannerOpenTs_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true;
+      boolean that_present_success = true;
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (this.success != that.success)
+          return false;
+      }
+
+      boolean this_present_io = true && this.isSetIo();
+      boolean that_present_io = true && that.isSetIo();
+      if (this_present_io || that_present_io) {
+        if (!(this_present_io && that_present_io))
+          return false;
+        if (!this.io.equals(that.io))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    public void read(TProtocol iprot) throws TException {
+      TField field;
+      iprot.readStructBegin();
+      while (true)
+      {
+        field = iprot.readFieldBegin();
+        if (field.type == TType.STOP) { 
+          break;
+        }
+        switch (field.id)
+        {
+          case SUCCESS:
+            if (field.type == TType.I32) {
+              this.success = iprot.readI32();
+              this.__isset.success = true;
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          case IO:
+            if (field.type == TType.STRUCT) {
+              this.io = new IOError();
+              this.io.read(iprot);
+            } else { 
+              TProtocolUtil.skip(iprot, field.type);
+            }
+            break;
+          default:
+            TProtocolUtil.skip(iprot, field.type);
+            break;
+        }
+        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 {
+      oprot.writeStructBegin(STRUCT_DESC);
+
+      if (this.isSetSuccess()) {
+        oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+        oprot.writeI32(this.success);
+        oprot.writeFieldEnd();
+      } else if (this.isSetIo()) {
+        oprot.writeFieldBegin(IO_FIELD_DESC);
+        this.io.write(oprot);
+        oprot.writeFieldEnd();
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("scannerOpenTs_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      sb.append(this.success);
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("io:");
+      if (this.io == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.io);
+      }
+      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
+    }
+
+  }
+
+  public static class scannerOpenWithStopTs_args implements TBase, java.io.Serializable, Cloneable   {
+    private static final TStruct STRUCT_DESC = new TStruct("scannerOpenWithStopTs_args");
+    private static final TField TABLE_NAME_FIELD_DESC = new TField("tableName", TType.STRING, (short)1);
+    private static final TField START_ROW_FIELD_DESC = new TField("startRow", TType.STRING, (short)2);
+    private static final TField STOP_ROW_FIELD_DESC = new TField("stopRow", TType.STRING, (short)3);
+    private static final TField COLUMNS_FIELD_DESC = new TField("columns", TType.LIST, (short)4);
+    private static final TField TIMESTAMP_FIELD_DESC = new TField("timestamp", TType.I64, (short)5);
+
+    public byte[] tableName;
+    public static final int TABLENAME = 1;
+    public byte[] startRow;
+    public static final int STARTROW = 2;
+    public byte[] stopRow;
+    public static final int STOPROW = 3;
+    public List<byte[]> columns;
+    public static final int COLUMNS = 4;
+    public long timestamp;
+    public static final int TIMESTAMP = 5;
+
+    private final Isset __isset = new Isset();
+    private static final class Isset implements java.io.Serializable {
+      public boolean timestamp = false;
+    }
+
+    public static final Map<Integer, FieldMetaData> metaDataMap = Collections.unmodifiableMap(new HashMap<Integer, FieldMetaData>() {{
+      put(TABLENAME, new FieldMetaData("tableName", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      put(STARTROW, new FieldMetaData("startRow", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      put(STOPROW, new FieldMetaData("stopRow", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.STRING)));
+      put(COLUMNS, new FieldMetaData("columns", TFieldRequirementType.DEFAULT, 
+          new ListMetaData(TType.LIST, 
+              new FieldValueMetaData(TType.STRING))));
+      put(TIMESTAMP, new FieldMetaData("timestamp", TFieldRequirementType.DEFAULT, 
+          new FieldValueMetaData(TType.I64)));
+    }});
+
+    static {
+      FieldMetaData.addStructMetaDataMap(scannerOpenWithStopTs_args.class, metaDataMap);
+    }
+
+    public scannerOpenWithStopTs_args() {
+    }
+
+    public scannerOpenWithStopTs_args(
+      byte[] tableName,
+      byte[] startRow,
+      byte[] stopRow,
+      List<byte[]> columns,
+      long timestamp)
+    {
+      this();
+      this.tableName = tableName;
+      this.startRow = startRow;
+      this.stopRow = stopRow;
+      this.columns = columns;
+      this.timestamp = timestamp;
+      this.__isset.timestamp = true;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public scannerOpenWithStopTs_args(scannerOpenWithStopTs_args other) {
+      if (other.isSetTableName()) {
+        this.tableName = other.tableName;
+      }
+      if (other.isSetStartRow()) {
+        this.startRow = other.startRow;
+      }
+      if (other.isSetStopRow()) {
+        this.stopRow = other.stopRow;
+      }
+      if (other.isSetColumns()) {
+        List<byte[]> __this__columns = new ArrayList<byte[]>();
+        for (byte[] other_element : other.columns) {
+          __this__columns.add(other_element);
+        }
+        this.columns = __this__columns;
+      }
+      __isset.timestamp = other.__isset.timestamp;
+      this.timestamp = other.timestamp;
+    }
+
+    @Override
+    public scannerOpenWithStopTs_args clone() {
+      return new scannerOpenWithStopTs_args(this);
+    }
+
+    public byte[] getTableName() {
+      return this.tableName;
+    }
+
+    public void setTableName(byte[] tableName) {
+      this.tableName = tableName;
+    }
+
+    public void unsetTableName() {
+      this.tableName = null;
+    }
+
+    // Returns true if field tableName is set (has been asigned a value) and false otherwise
+    public boolean isSetTableName() {
+      return this.tableName != null;
+    }
+
+    public void setTableNameIsSet(boolean value) {
+      if (!value) {
+        this.tableName = null;
+      }
+    }
+
+    public byte[] getStartRow() {
+      return this.startRow;
+    }
+
+    public void setStartRow(byte[] startRow) {
+      this.startRow = startRow;
+    }
+
+    public void unsetStartRow() {
+      this.startRow = null;
+    }
+
+    // Returns true if field startRow is set (has been asigned a value) and false otherwise
+    public boolean isSetStartRow() {
+      return this.startRow != null;
+    }
+
+    public void setStartRowIsSet(boolean value) {
+      if (!value) {
+        this.startRow = null;
+      }
+    }
+
+    public byte[] getStopRow() {
+      return this.stopRow;
     }
 
     public void setStopRow(byte[] stopRow) {
@@ -19934,13 +20771,13 @@
           case COLUMNS:
             if (field.type == TType.LIST) {
               {
-                TList _list90 = iprot.readListBegin();
-                this.columns = new ArrayList<byte[]>(_list90.size);
-                for (int _i91 = 0; _i91 < _list90.size; ++_i91)
+                TList _list94 = iprot.readListBegin();
+                this.columns = new ArrayList<byte[]>(_list94.size);
+                for (int _i95 = 0; _i95 < _list94.size; ++_i95)
                 {
-                  byte[] _elem92;
-                  _elem92 = iprot.readBinary();
-                  this.columns.add(_elem92);
+                  byte[] _elem96;
+                  _elem96 = iprot.readBinary();
+                  this.columns.add(_elem96);
                 }
                 iprot.readListEnd();
               }
@@ -19992,8 +20829,8 @@
         oprot.writeFieldBegin(COLUMNS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRING, this.columns.size()));
-          for (byte[] _iter93 : this.columns)          {
-            oprot.writeBinary(_iter93);
+          for (byte[] _iter97 : this.columns)          {
+            oprot.writeBinary(_iter97);
           }
           oprot.writeListEnd();
         }
@@ -20797,14 +21634,14 @@
           case SUCCESS:
             if (field.type == TType.LIST) {
               {
-                TList _list94 = iprot.readListBegin();
-                this.success = new ArrayList<TRowResult>(_list94.size);
-                for (int _i95 = 0; _i95 < _list94.size; ++_i95)
+                TList _list98 = iprot.readListBegin();
+                this.success = new ArrayList<TRowResult>(_list98.size);
+                for (int _i99 = 0; _i99 < _list98.size; ++_i99)
                 {
-                  TRowResult _elem96;
-                  _elem96 = new TRowResult();
-                  _elem96.read(iprot);
-                  this.success.add(_elem96);
+                  TRowResult _elem100;
+                  _elem100 = new TRowResult();
+                  _elem100.read(iprot);
+                  this.success.add(_elem100);
                 }
                 iprot.readListEnd();
               }
@@ -20848,8 +21685,8 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-          for (TRowResult _iter97 : this.success)          {
-            _iter97.write(oprot);
+          for (TRowResult _iter101 : this.success)          {
+            _iter101.write(oprot);
           }
           oprot.writeListEnd();
         }
@@ -21446,14 +22283,14 @@
           case SUCCESS:
             if (field.type == TType.LIST) {
               {
-                TList _list98 = iprot.readListBegin();
-                this.success = new ArrayList<TRowResult>(_list98.size);
-                for (int _i99 = 0; _i99 < _list98.size; ++_i99)
+                TList _list102 = iprot.readListBegin();
+                this.success = new ArrayList<TRowResult>(_list102.size);
+                for (int _i103 = 0; _i103 < _list102.size; ++_i103)
                 {
-                  TRowResult _elem100;
-                  _elem100 = new TRowResult();
-                  _elem100.read(iprot);
-                  this.success.add(_elem100);
+                  TRowResult _elem104;
+                  _elem104 = new TRowResult();
+                  _elem104.read(iprot);
+                  this.success.add(_elem104);
                 }
                 iprot.readListEnd();
               }
@@ -21497,8 +22334,8 @@
         oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
         {
           oprot.writeListBegin(new TList(TType.STRUCT, this.success.size()));
-          for (TRowResult _iter101 : this.success)          {
-            _iter101.write(oprot);
+          for (TRowResult _iter105 : this.success)          {
+            _iter105.write(oprot);
           }
           oprot.writeListEnd();
         }

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IOError.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/IllegalArgument.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/Mutation.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TCell.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TCell.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TCell.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TCell.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRegionInfo.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java (original)
+++ hadoop/hbase/trunk/src/java/org/apache/hadoop/hbase/thrift/generated/TRowResult.java Mon Jul 27 23:48:22 2009
@@ -1,4 +1,6 @@
-/**
+/*
+ * Copyright 2009 The Apache Software Foundation
+ *
  * 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
@@ -15,11 +17,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-/**
- * Autogenerated by Thrift
- *
- * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
- */
 package org.apache.hadoop.hbase.thrift.generated;
 
 import java.util.List;

Modified: hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=798334&r1=798333&r2=798334&view=diff
==============================================================================
--- hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original)
+++ hadoop/hbase/trunk/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java Mon Jul 27 23:48:22 2009
@@ -322,7 +322,7 @@
 
     // A slightly customized ColumnDescriptor (only 2 versions)
     ColumnDescriptor cDescB = new ColumnDescriptor(columnBname, 2, "NONE", 
-        false, 2147483647, "NONE", 0, 0, false, -1);
+        false, "NONE", 0, 0, false, -1);
     cDescriptors.add(cDescB);
 
     return cDescriptors;