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:26:51 UTC

svn commit: r942186 [2/18] - in /hadoop/hbase/trunk: ./ contrib/stargate/core/src/test/java/org/apache/hadoop/hbase/stargate/ core/src/main/java/org/apache/hadoop/hbase/ core/src/main/java/org/apache/hadoop/hbase/client/ core/src/main/java/org/apache/h...

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/KeyValue.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/KeyValue.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/KeyValue.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/KeyValue.java Fri May  7 19:26:45 2010
@@ -36,19 +36,19 @@ import org.apache.hadoop.io.Writable;
 
 /**
  * An HBase Key/Value.
- * 
+ *
  * <p>If being used client-side, the primary methods to access individual fields
- * are {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()}, 
+ * are {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()},
  * {@link #getTimestamp()}, and {@link #getValue()}.  These methods allocate new
  * byte arrays and return copies so they should be avoided server-side.
- * 
+ *
  * <p>Instances of this class are immutable.  They are not
  * comparable but Comparators are provided.  Comparators change with context,
  * whether user table or a catalog table comparison context.  Its
  * important that you use the appropriate comparator comparing rows in
  * particular.  There are Comparators for KeyValue instances and then for
  * just the Key portion of a KeyValue used mostly in {@link HFile}.
- * 
+ *
  * <p>KeyValue wraps a byte array and has offset and length for passed array
  * at where to start interpreting the content as a KeyValue blob.  The KeyValue
  * blob format inside the byte array is:
@@ -58,7 +58,7 @@ import org.apache.hadoop.io.Writable;
  * Rowlength maximum is Short.MAX_SIZE, column family length maximum is
  * Byte.MAX_SIZE, and column qualifier + key length must be < Integer.MAX_SIZE.
  * The column does not contain the family/qualifier delimiter.
- * 
+ *
  * <p>TODO: Group Key-only comparators and operations into a Key class, just
  * for neatness sake, if can figure what to call it.
  */
@@ -72,7 +72,7 @@ public class KeyValue implements Writabl
 
   public static final byte[] COLUMN_FAMILY_DELIM_ARRAY =
     new byte[]{COLUMN_FAMILY_DELIMITER};
-  
+
   /**
    * Comparator for plain key/values; i.e. non-catalog table key/values.
    */
@@ -110,10 +110,10 @@ public class KeyValue implements Writabl
 
   /**
    * Get the appropriate row comparator for the specified table.
-   * 
+   *
    * Hopefully we can get rid of this, I added this here because it's replacing
    * something in HSK.  We should move completely off of that.
-   * 
+   *
    * @param tableName  The table name.
    * @return The comparator.
    */
@@ -162,13 +162,13 @@ public class KeyValue implements Writabl
 
     // Maximum is used when searching; you look from maximum on down.
     Maximum((byte)255);
-    
+
     private final byte code;
-    
+
     Type(final byte c) {
       this.code = c;
     }
-    
+
     public byte getCode() {
       return this.code;
     }
@@ -194,9 +194,9 @@ public class KeyValue implements Writabl
    * Makes a Key with highest possible Timestamp, empty row and column.  No
    * key can be equal or lower than this one in memstore or in store file.
    */
-  public static final KeyValue LOWESTKEY = 
+  public static final KeyValue LOWESTKEY =
     new KeyValue(HConstants.EMPTY_BYTE_ARRAY, HConstants.LATEST_TIMESTAMP);
-  
+
   private byte [] bytes = null;
   private int offset = 0;
   private int length = 0;
@@ -238,7 +238,7 @@ public class KeyValue implements Writabl
   }
 
   /** Constructors that build a new backing byte array from fields */
-  
+
   /**
    * Constructs KeyValue structure filled with null value.
    * Sets type to {@link KeyValue.Type#Maximum}
@@ -265,7 +265,7 @@ public class KeyValue implements Writabl
    * @param family family name
    * @param qualifier column qualifier
    */
-  public KeyValue(final byte [] row, final byte [] family, 
+  public KeyValue(final byte [] row, final byte [] family,
       final byte [] qualifier) {
     this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Maximum);
   }
@@ -276,7 +276,7 @@ public class KeyValue implements Writabl
    * @param family family name
    * @param qualifier column qualifier
    */
-  public KeyValue(final byte [] row, final byte [] family, 
+  public KeyValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final byte [] value) {
     this(row, family, qualifier, HConstants.LATEST_TIMESTAMP, Type.Put, value);
   }
@@ -294,7 +294,7 @@ public class KeyValue implements Writabl
       final byte[] qualifier, final long timestamp, Type type) {
     this(row, family, qualifier, timestamp, type, null);
   }
-  
+
   /**
    * Constructs KeyValue structure filled with specified values.
    * @param row row key
@@ -308,7 +308,7 @@ public class KeyValue implements Writabl
       final byte[] qualifier, final long timestamp, final byte[] value) {
     this(row, family, qualifier, timestamp, Type.Put, value);
   }
-  
+
   /**
    * Constructs KeyValue structure filled with specified values.
    * @param row row key
@@ -322,9 +322,9 @@ public class KeyValue implements Writabl
   public KeyValue(final byte[] row, final byte[] family,
       final byte[] qualifier, final long timestamp, Type type,
       final byte[] value) {
-    this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length, 
+    this(row, family, qualifier, 0, qualifier==null ? 0 : qualifier.length,
         timestamp, type, value, 0, value==null ? 0 : value.length);
-  } 
+  }
 
   /**
    * Constructs KeyValue structure filled with specified values.
@@ -340,12 +340,12 @@ public class KeyValue implements Writabl
    * @param vlength value length
    * @throws IllegalArgumentException
    */
