You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2010/05/07 21:17:55 UTC

svn commit: r942184 [3/15] - in /hadoop/hbase/branches/0.20: ./ src/java/org/apache/hadoop/hbase/ src/java/org/apache/hadoop/hbase/client/ src/java/org/apache/hadoop/hbase/filter/ src/java/org/apache/hadoop/hbase/io/ src/java/org/apache/hadoop/hbase/io...

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTable.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTable.java Fri May  7 19:17:48 2010
@@ -62,15 +62,15 @@ import org.apache.hadoop.hbase.util.Writ
 /**
  * Used to communicate with a single HBase table.
  * This class is not thread safe. Use one instance per thread.
- * 
- * Puts, deletes, checkAndPut and incrementColumnValue are 
- * done in an exclusive (and thus serial) fashion for each row. 
+ *
+ * Puts, deletes, checkAndPut and incrementColumnValue are
+ * done in an exclusive (and thus serial) fashion for each row.
  * These calls acquire a row lock which is shared with the lockRow
- * calls. 
- * 
- * Gets and Scans will not return half written data. That is, 
+ * calls.
+ *
+ * Gets and Scans will not return half written data. That is,
  * all mutation operations are atomic on a row basis with
- * respect to other concurrent readers and writers. 
+ * respect to other concurrent readers and writers.
  */
 public class HTable {
   private final HConnection connection;
@@ -85,7 +85,7 @@ public class HTable {
   private int maxKeyValueSize;
 
   private long maxScannerResultSize;
-  
+
   /**
    * Creates an object to access a HBase table.
    *
@@ -110,7 +110,7 @@ public class HTable {
 
   /**
    * Creates an object to access a HBase table.
-   * 
+   *
    * @param conf Configuration object to use.
    * @param tableName Name of the table.
    * @throws IOException if a remote or network exception occurs
@@ -122,7 +122,7 @@ public class HTable {
 
   /**
    * Creates an object to access a HBase table.
-   * 
+   *
    * @param conf Configuration object to use.
    * @param tableName Name of the table.
    * @throws IOException if a remote or network exception occurs
@@ -144,9 +144,9 @@ public class HTable {
     this.autoFlush = true;
     this.currentWriteBufferSize = 0;
     this.scannerCaching = conf.getInt("hbase.client.scanner.caching", 1);
-    
+
     this.maxScannerResultSize = conf.getLong(
-      HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, 
+      HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
       HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
     this.maxKeyValueSize = conf.getInt("hbase.client.keyvalue.maxsize", -1);
 
@@ -197,7 +197,7 @@ public class HTable {
   public static boolean isTableEnabled(byte[] tableName) throws IOException {
     return isTableEnabled(new HBaseConfiguration(), tableName);
   }
-  
+
   /**
    * Tells whether or not a table is enabled or not.
    * @param conf The Configuration object to use.
@@ -262,7 +262,7 @@ public class HTable {
   public HConnection getConnection() {
     return this.connection;
   }
-  
+
   /**
    * Gets the number of rows that a scanner will fetch at once.
    * <p>
@@ -332,7 +332,7 @@ public class HTable {
     MetaScannerVisitor visitor = new MetaScannerVisitor() {
       public boolean processRow(Result rowResult) throws IOException {
         HRegionInfo info = Writables.getHRegionInfo(
-            rowResult.getValue(HConstants.CATALOG_FAMILY, 
+            rowResult.getValue(HConstants.CATALOG_FAMILY,
                 HConstants.REGIONINFO_QUALIFIER));
         if (Bytes.equals(info.getTableDesc().getName(), getTableName())) {
           if (!(info.isOffline() || info.isSplit())) {
@@ -362,21 +362,21 @@ public class HTable {
     MetaScannerVisitor visitor = new MetaScannerVisitor() {
       public boolean processRow(Result rowResult) throws IOException {
         HRegionInfo info = Writables.getHRegionInfo(
-            rowResult.getValue(HConstants.CATALOG_FAMILY, 
+            rowResult.getValue(HConstants.CATALOG_FAMILY,
                 HConstants.REGIONINFO_QUALIFIER));
-        
+
         if (!(Bytes.equals(info.getTableDesc().getName(), getTableName()))) {
           return false;
         }
 
         HServerAddress server = new HServerAddress();
-        byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY, 
+        byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
             HConstants.SERVER_QUALIFIER);
         if (value != null && value.length > 0) {
           String address = Bytes.toString(value);
           server = new HServerAddress(address);
         }
-        
+
         if (!(info.isOffline() || info.isSplit())) {
           regionMap.put(new UnmodifyableHRegionInfo(info), server);
         }
@@ -389,9 +389,9 @@ public class HTable {
   }
 
   /**
-   * Return the row that matches <i>row</i> exactly, 
+   * Return the row that matches <i>row</i> exactly,
    * or the one that immediately precedes it.
-   * 
+   *
    * @param row A row key.
    * @param family Column family to include in the {@link Result}.
    * @throws IOException if a remote or network exception occurs.
@@ -409,9 +409,9 @@ public class HTable {
    }
 
   /**
-  * Return the row that matches <i>row</i> exactly, 
+  * Return the row that matches <i>row</i> exactly,
   * or the one that immediately preceeds it.
-  * 
+  *
   * @param row row key
   * @param family Column family to look for row in.
   * @return map of values
@@ -426,7 +426,7 @@ public class HTable {
     return r == null || r.isEmpty()? null: r.getRowResult();
   }
 
-  /** 
+  /**
    * Returns a scanner on the current table as specified by the {@link Scan}
    * object.
    *
@@ -443,7 +443,7 @@ public class HTable {
 
   /**
    * Gets a scanner on the current table for the given family.
-   * 
+   *
    * @param family  The column family to scan.
    * @return A scanner.
    * @throws IOException if a remote or network exception occurs.
@@ -454,10 +454,10 @@ public class HTable {
     scan.addFamily(family);
     return getScanner(scan);
   }
-  
+
   /**
    * Gets a scanner on the current table for the given family and qualifier.
-   * 
+   *
    * @param family  The column family to scan.
    * @param qualifier  The column qualifier to scan.
    * @return A scanner.
@@ -489,10 +489,10 @@ public class HTable {
         }
     );
   }
-  
+
   /**
    * Deletes the specified cells/row.
-   * 
+   *
    * @param delete The object that specifies what to delete.
    * @throws IOException if a remote or network exception occurs.
    * @since 0.20.0
@@ -508,7 +508,7 @@ public class HTable {
         }
     );
   }
-  
+
   /**
    * Deletes the specified cells/rows in bulk.
    * @param deletes List of things to delete.  List gets modified by this
@@ -547,7 +547,7 @@ public class HTable {
       flushCommits();
     }
   }
-  
+
   /**
    * Puts some data in the table, in batch.
    * <p>
@@ -572,7 +572,7 @@ public class HTable {
       flushCommits();
     }
   }
-  
+
   /**
    * Atomically increments a column value.
    * <p>
@@ -587,7 +587,7 @@ public class HTable {
    * @return The new value, post increment.
    * @throws IOException if a remote or network exception occurs.
    */
-  public long incrementColumnValue(final byte [] row, final byte [] family, 
+  public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount)
   throws IOException {
     return incrementColumnValue(row, family, qualifier, amount, true);
@@ -598,7 +598,7 @@ public class HTable {
    * and is not a big-endian long, this could throw an exception. If the column
    * value does not yet exist it is initialized to <code>amount</code> and
    * written to the specified column.
-   * 
+   *
    * <p>Setting writeToWAL to false means that in a fail scenario, you will lose
    * any increments that have not been flushed.
    * @param row The row that contains the cell to increment.
@@ -616,7 +616,7 @@ public class HTable {
    * @return The new value, post increment.
    * @throws IOException if a remote or network exception occurs.
    */
-  public long incrementColumnValue(final byte [] row, final byte [] family, 
+  public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount, final boolean writeToWAL)
   throws IOException {
     NullPointerException npe = null;
@@ -634,7 +634,7 @@ public class HTable {
         new ServerCallable<Long>(connection, tableName, row) {
           public Long call() throws IOException {
             return server.incrementColumnValue(
-                location.getRegionInfo().getRegionName(), row, family, 
+                location.getRegionInfo().getRegionName(), row, family,
                 qualifier, amount, writeToWAL);
           }
         }
@@ -644,7 +644,7 @@ public class HTable {
   /**
    * Atomically checks if a row/family/qualifier value match the expectedValue.
    * If it does, it adds the put.
-   * 
+   *
    * @param row
    * @param family
    * @param qualifier
@@ -653,8 +653,8 @@ public class HTable {
    * @throws IOException
    * @return true if the new put was execute, false otherwise
    */
-  public boolean checkAndPut(final byte [] row, 
-      final byte [] family, final byte [] qualifier, final byte [] value, 
+  public boolean checkAndPut(final byte [] row,
+      final byte [] family, final byte [] qualifier, final byte [] value,
       final Put put)
   throws IOException {
     return connection.getRegionServerWithRetries(
@@ -666,12 +666,12 @@ public class HTable {
         }
       ).booleanValue();
   }
-  
+
   /**
    * Test for the existence of columns in the table, as specified in the Get.<p>
-   * 
+   *
    * This will return true if the Get matches one or more keys, false if not.<p>
-   * 
+   *
    * This is a server-side call so it prevents any data from being transfered
    * to the client.
    * @param get
@@ -688,7 +688,7 @@ public class HTable {
       }
     ).booleanValue();
   }
-  
+
   /**
    * Executes all the buffered {@link Put} operations.
    * <p>
@@ -712,17 +712,17 @@ public class HTable {
 
   /**
    * Releases any resources help or pending changes in internal buffers.
-   * 
+   *
    * @throws IOException if a remote or network exception occurs.
   */
   public void close() throws IOException{
     flushCommits();
     this.pool.shutdownNow();
   }
-  
+
   /**
    * Utility method that verifies Put is well formed.
-   * 
+   *
    * @param put
    * @throws IllegalArgumentException
    */
@@ -731,7 +731,7 @@ public class HTable {
       throw new IllegalArgumentException("No columns to insert");
     }
   }
-  
+
   /**
    * Obtains a lock on a row.
    *
@@ -775,7 +775,7 @@ public class HTable {
       }
     );
   }
-  
+
   /**
    * Tells whether or not 'auto-flush' is turned on.
    *
@@ -807,7 +807,7 @@ public class HTable {
 
   /**
    * Set the size of the buffer in bytes.
-   * If the new size is lower than the current size of data in the 
+   * If the new size is lower than the current size of data in the
    * write buffer, the buffer is flushed.
    * @param writeBufferSize
    * @throws IOException
@@ -828,10 +828,10 @@ public class HTable {
   }
 
   // Old API. Pre-hbase-880, hbase-1304.
-  
+
   /**
    * Get a single value for the specified row and column
-   * 
+   *
    * @param row row key
    * @param column column name
    * @return value for specified row/column
@@ -843,7 +843,7 @@ public class HTable {
     return get(Bytes.toBytes(row), Bytes.toBytes(column));
   }
 
-  /** 
+  /**
    * Get a single value for the specified row and column
    *
    * @param row row key
@@ -858,7 +858,7 @@ public class HTable {
     return get(Bytes.toBytes(row), Bytes.toBytes(column), numVersions);
   }
 
-  /** 
+  /**
    * Get a single value for the specified row and column
    *
    * @param row row key
@@ -876,7 +876,7 @@ public class HTable {
     return r == null || r.size() <= 0? null: r.getCellValue();
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column
    * @param row row key
    * @param column column name
@@ -891,7 +891,7 @@ public class HTable {
     return get(row, column, HConstants.LATEST_TIMESTAMP, numVersions);
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column with
    * the specified timestamp.
    *
@@ -909,7 +909,7 @@ public class HTable {
     return get(Bytes.toBytes(row), Bytes.toBytes(column), timestamp, numVersions);
   }
 
-  /** 
+  /**
    * Get the specified number of versions of the specified row and column with
    * the specified timestamp.
    *
@@ -932,15 +932,15 @@ public class HTable {
       g.addColumn(fq[0], fq[1]);
     }
     g.setMaxVersions(numVersions);
-    g.setTimeRange(0, 
+    g.setTimeRange(0,
         timestamp == HConstants.LATEST_TIMESTAMP ? timestamp : timestamp+1);
     Result r = get(g);
     return r == null || r.size() <= 0? null: r.getCellValues();
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
@@ -950,9 +950,9 @@ public class HTable {
     return getRow(Bytes.toBytes(row));
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
@@ -961,10 +961,10 @@ public class HTable {
   public RowResult getRow(final byte [] row) throws IOException {
     return getRow(row, HConstants.LATEST_TIMESTAMP);
   }
- 
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
-   * 
+   *
    * @param row row key
    * @param numVersions number of versions to return
    * @return RowResult is <code>null</code> if row does not exist.
@@ -973,13 +973,13 @@ public class HTable {
    */
   public RowResult getRow(final String row, final int numVersions)
   throws IOException {
-    return getRow(Bytes.toBytes(row), null, 
+    return getRow(Bytes.toBytes(row), null,
                   HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get more than one version of all columns for the specified row
-   * 
+   *
    * @param row row key
    * @param numVersions number of versions to return
    * @return RowResult is <code>null</code> if row does not exist.
@@ -991,38 +991,38 @@ public class HTable {
     return getRow(row, null, HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final long ts) 
+  public RowResult getRow(final String row, final long ts)
   throws IOException {
     return getRow(Bytes.toBytes(row), ts);
   }
 
-  /** 
+  /**
    * Get all the data for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final long ts) 
+  public RowResult getRow(final byte [] row, final long ts)
   throws IOException {
     return getRow(row,null,ts);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
    * at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param ts timestamp
    * @param numVersions number of versions to return
@@ -1034,11 +1034,11 @@ public class HTable {
       final int numVersions) throws IOException {
     return getRow(Bytes.toBytes(row), null, ts, numVersions, null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of all columns for the specified row
    * at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param timestamp timestamp
    * @param numVersions number of versions to return
@@ -1051,37 +1051,37 @@ public class HTable {
     return getRow(row, null, timestamp, numVersions, null);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final String [] columns) 
+  public RowResult getRow(final String row, final String [] columns)
   throws IOException {
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns));
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at the latest timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns) 
+  public RowResult getRow(final byte [] row, final byte [][] columns)
   throws IOException {
     return getRow(row, columns, HConstants.LATEST_TIMESTAMP);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -1094,10 +1094,10 @@ public class HTable {
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns),
                   HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -1110,9 +1110,9 @@ public class HTable {
     return getRow(row, columns, HConstants.LATEST_TIMESTAMP, numVersions, null);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
@@ -1120,15 +1120,15 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final String row, final String [] columns, 
-    final long ts) 
-  throws IOException {  
+  public RowResult getRow(final String row, final String [] columns,
+    final long ts)
+  throws IOException {
     return getRow(Bytes.toBytes(row), Bytes.toByteArrays(columns), ts);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
@@ -1136,16 +1136,16 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns, 
-    final long ts) 
-  throws IOException {       
+  public RowResult getRow(final byte [] row, final byte [][] columns,
+    final long ts)
+  throws IOException {
     return getRow(row,columns,ts,1,null);
   }
-  
-  /** 
+
+  /**
    * Get more than one version of selected columns for the specified row,
    * using an existing row lock.
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param numVersions number of versions to return
@@ -1161,21 +1161,21 @@ public class HTable {
                   numVersions, rowLock);
   }
 
-  /** 
+  /**
    * Get selected columns for the specified row at a specified timestamp
    * using existing row lock.
-   * 
+   *
    * @param row row key
    * @param columns Array of column names and families you want to retrieve.
    * @param ts timestamp
-   * @param numVersions 
+   * @param numVersions
    * @param rl row lock
    * @return RowResult is <code>null</code> if row does not exist.
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #get(Get)}
    */
-  public RowResult getRow(final byte [] row, final byte [][] columns, 
-    final long ts, final int numVersions, final RowLock rl) 
+  public RowResult getRow(final byte [] row, final byte [][] columns,
+    final long ts, final int numVersions, final RowLock rl)
   throws IOException {
     Get g = rl != null? new Get(row, rl): new Get(row);
     if (columns != null) {
@@ -1189,13 +1189,13 @@ public class HTable {
       }
     }
     g.setMaxVersions(numVersions);
-    g.setTimeRange(0,  
+    g.setTimeRange(0,
         ts == HConstants.LATEST_TIMESTAMP ? ts : ts+1);
     Result r = get(g);
     return r == null || r.size() <= 0? null: r.getRowResult();
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at first row.
    * Return the specified columns.
    *
@@ -1213,7 +1213,7 @@ public class HTable {
     return getScanner(Bytes.toByteArrays(columns), HConstants.EMPTY_START_ROW);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1232,7 +1232,7 @@ public class HTable {
     return getScanner(Bytes.toByteArrays(columns), Bytes.toBytes(startRow));
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at first row.
    * Return the specified columns.
    *
@@ -1251,7 +1251,7 @@ public class HTable {
       HConstants.LATEST_TIMESTAMP, null);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1269,8 +1269,8 @@ public class HTable {
   throws IOException {
     return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, null);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1290,8 +1290,8 @@ public class HTable {
   throws IOException {
     return getScanner(columns, startRow, timestamp, null);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1308,11 +1308,11 @@ public class HTable {
    */
   public Scanner getScanner(final byte[][] columns, final byte [] startRow,
     RowFilterInterface filter)
-  throws IOException { 
+  throws IOException {
     return getScanner(columns, startRow, HConstants.LATEST_TIMESTAMP, filter);
   }
-  
-  /** 
+
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1336,7 +1336,7 @@ public class HTable {
     return getScanner(columns, startRow, stopRow, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1362,7 +1362,7 @@ public class HTable {
       Bytes.toBytes(stopRow), timestamp);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row and
    * ending just before <code>stopRow<code>.
    * Return the specified columns.
@@ -1388,7 +1388,7 @@ public class HTable {
       new WhileMatchRowFilter(new StopRowFilter(stopRow)));
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1411,7 +1411,7 @@ public class HTable {
       timestamp, filter);
   }
 
-  /** 
+  /**
    * Get a scanner on the current table starting at the specified row.
    * Return the specified columns.
    *
@@ -1444,7 +1444,7 @@ public class HTable {
         scan.addColumn(splits[0], splits[1]);
       }
     }
-    scan.setTimeRange(0,  
+    scan.setTimeRange(0,
         timestamp == HConstants.LATEST_TIMESTAMP ? timestamp : timestamp+1);
     OldClientScanner s = new OldClientScanner(new ClientScanner(scan));
     s.initialize();
@@ -1472,7 +1472,7 @@ public class HTable {
   public void deleteAll(final String row) throws IOException {
     deleteAll(row, null);
   }
-  
+
   /**
    * Completely delete the row's cells.
    *
@@ -1512,11 +1512,11 @@ public class HTable {
     deleteAll(row, null, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final String row, final String column)
@@ -1524,13 +1524,13 @@ public class HTable {
     deleteAll(row, column, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final String row, final String column, final long ts)
@@ -1539,13 +1539,13 @@ public class HTable {
       column != null? Bytes.toBytes(column): null, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final byte [] row, final byte [] column, final long ts)
@@ -1553,7 +1553,7 @@ public class HTable {
     deleteAll(row,column,ts,null);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp, using an
    * existing row lock.
@@ -1561,7 +1561,7 @@ public class HTable {
    * @param column name of column whose value is to be deleted
    * @param ts Delete all cells of the same timestamp or older.
    * @param rl Existing row lock
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAll(final byte [] row, final byte [] column, final long ts,
@@ -1573,12 +1573,12 @@ public class HTable {
     }
     delete(d);
   }
-  
-  /** 
+
+  /**
    * Delete all cells that match the passed row and column.
    * @param row Row to update
    * @param colRegex column regex expression
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
   public void deleteAllByRegex(final String row, final String colRegex)
@@ -1586,35 +1586,35 @@ public class HTable {
     deleteAllByRegex(row, colRegex, HConstants.LATEST_TIMESTAMP);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param colRegex Column Regex expression
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final String row, final String colRegex, 
+  public void deleteAllByRegex(final String row, final String colRegex,
       final long ts) throws IOException {
     deleteAllByRegex(Bytes.toBytes(row), colRegex, ts);
   }
 
-  /** 
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp.
    * @param row Row to update
    * @param colRegex Column Regex expression
    * @param ts Delete all cells of the same timestamp or older.
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+  public void deleteAllByRegex(final byte [] row, final String colRegex,
       final long ts) throws IOException {
     deleteAllByRegex(row, colRegex, ts, null);
   }
-  
-  /** 
+
+  /**
    * Delete all cells that match the passed row and column and whose
    * timestamp is equal-to or older than the passed timestamp, using an
    * existing row lock.
@@ -1622,10 +1622,10 @@ public class HTable {
    * @param colRegex Column regex expression
    * @param ts Delete all cells of the same timestamp or older.
    * @param rl Existing row lock
-   * @throws IOException 
+   * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteAllByRegex(final byte [] row, final String colRegex, 
+  public void deleteAllByRegex(final byte [] row, final String colRegex,
       final long ts, final RowLock rl)
   throws IOException {
     throw new UnsupportedOperationException("TODO: Not yet implemented");
@@ -1639,7 +1639,7 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final String row, final String family) 
+  public void deleteFamily(final String row, final String family)
   throws IOException {
     deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
   }
@@ -1652,7 +1652,7 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte[] row, final byte[] family) 
+  public void deleteFamily(final byte[] row, final byte[] family)
   throws IOException {
     deleteFamily(row, family, HConstants.LATEST_TIMESTAMP);
   }
@@ -1666,7 +1666,7 @@ public class HTable {
    * @param timestamp Timestamp to match
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
-   */  
+   */
   public void deleteFamily(final String row, final String family,
       final long timestamp)
   throws IOException{
@@ -1683,7 +1683,7 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte [] row, final byte [] family, 
+  public void deleteFamily(final byte [] row, final byte [] family,
     final long timestamp)
   throws IOException {
     deleteFamily(row,family,timestamp,null);
@@ -1700,16 +1700,16 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamily(final byte [] row, final byte [] family, 
+  public void deleteFamily(final byte [] row, final byte [] family,
     final long timestamp, final RowLock rl)
   throws IOException {
     Delete d = new Delete(row, HConstants.LATEST_TIMESTAMP, rl);
     d.deleteFamily(stripColon(family), timestamp);
     delete(d);
   }
-  
+
   /**
-   * Delete all cells for a row with matching column family regex 
+   * Delete all cells for a row with matching column family regex
    * at all timestamps.
    *
    * @param row The row to operate on
@@ -1717,13 +1717,13 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final String row, final String familyRegex) 
+  public void deleteFamilyByRegex(final String row, final String familyRegex)
   throws IOException {
     deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
   }
 
   /**
-   * Delete all cells for a row with matching column family regex 
+   * Delete all cells for a row with matching column family regex
    * at all timestamps.
    *
    * @param row The row to operate on
@@ -1731,7 +1731,7 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final byte[] row, final String familyRegex) 
+  public void deleteFamilyByRegex(final byte[] row, final String familyRegex)
   throws IOException {
     deleteFamilyByRegex(row, familyRegex, HConstants.LATEST_TIMESTAMP);
   }
@@ -1745,7 +1745,7 @@ public class HTable {
    * @param timestamp Timestamp to match
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
-   */  
+   */
   public void deleteFamilyByRegex(final String row, final String familyRegex,
       final long timestamp)
   throws IOException{
@@ -1762,17 +1762,17 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)}
    */
-  public void deleteFamilyByRegex(final byte [] row, final String familyRegex, 
+  public void deleteFamilyByRegex(final byte [] row, final String familyRegex,
     final long timestamp)
   throws IOException {
     deleteFamilyByRegex(row,familyRegex,timestamp,null);
   }
-  
+
   /**
    * Delete all cells for a row with matching column family regex with
    * timestamps less than or equal to <i>timestamp</i>, using existing
    * row lock.
-   * 
+   *
    * @param row The row to operate on
    * @param familyRegex Column Family Regex
    * @param timestamp Timestamp to match
@@ -1788,7 +1788,7 @@ public class HTable {
 
   /**
    * Test for the existence of a row in the table.
-   * 
+   *
    * @param row The row
    * @return true if the row exists, false otherwise
    * @throws IOException
@@ -1800,7 +1800,7 @@ public class HTable {
 
   /**
    * Test for the existence of a row and column in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @return true if the row exists, false otherwise
@@ -1814,7 +1814,7 @@ public class HTable {
 
   /**
    * Test for the existence of a coordinate in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @param timestamp The timestamp
@@ -1829,7 +1829,7 @@ public class HTable {
 
   /**
    * Test for the existence of a coordinate in the table.
-   * 
+   *
    * @param row The row
    * @param column The column
    * @param timestamp The timestamp
@@ -1842,7 +1842,7 @@ public class HTable {
       final long timestamp, final RowLock rl) throws IOException {
     final Get g = new Get(row, rl);
     g.addColumn(column);
-    g.setTimeRange(0,  
+    g.setTimeRange(0,
         timestamp == HConstants.LATEST_TIMESTAMP ? timestamp : timestamp+1);
     return exists(g);
   }
@@ -1854,12 +1854,12 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(Put)}
-   */ 
-  public synchronized void commit(final BatchUpdate batchUpdate) 
+   */
+  public synchronized void commit(final BatchUpdate batchUpdate)
   throws IOException {
     commit(batchUpdate, null);
   }
-  
+
   /**
    * Commit a BatchUpdate to the table using existing row lock.
    * If autoFlush is false, the update is buffered
@@ -1868,9 +1868,9 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(Put)}
-   */ 
+   */
   public synchronized void commit(final BatchUpdate batchUpdate,
-      final RowLock rl) 
+      final RowLock rl)
   throws IOException {
     for (BatchOperation bo: batchUpdate) {
       if (!bo.isPut()) throw new IOException("Only Puts in BU as of 0.20.0");
@@ -1887,7 +1887,7 @@ public class HTable {
    * @throws IOException
    * @deprecated As of hbase 0.20.0, replaced by {@link #delete(Delete)} or
    * {@link #put(List)}
-   */ 
+   */
   public synchronized void commit(final List<BatchUpdate> batchUpdates)
       throws IOException {
     // Am I breaking something here in old API by doing this?
@@ -1895,11 +1895,11 @@ public class HTable {
       commit(bu);
     }
   }
-  
+
   /**
-   * Atomically checks if a row's values match the expectedValues. 
+   * Atomically checks if a row's values match the expectedValues.
    * If it does, it uses the batchUpdate to update the row.<p>
-   * 
+   *
    * This operation is not currently supported, use {@link #checkAndPut}
    * @param batchUpdate batchupdate to apply if check is successful
    * @param expectedValues values to check
@@ -1932,11 +1932,11 @@ public class HTable {
     private long lastNext;
     // Keep lastResult returned successfully in case we have to reset scanner.
     private Result lastResult = null;
-    
+
     protected ClientScanner(final Scan scan) {
       if (CLIENT_LOG.isDebugEnabled()) {
-        CLIENT_LOG.debug("Creating scanner over " 
-            + Bytes.toString(getTableName()) 
+        CLIENT_LOG.debug("Creating scanner over "
+            + Bytes.toString(getTableName())
             + " starting at key '" + Bytes.toStringBinary(scan.getStartRow()) + "'");
       }
       this.scan = scan;
@@ -1964,7 +1964,7 @@ public class HTable {
     protected Scan getScan() {
       return scan;
     }
-    
+
     protected long getTimestamp() {
       return lastNext;
     }
@@ -2005,7 +2005,7 @@ public class HTable {
         getConnection().getRegionServerWithRetries(callable);
         this.callable = null;
       }
-      
+
       // Where to start the next scanner
       byte [] localStartKey = null;
 
@@ -2033,10 +2033,10 @@ public class HTable {
       if (CLIENT_LOG.isDebugEnabled()) {
         CLIENT_LOG.debug("Advancing internal scanner to startKey at '" +
           Bytes.toStringBinary(localStartKey) + "'");
-      }            
+      }
       try {
         callable = getScannerCallable(localStartKey, nbRows);
-        // Open a scanner on the region server starting at the 
+        // Open a scanner on the region server starting at the
         // beginning of the region
         getConnection().getRegionServerWithRetries(callable);
         this.currentRegion = callable.getHRegionInfo();
@@ -2046,11 +2046,11 @@ public class HTable {
       }
       return true;
     }
-    
+
     protected ScannerCallable getScannerCallable(byte [] localStartKey,
         int nbRows) {
       scan.setStartRow(localStartKey);
-      ScannerCallable s = new ScannerCallable(getConnection(), 
+      ScannerCallable s = new ScannerCallable(getConnection(),
         getTableName(), scan);
       s.setCaching(nbRows);
       return s;
@@ -2066,7 +2066,7 @@ public class HTable {
         Result [] values = null;
         long remainingResultSize = maxScannerResultSize;
         int countdown = this.caching;
-        // We need to reset it if it's a new callable that was created 
+        // We need to reset it if it's a new callable that was created
         // with a countdown in nextScanner
         callable.setCaching(this.caching);
         // This flag is set when we want to skip the result returned.  We do
@@ -2174,7 +2174,7 @@ public class HTable {
       return new Iterator<Result>() {
         // The next RowResult, possibly pre-read
         Result next = null;
-        
+
         // return true if there is another item pending, false if there isn't.
         // this method is where the actual advancing takes place, but you need
         // to call next() to consume it. hasNext() will only advance if there
@@ -2199,7 +2199,7 @@ public class HTable {
           if (!hasNext()) {
             return null;
           }
-          
+
           // if we get to here, then hasNext() has given us an item to return.
           // we want to return the item and then null out the next pointer, so
           // we use a temporary variable.
@@ -2220,7 +2220,7 @@ public class HTable {
    */
   protected class OldClientScanner implements Scanner {
     private final ClientScanner cs;
- 
+
     OldClientScanner(final ClientScanner cs) {
       this.cs = cs;
     }
@@ -2256,7 +2256,7 @@ public class HTable {
       return new Iterator<RowResult>() {
         // The next RowResult, possibly pre-read
         RowResult next = null;
-        
+
         // return true if there is another item pending, false if there isn't.
         // this method is where the actual advancing takes place, but you need
         // to call next() to consume it. hasNext() will only advance if there
@@ -2268,7 +2268,7 @@ public class HTable {
               return next != null;
             } catch (IOException e) {
               throw new RuntimeException(e);
-            }            
+            }
           }
           return true;
         }
@@ -2281,7 +2281,7 @@ public class HTable {
           if (!hasNext()) {
             return null;
           }
-          
+
           // if we get to here, then hasNext() has given us an item to return.
           // we want to return the item and then null out the next pointer, so
           // we use a temporary variable.
@@ -2296,7 +2296,7 @@ public class HTable {
       };
     }
   }
-  
+
   private static byte [] stripColon(final byte [] n) {
     byte col = n[n.length-1];
     if (col == ':') {

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/HTablePool.java Fri May  7 19:17:48 2010
@@ -30,17 +30,17 @@ import org.apache.hadoop.hbase.util.Byte
 
 /**
  * A simple pool of HTable instances.<p>
- * 
+ *
  * Each HTablePool acts as a pool for all tables.  To use, instantiate an
  * HTablePool and use {@link #getTable(String)} to get an HTable from the pool.
  * Once you are done with it, return it to the pool with {@link #putTable(HTable)}.<p>
- * 
+ *
  * A pool can be created with a <i>maxSize</i> which defines the most HTable
  * references that will ever be retained for each table.  Otherwise the default
  * is {@link Integer#MAX_VALUE}.<p>
  */
 public class HTablePool {
-  private final Map<String, LinkedList<HTable>> tables = 
+  private final Map<String, LinkedList<HTable>> tables =
       Collections.synchronizedMap(new HashMap<String, LinkedList<HTable>>());
   private final HBaseConfiguration config;
   private final int maxSize;
@@ -64,7 +64,7 @@ public class HTablePool {
 
   /**
    * Get a reference to the specified table from the pool.<p>
-   * 
+   *
    * Create a new one if one is not available.
    * @param tableName
    * @return a reference to the specified table
@@ -89,7 +89,7 @@ public class HTablePool {
 
   /**
    * Get a reference to the specified table from the pool.<p>
-   * 
+   *
    * Create a new one if one is not available.
    * @param tableName
    * @return a reference to the specified table
@@ -101,7 +101,7 @@ public class HTablePool {
 
   /**
    * Puts the specified HTable back into the pool.<p>
-   * 
+   *
    * If the pool already contains <i>maxSize</i> references to the table,
    * then nothing happens.
    * @param table

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MetaScanner.java Fri May  7 19:17:48 2010
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.HRegionIn
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
- * Scanner class that contains the <code>.META.</code> table scanning logic 
+ * Scanner class that contains the <code>.META.</code> table scanning logic
  * and uses a Retryable scanner. Provided visitors will be called
  * for each row.
  */
@@ -37,7 +37,7 @@ class MetaScanner implements HConstants 
   /**
    * Scans the meta table and calls a visitor on each RowResult and uses a empty
    * start row value as table name.
-   * 
+   *
    * @param configuration
    * @param visitor A custom visitor
    * @throws IOException
@@ -51,7 +51,7 @@ class MetaScanner implements HConstants 
   /**
    * Scans the meta table and calls a visitor on each RowResult. Uses a table
    * name to locate meta regions.
-   * 
+   *
    * @param configuration
    * @param visitor
    * @param tableName
@@ -62,12 +62,12 @@ class MetaScanner implements HConstants 
   throws IOException {
     HConnection connection = HConnectionManager.getConnection(configuration);
     byte [] startRow = tableName == null || tableName.length == 0 ?
-        HConstants.EMPTY_START_ROW : 
+        HConstants.EMPTY_START_ROW :
           HRegionInfo.createRegionName(tableName, null, ZEROES);
-      
+
     // Scan over each meta region
     ScannerCallable callable = null;
-    int rows = configuration.getInt("hbase.meta.scanner.caching", 100); 
+    int rows = configuration.getInt("hbase.meta.scanner.caching", 100);
     do {
       Scan scan = new Scan(startRow).addFamily(CATALOG_FAMILY);
       callable = new ScannerCallable(connection, META_TABLE_NAME, scan);
@@ -76,7 +76,7 @@ class MetaScanner implements HConstants 
       try {
         callable.setCaching(rows);
         done: do {
-          //we have all the rows here 
+          //we have all the rows here
           Result [] rrs = connection.getRegionServerWithRetries(callable);
           if (rrs == null || rrs.length == 0 || rrs[0].size() == 0) {
             break done; //exit completely
@@ -105,7 +105,7 @@ class MetaScanner implements HConstants 
      * Visitor method that accepts a RowResult and the meta region location.
      * Implementations can return false to stop the region's loop if it becomes
      * unnecessary for some reason.
-     * 
+     *
      * @param rowResult
      * @return A boolean to know if it should continue to loop in the region
      * @throws IOException

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPut.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPut.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPut.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPut.java Fri May  7 19:17:48 2010
@@ -52,7 +52,7 @@ public class MultiPut implements Writabl
     }
     return size;
   }
-  
+
   public void add(byte[] regionName, Put aPut) {
     List<Put> rsput = puts.get(regionName);
     if (rsput == null) {

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPutResponse.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPutResponse.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPutResponse.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/MultiPutResponse.java Fri May  7 19:17:48 2010
@@ -46,7 +46,7 @@ public class MultiPutResponse implements
   public Integer getAnswer(byte[] region) {
     return answers.get(region);
   }
-  
+
   @Override
   public void write(DataOutput out) throws IOException {
     out.writeInt(answers.size());

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Put.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Put.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Put.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Put.java Fri May  7 19:17:48 2010
@@ -37,7 +37,7 @@ import org.apache.hadoop.hbase.util.Byte
 import org.apache.hadoop.hbase.util.ClassSize;
 
 
-/** 
+/**
  * Used to perform Put operations for a single row.
  * <p>
  * To perform a Put, instantiate a Put object with the row to insert to and
@@ -49,18 +49,18 @@ public class Put implements HeapSize, Wr
   private long timestamp = HConstants.LATEST_TIMESTAMP;
   private long lockId = -1L;
   private boolean writeToWAL = true;
-  
+
   private Map<byte [], List<KeyValue>> familyMap =
     new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
-  
+
   private static final long OVERHEAD = ClassSize.align(
-      ClassSize.OBJECT + ClassSize.REFERENCE + 
-      2 * Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN + 
+      ClassSize.OBJECT + ClassSize.REFERENCE +
+      2 * Bytes.SIZEOF_LONG + Bytes.SIZEOF_BOOLEAN +
       ClassSize.REFERENCE + ClassSize.TREEMAP);
-  
+
   /** Constructor for Writable. DO NOT USE */
   public Put() {}
-  
+
   /**
    * Create a Put operation for the specified row.
    * @param row row key
@@ -92,7 +92,7 @@ public class Put implements HeapSize, Wr
     this(putToCopy.getRow(), putToCopy.getRowLock());
     this.timestamp = putToCopy.timestamp;
     this.writeToWAL = putToCopy.writeToWAL;
-    this.familyMap = 
+    this.familyMap =
       new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
     for(Map.Entry<byte [], List<KeyValue>> entry :
       putToCopy.getFamilyMap().entrySet()) {
@@ -111,7 +111,7 @@ public class Put implements HeapSize, Wr
   }
 
   /**
-   * Add the specified column and value, with the specified timestamp as 
+   * Add the specified column and value, with the specified timestamp as
    * its version to this Put operation.
    * @param column Old style column name with family and qualifier put together
    * with a colon.
@@ -124,7 +124,7 @@ public class Put implements HeapSize, Wr
   }
 
   /**
-   * Add the specified column and value, with the specified timestamp as 
+   * Add the specified column and value, with the specified timestamp as
    * its version to this Put operation.
    * @param family family name
    * @param qualifier column qualifier
@@ -138,9 +138,9 @@ public class Put implements HeapSize, Wr
     familyMap.put(kv.getFamily(), list);
     return this;
   }
-  
+
   /**
-   * Add the specified KeyValue to this Put operation.  Operation assumes that 
+   * Add the specified KeyValue to this Put operation.  Operation assumes that
    * the passed KeyValue is immutable and its backing array will not be modified
    * for the duration of this Put.
    * @param kv
@@ -149,12 +149,12 @@ public class Put implements HeapSize, Wr
     byte [] family = kv.getFamily();
     List<KeyValue> list = getKeyValueList(family);
     //Checking that the row of the kv is the same as the put
-    int res = Bytes.compareTo(this.row, 0, row.length, 
+    int res = Bytes.compareTo(this.row, 0, row.length,
         kv.getBuffer(), kv.getRowOffset(), kv.getRowLength());
     if(res != 0) {
-      throw new IOException("The row in the recently added KeyValue " + 
-          Bytes.toStringBinary(kv.getBuffer(), kv.getRowOffset(), 
-        kv.getRowLength()) + " doesn't match the original one " + 
+      throw new IOException("The row in the recently added KeyValue " +
+          Bytes.toStringBinary(kv.getBuffer(), kv.getRowOffset(),
+        kv.getRowLength()) + " doesn't match the original one " +
         Bytes.toStringBinary(this.row));
     }
     list.add(kv);
@@ -164,7 +164,7 @@ public class Put implements HeapSize, Wr
 
   /**
    * Create a KeyValue with this objects row key and the Put identifier.
-   * 
+   *
    * @param family
    * @param qualifier
    * @param ts
@@ -173,15 +173,15 @@ public class Put implements HeapSize, Wr
    */
   private KeyValue createPutKeyValue(byte[] family, byte[] qualifier, long ts,
       byte[] value) {
-  return  new KeyValue(this.row, family, qualifier, ts, KeyValue.Type.Put, 
+  return  new KeyValue(this.row, family, qualifier, ts, KeyValue.Type.Put,
       value);
   }
-  
+
   /**
-   * A convenience method to determine if this object's familyMap contains 
+   * A convenience method to determine if this object's familyMap contains
    * a value assigned to the given family & qualifier.
    * Both given arguments must match the KeyValue object to return true.
-   * 
+   *
    * @param family
    * @param qualifier
    * @return returns true if the given family and qualifier already has an
@@ -190,12 +190,12 @@ public class Put implements HeapSize, Wr
   public boolean has(byte [] family, byte [] qualifier) {
   return has(family, qualifier, this.timestamp, new byte[0], true, true);
   }
-  
+
   /**
-   * A convenience method to determine if this object's familyMap contains 
+   * A convenience method to determine if this object's familyMap contains
    * a value assigned to the given family, qualifier and timestamp.
    * All 3 given arguments must match the KeyValue object to return true.
-   * 
+   *
    * @param family
    * @param qualifier
    * @param ts
@@ -205,12 +205,12 @@ public class Put implements HeapSize, Wr
   public boolean has(byte [] family, byte [] qualifier, long ts) {
   return has(family, qualifier, ts, new byte[0], false, true);
   }
-  
+
   /**
-   * A convenience method to determine if this object's familyMap contains 
+   * A convenience method to determine if this object's familyMap contains
    * a value assigned to the given family, qualifier and timestamp.
    * All 3 given arguments must match the KeyValue object to return true.
-   * 
+   *
    * @param family
    * @param qualifier
    * @param value
@@ -220,38 +220,38 @@ public class Put implements HeapSize, Wr
   public boolean has(byte [] family, byte [] qualifier, byte [] value) {
     return has(family, qualifier, this.timestamp, value, true, false);
   }
-  
+
   /**
-   * A convenience method to determine if this object's familyMap contains 
+   * A convenience method to determine if this object's familyMap contains
    * the given value assigned to the given family, qualifier and timestamp.
    * All 4 given arguments must match the KeyValue object to return true.
-   * 
+   *
    * @param family
    * @param qualifier
    * @param ts
    * @param value
-   * @return returns true if the given family, qualifier timestamp and value 
+   * @return returns true if the given family, qualifier timestamp and value
    * already has an existing KeyValue object in the family map.
    */
   public boolean has(byte [] family, byte [] qualifier, long ts, byte [] value) {
       return has(family, qualifier, ts, value, false, false);
   }
-  
+
   /**
-   * Private method to determine if this object's familyMap contains 
+   * Private method to determine if this object's familyMap contains
    * the given value assigned to the given family, qualifier and timestamp
    * respecting the 2 boolean arguments
-   * 
+   *
    * @param family
    * @param qualifier
    * @param ts
    * @param value
    * @param ignoreTS
    * @param ignoreValue
-   * @return returns true if the given family, qualifier timestamp and value 
+   * @return returns true if the given family, qualifier timestamp and value
    * already has an existing KeyValue object in the family map.
    */
-  private boolean has(byte [] family, byte [] qualifier, long ts, byte [] value, 
+  private boolean has(byte [] family, byte [] qualifier, long ts, byte [] value,
       boolean ignoreTS, boolean ignoreValue) {
     List<KeyValue> list = getKeyValueList(family);
     if (list.size() == 0 ) {
@@ -284,13 +284,13 @@ public class Put implements HeapSize, Wr
     }
     return false;
   }
-  
+
   /**
    * Returns a list of all KeyValue objects with matching column family and qualifier.
-   * 
+   *
    * @param family
    * @param qualifier
-   * @return a list of KeyValue objects with the matching family and qualifier, 
+   * @return a list of KeyValue objects with the matching family and qualifier,
    * returns an empty list if one doesnt exist for the given family.
    */
   public List<KeyValue> get(byte[] family, byte[] qualifier) {
@@ -306,7 +306,7 @@ public class Put implements HeapSize, Wr
   /**
    * Creates an empty list if one doesnt exist for the given column family
    * or else it returns the associated list of KeyValue objects.
-   * 
+   *
    * @param family
    * @return a list of KeyValue objects, returns an empty list if one doesnt exist.
    */
@@ -317,7 +317,7 @@ public class Put implements HeapSize, Wr
     }
     return list;
   }
-  
+
   /**
    * Method for retrieving the put's familyMap
    * @return familyMap
@@ -325,15 +325,15 @@ public class Put implements HeapSize, Wr
   public Map<byte [], List<KeyValue>> getFamilyMap() {
     return this.familyMap;
   }
-  
+
   /**
    * Method for retrieving the put's row
-   * @return row 
+   * @return row
    */
   public byte [] getRow() {
     return this.row;
   }
-  
+
   /**
    * Method for retrieving the put's RowLock
    * @return RowLock
@@ -341,7 +341,7 @@ public class Put implements HeapSize, Wr
   public RowLock getRowLock() {
     return new RowLock(this.row, this.lockId);
   }
-  
+
   /**
    * Method for retrieving the put's lockId
    * @return lockId
@@ -349,7 +349,7 @@ public class Put implements HeapSize, Wr
   public long getLockId() {
   	return this.lockId;
   }
-  
+
   /**
    * Method to check if the familyMap is empty
    * @return true if empty, false otherwise
@@ -357,14 +357,14 @@ public class Put implements HeapSize, Wr
   public boolean isEmpty() {
     return familyMap.isEmpty();
   }
-  
+
   /**
    * @return Timestamp
    */
   public long getTimeStamp() {
     return this.timestamp;
   }
-  
+
   /**
    * Method for setting the timestamp
    * NOTE - This does not affect the timestamp for values previously added to this Put.
@@ -375,9 +375,9 @@ public class Put implements HeapSize, Wr
     this.timestamp = timestamp;
     return this;
   }
-  
+
   /**
-   * @return the number of different families included in this put 
+   * @return the number of different families included in this put
    */
   public int numFamilies() {
     return familyMap.size();
@@ -393,14 +393,14 @@ public class Put implements HeapSize, Wr
     }
     return size;
   }
-  
+
   /**
    * @return true if edits should be applied to WAL, false if not
    */
   public boolean getWriteToWAL() {
     return this.writeToWAL;
   }
-  
+
   /**
    * Set whether this Put should be written to the WAL or not.
    * Not writing the WAL means you may lose edits on server crash.
@@ -409,9 +409,9 @@ public class Put implements HeapSize, Wr
   public void setWriteToWAL(boolean write) {
     this.writeToWAL = write;
   }
-  
+
   /**
-   * @return String 
+   * @return String
    */
   @Override
   public String toString() {
@@ -443,40 +443,40 @@ public class Put implements HeapSize, Wr
     sb.append("}");
     return sb.toString();
   }
-  
+
   public int compareTo(Row p) {
     return Bytes.compareTo(this.getRow(), p.getRow());
   }
-  
+
   //HeapSize
   public long heapSize() {
     long heapsize = OVERHEAD;
     //Adding row
     heapsize += ClassSize.align(ClassSize.ARRAY + this.row.length);
-    
+
     //Adding map overhead
-    heapsize += 
+    heapsize +=
       ClassSize.align(this.familyMap.size() * ClassSize.MAP_ENTRY);
     for(Map.Entry<byte [], List<KeyValue>> entry : this.familyMap.entrySet()) {
       //Adding key overhead
-      heapsize += 
+      heapsize +=
         ClassSize.align(ClassSize.ARRAY + entry.getKey().length);
-      
+
       //This part is kinds tricky since the JVM can reuse references if you
       //store the same value, but have a good match with SizeOf at the moment
       //Adding value overhead
       heapsize += ClassSize.align(ClassSize.ARRAYLIST);
       int size = entry.getValue().size();
-      heapsize += ClassSize.align(ClassSize.ARRAY + 
+      heapsize += ClassSize.align(ClassSize.ARRAY +
           size * ClassSize.REFERENCE);
-      
+
       for(KeyValue kv : entry.getValue()) {
         heapsize += kv.heapSize();
       }
     }
     return ClassSize.align((int)heapsize);
   }
-  
+
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
@@ -502,7 +502,7 @@ public class Put implements HeapSize, Wr
       this.familyMap.put(family, keys);
     }
   }
-  
+
   public void write(final DataOutput out)
   throws IOException {
     Bytes.writeByteArray(out, this.row);

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Result.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Result.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Result.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Result.java Fri May  7 19:17:48 2010
@@ -41,32 +41,32 @@ import org.apache.hadoop.io.Writable;
 
 /**
  * Single row result of a {@link Get} or {@link Scan} query.<p>
- * 
+ *
  * Convenience methods are available that return various {@link Map}
  * structures and values directly.<p>
- * 
- * To get a complete mapping of all cells in the Result, which can include 
+ *
+ * To get a complete mapping of all cells in the Result, which can include
  * multiple families and multiple versions, use {@link #getMap()}.<p>
- * 
- * To get a mapping of each family to its columns (qualifiers and values), 
+ *
+ * To get a mapping of each family to its columns (qualifiers and values),
  * including only the latest version of each, use {@link #getNoVersionMap()}.
- * 
- * To get a mapping of qualifiers to latest values for an individual family use 
+ *
+ * To get a mapping of qualifiers to latest values for an individual family use
  * {@link #getFamilyMap(byte[])}.<p>
- * 
+ *
  * To get the latest value for a specific family and qualifier use {@link #getValue(byte[], byte[])}.
  *
  * A Result is backed by an array of {@link KeyValue} objects, each representing
  * an HBase cell defined by the row, family, qualifier, timestamp, and value.<p>
- * 
+ *
  * The underlying {@link KeyValue} objects can be accessed through the methods
  * {@link #sorted()} and {@link #list()}.  Each KeyValue can then be accessed
- * through {@link KeyValue#getRow()}, {@link KeyValue#getFamily()}, {@link KeyValue#getQualifier()}, 
+ * through {@link KeyValue#getRow()}, {@link KeyValue#getFamily()}, {@link KeyValue#getQualifier()},
  * {@link KeyValue#getTimestamp()}, and {@link KeyValue#getValue()}.
  */
 public class Result implements Writable {
   private KeyValue [] kvs = null;
-  private NavigableMap<byte[], 
+  private NavigableMap<byte[],
      NavigableMap<byte[], NavigableMap<Long, byte[]>>> familyMap = null;
   // We're not using java serialization.  Transient here is just a marker to say
   // that this is where we cache row if we're ever asked for it.
@@ -95,7 +95,7 @@ public class Result implements Writable 
   public Result(List<KeyValue> kvs) {
     this(kvs.toArray(new KeyValue[0]));
   }
-  
+
   /**
    * Instantiate a Result from the specified raw binary format.
    * @param bytes raw binary format of Result
@@ -131,7 +131,7 @@ public class Result implements Writable 
 
   /**
    * Create a sorted list of the KeyValue's in this result.
-   * 
+   *
    * @return The sorted list of KeyValue's.
    */
   public List<KeyValue> list() {
@@ -159,10 +159,10 @@ public class Result implements Writable 
   /**
    * Map of families to all versions of its qualifiers and values.
    * <p>
-   * Returns a three level Map of the form: 
+   * Returns a three level Map of the form:
    * <code>Map<family,Map&lt;qualifier,Map&lt;timestamp,value>>></code>
    * <p>
-   * Note: All other map returning methods make use of this map internally. 
+   * Note: All other map returning methods make use of this map internally.
    * @return map from families to qualifiers to versions
    */
   public NavigableMap<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> getMap() {
@@ -178,7 +178,7 @@ public class Result implements Writable 
     for(KeyValue kv : this.kvs) {
       SplitKeyValue splitKV = kv.split();
       byte [] family = splitKV.getFamily();
-      NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap = 
+      NavigableMap<byte[], NavigableMap<Long, byte[]>> columnMap =
         familyMap.get(family);
       if(columnMap == null) {
         columnMap = new TreeMap<byte[], NavigableMap<Long, byte[]>>
@@ -217,15 +217,15 @@ public class Result implements Writable 
     if(isEmpty()) {
       return null;
     }
-    NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap = 
+    NavigableMap<byte[], NavigableMap<byte[], byte[]>> returnMap =
       new TreeMap<byte[], NavigableMap<byte[], byte[]>>(Bytes.BYTES_COMPARATOR);
-    for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>> 
+    for(Map.Entry<byte[], NavigableMap<byte[], NavigableMap<Long, byte[]>>>
       familyEntry : familyMap.entrySet()) {
-      NavigableMap<byte[], byte[]> qualifierMap = 
+      NavigableMap<byte[], byte[]> qualifierMap =
         new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
       for(Map.Entry<byte[], NavigableMap<Long, byte[]>> qualifierEntry :
         familyEntry.getValue().entrySet()) {
-        byte [] value = 
+        byte [] value =
           qualifierEntry.getValue().get(qualifierEntry.getValue().firstKey());
         qualifierMap.put(qualifierEntry.getKey(), value);
       }
@@ -247,16 +247,16 @@ public class Result implements Writable 
     if(isEmpty()) {
       return null;
     }
-    NavigableMap<byte[], byte[]> returnMap = 
+    NavigableMap<byte[], byte[]> returnMap =
       new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
-    NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap = 
+    NavigableMap<byte[], NavigableMap<Long, byte[]>> qualifierMap =
       familyMap.get(family);
     if(qualifierMap == null) {
       return returnMap;
     }
-    for(Map.Entry<byte[], NavigableMap<Long, byte[]>> entry : 
+    for(Map.Entry<byte[], NavigableMap<Long, byte[]>> entry :
       qualifierMap.entrySet()) {
-      byte [] value = 
+      byte [] value =
         entry.getValue().get(entry.getValue().firstKey());
       returnMap.put(entry.getKey(), value);
     }
@@ -319,13 +319,13 @@ public class Result implements Writable 
     }
     return versionMap.firstEntry();
   }
-  
+
   private NavigableMap<Long, byte[]> getVersionMap(
       NavigableMap<byte [], NavigableMap<Long, byte[]>> qualifierMap, byte [] qualifier) {
     return qualifier != null?
       qualifierMap.get(qualifier): qualifierMap.get(new byte[0]);
   }
-  
+
   /**
    * Get the latest version of the specified column,
    * using <pre>family:qualifier</pre> notation.
@@ -340,7 +340,7 @@ public class Result implements Writable 
       return null;
     }
   }
-  
+
   /**
    * Checks for existence of the specified column.
    * @param family family name
@@ -365,7 +365,7 @@ public class Result implements Writable 
     }
     return true;
   }
-  
+
   /**
    * Returns this Result in the old return format, {@link RowResult}.
    * @return a RowResult
@@ -376,7 +376,7 @@ public class Result implements Writable 
     }
     return RowResult.createRowResult(Arrays.asList(kvs));
   }
-  
+
   /**
    * Returns the value of the first column in the Result.
    * @return value of the first column
@@ -387,19 +387,19 @@ public class Result implements Writable 
     }
     return kvs[0].getValue();
   }
-  
+
   /**
    * Returns the raw binary encoding of this Result.<p>
-   * 
+   *
    * Please note, there may be an offset into the underlying byte array of the
-   * returned ImmutableBytesWritable.  Be sure to use both 
+   * returned ImmutableBytesWritable.  Be sure to use both
    * {@link ImmutableBytesWritable#get()} and {@link ImmutableBytesWritable#getOffset()}
    * @return pointer to raw binary of Result
    */
   public ImmutableBytesWritable getBytes() {
     return this.bytes;
   }
-  
+
   /**
    * Check if the underlying KeyValue [] is empty or not
    * @return true if empty
@@ -410,7 +410,7 @@ public class Result implements Writable 
     }
     return this.kvs == null || this.kvs.length == 0;
   }
-  
+
   /**
    * @return the size of the underlying KeyValue []
    */
@@ -420,7 +420,7 @@ public class Result implements Writable 
     }
     return this.kvs == null? 0: this.kvs.length;
   }
-  
+
   /**
    * @return String
    */
@@ -445,7 +445,7 @@ public class Result implements Writable 
     sb.append("}");
     return sb.toString();
   }
-  
+
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
@@ -461,7 +461,7 @@ public class Result implements Writable 
     in.readFully(raw, 0, totalBuffer);
     bytes = new ImmutableBytesWritable(raw, 0, totalBuffer);
   }
-  
+
   //Create KeyValue[] when needed
   private void readFields() {
     if (bytes == null) {
@@ -480,7 +480,7 @@ public class Result implements Writable 
     }
     this.kvs = kvs.toArray(new KeyValue[kvs.size()]);
   }
-  
+
   public void write(final DataOutput out)
   throws IOException {
     if(isEmpty()) {
@@ -497,7 +497,7 @@ public class Result implements Writable 
       }
     }
   }
-  
+
   public static void writeArray(final DataOutput out, Result [] results)
   throws IOException {
     if(results == null || results.length == 0) {
@@ -528,7 +528,7 @@ public class Result implements Writable 
       }
     }
   }
-  
+
   public static Result [] readArray(final DataInput in)
   throws IOException {
     int numResults = in.readInt();
@@ -555,7 +555,7 @@ public class Result implements Writable 
         offset += keyLen;
       }
       int totalLength = offset - initialOffset;
-      results[i] = new Result(new ImmutableBytesWritable(buf, initialOffset, 
+      results[i] = new Result(new ImmutableBytesWritable(buf, initialOffset,
           totalLength));
     }
     return results;

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ResultScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ResultScanner.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ResultScanner.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ResultScanner.java Fri May  7 19:17:48 2010
@@ -35,18 +35,18 @@ public interface ResultScanner extends C
    * @return Result object if there is another row, null if the scanner is
    * exhausted.
    * @throws IOException
-   */  
+   */
   public Result next() throws IOException;
- 
+
   /**
    * @param nbRows number of rows to return
    * @return Between zero and <param>nbRows</param> Results
    * @throws IOException
    */
   public Result [] next(int nbRows) throws IOException;
- 
+
   /**
    * Closes the scanner and releases any resources it has allocated
    */
-  public void close();  
+  public void close();
 }
\ No newline at end of file

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RetriesExhaustedException.java Fri May  7 19:17:48 2010
@@ -20,10 +20,10 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.util.Bytes;
 
-/** 
+/**
  * Exception thrown by HTable methods when an attempt to do something (like
- * commit changes) fails after a bunch of retries. 
- */ 
+ * commit changes) fails after a bunch of retries.
+ */
 public class RetriesExhaustedException extends IOException {
   private static final long serialVersionUID = 1876775844L;
 
@@ -31,14 +31,14 @@ public class RetriesExhaustedException e
     super(msg);
   }
 
-  /** 
+  /**
    * Create a new RetriesExhaustedException from the list of prior failures.
    * @param serverName name of HRegionServer
    * @param regionName name of region
    * @param row The row we were pursuing when we ran out of retries
    * @param numTries The number of tries we made
    * @param exceptions List of exceptions that failed before giving up
-   */ 
+   */
   public RetriesExhaustedException(String serverName, final byte [] regionName,
       final byte []  row, int numTries, List<Throwable> exceptions) {
     super(getMessage(serverName, regionName, row, numTries, exceptions));

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RowLock.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RowLock.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RowLock.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/RowLock.java Fri May  7 19:17:48 2010
@@ -35,7 +35,7 @@ public class RowLock {
     this.row = row;
     this.lockId = lockId;
   }
-  
+
   /**
    * Creates a RowLock with only a lock id
    * @param lockId

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scan.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scan.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scan.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scan.java Fri May  7 19:17:48 2010
@@ -55,7 +55,7 @@ import org.apache.hadoop.io.WritableFact
  * <p>
  * To modify scanner caching for just this scan, use {@link #setCaching(int) setCaching}.
  * <p>
- * To further define the scope of what to get when scanning, perform additional 
+ * To further define the scope of what to get when scanning, perform additional
  * methods as outlined below.
  * <p>
  * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
@@ -73,12 +73,12 @@ import org.apache.hadoop.io.WritableFact
  * To limit the number of versions of each column to be returned, execute
  * {@link #setMaxVersions(int) setMaxVersions}.
  * <p>
- * To limit the maximum number of values returned for each call to next(), execute 
+ * To limit the maximum number of values returned for each call to next(), execute
  * {@link #setBatch(int) setBatch}.
  * <p>
  * To add a filter, execute {@link #setFilter(org.apache.hadoop.hbase.filter.Filter) setFilter}.
  * <p>
- * Expert: To explicitly disable server-side block caching for this scan, 
+ * Expert: To explicitly disable server-side block caching for this scan,
  * execute {@link #setCacheBlocks(boolean)}.
  */
 public class Scan implements Writable {
@@ -105,7 +105,7 @@ public class Scan implements Writable {
   // additional data for the scan
   protected Map<ImmutableBytesWritable, ImmutableBytesWritable> values =
     new HashMap<ImmutableBytesWritable, ImmutableBytesWritable>();
-  
+
   /**
    * Create a Scan operation across all rows.
    */
@@ -115,7 +115,7 @@ public class Scan implements Writable {
     this(startRow);
     this.filter = filter;
   }
-  
+
   /**
    * Create a Scan operation starting at the specified row.
    * <p>
@@ -126,7 +126,7 @@ public class Scan implements Writable {
   public Scan(byte [] startRow) {
     this.startRow = startRow;
   }
-  
+
   /**
    * Create a Scan operation for the range of rows specified.
    * @param startRow row to start scanner at or after (inclusive)
@@ -136,10 +136,10 @@ public class Scan implements Writable {
     this.startRow = startRow;
     this.stopRow = stopRow;
   }
-  
+
   /**
    * Creates a new instance of this class while copying all values.
-   * 
+   *
    * @param scan  The scan instance to copy from.
    * @throws IOException When copying the values fails.
    */
@@ -197,7 +197,7 @@ public class Scan implements Writable {
     familyMap.put(family, EMPTY_NAVIGABLE_SET);
     return this;
   }
-  
+
   /**
    * Get the column from the specified family with the specified qualifier.
    * <p>
@@ -218,12 +218,12 @@ public class Scan implements Writable {
   }
 
   /**
-   * Parses a combined family and qualifier and adds either both or just the 
-   * family in case there is not qualifier. This assumes the older colon 
+   * Parses a combined family and qualifier and adds either both or just the
+   * family in case there is not qualifier. This assumes the older colon
    * divided notation, e.g. "data:contents" or "meta:".
    * <p>
    * Note: It will through an error when the colon is missing.
-   * 
+   *
    * @param familyAndQualifier
    * @return A reference to this instance.
    * @throws IllegalArgumentException When the colon is missing.
@@ -231,18 +231,18 @@ public class Scan implements Writable {
   public Scan addColumn(byte[] familyAndQualifier) {
     byte [][] fq = KeyValue.parseColumn(familyAndQualifier);
     if (fq.length > 1 && fq[1] != null && fq[1].length > 0) {
-      addColumn(fq[0], fq[1]);  
+      addColumn(fq[0], fq[1]);
     } else {
       addFamily(fq[0]);
     }
     return this;
   }
-  
+
   /**
    * Adds an array of columns specified using old format, family:qualifier.
    * <p>
    * Overrides previous calls to addFamily for any families in the input.
-   * 
+   *
    * @param columns array of columns, formatted as <pre>family:qualifier</pre>
    */
   public Scan addColumns(byte [][] columns) {
@@ -257,7 +257,7 @@ public class Scan implements Writable {
    * command line) column definitions, e.g. "data:contents mime:". The columns
    * must be space delimited and always have a colon (":") to denote family
    * and qualifier.
-   * 
+   *
    * @param columns  The columns to parse.
    * @return A reference to this instance.
    */
@@ -270,15 +270,15 @@ public class Scan implements Writable {
   }
 
   /**
-   * Helps to convert the binary column families and qualifiers to a text 
+   * Helps to convert the binary column families and qualifiers to a text
    * representation, e.g. "data:mimetype data:contents meta:". Binary values
    * are properly encoded using {@link Bytes#toBytesBinary(String)}.
-   * 
+   *
    * @return The columns in an old style string format.
    */
   public String getInputColumns() {
     StringBuilder cols = new StringBuilder();
-    for (Map.Entry<byte[], NavigableSet<byte[]>> e : 
+    for (Map.Entry<byte[], NavigableSet<byte[]>> e :
       familyMap.entrySet()) {
       byte[] fam = e.getKey();
       if (cols.length() > 0) {
@@ -297,14 +297,14 @@ public class Scan implements Writable {
           cols.append(Bytes.toStringBinary(qual));
         }
       } else {
-        // only add the family but with old style delimiter 
+        // only add the family but with old style delimiter
         cols.append(Bytes.toStringBinary(fam));
         cols.append(":");
       }
     }
     return cols.toString();
   }
-  
+
   /**
    * Get versions of columns only within the specified timestamp range,
    * [minStamp, maxStamp).  Note, default maximum versions to return is 1.  If
@@ -321,7 +321,7 @@ public class Scan implements Writable {
     tr = new TimeRange(minStamp, maxStamp);
     return this;
   }
-  
+
   /**
    * Get versions of columns with the specified timestamp. Note, default maximum
    * versions to return is 1.  If your time range spans more than one version
@@ -348,7 +348,7 @@ public class Scan implements Writable {
     this.startRow = startRow;
     return this;
   }
-  
+
   /**
    * Set the stop row.
    * @param stopRow
@@ -357,7 +357,7 @@ public class Scan implements Writable {
     this.stopRow = stopRow;
     return this;
   }
-  
+
   /**
    * Get all available versions.
    */
@@ -405,16 +405,16 @@ public class Scan implements Writable {
   /**
    * Set an old-style filter interface to use. Note: not all features of the
    * old style filters are supported.
-   * 
+   *
    * @deprecated
    * @param filter
-   * @return The scan instance. 
+   * @return The scan instance.
    */
   public Scan setOldFilter(RowFilterInterface filter) {
     oldFilter = filter;
     return this;
   }
-  
+
   /**
    * Setting the familyMap
    * @param familyMap
@@ -423,7 +423,7 @@ public class Scan implements Writable {
     this.familyMap = familyMap;
     return this;
   }
-  
+
   /**
    * Getting the familyMap
    * @return familyMap
@@ -431,7 +431,7 @@ public class Scan implements Writable {
   public Map<byte [], NavigableSet<byte []>> getFamilyMap() {
     return this.familyMap;
   }
-  
+
   /**
    * @return the number of families in familyMap
    */
@@ -448,7 +448,7 @@ public class Scan implements Writable {
   public boolean hasFamilies() {
     return !this.familyMap.isEmpty();
   }
-  
+
   /**
    * @return the keys of the familyMap
    */
@@ -458,7 +458,7 @@ public class Scan implements Writable {
     }
     return null;
   }
-  
+
   /**
    * @return the startrow
    */
@@ -472,13 +472,13 @@ public class Scan implements Writable {
   public byte [] getStopRow() {
     return this.stopRow;
   }
-  
+
   /**
    * @return the max number of versions to fetch
    */
   public int getMaxVersions() {
     return this.maxVersions;
-  } 
+  }
 
   /**
    * @return maximum number of values to return for a single call to next()
@@ -492,15 +492,15 @@ public class Scan implements Writable {
    */
   public int getCaching() {
     return this.caching;
-  } 
+  }
 
   /**
    * @return TimeRange
    */
   public TimeRange getTimeRange() {
     return this.tr;
-  } 
-  
+  }
+
   /**
    * @return RowFilter
    */
@@ -516,28 +516,28 @@ public class Scan implements Writable {
   public RowFilterInterface getOldFilter() {
     return oldFilter;
   }
-  
+
   /**
    * @return true is a filter has been specified, false if not
    */
   public boolean hasFilter() {
     return filter != null || oldFilter != null;
   }
-  
+
   /**
    * Set whether blocks should be cached for this Scan.
    * <p>
    * This is true by default.  When true, default settings of the table and
    * family are used (this will never override caching blocks if the block
    * cache is disabled for that family or entirely).
-   * 
+   *
    * @param cacheBlocks if false, default settings are overridden and blocks
    * will not be cached
    */
   public void setCacheBlocks(boolean cacheBlocks) {
     this.cacheBlocks = cacheBlocks;
   }
-  
+
   /**
    * Get whether blocks should be cached for this Scan.
    * @return true if default caching should be used, false if blocks should not
@@ -620,7 +620,7 @@ public class Scan implements Writable {
   public void remove(final byte [] key) {
     values.remove(new ImmutableBytesWritable(key));
   }
-  
+
   /**
    * @return String
    */
@@ -674,7 +674,7 @@ public class Scan implements Writable {
       }
     }
     sb.append("}");
-    
+
     for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e:
         values.entrySet()) {
       String key = Bytes.toString(e.getKey().get());
@@ -691,7 +691,7 @@ public class Scan implements Writable {
 
     return sb.toString();
   }
-  
+
   @SuppressWarnings("unchecked")
   private Writable createForName(String className) {
     try {
@@ -700,9 +700,9 @@ public class Scan implements Writable {
       return WritableFactories.newInstance(clazz, new Configuration());
     } catch (ClassNotFoundException e) {
       throw new RuntimeException("Can't find class " + className);
-    }    
+    }
   }
-  
+
   //Writable
   public void readFields(final DataInput in)
   throws IOException {
@@ -730,7 +730,7 @@ public class Scan implements Writable {
     this.tr = new TimeRange();
     tr.readFields(in);
     int numFamilies = in.readInt();
-    this.familyMap = 
+    this.familyMap =
       new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
     for(int i=0; i<numFamilies; i++) {
       byte [] family = Bytes.readByteArray(in);

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scanner.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scanner.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/Scanner.java Fri May  7 19:17:48 2010
@@ -35,16 +35,16 @@ public interface Scanner extends Closeab
    * @return RowResult object if there is another row, null if the scanner is
    * exhausted.
    * @throws IOException
-   */  
+   */
   public RowResult next() throws IOException;
- 
+
   /**
    * @param nbRows number of rows to return
    * @return Between zero and <param>nbRows</param> Results
    * @throws IOException
    */
   public RowResult [] next(int nbRows) throws IOException;
- 
+
   /**
    * Closes the scanner and releases any resources it has allocated
    */

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerCallable.java Fri May  7 19:17:48 2010
@@ -51,7 +51,7 @@ public class ScannerCallable extends Ser
     super(connection, tableName, scan.getStartRow());
     this.scan = scan;
   }
-  
+
   /**
    * @param reload
    * @throws IOException
@@ -96,7 +96,7 @@ public class ScannerCallable extends Ser
     }
     return null;
   }
-  
+
   private void close() {
     if (this.scannerId == -1L) {
       return;
@@ -113,18 +113,18 @@ public class ScannerCallable extends Ser
     return this.server.openScanner(this.location.getRegionInfo().getRegionName(),
       this.scan);
   }
-  
+
   protected Scan getScan() {
     return scan;
   }
-  
+
   /**
    * Call this when the next invocation of call should close the scanner
    */
   public void setClose() {
     this.closed = true;
   }
-  
+
   /**
    * @return the HRegionInfo for the current region
    */

Modified: hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerTimeoutException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerTimeoutException.java?rev=942184&r1=942183&r2=942184&view=diff
==============================================================================
--- hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerTimeoutException.java (original)
+++ hadoop/hbase/branches/0.20/src/java/org/apache/hadoop/hbase/client/ScannerTimeoutException.java Fri May  7 19:17:48 2010
@@ -23,7 +23,7 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 
 /**
- * Thrown when a scanner has timed out. 
+ * Thrown when a scanner has timed out.
  */
 public class ScannerTimeoutException extends DoNotRetryIOException {