-  public KeyValue(byte [] row, byte [] family, 
-      byte [] qualifier, int qoffset, int qlength, long timestamp, Type type, 
+  public KeyValue(byte [] row, byte [] family,
+      byte [] qualifier, int qoffset, int qlength, long timestamp, Type type,
       byte [] value, int voffset, int vlength) {
-    this(row, 0, row==null ? 0 : row.length, 
+    this(row, 0, row==null ? 0 : row.length,
         family, 0, family==null ? 0 : family.length,
-        qualifier, qoffset, qlength, timestamp, type, 
+        qualifier, qoffset, qlength, timestamp, type,
         value, voffset, vlength);
   }
 
@@ -374,7 +374,7 @@ public class KeyValue implements Writabl
       final byte [] qualifier, final int qoffset, final int qlength,
       final long timestamp, final Type type,
       final byte [] value, final int voffset, final int vlength) {
-    this.bytes = createByteArray(row, roffset, rlength, 
+    this.bytes = createByteArray(row, roffset, rlength,
         family, foffset, flength, qualifier, qoffset, qlength,
         timestamp, type, value, voffset, vlength);
     this.length = bytes.length;
@@ -383,7 +383,7 @@ public class KeyValue implements Writabl
 
   /**
    * Write KeyValue format into a byte array.
-   * 
+   *
    * @param row row key
    * @param roffset row offset
    * @param rlength row length
@@ -398,7 +398,7 @@ public class KeyValue implements Writabl
    * @param value column value
    * @param voffset value offset
    * @param vlength value length
-   * @return The newly created byte array. 
+   * @return The newly created byte array.
    */
   static byte [] createByteArray(final byte [] row, final int roffset,
       final int rlength, final byte [] family, final int foffset, int flength,
@@ -431,10 +431,10 @@ public class KeyValue implements Writabl
     // Value length
     vlength = value == null? 0 : vlength;
     if (vlength > HConstants.MAXIMUM_VALUE_LENGTH) { // FindBugs INT_VACUOUS_COMPARISON
-      throw new IllegalArgumentException("Valuer > " + 
+      throw new IllegalArgumentException("Valuer > " +
           HConstants.MAXIMUM_VALUE_LENGTH);
     }
-    
+
     // Allocate right-sized byte array.
     byte [] bytes = new byte[KEYVALUE_INFRASTRUCTURE_SIZE + keylength + vlength];
     // Write key, value and key row length.
@@ -457,7 +457,7 @@ public class KeyValue implements Writabl
     }
     return bytes;
   }
-  
+
   /**
    * Write KeyValue format into a byte array.
    * <p>
@@ -473,7 +473,7 @@ public class KeyValue implements Writabl
    * @param value
    * @param voffset
    * @param vlength
-   * @return The newly created byte array. 
+   * @return The newly created byte array.
    */
   static byte [] createByteArray(final byte [] row, final int roffset,
         final int rlength,
@@ -528,7 +528,7 @@ public class KeyValue implements Writabl
   //  KeyValue cloning
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * Clones a KeyValue.  This creates a copy, re-allocating the buffer.
    * @return Fully copied clone of this KeyValue
@@ -544,7 +544,7 @@ public class KeyValue implements Writabl
   //  String representation
   //
   //---------------------------------------------------------------------------
-  
+
   public String toString() {
     if (this.bytes == null || this.bytes.length == 0) {
       return "empty";
@@ -595,7 +595,7 @@ public class KeyValue implements Writabl
   //  Public Member Accessors
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * @return The byte array backing this KeyValue.
    */
@@ -622,7 +622,7 @@ public class KeyValue implements Writabl
   //  Length and Offset Calculators
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * Determines the total length of the KeyValue stored in the specified
    * byte array and offset.  Includes all headers.
@@ -631,7 +631,7 @@ public class KeyValue implements Writabl
    * @return length of entire KeyValue, in bytes
    */
   private static int getLength(byte [] bytes, int offset) {
-    return (2 * Bytes.SIZEOF_INT) + 
+    return (2 * Bytes.SIZEOF_INT) +
         Bytes.toInt(bytes, offset) +
         Bytes.toInt(bytes, offset + Bytes.SIZEOF_INT);
   }
@@ -660,7 +660,7 @@ public class KeyValue implements Writabl
   public int getValueOffset() {
     return getKeyOffset() + getKeyLength();
   }
-  
+
   /**
    * @return Value length
    */
@@ -674,7 +674,7 @@ public class KeyValue implements Writabl
   public int getRowOffset() {
     return getKeyOffset() + Bytes.SIZEOF_SHORT;
   }
-  
+
   /**
    * @return Row length
    */
@@ -688,21 +688,21 @@ public class KeyValue implements Writabl
   public int getFamilyOffset() {
     return getFamilyOffset(getRowLength());
   }
-  
+
   /**
    * @return Family offset
    */
   public int getFamilyOffset(int rlength) {
     return this.offset + ROW_OFFSET + Bytes.SIZEOF_SHORT + rlength + Bytes.SIZEOF_BYTE;
   }
-  
+
   /**
    * @return Family length
    */
   public byte getFamilyLength() {
     return getFamilyLength(getFamilyOffset());
   }
-  
+
   /**
    * @return Family length
    */
@@ -716,29 +716,29 @@ public class KeyValue implements Writabl
   public int getQualifierOffset() {
     return getQualifierOffset(getFamilyOffset());
   }
-  
+
   /**
    * @return Qualifier offset
    */
   public int getQualifierOffset(int foffset) {
     return foffset + getFamilyLength(foffset);
   }
-  
+
   /**
    * @return Qualifier length
    */
   public int getQualifierLength() {
     return getQualifierLength(getRowLength(),getFamilyLength());
   }
-  
+
   /**
    * @return Qualifier length
    */
   public int getQualifierLength(int rlength, int flength) {
-    return getKeyLength() - 
+    return getKeyLength() -
       (KEY_INFRASTRUCTURE_SIZE + rlength + flength);
   }
-  
+
   /**
    * @return Column (family + qualifier) length
    */
@@ -747,7 +747,7 @@ public class KeyValue implements Writabl
     int foffset = getFamilyOffset(rlength);
     return getTotalColumnLength(rlength,foffset);
   }
-  
+
   /**
    * @return Column (family + qualifier) length
    */
@@ -756,14 +756,14 @@ public class KeyValue implements Writabl
     int qlength = getQualifierLength(rlength,flength);
     return flength + qlength;
   }
-  
+
   /**
    * @return Timestamp offset
    */
   public int getTimestampOffset() {
     return getTimestampOffset(getKeyLength());
   }
-  
+
   /**
    * @param keylength Pass if you have it to save on a int creation.
    * @return Timestamp offset
@@ -776,7 +776,7 @@ public class KeyValue implements Writabl
    * @return True if this KeyValue has a LATEST_TIMESTAMP timestamp.
    */
   public boolean isLatestTimestamp() {
-    return  Bytes.compareTo(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG, 
+    return  Bytes.compareTo(getBuffer(), getTimestampOffset(), Bytes.SIZEOF_LONG,
       HConstants.LATEST_TIMESTAMP_BYTES, 0, Bytes.SIZEOF_LONG) == 0;
   }
 
@@ -788,17 +788,17 @@ public class KeyValue implements Writabl
     }
     return false;
   }
-  
+
   //---------------------------------------------------------------------------
   //
   //  Methods that return copies of fields
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * Do not use unless you have to.  Used internally for compacting and testing.
-   * 
-   * Use {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()}, and 
+   *
+   * Use {@link #getRow()}, {@link #getFamily()}, {@link #getQualifier()}, and
    * {@link #getValue()} if accessing a KeyValue client-side.
    * @return Copy of the key portion only.
    */
@@ -808,7 +808,7 @@ public class KeyValue implements Writabl
     System.arraycopy(getBuffer(), getKeyOffset(), key, 0, keylength);
     return key;
   }
-  
+
   /**
    * Returns value in a new byte array.
    * Primarily for use client-side. If server-side, use
@@ -823,12 +823,12 @@ public class KeyValue implements Writabl
     System.arraycopy(getBuffer(), o, result, 0, l);
     return result;
   }
-  
+
   /**
    * Primarily for use client-side.  Returns the row of this KeyValue in a new
    * byte array.<p>
-   * 
-   * If server-side, use {@link #getBuffer()} with appropriate offsets and 
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
    * lengths instead.
    * @return Row in a new byte array.
    */
@@ -841,7 +841,7 @@ public class KeyValue implements Writabl
   }
 
   /**
-   * 
+   *
    * @return Timestamp
    */
   public long getTimestamp() {
@@ -897,10 +897,10 @@ public class KeyValue implements Writabl
   }
 
   /**
-   * Primarily for use client-side.  Returns the family of this KeyValue in a 
+   * Primarily for use client-side.  Returns the family of this KeyValue in a
    * new byte array.<p>
-   * 
-   * If server-side, use {@link #getBuffer()} with appropriate offsets and 
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
    * lengths instead.
    * @return Returns family. Makes a copy.
    */
@@ -913,10 +913,10 @@ public class KeyValue implements Writabl
   }
 
   /**
-   * Primarily for use client-side.  Returns the column qualifier of this 
+   * Primarily for use client-side.  Returns the column qualifier of this
    * KeyValue in a new byte array.<p>
-   * 
-   * If server-side, use {@link #getBuffer()} with appropriate offsets and 
+   *
+   * If server-side, use {@link #getBuffer()} with appropriate offsets and
    * lengths instead.
    * Use {@link #getBuffer()} with appropriate offsets and lengths instead.
    * @return Returns qualifier. Makes a copy.
@@ -934,7 +934,7 @@ public class KeyValue implements Writabl
   //  KeyValue splitter
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * Utility class that splits a KeyValue buffer into separate byte arrays.
    * <p>
@@ -958,7 +958,7 @@ public class KeyValue implements Writabl
     public byte [] getType() { return this.split[4]; }
     public byte [] getValue() { return this.split[5]; }
   }
-  
+
   public SplitKeyValue split() {
     SplitKeyValue split = new SplitKeyValue();
     int splitOffset = this.offset;
@@ -998,13 +998,13 @@ public class KeyValue implements Writabl
     split.setValue(value);
     return split;
   }
-  
+
   //---------------------------------------------------------------------------
   //
-  //  Compare specified fields against those contained in this KeyValue 
+  //  Compare specified fields against those contained in this KeyValue
   //
   //---------------------------------------------------------------------------
-  
+
   /**
    * @param family
    * @return True if matching families.
@@ -1025,7 +1025,7 @@ public class KeyValue implements Writabl
   public boolean matchingQualifier(final byte [] qualifier) {
     int o = getQualifierOffset();
     int l = getQualifierLength();
-    return Bytes.compareTo(qualifier, 0, qualifier.length, 
+    return Bytes.compareTo(qualifier, 0, qualifier.length,
         this.bytes, o, l) == 0;
   }
 
@@ -1135,7 +1135,7 @@ public class KeyValue implements Writabl
       len);
     return result;
   }
-  
+
   /**
    * Makes a column in family:qualifier form from separate byte arrays.
    * <p>
@@ -1147,7 +1147,7 @@ public class KeyValue implements Writabl
   public static byte [] makeColumn(byte [] family, byte [] qualifier) {
     return Bytes.add(family, COLUMN_FAMILY_DELIM_ARRAY, qualifier);
   }
-  
+
   /**
    * @param b
    * @return Index of the family-qualifier colon delimiter character in passed
@@ -1226,7 +1226,7 @@ public class KeyValue implements Writabl
    */
   public static class RootComparator extends MetaComparator {
     private final KeyComparator rawcomparator = new RootKeyComparator();
-    
+
     public KeyComparator getRawComparator() {
       return this.rawcomparator;
     }
@@ -1297,7 +1297,7 @@ public class KeyValue implements Writabl
      * @return Result comparing rows.
      */
     public int compareRows(final KeyValue left, final KeyValue right) {
-      return compareRows(left, left.getRowLength(), right, 
+      return compareRows(left, left.getRowLength(), right,
           right.getRowLength());
     }
 
@@ -1330,7 +1330,7 @@ public class KeyValue implements Writabl
       return getRawComparator().compareRows(left, loffset, llength,
         right, roffset, rlength);
     }
-    
+
     public int compareColumns(final KeyValue left, final byte [] right,
         final int roffset, final int rlength, final int rfamilyoffset) {
       int offset = left.getFamilyOffset();
@@ -1408,7 +1408,7 @@ public class KeyValue implements Writabl
     public boolean matchingRows(final byte [] left, final int loffset,
         final int llength,
         final byte [] right, final int roffset, final int rlength) {
-      int compare = compareRows(left, loffset, llength, 
+      int compare = compareRows(left, loffset, llength,
           right, roffset, rlength);
       if (compare != 0) {
         return false;
@@ -1437,7 +1437,7 @@ public class KeyValue implements Writabl
     protected Object clone() throws CloneNotSupportedException {
       return new KVComparator();
     }
- 
+
     /**
      * @return Comparator that ignores timestamps; useful counting versions.
      */
@@ -1518,7 +1518,7 @@ public class KeyValue implements Writabl
       final byte [] q, final long ts) {
     return new KeyValue(row, f, q, ts, Type.Maximum);
   }
-  
+
   /**
    * @param b
    * @return A KeyValue made of a byte array that holds the key-only part.
@@ -1527,7 +1527,7 @@ public class KeyValue implements Writabl
   public static KeyValue createKeyValueFromKey(final byte [] b) {
     return createKeyValueFromKey(b, 0, b.length);
   }
-  
+
   /**
    * @param bb
    * @return A KeyValue made of a byte buffer that holds the key-only part.
@@ -1536,7 +1536,7 @@ public class KeyValue implements Writabl
   public static KeyValue createKeyValueFromKey(final ByteBuffer bb) {
     return createKeyValueFromKey(bb.array(), bb.arrayOffset(), bb.limit());
   }
-  
+
   /**
    * @param b
    * @param o
@@ -1565,7 +1565,7 @@ public class KeyValue implements Writabl
       //          "---" + Bytes.toString(right, roffset, rlength));
       final int metalength = 7; // '.META.' length
       int lmetaOffsetPlusDelimiter = loffset + metalength;
-      int leftFarDelimiter = getDelimiterInReverse(left, 
+      int leftFarDelimiter = getDelimiterInReverse(left,
           lmetaOffsetPlusDelimiter,
           llength - metalength, HRegionInfo.DELIMITER);
       int rmetaOffsetPlusDelimiter = roffset + metalength;
@@ -1716,7 +1716,7 @@ public class KeyValue implements Writabl
       if (compare != 0) {
         return compare;
       }
-      
+
       if (!this.ignoreTimestamp) {
         // Get timestamps.
         long ltimestamp = Bytes.toLong(left,
@@ -1769,14 +1769,14 @@ public class KeyValue implements Writabl
       return 0;
     }
   }
-  
+
   // HeapSize
   public long heapSize() {
-    return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE + 
-        ClassSize.align(ClassSize.ARRAY + length) + 
+    return ClassSize.align(ClassSize.OBJECT + ClassSize.REFERENCE +
+        ClassSize.align(ClassSize.ARRAY + length) +
         (2 * Bytes.SIZEOF_INT));
   }
-  
+
   // this overload assumes that the length bytes have already been read,
   // and it expects the length of the KeyValue to be explicitly passed
   // to it.
@@ -1786,13 +1786,13 @@ public class KeyValue implements Writabl
     this.bytes = new byte[this.length];
     in.readFully(this.bytes, 0, this.length);
   }
-  
+
   // Writable
   public void readFields(final DataInput in) throws IOException {
     int length = in.readInt();
     readFields(length, in);
   }
-  
+
   public void write(final DataOutput out) throws IOException {
     out.writeInt(this.length);
     out.write(this.bytes, this.offset, this.length);

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LeaseListener.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LeaseListener.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LeaseListener.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LeaseListener.java Fri May  7 19:26:45 2010
@@ -21,11 +21,11 @@ package org.apache.hadoop.hbase;
 
 
 /**
- * LeaseListener is an interface meant to be implemented by users of the Leases 
+ * LeaseListener is an interface meant to be implemented by users of the Leases
  * class.
  *
  * It receives events from the Leases class about the status of its accompanying
- * lease.  Users of the Leases class can use a LeaseListener subclass to, for 
+ * lease.  Users of the Leases class can use a LeaseListener subclass to, for
  * example, clean up resources after a lease has expired.
  */
 public interface LeaseListener {

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/Leases.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/Leases.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/Leases.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/Leases.java Fri May  7 19:26:45 2010
@@ -36,15 +36,15 @@ import java.io.IOException;
  *
  * There are several server classes in HBase that need to track external
  * clients that occasionally send heartbeats.
- * 
+ *
  * <p>These external clients hold resources in the server class.
  * Those resources need to be released if the external client fails to send a
  * heartbeat after some interval of time passes.
  *
  * <p>The Leases class is a general reusable class for this kind of pattern.
- * An instance of the Leases class will create a thread to do its dirty work.  
+ * An instance of the Leases class will create a thread to do its dirty work.
  * You should close() the instance if you want to clean up the thread properly.
- * 
+ *
  * <p>
  * NOTE: This class extends Thread rather than Chore because the sleep time
  * can be interrupted when there is something to do, rather than the Chore
@@ -60,7 +60,7 @@ public class Leases extends Thread {
 
   /**
    * Creates a lease monitor
-   * 
+   *
    * @param leasePeriod - length of time (milliseconds) that the lease is valid
    * @param leaseCheckFrequency - how often the lease should be checked
    * (milliseconds)
@@ -114,9 +114,9 @@ public class Leases extends Thread {
   public void closeAfterLeasesExpire() {
     this.stopRequested = true;
   }
-  
+
   /**
-   * Shut down this Leases instance.  All pending leases will be destroyed, 
+   * Shut down this Leases instance.  All pending leases will be destroyed,
    * without any cancellation calls.
    */
   public void close() {
@@ -132,10 +132,10 @@ public class Leases extends Thread {
 
   /**
    * Obtain a lease
-   * 
+   *
    * @param leaseName name of the lease
    * @param listener listener that will process lease expirations
-   * @throws LeaseStillHeldException 
+   * @throws LeaseStillHeldException
    */
   public void createLease(String leaseName, final LeaseListener listener)
   throws LeaseStillHeldException {
@@ -160,25 +160,25 @@ public class Leases extends Thread {
   @SuppressWarnings("serial")
   public static class LeaseStillHeldException extends IOException {
     private final String leaseName;
-    
+
     /**
      * @param name
      */
     public LeaseStillHeldException(final String name) {
       this.leaseName = name;
     }
-    
+
     /** @return name of lease */
     public String getName() {
       return this.leaseName;
     }
   }
-  
+
   /**
    * Renew a lease
-   * 
+   *
    * @param leaseName name of lease
-   * @throws LeaseException 
+   * @throws LeaseException
    */
   public void renewLease(final String leaseName) throws LeaseException {
     synchronized (leaseQueue) {
@@ -197,9 +197,9 @@ public class Leases extends Thread {
 
   /**
    * Client explicitly cancels a lease.
-   * 
+   *
    * @param leaseName name of lease
-   * @throws LeaseException 
+   * @throws LeaseException
    */
   public void cancelLease(final String leaseName) throws LeaseException {
     synchronized (leaseQueue) {
@@ -227,7 +227,7 @@ public class Leases extends Thread {
     public String getLeaseName() {
       return leaseName;
     }
-    
+
     /** @return listener */
     public LeaseListener getListener() {
       return this.listener;
@@ -246,7 +246,7 @@ public class Leases extends Thread {
       }
       return this.hashCode() == ((Lease) obj).hashCode();
     }
-    
+
     @Override
     public int hashCode() {
       return this.leaseName.hashCode();

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java Fri May  7 19:26:45 2010
@@ -36,18 +36,18 @@ import org.apache.hadoop.hbase.util.JVMC
 /**
  * This class creates a single process HBase cluster. One thread is created for
  * a master and one per region server.
- * 
+ *
  * Call {@link #startup()} to start the cluster running and {@link #shutdown()}
  * to close it all down. {@link #join} the cluster is you want to wait on
  * shutdown completion.
- * 
+ *
  * <p>Runs master on port 60000 by default.  Because we can't just kill the
  * process -- not till HADOOP-1700 gets fixed and even then.... -- we need to
  * be able to find the master with a remote client to run shutdown.  To use a
  * port other than 60000, set the hbase.master to a value of 'local:PORT':
  * that is 'local', not 'localhost', and the port number the master should use
  * instead of 60000.
- * 
+ *
  * <p>To make 'local' mode more responsive, make values such as
  * <code>hbase.regionserver.msginterval</code>,
  * <code>hbase.master.meta.thread.rescanfrequency</code>, and
@@ -203,7 +203,7 @@ public class LocalHBaseCluster implement
       }
     }
   }
-  
+
   /**
    * Start the cluster.
    */

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/NotServingRegionException.java Fri May  7 19:26:45 2010
@@ -42,7 +42,7 @@ public class NotServingRegionException e
   public NotServingRegionException(String s) {
     super(s);
   }
-  
+
   /**
    * Constructor
    * @param s message

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/RemoteExceptionHandler.java Fri May  7 19:26:45 2010
@@ -25,14 +25,14 @@ import java.lang.reflect.InvocationTarge
 
 import org.apache.hadoop.ipc.RemoteException;
 
-/** 
+/**
  * An immutable class which contains a static method for handling
  * org.apache.hadoop.ipc.RemoteException exceptions.
  */
 public class RemoteExceptionHandler {
   /* Not instantiable */
   private RemoteExceptionHandler() {super();}
-  
+
   /**
    * Examine passed Throwable.  See if its carrying a RemoteException. If so,
    * run {@link #decodeRemoteException(RemoteException)} on it.  Otherwise,
@@ -53,7 +53,7 @@ public class RemoteExceptionHandler {
     }
     return result;
   }
-  
+
   /**
    * Examine passed IOException.  See if its carrying a RemoteException. If so,
    * run {@link #decodeRemoteException(RemoteException)} on it.  Otherwise,
@@ -66,17 +66,17 @@ public class RemoteExceptionHandler {
     Throwable t = checkThrowable(e);
     return t instanceof IOException? (IOException)t: new IOException(t);
   }
-  
+
   /**
    * Converts org.apache.hadoop.ipc.RemoteException into original exception,
    * if possible. If the original exception is an Error or a RuntimeException,
    * throws the original exception.
-   * 
+   *
    * @param re original exception
    * @return decoded RemoteException if it is an instance of or a subclass of
    *         IOException, or the original RemoteException if it cannot be decoded.
-   * 
-   * @throws IOException indicating a server error ocurred if the decoded 
+   *
+   * @throws IOException indicating a server error ocurred if the decoded
    *         exception is not an IOException. The decoded exception is set as
    *         the cause.
    */
@@ -89,10 +89,10 @@ public class RemoteExceptionHandler {
 
       Class<?>[] parameterTypes = { String.class };
       Constructor<?> ctor = c.getConstructor(parameterTypes);
-      
+
       Object[] arguments = { re.getMessage() };
       Throwable t = (Throwable) ctor.newInstance(arguments);
-      
+
       if (t instanceof IOException) {
         i = (IOException) t;
 

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/TableExistsException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/TableExistsException.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/TableExistsException.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/TableExistsException.java Fri May  7 19:26:45 2010
@@ -29,7 +29,7 @@ public class TableExistsException extend
 
   /**
    * Constructor
-   * 
+   *
    * @param s message
    */
   public TableExistsException(String s) {

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/ValueOverMaxLengthException.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/ValueOverMaxLengthException.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/ValueOverMaxLengthException.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/ValueOverMaxLengthException.java Fri May  7 19:26:45 2010
@@ -23,7 +23,7 @@ package org.apache.hadoop.hbase;
  * Thrown when a value is longer than the specified LENGTH
  */
 public class ValueOverMaxLengthException extends DoNotRetryIOException {
-  
+
   private static final long serialVersionUID = -5525656352372008316L;
 
   /**

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/VersionAnnotation.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/VersionAnnotation.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/VersionAnnotation.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/VersionAnnotation.java Fri May  7 19:26:45 2010
@@ -26,29 +26,29 @@ import java.lang.annotation.*;
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.PACKAGE)
 public @interface VersionAnnotation {
- 
+
   /**
    * Get the Hadoop version
    * @return the version string "0.6.3-dev"
    */
   String version();
-  
+
   /**
    * Get the username that compiled Hadoop.
    */
   String user();
-  
+
   /**
    * Get the date when Hadoop was compiled.
    * @return the date in unix 'date' format
    */
   String date();
-  
+
   /**
    * Get the url for the subversion repository.
    */
   String url();
-  
+
   /**
    * Get the subversion revision.
    * @return the revision number as a string (eg. "451451")

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Delete.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Delete.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Delete.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Delete.java Fri May  7 19:26:45 2010
@@ -36,7 +36,7 @@ import java.util.TreeMap;
 /**
  * Used to perform Delete operations on a single row.
  * <p>
- * To delete an entire row, instantiate a Delete object with the row 
+ * To delete an entire row, instantiate a Delete object with the row
  * to delete.  To further define the scope of what to delete, perform
  * additional methods as outlined below.
  * <p>
@@ -45,7 +45,7 @@ import java.util.TreeMap;
  * <p>
  * To delete multiple versions of specific columns, execute
  * {@link #deleteColumns(byte[], byte[]) deleteColumns}
- * for each column to delete.  
+ * for each column to delete.
  * <p>
  * To delete specific versions of specific columns, execute
  * {@link #deleteColumn(byte[], byte[], long) deleteColumn}
@@ -69,10 +69,10 @@ public class Delete implements Writable,
   private static final byte DELETE_VERSION = (byte)1;
 
   private byte [] row = null;
-  // This ts is only used when doing a deleteRow.  Anything less, 
+  // This ts is only used when doing a deleteRow.  Anything less,
   private long ts;
   private long lockId = -1L;
-  private final Map<byte [], List<KeyValue>> familyMap = 
+  private final Map<byte [], List<KeyValue>> familyMap =
     new TreeMap<byte [], List<KeyValue>>(Bytes.BYTES_COMPARATOR);
 
   /** Constructor for Writable.  DO NOT USE */
@@ -95,12 +95,12 @@ public class Delete implements Writable,
   /**
    * Create a Delete operation for the specified row and timestamp, using
    * an optional row lock.<p>
-   * 
+   *
    * If no further operations are done, this will delete all columns in all
-   * families of the specified row with a timestamp less than or equal to the 
+   * families of the specified row with a timestamp less than or equal to the
    * specified timestamp.<p>
-   * 
-   * This timestamp is ONLY used for a delete row operation.  If specifying 
+   *
+   * This timestamp is ONLY used for a delete row operation.  If specifying
    * families or columns, you must specify each timestamp individually.
    * @param row row key
    * @param timestamp maximum version timestamp (only for delete row)
@@ -170,7 +170,7 @@ public class Delete implements Writable,
     familyMap.put(family, list);
     return this;
   }
-  
+
   /**
    * Delete all versions of the specified column.
    * @param family family name
@@ -181,7 +181,7 @@ public class Delete implements Writable,
     this.deleteColumns(family, qualifier, HConstants.LATEST_TIMESTAMP);
     return this;
   }
-  
+
   /**
    * Delete all versions of the specified column with a timestamp less than
    * or equal to the specified timestamp.
@@ -200,7 +200,7 @@ public class Delete implements Writable,
     familyMap.put(family, list);
     return this;
   }
-  
+
   /**
    * Delete the latest version of the specified column.
    * This is an expensive call in that on the server-side, it first does a
@@ -214,7 +214,7 @@ public class Delete implements Writable,
     this.deleteColumn(family, qualifier, HConstants.LATEST_TIMESTAMP);
     return this;
   }
-  
+
   /**
    * Delete the specified version of the specified column.
    * @param family family name
@@ -232,15 +232,15 @@ public class Delete implements Writable,
     familyMap.put(family, list);
     return this;
   }
-  
+
   /**
-   * Method for retrieving the delete's familyMap 
+   * Method for retrieving the delete's familyMap
    * @return familyMap
    */
   public Map<byte [], List<KeyValue>> getFamilyMap() {
     return this.familyMap;
   }
-  
+
   /**
    *  Method for retrieving the delete's row
    * @return row
@@ -248,7 +248,7 @@ public class Delete implements Writable,
   public byte [] getRow() {
     return this.row;
   }
-  
+
   /**
    * Method for retrieving the delete's RowLock
    * @return RowLock
@@ -256,16 +256,16 @@ public class Delete implements Writable,
   public RowLock getRowLock() {
     return new RowLock(this.row, this.lockId);
   }
-  
+
   /**
    * Method for retrieving the delete's lock ID.
-   * 
+   *
    * @return The lock ID.
    */
   public long getLockId() {
 	return this.lockId;
   }
-  
+
   /**
    * Method for retrieving the delete's timestamp
    * @return timestamp
@@ -273,7 +273,7 @@ public class Delete implements Writable,
   public long getTimeStamp() {
     return this.ts;
   }
-  
+
   /**
    * @return string
    */
@@ -309,7 +309,7 @@ public class Delete implements Writable,
     sb.append("}");
     return sb.toString();
   }
-  
+
   //Writable
   public void readFields(final DataInput in) throws IOException {
     int version = in.readByte();
@@ -332,8 +332,8 @@ public class Delete implements Writable,
       }
       this.familyMap.put(family, list);
     }
-  }  
-  
+  }
+
   public void write(final DataOutput out) throws IOException {
     out.writeByte(DELETE_VERSION);
     Bytes.writeByteArray(out, this.row);

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Get.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Get.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Get.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/Get.java Fri May  7 19:26:45 2010
@@ -40,7 +40,7 @@ import java.util.TreeSet;
  * Used to perform Get operations on a single row.
  * <p>
  * To get everything for a row, instantiate a Get object with the row to get.
- * To further define the scope of what to get, perform additional methods as 
+ * To further define the scope of what to get, perform additional methods as
  * outlined below.
  * <p>
  * To get all columns from specific families, execute {@link #addFamily(byte[]) addFamily}
@@ -68,7 +68,7 @@ public class Get implements Writable {
   private int maxVersions = 1;
   private Filter filter = null;
   private TimeRange tr = new TimeRange();
-  private Map<byte [], NavigableSet<byte []>> familyMap = 
+  private Map<byte [], NavigableSet<byte []>> familyMap =
     new TreeMap<byte [], NavigableSet<byte []>>(Bytes.BYTES_COMPARATOR);
 
   /** Constructor for Writable.  DO NOT USE */
@@ -205,7 +205,7 @@ public class Get implements Writable {
 
   /**
    * Method for retrieving the get's row
-   * @return row 
+   * @return row
    */
   public byte [] getRow() {
     return this.row;
@@ -233,7 +233,7 @@ public class Get implements Writable {
    */
   public int getMaxVersions() {
     return this.maxVersions;
-  } 
+  }
 
   /**
    * Method for retrieving the get's TimeRange
@@ -294,7 +294,7 @@ public class Get implements Writable {
       return sb.toString();
     }
     boolean moreThanOne = false;
-    for(Map.Entry<byte [], NavigableSet<byte[]>> entry : 
+    for(Map.Entry<byte [], NavigableSet<byte[]>> entry :
       this.familyMap.entrySet()) {
       if(moreThanOne) {
         sb.append("), ");
@@ -343,7 +343,7 @@ public class Get 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);
@@ -376,7 +376,7 @@ public class Get implements Writable {
     }
     tr.write(out);
     out.writeInt(familyMap.size());
-    for(Map.Entry<byte [], NavigableSet<byte []>> entry : 
+    for(Map.Entry<byte [], NavigableSet<byte []>> entry :
       familyMap.entrySet()) {
       Bytes.writeByteArray(out, entry.getKey());
       NavigableSet<byte []> columnSet = entry.getValue();
@@ -400,7 +400,7 @@ public class Get implements Writable {
       return WritableFactories.newInstance(clazz, new Configuration());
     } catch (ClassNotFoundException e) {
       throw new RuntimeException("Can't find class " + className);
-    }    
+    }
   }
 
   /**

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java Fri May  7 19:26:45 2010
@@ -62,7 +62,7 @@ public class HBaseAdmin {
 
   /**
    * Constructor
-   * 
+   *
    * @param conf Configuration object
    * @throws MasterNotRunningException if the master is not running
    */
@@ -86,7 +86,7 @@ public class HBaseAdmin {
   public HMasterInterface getMaster() throws MasterNotRunningException{
     return this.connection.getMaster();
   }
-  
+
   /** @return - true if the master server is running */
   public boolean isMasterRunning() {
     return this.connection.isMasterRunning();
@@ -122,14 +122,14 @@ public class HBaseAdmin {
    * catalog table that just contains table names and their descriptors.
    * Right now, it only exists as part of the META table's region info.
    *
-   * @return - returns an array of HTableDescriptors 
+   * @return - returns an array of HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   public HTableDescriptor[] listTables() throws IOException {
     return this.connection.listTables();
   }
 
-  
+
   /**
    * Method for getting the tableDescriptor
    * @param tableName as a byte []
@@ -140,20 +140,20 @@ public class HBaseAdmin {
   throws IOException {
     return this.connection.getHTableDescriptor(tableName);
   }
-  
+
   private long getPauseTime(int tries) {
-	int triesCount = tries;
+    int triesCount = tries;
     if (triesCount >= HConstants.RETRY_BACKOFF.length)
-    	triesCount = HConstants.RETRY_BACKOFF.length - 1;
+      triesCount = HConstants.RETRY_BACKOFF.length - 1;
     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
   }
 
   /**
    * Creates a new table.
    * Synchronous operation.
-   * 
+   *
    * @param desc table descriptor for table
-   * 
+   *
    * @throws IllegalArgumentException if the table name is reserved
    * @throws MasterNotRunningException if master is not running
    * @throws TableExistsException if table already exists (If concurrent
@@ -287,7 +287,7 @@ public class HBaseAdmin {
   /**
    * Deletes a table.
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of table to delete
    * @throws IOException if a remote or network exception occurs
    */
@@ -298,7 +298,7 @@ public class HBaseAdmin {
   /**
    * Deletes a table.
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of table to delete
    * @throws IOException if a remote or network exception occurs
    */
@@ -376,12 +376,12 @@ public class HBaseAdmin {
     LOG.info("Deleted " + Bytes.toString(tableName));
   }
 
-  
+
 
   /**
    * Brings a table on-line (enables it).
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of the table
    * @throws IOException if a remote or network exception occurs
    */
@@ -392,7 +392,7 @@ public class HBaseAdmin {
   /**
    * Brings a table on-line (enables it).
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of the table
    * @throws IOException if a remote or network exception occurs
    */
@@ -437,7 +437,7 @@ public class HBaseAdmin {
    * Disables a table (takes it off-line) If it is being served, the master
    * will tell the servers to stop serving it.
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @throws IOException if a remote or network exception occurs
    */
@@ -449,7 +449,7 @@ public class HBaseAdmin {
    * Disables a table (takes it off-line) If it is being served, the master
    * will tell the servers to stop serving it.
    * Synchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @throws IOException if a remote or network exception occurs
    */
@@ -488,7 +488,7 @@ public class HBaseAdmin {
     }
     LOG.info("Disabled " + Bytes.toString(tableName));
   }
-  
+
   /**
    * @param tableName name of table to check
    * @return true if table is on-line
@@ -505,7 +505,7 @@ public class HBaseAdmin {
   public boolean isTableEnabled(byte[] tableName) throws IOException {
     return connection.isTableEnabled(tableName);
   }
-  
+
   /**
    * @param tableName name of table to check
    * @return true if table is off-line
@@ -536,7 +536,7 @@ public class HBaseAdmin {
   /**
    * Add a column to an existing table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of the table to add column to
    * @param column column descriptor of column to be added
    * @throws IOException if a remote or network exception occurs
@@ -549,7 +549,7 @@ public class HBaseAdmin {
   /**
    * Add a column to an existing table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of the table to add column to
    * @param column column descriptor of column to be added
    * @throws IOException if a remote or network exception occurs
@@ -570,7 +570,7 @@ public class HBaseAdmin {
   /**
    * Delete a column from a table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @param columnName name of column to be deleted
    * @throws IOException if a remote or network exception occurs
@@ -583,7 +583,7 @@ public class HBaseAdmin {
   /**
    * Delete a column from a table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @param columnName name of column to be deleted
    * @throws IOException if a remote or network exception occurs
@@ -604,13 +604,13 @@ public class HBaseAdmin {
   /**
    * Modify an existing column family on a table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @param columnName name of column to be modified
    * @param descriptor new column descriptor to use
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyColumn(final String tableName, final String columnName, 
+  public void modifyColumn(final String tableName, final String columnName,
       HColumnDescriptor descriptor)
   throws IOException {
     modifyColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName),
@@ -620,13 +620,13 @@ public class HBaseAdmin {
   /**
    * Modify an existing column family on a table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table
    * @param columnName name of column to be modified
    * @param descriptor new column descriptor to use
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyColumn(final byte [] tableName, final byte [] columnName, 
+  public void modifyColumn(final byte [] tableName, final byte [] columnName,
     HColumnDescriptor descriptor)
   throws IOException {
     if (this.master == null) {
@@ -643,7 +643,7 @@ public class HBaseAdmin {
   /**
    * Close a region. For expert-admins.
    * Asynchronous operation.
-   * 
+   *
    * @param regionname region name to close
    * @param args Optional server name.  Otherwise, we'll send close to the
    * server registered in .META.
@@ -657,7 +657,7 @@ public class HBaseAdmin {
   /**
    * Close a region.  For expert-admins.
    * Asynchronous operation.
-   * 
+   *
    * @param regionname region name to close
    * @param args Optional server name.  Otherwise, we'll send close to the
    * server registered in .META.
@@ -676,11 +676,11 @@ public class HBaseAdmin {
     modifyTable(HConstants.META_TABLE_NAME, HConstants.Modify.CLOSE_REGION,
       newargs);
   }
-  
+
   /**
    * Flush a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to flush
    * @throws IOException if a remote or network exception occurs
    */
@@ -691,7 +691,7 @@ public class HBaseAdmin {
   /**
    * Flush a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to flush
    * @throws IOException if a remote or network exception occurs
    */
@@ -702,7 +702,7 @@ public class HBaseAdmin {
   /**
    * Compact a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -713,18 +713,18 @@ public class HBaseAdmin {
   /**
    * Compact a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to compact
    * @throws IOException if a remote or network exception occurs
    */
   public void compact(final byte [] tableNameOrRegionName) throws IOException {
     modifyTable(tableNameOrRegionName, HConstants.Modify.TABLE_COMPACT);
   }
-  
+
   /**
    * Major compact a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to major compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -736,7 +736,7 @@ public class HBaseAdmin {
   /**
    * Major compact a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to major compact
    * @throws IOException if a remote or network exception occurs
    */
@@ -748,7 +748,7 @@ public class HBaseAdmin {
   /**
    * Split a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table or region to split
    * @throws IOException if a remote or network exception occurs
    */
@@ -759,7 +759,7 @@ public class HBaseAdmin {
   /**
    * Split a table or an individual region.
    * Asynchronous operation.
-   * 
+   *
    * @param tableNameOrRegionName table to region to split
    * @throws IOException if a remote or network exception occurs
    */
@@ -774,7 +774,7 @@ public class HBaseAdmin {
    * @param op
    * @throws IOException
    */
-  private void modifyTable(final byte [] tableNameOrRegionName, 
+  private void modifyTable(final byte [] tableNameOrRegionName,
       final HConstants.Modify op)
   throws IOException {
     if (tableNameOrRegionName == null) {
@@ -786,16 +786,16 @@ public class HBaseAdmin {
     Object [] args = regionName == null? null: new byte [][] {regionName};
     modifyTable(tableName == null? null: tableName, op, args);
   }
-  
+
   /**
    * Modify an existing table, more IRB friendly version.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table.
    * @param htd modified description of the table
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyTable(final byte [] tableName, HTableDescriptor htd) 
+  public void modifyTable(final byte [] tableName, HTableDescriptor htd)
   throws IOException {
     modifyTable(tableName, HConstants.Modify.TABLE_SET_HTD, htd);
   }
@@ -803,14 +803,14 @@ public class HBaseAdmin {
   /**
    * Modify an existing table.
    * Asynchronous operation.
-   * 
+   *
    * @param tableName name of table.  May be null if we are operating on a
    * region.
    * @param op table modification operation
    * @param args operation specific arguments
    * @throws IOException if a remote or network exception occurs
    */
-  public void modifyTable(final byte [] tableName, HConstants.Modify op, 
+  public void modifyTable(final byte [] tableName, HConstants.Modify op,
       Object... args)
       throws IOException {
     if (this.master == null) {
@@ -825,7 +825,7 @@ public class HBaseAdmin {
     try {
       switch (op) {
       case TABLE_SET_HTD:
-        if (args == null || args.length < 1 || 
+        if (args == null || args.length < 1 ||
             !(args[0] instanceof HTableDescriptor)) {
           throw new IllegalArgumentException("SET_HTD requires a HTableDescriptor");
         }
@@ -884,8 +884,8 @@ public class HBaseAdmin {
     }
   }
 
-  /** 
-   * Shuts down the HBase instance 
+  /**
+   * Shuts down the HBase instance
    * @throws IOException if a remote or network exception occurs
    */
   public synchronized void shutdown() throws IOException {

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Fri May  7 19:26:45 2010
@@ -52,7 +52,7 @@ public interface HConnection {
 
   /** @return - true if the master server is running */
   public boolean isMasterRunning();
-  
+
   /**
    * Checks if <code>tableName</code> exists.
    * @param tableName Table to check.
@@ -71,7 +71,7 @@ public interface HConnection {
    * @throws IOException if a remote or network exception occurs
    */
   public boolean isTableEnabled(byte[] tableName) throws IOException;
-  
+
   /**
    * @param tableName table name
    * @return true if the table is disabled, false otherwise
@@ -93,51 +93,51 @@ public interface HConnection {
    * catalog table that just contains table names and their descriptors.
    * Right now, it only exists as part of the META table's region info.
    *
-   * @return - returns an array of HTableDescriptors 
+   * @return - returns an array of HTableDescriptors
    * @throws IOException if a remote or network exception occurs
    */
   public HTableDescriptor[] listTables() throws IOException;
-  
+
   /**
    * @param tableName table name
-   * @return table metadata 
+   * @return table metadata
    * @throws IOException if a remote or network exception occurs
    */
   public HTableDescriptor getHTableDescriptor(byte[] tableName)
   throws IOException;
-  
+
   /**
    * Find the location of the region of <i>tableName</i> that <i>row</i>
    * lives in.
    * @param tableName name of the table <i>row</i> is in
    * @param row row key you're trying to find the region of
-   * @return HRegionLocation that describes where to find the reigon in 
+   * @return HRegionLocation that describes where to find the reigon in
    * question
    * @throws IOException if a remote or network exception occurs
    */
   public HRegionLocation locateRegion(final byte [] tableName,
       final byte [] row)
   throws IOException;
-  
+
   /**
    * Allows flushing the region cache.
    */
-  public void clearRegionCache(); 
-  
+  public void clearRegionCache();
+
   /**
    * Find the location of the region of <i>tableName</i> that <i>row</i>
    * lives in, ignoring any value that might be in the cache.
    * @param tableName name of the table <i>row</i> is in
    * @param row row key you're trying to find the region of
-   * @return HRegionLocation that describes where to find the reigon in 
+   * @return HRegionLocation that describes where to find the reigon in
    * question
    * @throws IOException if a remote or network exception occurs
    */
   public HRegionLocation relocateRegion(final byte [] tableName,
       final byte [] row)
-  throws IOException;  
-  
-  /** 
+  throws IOException;
+
+  /**
    * Establishes a connection to the region server at the specified address.
    * @param regionServer - the server to connect to
    * @return proxy for HRegionServer
@@ -145,8 +145,8 @@ public interface HConnection {
    */
   public HRegionInterface getHRegionConnection(HServerAddress regionServer)
   throws IOException;
-  
-  /** 
+
+  /**
    * Establishes a connection to the region server at the specified address.
    * @param regionServer - the server to connect to
    * @param getMaster - do we check if master is alive
@@ -156,7 +156,7 @@ public interface HConnection {
   public HRegionInterface getHRegionConnection(
       HServerAddress regionServer, boolean getMaster)
   throws IOException;
-  
+
   /**
    * Find region location hosting passed row
    * @param tableName table name
@@ -170,8 +170,8 @@ public interface HConnection {
   throws IOException;
 
   /**
-   * Pass in a ServerCallable with your particular bit of logic defined and 
-   * this method will manage the process of doing retries with timed waits 
+   * Pass in a ServerCallable with your particular bit of logic defined and
+   * this method will manage the process of doing retries with timed waits
    * and refinds of missing regions.
    *
    * @param <T> the type of the return value
@@ -180,9 +180,9 @@ public interface HConnection {
    * @throws IOException if a remote or network exception occurs
    * @throws RuntimeException other unspecified error
    */
-  public <T> T getRegionServerWithRetries(ServerCallable<T> callable) 
+  public <T> T getRegionServerWithRetries(ServerCallable<T> callable)
   throws IOException, RuntimeException;
-  
+
   /**
    * Pass in a ServerCallable with your particular bit of logic defined and
    * this method will pass it to the defined region server.
@@ -192,10 +192,10 @@ public interface HConnection {
    * @throws IOException if a remote or network exception occurs
    * @throws RuntimeException other unspecified error
    */
-  public <T> T getRegionServerForWithoutRetries(ServerCallable<T> callable) 
+  public <T> T getRegionServerForWithoutRetries(ServerCallable<T> callable)
   throws IOException, RuntimeException;
-  
-    
+
+
   /**
    * Process a batch of Puts. Does the retries.
    * @param list A batch of Puts to process.
@@ -215,9 +215,9 @@ public interface HConnection {
    */
   public int processBatchOfDeletes(List<Delete> list, byte[] tableName)
   throws IOException;
-  
+
   public void processBatchOfPuts(List<Put> list,
                                  final byte[] tableName, ExecutorService pool) throws IOException;
 
-  
+
 }

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Fri May  7 19:26:45 2010
@@ -67,7 +67,7 @@ import java.util.concurrent.atomic.Atomi
 /**
  * A non-instantiable class that manages connections to multiple tables in
  * multiple HBase instances.
- * 
+ *
  * Used by {@link HTable} and {@link HBaseAdmin}
  */
 public class HConnectionManager implements HConstants {
@@ -90,13 +90,13 @@ public class HConnectionManager implemen
   protected HConnectionManager() {
     super();
   }
-  
+
   private static final int MAX_CACHED_HBASE_INSTANCES=31;
-  // A LRU Map of master HBaseConfiguration -> connection information for that 
+  // A LRU Map of master HBaseConfiguration -> connection information for that
   // instance. The objects it contains are mutable and hence require
   // synchronized access to them.  We set instances to 31.  The zk default max
   // connections is 30 so should run into zk issues before hit this value of 31.
-  private static 
+  private static
   final Map<Integer, TableServers> HBASE_INSTANCES =
     new LinkedHashMap<Integer, TableServers>
       ((int) (MAX_CACHED_HBASE_INSTANCES/0.75F)+1, 0.75F, true) {
@@ -105,10 +105,10 @@ public class HConnectionManager implemen
         return size() > MAX_CACHED_HBASE_INSTANCES;
       }
   };
-  
-  private static final Map<String, ClientZKWatcher> ZK_WRAPPERS = 
+
+  private static final Map<String, ClientZKWatcher> ZK_WRAPPERS =
     new HashMap<String, ClientZKWatcher>();
-  
+
   /**
    * Get the connection object for the instance specified by the configuration
    * If no current connection exists, create a new connection for that instance
@@ -127,7 +127,7 @@ public class HConnectionManager implemen
     }
     return connection;
   }
-  
+
   /**
    * Delete connection information for the instance specified by configuration
    * @param conf configuration
@@ -180,7 +180,7 @@ public class HConnectionManager implemen
     }
     return ZK_WRAPPERS.get(ZooKeeperWrapper.getZookeeperClusterKey(conf));
   }
-  
+
   /**
    * This class is responsible to handle connection and reconnection
    * to a zookeeper quorum.
@@ -220,7 +220,7 @@ public class HConnectionManager implemen
         resetZooKeeper();
       }
     }
-    
+
     /**
      * Get this watcher's ZKW, instanciate it if necessary.
      * @return ZKW
@@ -229,10 +229,10 @@ public class HConnectionManager implemen
     public synchronized ZooKeeperWrapper getZooKeeperWrapper() throws IOException {
       if(zooKeeperWrapper == null) {
         zooKeeperWrapper = new ZooKeeperWrapper(conf, this);
-      } 
+      }
       return zooKeeperWrapper;
     }
-    
+
     /**
      * Clear this connection to zookeeper.
      */
@@ -257,25 +257,25 @@ public class HConnectionManager implemen
     private volatile boolean closed;
     private volatile HMasterInterface master;
     private volatile boolean masterChecked;
-    
+
     private final Object rootRegionLock = new Object();
     private final Object metaRegionLock = new Object();
     private final Object userRegionLock = new Object();
-        
+
     private volatile Configuration conf;
 
-    // Known region HServerAddress.toString() -> HRegionInterface 
+    // Known region HServerAddress.toString() -> HRegionInterface
     private final Map<String, HRegionInterface> servers =
       new ConcurrentHashMap<String, HRegionInterface>();
 
     // Used by master and region servers during safe mode only
-    private volatile HRegionLocation rootRegionLocation; 
-    
-    private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>> 
+    private volatile HRegionLocation rootRegionLocation;
+
+    private final Map<Integer, SoftValueSortedMap<byte [], HRegionLocation>>
       cachedRegionLocations =
         new HashMap<Integer, SoftValueSortedMap<byte [], HRegionLocation>>();
 
-    /** 
+    /**
      * constructor
      * @param conf Configuration object
      */
@@ -287,11 +287,11 @@ public class HConnectionManager implemen
         conf.get(REGION_SERVER_CLASS, DEFAULT_REGION_SERVER_CLASS);
 
       this.closed = false;
-      
+
       try {
         this.serverInterfaceClass =
           (Class<? extends HRegionInterface>) Class.forName(serverClassName);
-        
+
       } catch (ClassNotFoundException e) {
         throw new UnsupportedOperationException(
             "Unable to find region server interface " + serverClassName, e);
@@ -301,7 +301,7 @@ public class HConnectionManager implemen
       this.numRetries = conf.getInt("hbase.client.retries.number", 10);
       this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts", 1);
       this.rpcTimeout = conf.getLong(HBASE_REGIONSERVER_LEASE_PERIOD_KEY, DEFAULT_HBASE_REGIONSERVER_LEASE_PERIOD);
-      
+
       this.master = null;
       this.masterChecked = false;
     }
@@ -317,7 +317,7 @@ public class HConnectionManager implemen
     public void unsetRootRegionLocation() {
       this.rootRegionLocation = null;
     }
-    
+
     // Used by master and region servers during safe mode only
     public void setRootRegionLocation(HRegionLocation rootRegion) {
       if (rootRegion == null) {
@@ -326,7 +326,7 @@ public class HConnectionManager implemen
       }
       this.rootRegionLocation = rootRegion;
     }
-    
+
     public HMasterInterface getMaster() throws MasterNotRunningException {
       ZooKeeperWrapper zk;
       try {
@@ -347,15 +347,15 @@ public class HConnectionManager implemen
             masterLocation = zk.readMasterAddressOrThrow();
 
             HMasterInterface tryMaster = (HMasterInterface)HBaseRPC.getProxy(
-                HMasterInterface.class, HBaseRPCProtocolVersion.versionID, 
+                HMasterInterface.class, HBaseRPCProtocolVersion.versionID,
                 masterLocation.getInetSocketAddress(), this.conf);
-            
+
             if (tryMaster.isMasterRunning()) {
               this.master = tryMaster;
               this.masterLock.notifyAll();
               break;
             }
-            
+
           } catch (IOException e) {
             if (tries == numRetries - 1) {
               // This was our last chance - don't bother sleeping
@@ -390,7 +390,7 @@ public class HConnectionManager implemen
       if (this.master == null) {
         try {
           getMaster();
-          
+
         } catch (MasterNotRunningException e) {
           return false;
         }
@@ -420,7 +420,7 @@ public class HConnectionManager implemen
       }
       return exists;
     }
-    
+
     /*
      * @param n
      * @return Truen if passed tablename <code>n</code> is equal to the name
@@ -463,11 +463,11 @@ public class HConnectionManager implemen
 
       return uniqueTables.toArray(new HTableDescriptor[uniqueTables.size()]);
     }
-    
+
     public boolean isTableEnabled(byte[] tableName) throws IOException {
       return testTableOnlineState(tableName, true);
     }
-    
+
     public boolean isTableDisabled(byte[] tableName) throws IOException {
       return testTableOnlineState(tableName, false);
     }
@@ -489,7 +489,7 @@ public class HConnectionManager implemen
             }
           }
           return true;
-        }        
+        }
       };
       MetaScanner.metaScan(conf, visitor);
       return available.get();
@@ -522,7 +522,7 @@ public class HConnectionManager implemen
       scan.addColumn(CATALOG_FAMILY, REGIONINFO_QUALIFIER);
       int rows = this.conf.getInt("hbase.meta.scanner.caching", 100);
       scan.setCaching(rows);
-      ScannerCallable s = new ScannerCallable(this, 
+      ScannerCallable s = new ScannerCallable(this,
           (Bytes.equals(tableName, HConstants.META_TABLE_NAME) ?
               HConstants.ROOT_TABLE_NAME : HConstants.META_TABLE_NAME), scan);
       try {
@@ -560,7 +560,7 @@ public class HConnectionManager implemen
       return rowsScanned > 0 && onOffLine;
     }
 
-    private static class HTableDescriptorFinder 
+    private static class HTableDescriptorFinder
     implements MetaScanner.MetaScannerVisitor {
         byte[] tableName;
         HTableDescriptor result;
@@ -618,18 +618,18 @@ public class HConnectionManager implemen
         throw new IllegalArgumentException(
             "table name cannot be null or zero length");
       }
-            
+
       if (Bytes.equals(tableName, ROOT_TABLE_NAME)) {
         synchronized (rootRegionLock) {
           // This block guards against two threads trying to find the root
-          // region at the same time. One will go do the find while the 
+          // region at the same time. One will go do the find while the
           // second waits. The second thread will not do find.
-          
+
           if (!useCache || rootRegionLocation == null) {
             this.rootRegionLocation = locateRootRegion();
           }
           return this.rootRegionLocation;
-        }        
+        }
       } else if (Bytes.equals(tableName, META_TABLE_NAME)) {
         return locateRegionInMeta(ROOT_TABLE_NAME, tableName, row, useCache,
                                   metaRegionLock);
@@ -658,7 +658,7 @@ public class HConnectionManager implemen
           return location;
         }
       }
-      
+
       // build the key of the meta region we should be looking for.
       // the extra 9's on the end are necessary to allow "exact" matches
       // without knowing the precise region names.
@@ -666,7 +666,7 @@ public class HConnectionManager implemen
         HConstants.NINES);
       for (int tries = 0; true; tries++) {
         if (tries >= numRetries) {
-          throw new NoServerForRegionException("Unable to find region for " 
+          throw new NoServerForRegionException("Unable to find region for "
             + Bytes.toStringBinary(row) + " after " + numRetries + " tries.");
         }
 
@@ -702,10 +702,10 @@ public class HConnectionManager implemen
           if (regionInfoRow == null) {
             throw new TableNotFoundException(Bytes.toString(tableName));
           }
-          byte [] value = regionInfoRow.getValue(CATALOG_FAMILY, 
+          byte [] value = regionInfoRow.getValue(CATALOG_FAMILY,
               REGIONINFO_QUALIFIER);
           if (value == null || value.length == 0) {
-            throw new IOException("HRegionInfo was null or empty in " + 
+            throw new IOException("HRegionInfo was null or empty in " +
               Bytes.toString(parentTable) + ", row=" + regionInfoRow);
           }
           // convert the row result into the HRegionLocation we need!
@@ -717,21 +717,21 @@ public class HConnectionManager implemen
               "Table '" + Bytes.toString(tableName) + "' was not found.");
           }
           if (regionInfo.isOffline()) {
-            throw new RegionOfflineException("region offline: " + 
+            throw new RegionOfflineException("region offline: " +
               regionInfo.getRegionNameAsString());
           }
-          
+
           value = regionInfoRow.getValue(CATALOG_FAMILY, SERVER_QUALIFIER);
           String serverAddress = "";
           if(value != null) {
             serverAddress = Bytes.toString(value);
           }
-          if (serverAddress.equals("")) { 
+          if (serverAddress.equals("")) {
             throw new NoServerForRegionException("No server address listed " +
               "in " + Bytes.toString(parentTable) + " for region " +
               regionInfo.getRegionNameAsString());
           }
-        
+
           // instantiate the location
           location = new HRegionLocation(regionInfo,
             new HServerAddress(serverAddress));
@@ -773,10 +773,10 @@ public class HConnectionManager implemen
     /*
      * Search the cache for a location that fits our table and row key.
      * Return null if no suitable region is located. TODO: synchronization note
-     * 
+     *
      * <p>TODO: This method during writing consumes 15% of CPU doing lookup
      * into the Soft Reference SortedMap.  Improve.
-     * 
+     *
      * @param tableName
      * @param row
      * @return Null or region location found in cache.
@@ -883,7 +883,7 @@ public class HConnectionManager implemen
         }
       }
     }
-    
+
     /*
      * @param tableName
      * @return Map of cached locations for passed <code>tableName</code>
@@ -909,9 +909,9 @@ public class HConnectionManager implemen
      * Allows flushing the region cache.
      */
     public void clearRegionCache() {
-     cachedRegionLocations.clear();  
+     cachedRegionLocations.clear();
     }
-    
+
     /*
      * Put a newly discovered HRegionLocation into the cache.
      */
@@ -928,7 +928,7 @@ public class HConnectionManager implemen
     }
 
     public HRegionInterface getHRegionConnection(
-        HServerAddress regionServer, boolean getMaster) 
+        HServerAddress regionServer, boolean getMaster)
     throws IOException {
       if (getMaster) {
         getMaster();
@@ -941,7 +941,7 @@ public class HConnectionManager implemen
           try {
             server = (HRegionInterface)HBaseRPC.waitForProxy(
                 serverInterfaceClass, HBaseRPCProtocolVersion.versionID,
-                regionServer.getInetSocketAddress(), this.conf, 
+                regionServer.getInetSocketAddress(), this.conf,
                 this.maxRPCAttempts, this.rpcTimeout);
           } catch (RemoteException e) {
             throw RemoteExceptionHandler.decodeRemoteException(e);
@@ -951,9 +951,9 @@ public class HConnectionManager implemen
       }
       return server;
     }
-    
+
     public HRegionInterface getHRegionConnection(
-        HServerAddress regionServer) 
+        HServerAddress regionServer)
     throws IOException {
       return getHRegionConnection(regionServer, false);
     }
@@ -969,7 +969,7 @@ public class HConnectionManager implemen
      * @return HRegionLocation for root region if found
      * @throws NoServerForRegionException - if the root region can not be
      * located after retrying
-     * @throws IOException 
+     * @throws IOException
      */
     private HRegionLocation locateRootRegion()
     throws IOException {
@@ -1022,7 +1022,7 @@ public class HConnectionManager implemen
             throw new NoServerForRegionException("Timed out trying to locate "+
                 "root region because: " + t.getMessage());
           }
-          
+
           // Sleep and retry finding root region.
           try {
             if (LOG.isDebugEnabled()) {
@@ -1036,17 +1036,17 @@ public class HConnectionManager implemen
             // continue
           }
         }
-        
+
         rootRegionAddress = null;
       }
-      
+
       // if the address is null by this point, then the retries have failed,
       // and we're sort of sunk
       if (rootRegionAddress == null) {
         throw new NoServerForRegionException(
           "unable to locate root region server");
       }
-      
+
       // return the region location
       return new HRegionLocation(
         HRegionInfo.ROOT_REGIONINFO, rootRegionAddress);
@@ -1074,9 +1074,9 @@ public class HConnectionManager implemen
           // continue
         }
       }
-      return null;    
+      return null;
     }
-    
+
     public <T> T getRegionServerForWithoutRetries(ServerCallable<T> callable)
         throws IOException, RuntimeException {
       try {

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTable.java Fri May  7 19:26:45 2010
@@ -74,7 +74,7 @@ public class HTable implements HTableInt
   private int maxKeyValueSize;
 
   private long maxScannerResultSize;
-  
+
   /**
    * Creates an object to access a HBase table.
    *
@@ -134,9 +134,9 @@ public class HTable implements HTableInt
     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);
 
@@ -454,7 +454,7 @@ public class HTable implements HTableInt
       flushCommits();
     }
   }
-  
+
   public long incrementColumnValue(final byte [] row, final byte [] family,
       final byte [] qualifier, final long amount)
   throws IOException {
@@ -653,7 +653,7 @@ public class HTable implements HTableInt
   public ArrayList<Put> getWriteBuffer() {
     return writeBuffer;
   }
-  
+
   /**
    * Implements the scanner interface for the HBase client.
    * If there are multiple regions in a table, this scanner will iterate

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableFactory.java Fri May  7 19:26:45 2010
@@ -42,14 +42,14 @@ public class HTableFactory implements HT
 
   @Override
   public void releaseHTableInterface(HTableInterface table) {
-    try { 
+    try {
       table.close();
     } catch (IOException ioe) {
       throw new RuntimeException(ioe);
     }
-    
+
   }
-  
-  
-  
+
+
+
 }

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Fri May  7 19:26:45 2010
@@ -230,7 +230,7 @@ public interface HTableInterface {
    * @throws IOException if a remote or network exception occurs.
    */
   long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
-      long amount, boolean writeToWAL) throws IOException; 
+      long amount, boolean writeToWAL) throws IOException;
 
   /**
    * Tells whether or not 'auto-flush' is turned on.

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTableInterfaceFactory.java Fri May  7 19:26:45 2010
@@ -37,8 +37,8 @@ public interface HTableInterfaceFactory 
    * @return HTableInterface instance.
    */
   HTableInterface createHTableInterface(Configuration config, byte[] tableName);
-  
-  
+
+
   /**
    * Release the HTable resource represented by the table.
    * @param table

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Fri May  7 19:26:45 2010
@@ -125,15 +125,15 @@ public class HTablePool {
   protected HTableInterface createHTable(String tableName) {
     return this.tableFactory.createHTableInterface(config, Bytes.toBytes(tableName));
   }
-  
+
   /**
    * Closes all the HTable instances , belonging to the given table, in the table pool.
    * <p>
-   * Note: this is a 'shutdown' of the given table pool and different from 
-   * {@link #putTable(HTableInterface)}, that is used to return the table 
+   * Note: this is a 'shutdown' of the given table pool and different from
+   * {@link #putTable(HTableInterface)}, that is used to return the table
    * instance to the pool for future re-use.
-   *  
-   * @param tableName 
+   *
+   * @param tableName
    */
   public void closeTablePool(final String tableName)  {
     Queue<HTableInterface> queue = tables.get(tableName);
@@ -149,7 +149,7 @@ public class HTablePool {
 
   /**
    * See {@link #closeTablePool(String)}.
-   * 
+   *
    * @param tableName
    */
   public void closeTablePool(final byte[] tableName)  {
@@ -161,5 +161,5 @@ public class HTablePool {
     synchronized(queue) {
       return queue.size();
     }
-  }  
+  }
 }

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java Fri May  7 19:26:45 2010
@@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.util.Byte
 import java.io.IOException;
 
 /**
- * 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 conf
    * @param visitor A custom visitor
    * @throws IOException e
@@ -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 config
    * @param visitor visitor object
    * @param tableName table name
@@ -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;
-    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; //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 result
      * @return A boolean to know if it should continue to loop in the region
      * @throws IOException e

Modified: hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPut.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPut.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPut.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPut.java Fri May  7 19:26:45 2010
@@ -62,7 +62,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/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPutResponse.java
URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPutResponse.java?rev=942186&r1=942185&r2=942186&view=diff
==============================================================================
--- hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPutResponse.java (original)
+++ hadoop/hbase/trunk/core/src/main/java/org/apache/hadoop/hbase/client/MultiPutResponse.java Fri May  7 19:26:45 2010
@@ -47,7 +47,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());