浏览代码

HADOOP-1466 Clean up warnings, visibility and javadoc issues in HBase.

Works in my environment. Since no changes were made to the code aside from white space adjustment, not testing with Hudson.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@562608 13f79535-47bb-0310-9956-ffa450edef68
Jim Kellerman 18 年之前
父节点
当前提交
719b1dbe95
共有 55 个文件被更改,包括 629 次插入318 次删除
  1. 1 0
      src/contrib/hbase/CHANGES.txt
  2. 6 9
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java
  3. 11 14
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java
  4. 4 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseConfiguration.java
  5. 9 13
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java
  6. 2 1
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java
  7. 17 4
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java
  8. 44 11
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java
  9. 19 2
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java
  10. 16 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java
  11. 24 8
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HServerInfo.java
  12. 105 48
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java
  13. 13 17
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreKey.java
  14. 9 6
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java
  15. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/InvalidColumnNameException.java
  16. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/LockException.java
  17. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/MasterNotRunningException.java
  18. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/NoServerForRegionException.java
  19. 10 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/NotServingRegionException.java
  20. 10 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/TableExistsException.java
  21. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/TableNotDisabledException.java
  22. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/UnknownScannerException.java
  23. 8 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/WrongRegionException.java
  24. 6 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java
  25. 3 5
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java
  26. 3 8
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java
  27. 1 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java
  28. 15 41
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java
  29. 11 25
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java
  30. 13 36
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java
  31. 19 13
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java
  32. 2 8
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/KeyedData.java
  33. 2 0
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/KeyedDataArrayWritable.java
  34. 1 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java
  35. 1 3
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/TableMap.java
  36. 5 12
      src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java
  37. 13 2
      src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java
  38. 13 0
      src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java
  39. 13 0
      src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java
  40. 4 9
      src/contrib/hbase/src/java/org/onelab/filter/Filter.java
  41. 5 9
      src/contrib/hbase/src/java/org/onelab/filter/Key.java
  42. 6 0
      src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java
  43. 15 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java
  44. 1 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java
  45. 2 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java
  46. 3 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java
  47. 7 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java
  48. 11 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java
  49. 13 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java
  50. 20 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java
  51. 18 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java
  52. 14 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java
  53. 18 1
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java
  54. 4 4
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java
  55. 11 0
      src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestKeying.java

+ 1 - 0
src/contrib/hbase/CHANGES.txt

@@ -84,3 +84,4 @@ Trunk (unreleased changes)
      (except TestHClient and HBaseShell) have been converted to use the new client
      (except TestHClient and HBaseShell) have been converted to use the new client
      side objects (HTable/HBaseAdmin/HConnection) instead of HClient.
      side objects (HTable/HBaseAdmin/HConnection) instead of HClient.
  53. HADOOP-1528 HClient for multiple tables - expose close table function
  53. HADOOP-1528 HClient for multiple tables - expose close table function
+ 54. HADOOP-1466 Clean up warnings, visibility and javadoc issues in HBase.

+ 6 - 9
src/contrib/hbase/src/java/org/apache/hadoop/hbase/BloomFilterDescriptor.java

@@ -79,6 +79,7 @@ public class BloomFilterDescriptor implements WritableComparable {
   int vectorSize;
   int vectorSize;
   int nbHash;
   int nbHash;
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     StringBuilder value = new StringBuilder();
     StringBuilder value = new StringBuilder();
@@ -103,11 +104,13 @@ public class BloomFilterDescriptor implements WritableComparable {
     return value.toString();
     return value.toString();
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     return compareTo(obj) == 0;
     return compareTo(obj) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = Integer.valueOf(this.filterType).hashCode();
     int result = Integer.valueOf(this.filterType).hashCode();
@@ -118,18 +121,14 @@ public class BloomFilterDescriptor implements WritableComparable {
 
 
   // Writable
   // Writable
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     filterType = in.readInt();
     filterType = in.readInt();
     vectorSize = in.readInt();
     vectorSize = in.readInt();
     nbHash = in.readInt();
     nbHash = in.readInt();
   }
   }
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeInt(filterType);
     out.writeInt(filterType);
     out.writeInt(vectorSize);
     out.writeInt(vectorSize);
@@ -138,9 +137,7 @@ public class BloomFilterDescriptor implements WritableComparable {
   
   
   // Comparable
   // Comparable
   
   
-  /* (non-Javadoc)
-   * @see java.lang.Comparable#compareTo(java.lang.Object)
-   */
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     BloomFilterDescriptor other = (BloomFilterDescriptor)o;
     BloomFilterDescriptor other = (BloomFilterDescriptor)o;
     int result = this.filterType - other.filterType;
     int result = this.filterType - other.filterType;

+ 11 - 14
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HAbstractScanner.java

@@ -41,7 +41,7 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
   static Pattern isRegexPattern =
   static Pattern isRegexPattern =
     Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
     Pattern.compile("^.*[\\\\+|^&*$\\[\\]\\}{)(]+.*$");
   
   
-  // The kind of match we are doing on a column:
+  /** The kind of match we are doing on a column: */
   private static enum MATCH_TYPE {
   private static enum MATCH_TYPE {
     /** Just check the column family name */
     /** Just check the column family name */
     FAMILY_ONLY,
     FAMILY_ONLY,
@@ -51,11 +51,13 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
     SIMPLE
     SIMPLE
   }
   }
 
 
-  // This class provides column matching functions that are more sophisticated
-  // than a simple string compare. There are three types of matching:
-  // 1. Match on the column family name only
-  // 2. Match on the column family + column key regex
-  // 3. Simple match: compare column family + column key literally
+  /**
+   * This class provides column matching functions that are more sophisticated
+   * than a simple string compare. There are three types of matching:
+   * 1. Match on the column family name only
+   * 2. Match on the column family + column key regex
+   * 3. Simple match: compare column family + column key literally
+   */
   private static class ColumnMatcher {
   private static class ColumnMatcher {
     private boolean wildCardmatch;
     private boolean wildCardmatch;
     private MATCH_TYPE matchType;
     private MATCH_TYPE matchType;
@@ -84,8 +86,7 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
       }
       }
     }
     }
     
     
-    // Matching method
-    
+    /** Matching method */
     boolean matches(Text c) throws IOException {
     boolean matches(Text c) throws IOException {
       if(this.matchType == MATCH_TYPE.SIMPLE) {
       if(this.matchType == MATCH_TYPE.SIMPLE) {
         return c.equals(this.col);
         return c.equals(this.col);
@@ -187,16 +188,12 @@ public abstract class HAbstractScanner implements HInternalScannerInterface {
   /** Mechanism used to shut down the whole scan */
   /** Mechanism used to shut down the whole scan */
   public abstract void close();
   public abstract void close();
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hbase.HInternalScannerInterface#isWildcardScanner()
-   */
+  /** {@inheritDoc} */
   public boolean isWildcardScanner() {
   public boolean isWildcardScanner() {
     return this.wildcardMatch;
     return this.wildcardMatch;
   }
   }
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hbase.HInternalScannerInterface#isMultipleMatchScanner()
-   */
+  /** {@inheritDoc} */
   public boolean isMultipleMatchScanner() {
   public boolean isMultipleMatchScanner() {
     return this.multipleMatchers;
     return this.multipleMatchers;
   }
   }

+ 4 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HBaseConfiguration.java

@@ -21,7 +21,11 @@ package org.apache.hadoop.hbase;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 
 
+/**
+ * Adds HBase configuration files to a Configuration
+ */
 public class HBaseConfiguration extends Configuration {
 public class HBaseConfiguration extends Configuration {
+  /** constructor */
   public HBaseConfiguration() {
   public HBaseConfiguration() {
     super();
     super();
     addDefaultResource("hbase-default.xml");
     addDefaultResource("hbase-default.xml");

+ 9 - 13
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HColumnDescriptor.java

@@ -148,16 +148,12 @@ public class HColumnDescriptor implements WritableComparable {
     this.versionNumber = COLUMN_DESCRIPTOR_VERSION;
     this.versionNumber = COLUMN_DESCRIPTOR_VERSION;
   }
   }
   
   
-  /**
-   * @return    - name of column family
-   */
+  /** @return name of column family */
   public Text getName() {
   public Text getName() {
     return name;
     return name;
   }
   }
   
   
-  /**
-   * @return    - compression type being used for the column family
-   */
+  /** @return compression type being used for the column family */
   public CompressionType getCompression() {
   public CompressionType getCompression() {
     CompressionType value = null;
     CompressionType value = null;
 
 
@@ -176,13 +172,12 @@ public class HColumnDescriptor implements WritableComparable {
     return value;
     return value;
   }
   }
   
   
-  /**
-   * @return    - maximum number of versions
-   */
+  /** @return maximum number of versions */
   public int getMaxVersions() {
   public int getMaxVersions() {
     return this.maxVersions;
     return this.maxVersions;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     String compression = "none";
     String compression = "none";
@@ -205,11 +200,13 @@ public class HColumnDescriptor implements WritableComparable {
       + (bloomFilterSpecified ? bloomFilter.toString() : "none") + ")";
       + (bloomFilterSpecified ? bloomFilter.toString() : "none") + ")";
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     return compareTo(obj) == 0;
     return compareTo(obj) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = this.name.hashCode();
     int result = this.name.hashCode();
@@ -225,10 +222,9 @@ public class HColumnDescriptor implements WritableComparable {
     return result;
     return result;
   }
   }
   
   
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.versionNumber = in.readByte();
     this.versionNumber = in.readByte();
     this.name.readFields(in);
     this.name.readFields(in);
@@ -244,6 +240,7 @@ public class HColumnDescriptor implements WritableComparable {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeByte(this.versionNumber);
     out.writeByte(this.versionNumber);
     this.name.write(out);
     this.name.write(out);
@@ -258,10 +255,9 @@ public class HColumnDescriptor implements WritableComparable {
     }
     }
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Comparable
   // Comparable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     // NOTE: we don't do anything with the version number yet.
     // NOTE: we don't do anything with the version number yet.
     // Version numbers will come into play when we introduce an incompatible
     // Version numbers will come into play when we introduce an incompatible

+ 2 - 1
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HConnectionManager.java

@@ -113,7 +113,8 @@ public class HConnectionManager implements HConstants {
     // Known region HServerAddress.toString() -> HRegionInterface 
     // Known region HServerAddress.toString() -> HRegionInterface 
     private HashMap<String, HRegionInterface> servers;
     private HashMap<String, HRegionInterface> servers;
 
 
-    /** constructor
+    /** 
+     * constructor
      * @param conf Configuration object
      * @param conf Configuration object
      */
      */
     @SuppressWarnings("unchecked")
     @SuppressWarnings("unchecked")

+ 17 - 4
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HLogEdit.java

@@ -23,50 +23,62 @@ import org.apache.hadoop.io.*;
 
 
 import java.io.*;
 import java.io.*;
 
 
-/*******************************************************************************
+/**
  * A log value.
  * A log value.
  *
  *
  * These aren't sortable; you need to sort by the matching HLogKey.
  * These aren't sortable; you need to sort by the matching HLogKey.
  * The table and row are already identified in HLogKey.
  * The table and row are already identified in HLogKey.
  * This just indicates the column and value.
  * This just indicates the column and value.
- ******************************************************************************/
+ */
 public class HLogEdit implements Writable {
 public class HLogEdit implements Writable {
   private Text column = new Text();
   private Text column = new Text();
   private byte [] val;
   private byte [] val;
   private long timestamp;
   private long timestamp;
 
 
+  /**
+   * Default constructor used by Writable
+   */
   public HLogEdit() {
   public HLogEdit() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Construct a fully initialized HLogEdit
+   * @param column column name
+   * @param bval value
+   * @param timestamp timestamp for modification
+   */
   public HLogEdit(Text column, byte [] bval, long timestamp) {
   public HLogEdit(Text column, byte [] bval, long timestamp) {
     this.column.set(column);
     this.column.set(column);
     this.val = bval;
     this.val = bval;
     this.timestamp = timestamp;
     this.timestamp = timestamp;
   }
   }
 
 
+  /** @return the column */
   public Text getColumn() {
   public Text getColumn() {
     return this.column;
     return this.column;
   }
   }
 
 
+  /** @return the value */
   public byte [] getVal() {
   public byte [] getVal() {
     return this.val;
     return this.val;
   }
   }
 
 
+  /** @return the timestamp */
   public long getTimestamp() {
   public long getTimestamp() {
     return this.timestamp;
     return this.timestamp;
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     return getColumn().toString() + " " + this.getTimestamp() + " " +
     return getColumn().toString() + " " + this.getTimestamp() + " " +
       new String(getVal()).trim();
       new String(getVal()).trim();
   }
   }
   
   
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     this.column.write(out);
     this.column.write(out);
     out.writeShort(this.val.length);
     out.writeShort(this.val.length);
@@ -74,6 +86,7 @@ public class HLogEdit implements Writable {
     out.writeLong(timestamp);
     out.writeLong(timestamp);
   }
   }
   
   
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.column.readFields(in);
     this.column.readFields(in);
     this.val = new byte[in.readShort()];
     this.val = new byte[in.readShort()];

+ 44 - 11
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterInterface.java

@@ -30,35 +30,68 @@ import java.io.IOException;
  * tables.
  * tables.
  */
  */
 public interface HMasterInterface extends VersionedProtocol {
 public interface HMasterInterface extends VersionedProtocol {
-  public static final long versionID = 1L; // initial version
-
-  //////////////////////////////////////////////////////////////////////////////
-  // Check to see if master is available
-  //////////////////////////////////////////////////////////////////////////////
+  /** Interface version */
+  public static final long versionID = 1L;
 
 
+  /** @return true if master is available */
   public boolean isMasterRunning();
   public boolean isMasterRunning();
   
   
-  //////////////////////////////////////////////////////////////////////////////
   // Admin tools would use these cmds
   // Admin tools would use these cmds
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /**
+   * Creates a new table
+   * @param desc table descriptor
+   * @throws IOException
+   */
   public void createTable(HTableDescriptor desc) throws IOException;
   public void createTable(HTableDescriptor desc) throws IOException;
+
+  /**
+   * Deletes a table
+   * @param tableName
+   * @throws IOException
+   */
   public void deleteTable(Text tableName) throws IOException;
   public void deleteTable(Text tableName) throws IOException;
   
   
+  /**
+   * Adds a column to the specified table
+   * @param tableName
+   * @param column column descriptor
+   * @throws IOException
+   */
   public void addColumn(Text tableName, HColumnDescriptor column) throws IOException;
   public void addColumn(Text tableName, HColumnDescriptor column) throws IOException;
+
+  /**
+   * Deletes a column from the specified table
+   * @param tableName
+   * @param columnName
+   * @throws IOException
+   */
   public void deleteColumn(Text tableName, Text columnName) throws IOException;
   public void deleteColumn(Text tableName, Text columnName) throws IOException;
   
   
+  /**
+   * Puts the table on-line (only needed if table has been previously taken offline)
+   * @param tableName
+   * @throws IOException
+   */
   public void enableTable(Text tableName) throws IOException;
   public void enableTable(Text tableName) throws IOException;
+  
+  /**
+   * Take table offline
+   * 
+   * @param tableName
+   * @throws IOException
+   */
   public void disableTable(Text tableName) throws IOException;
   public void disableTable(Text tableName) throws IOException;
   
   
   /**
   /**
    * Shutdown an HBase cluster.
    * Shutdown an HBase cluster.
+   * @throws IOException
    */
    */
   public void shutdown() throws IOException;
   public void shutdown() throws IOException;
 
 
-  //////////////////////////////////////////////////////////////////////////////
-  // These are the method calls of last resort when trying to find an HRegion
-  //////////////////////////////////////////////////////////////////////////////
-
+  /**
+   * Get the location of the root region
+   * @return address of server that serves the root region
+   */
   public HServerAddress findRootRegion();
   public HServerAddress findRootRegion();
 }
 }

+ 19 - 2
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HMasterRegionInterface.java

@@ -22,13 +22,30 @@ package org.apache.hadoop.hbase;
 import java.io.IOException;
 import java.io.IOException;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.ipc.VersionedProtocol;
 
 
-/*******************************************************************************
+/**
  * HRegionServers interact with the HMasterRegionInterface to report on local 
  * HRegionServers interact with the HMasterRegionInterface to report on local 
  * goings-on and to obtain data-handling instructions from the HMaster.
  * goings-on and to obtain data-handling instructions from the HMaster.
- *********************************************/
+ */
 public interface HMasterRegionInterface extends VersionedProtocol {
 public interface HMasterRegionInterface extends VersionedProtocol {
+  /** Interface version number */
   public static final long versionID = 1L;
   public static final long versionID = 1L;
+  
+  /**
+   * Called when a region server first starts
+   * @param info
+   * @throws IOException
+   */
   public void regionServerStartup(HServerInfo info) throws IOException;
   public void regionServerStartup(HServerInfo info) throws IOException;
+  
+  /**
+   * Called to renew lease, tell master what the region server is doing and to
+   * receive new instructions from the master
+   * 
+   * @param info server's address and start code
+   * @param msgs things the region server wants to tell the master
+   * @return instructions from the master to the region server
+   * @throws IOException
+   */
   public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
   public HMsg[] regionServerReport(HServerInfo info, HMsg msgs[])
   throws IOException;
   throws IOException;
 }
 }

+ 16 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HScannerInterface.java

@@ -24,11 +24,24 @@ import org.apache.hadoop.io.*;
 import java.io.*;
 import java.io.*;
 import java.util.*;
 import java.util.*;
 
 
-/*******************************************************************************
- * HScannerInterface iterates through a set of rows.  It's implemented by several classes.
- ******************************************************************************/
+/**
+ * HScannerInterface iterates through a set of rows.  It's implemented by
+ * several classes.
+ */
 public interface HScannerInterface {
 public interface HScannerInterface {
+  /**
+   * Get the next set of values
+   * @param key will contain the row and timestamp upon return
+   * @param results will contain an entry for each column family member and its value
+   * @return true if data was returned
+   * @throws IOException
+   */
   public boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
   public boolean next(HStoreKey key, TreeMap<Text, byte[]> results)
   throws IOException;
   throws IOException;
+  
+  /**
+   * Closes a scanner and releases any resources it has allocated
+   * @throws IOException
+   */
   public void close() throws IOException;
   public void close() throws IOException;
 }
 }

+ 24 - 8
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HServerInfo.java

@@ -23,51 +23,67 @@ import org.apache.hadoop.io.*;
 
 
 import java.io.*;
 import java.io.*;
 
 
-/*******************************************************************************
- * HRSInfo contains metainfo about an HRegionServer, including details about the
- * source machine and load statistics.
- ******************************************************************************/
+/**
+ * HServerInfo contains metainfo about an HRegionServer, Currently it only
+ * contains the server start code.
+ * 
+ * In the future it will contain information about the source machine and
+ * load statistics.
+ */
 public class HServerInfo implements Writable {
 public class HServerInfo implements Writable {
   private HServerAddress serverAddress;
   private HServerAddress serverAddress;
   private long startCode;
   private long startCode;
 
 
+  /** default constructor - used by Writable */
   public HServerInfo() {
   public HServerInfo() {
     this.serverAddress = new HServerAddress();
     this.serverAddress = new HServerAddress();
     this.startCode = 0;
     this.startCode = 0;
   }
   }
   
   
+  /**
+   * Constructs a fully initialized object
+   * @param serverAddress
+   * @param startCode
+   */
   public HServerInfo(HServerAddress serverAddress, long startCode) {
   public HServerInfo(HServerAddress serverAddress, long startCode) {
     this.serverAddress = new HServerAddress(serverAddress);
     this.serverAddress = new HServerAddress(serverAddress);
     this.startCode = startCode;
     this.startCode = startCode;
   }
   }
   
   
+  /**
+   * Construct a new object using another as input (like a copy constructor)
+   * @param other
+   */
   public HServerInfo(HServerInfo other) {
   public HServerInfo(HServerInfo other) {
     this.serverAddress = new HServerAddress(other.getServerAddress());
     this.serverAddress = new HServerAddress(other.getServerAddress());
     this.startCode = other.getStartCode();
     this.startCode = other.getStartCode();
   }
   }
-  
+
+  /** @return the server address */
   public HServerAddress getServerAddress() {
   public HServerAddress getServerAddress() {
     return serverAddress;
     return serverAddress;
   }
   }
-  
+ 
+  /** @return the server start code */
   public long getStartCode() {
   public long getStartCode() {
     return startCode;
     return startCode;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "address: " + this.serverAddress + ", startcode: " + this.startCode;
     return "address: " + this.serverAddress + ", startcode: " + this.startCode;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.serverAddress.readFields(in);
     this.serverAddress.readFields(in);
     this.startCode = in.readLong();
     this.startCode = in.readLong();
   }
   }
 
 
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     this.serverAddress.write(out);
     this.serverAddress.write(out);
     out.writeLong(this.startCode);
     out.writeLong(this.startCode);

+ 105 - 48
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreFile.java

@@ -29,14 +29,21 @@ import org.apache.hadoop.conf.*;
 import java.io.*;
 import java.io.*;
 import java.util.*;
 import java.util.*;
 
 
-/*******************************************************************************
+/**
  * Each HStore maintains a bunch of different data files.
  * Each HStore maintains a bunch of different data files.
  *
  *
+ * An HStoreFile tracks 4 things: its parent dir, the region identifier, the 
+ * column family, and the file identifier.  If you know those four things, you
+ * know how to obtain the right HStoreFile.
+ *
+ * When merging or splitting HRegions, we might want to modify one of the 
+ * params for an HStoreFile (effectively moving it elsewhere).
+ * 
  * The filename is a mix of the parent dir, the region name, the column name, 
  * The filename is a mix of the parent dir, the region name, the column name, 
  * and the file identifier.
  * and the file identifier.
  * 
  * 
  * This class handles all that path-building stuff for you.
  * This class handles all that path-building stuff for you.
- ******************************************************************************/
+ */
 public class HStoreFile implements HConstants, WritableComparable {
 public class HStoreFile implements HConstants, WritableComparable {
   private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
   private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
   static final byte INFO_SEQ_NUM = 0;
   static final byte INFO_SEQ_NUM = 0;
@@ -53,14 +60,7 @@ public class HStoreFile implements HConstants, WritableComparable {
   long fileId;
   long fileId;
   Configuration conf;
   Configuration conf;
 
 
-  /**
-   * An HStoreFile tracks 4 things: its parent dir, the region identifier, the 
-   * column family, and the file identifier.  If you know those four things, you
-   * know how to obtain the right HStoreFile.
-   *
-   * When merging or splitting HRegions, we might want to modify one of the 
-   * params for an HStoreFile (effectively moving it elsewhere).
-   */
+  /** Constructor used by Writable */
   HStoreFile(Configuration conf) {
   HStoreFile(Configuration conf) {
     this.conf = conf;
     this.conf = conf;
     this.dir = new Path(Path.CUR_DIR);
     this.dir = new Path(Path.CUR_DIR);
@@ -69,6 +69,14 @@ public class HStoreFile implements HConstants, WritableComparable {
     this.fileId = 0;
     this.fileId = 0;
   }
   }
   
   
+  /**
+   * Constructor that fully initializes the object
+   * @param conf Configuration object
+   * @param dir directory path
+   * @param regionName name of the region
+   * @param colFamily name of the column family
+   * @param fileId file identifier
+   */
   HStoreFile(Configuration conf, Path dir, Text regionName, 
   HStoreFile(Configuration conf, Path dir, Text regionName, 
       Text colFamily, long fileId) {
       Text colFamily, long fileId) {
     
     
@@ -79,31 +87,35 @@ public class HStoreFile implements HConstants, WritableComparable {
     this.fileId = fileId;
     this.fileId = fileId;
   }
   }
 
 
-  // Get the individual components
-  
+  /** @return the directory path */
   Path getDir() {
   Path getDir() {
     return dir;
     return dir;
   }
   }
-  
+
+  /** @return the region name */
   Text getRegionName() {
   Text getRegionName() {
     return regionName;
     return regionName;
   }
   }
-  
+
+  /** @return the column family */
   Text getColFamily() {
   Text getColFamily() {
     return colFamily;
     return colFamily;
   }
   }
-  
+
+  /** @return the file identifier */
   long fileId() {
   long fileId() {
     return fileId;
     return fileId;
   }
   }
 
 
   // Build full filenames from those components
   // Build full filenames from those components
   
   
+  /** @return path for MapFile */
   Path getMapFilePath() {
   Path getMapFilePath() {
     return new Path(HStoreFile.getMapDir(dir, regionName, colFamily), 
     return new Path(HStoreFile.getMapDir(dir, regionName, colFamily), 
         HSTORE_DATFILE_PREFIX + fileId);
         HSTORE_DATFILE_PREFIX + fileId);
   }
   }
   
   
+  /** @return path for info file */
   Path getInfoFilePath() {
   Path getInfoFilePath() {
     return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily), 
     return new Path(HStoreFile.getInfoDir(dir, regionName, colFamily), 
         HSTORE_INFOFILE_PREFIX + fileId);
         HSTORE_INFOFILE_PREFIX + fileId);
@@ -111,34 +123,41 @@ public class HStoreFile implements HConstants, WritableComparable {
 
 
   // Static methods to build partial paths to internal directories.  Useful for 
   // Static methods to build partial paths to internal directories.  Useful for 
   // HStore construction and log-rebuilding.
   // HStore construction and log-rebuilding.
-  
+
+  /** @return the map directory path */
   static Path getMapDir(Path dir, Text regionName, Text colFamily) {
   static Path getMapDir(Path dir, Text regionName, Text colFamily) {
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
         new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
         new Path(colFamily.toString(), HSTORE_DATFILE_DIR)));
   }
   }
 
 
+  /** @return the info directory path */
   static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
   static Path getInfoDir(Path dir, Text regionName, Text colFamily) {
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
         new Path(colFamily.toString(), HSTORE_INFO_DIR)));
         new Path(colFamily.toString(), HSTORE_INFO_DIR)));
   }
   }
-  
+
+  /** @return the bloom filter directory path */
   static Path getFilterDir(Path dir, Text regionName, Text colFamily) {
   static Path getFilterDir(Path dir, Text regionName, Text colFamily) {
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName,
         new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
         new Path(colFamily.toString(), HSTORE_FILTER_DIR)));
   }
   }
 
 
+  /** @return the HStore directory path */
   static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
   static Path getHStoreDir(Path dir, Text regionName, Text colFamily) {
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName, 
         colFamily.toString()));
         colFamily.toString()));
   }
   }
 
 
+  /** @return the HRegion directory path */
   static Path getHRegionDir(Path dir, Text regionName) {
   static Path getHRegionDir(Path dir, Text regionName) {
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
     return new Path(dir, new Path(HREGIONDIR_PREFIX + regionName));
   }
   }
 
 
   /**
   /**
-   * Obtain a brand-new randomly-named HStoreFile.  Checks the existing
-   * filesystem if the file already exists.
+   * @return a brand-new randomly-named HStoreFile.
+   * 
+   * Checks the filesystem to determine if the file already exists. If so, it
+   * will keep generating names until it generates a name that does not exist.
    */
    */
   static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir, 
   static HStoreFile obtainNewHStoreFile(Configuration conf, Path dir, 
       Text regionName, Text colFamily, FileSystem fs) throws IOException {
       Text regionName, Text colFamily, FileSystem fs) throws IOException {
@@ -157,10 +176,18 @@ public class HStoreFile implements HConstants, WritableComparable {
   }
   }
 
 
   /**
   /**
-   * Create a series of HStoreFiles loaded from the given directory.
+   * Creates a series of HStoreFiles loaded from the given directory.
    * 
    * 
    * There must be a matching 'mapdir' and 'loginfo' pair of files.
    * There must be a matching 'mapdir' and 'loginfo' pair of files.
    * If only one exists, we'll delete it.
    * If only one exists, we'll delete it.
+   *
+   * @param conf Configuration object
+   * @param dir directory path
+   * @param regionName region name
+   * @param colFamily column family
+   * @param fs file system
+   * @return Vector of HStoreFiles
+   * @throws IOException
    */
    */
   static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir, 
   static Vector<HStoreFile> loadHStoreFiles(Configuration conf, Path dir, 
       Text regionName, Text colFamily, FileSystem fs) throws IOException {
       Text regionName, Text colFamily, FileSystem fs) throws IOException {
@@ -173,8 +200,12 @@ public class HStoreFile implements HConstants, WritableComparable {
       String name = datfiles[i].getName();
       String name = datfiles[i].getName();
       
       
       if(name.startsWith(HSTORE_DATFILE_PREFIX)) {
       if(name.startsWith(HSTORE_DATFILE_PREFIX)) {
-        Long fileId = Long.parseLong(name.substring(HSTORE_DATFILE_PREFIX.length()));
-        HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
+        Long fileId =
+          Long.parseLong(name.substring(HSTORE_DATFILE_PREFIX.length()));
+
+        HStoreFile curfile =
+          new HStoreFile(conf, dir, regionName, colFamily, fileId);
+
         Path mapfile = curfile.getMapFilePath();
         Path mapfile = curfile.getMapFilePath();
         Path infofile = curfile.getInfoFilePath();
         Path infofile = curfile.getInfoFilePath();
         
         
@@ -193,8 +224,12 @@ public class HStoreFile implements HConstants, WritableComparable {
       String name = infofiles[i].getName();
       String name = infofiles[i].getName();
       
       
       if(name.startsWith(HSTORE_INFOFILE_PREFIX)) {
       if(name.startsWith(HSTORE_INFOFILE_PREFIX)) {
-        long fileId = Long.parseLong(name.substring(HSTORE_INFOFILE_PREFIX.length()));
-        HStoreFile curfile = new HStoreFile(conf, dir, regionName, colFamily, fileId);
+        long fileId =
+          Long.parseLong(name.substring(HSTORE_INFOFILE_PREFIX.length()));
+
+        HStoreFile curfile =
+          new HStoreFile(conf, dir, regionName, colFamily, fileId);
+
         Path mapfile = curfile.getMapFilePath();
         Path mapfile = curfile.getMapFilePath();
         
         
         if(! fs.exists(mapfile)) {
         if(! fs.exists(mapfile)) {
@@ -205,31 +240,40 @@ public class HStoreFile implements HConstants, WritableComparable {
     return results;
     return results;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // File handling
   // File handling
-  //////////////////////////////////////////////////////////////////////////////
 
 
   /**
   /**
    * Break this HStoreFile file into two new parts, which live in different 
    * Break this HStoreFile file into two new parts, which live in different 
    * brand-new HRegions.
    * brand-new HRegions.
+   *
+   * @param midKey the key which will be the starting key of the second region
+   * @param dstA the file which will contain keys from the start of the source
+   * @param dstB the file which will contain keys from midKey to end of source
+   * @param fs file system
+   * @param c configuration
+   * @throws IOException
    */
    */
   void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
   void splitStoreFile(Text midKey, HStoreFile dstA, HStoreFile dstB,
-      FileSystem fs, Configuration c)
-  throws IOException {
+      FileSystem fs, Configuration c) throws IOException {
+    
     // Copy the appropriate tuples to one MapFile or the other.
     // Copy the appropriate tuples to one MapFile or the other.
+    
     MapFile.Reader in = new MapFile.Reader(fs, getMapFilePath().toString(), c);
     MapFile.Reader in = new MapFile.Reader(fs, getMapFilePath().toString(), c);
     try {
     try {
       MapFile.Writer outA = new MapFile.Writer(c, fs, 
       MapFile.Writer outA = new MapFile.Writer(c, fs, 
         dstA.getMapFilePath().toString(), HStoreKey.class,
         dstA.getMapFilePath().toString(), HStoreKey.class,
         ImmutableBytesWritable.class);
         ImmutableBytesWritable.class);
+      
       try {
       try {
         MapFile.Writer outB = new MapFile.Writer(c, fs, 
         MapFile.Writer outB = new MapFile.Writer(c, fs, 
           dstB.getMapFilePath().toString(), HStoreKey.class,
           dstB.getMapFilePath().toString(), HStoreKey.class,
           ImmutableBytesWritable.class);
           ImmutableBytesWritable.class);
+        
         try {
         try {
           long count = 0;
           long count = 0;
           HStoreKey readkey = new HStoreKey();
           HStoreKey readkey = new HStoreKey();
           ImmutableBytesWritable readval = new ImmutableBytesWritable();
           ImmutableBytesWritable readval = new ImmutableBytesWritable();
+          
           while(in.next(readkey, readval)) {
           while(in.next(readkey, readval)) {
             if(readkey.getRow().compareTo(midKey) < 0) {
             if(readkey.getRow().compareTo(midKey) < 0) {
               outA.append(readkey, readval);
               outA.append(readkey, readval);
@@ -243,12 +287,15 @@ public class HStoreFile implements HConstants, WritableComparable {
               }
               }
             }
             }
           }
           }
+          
         } finally {
         } finally {
           outB.close();
           outB.close();
         }
         }
+        
       } finally {
       } finally {
         outA.close();
         outA.close();
       }
       }
+      
     } finally {
     } finally {
       in.close();
       in.close();
     }
     }
@@ -260,8 +307,12 @@ public class HStoreFile implements HConstants, WritableComparable {
   }
   }
 
 
   /**
   /**
-   * Write to this HStoreFile with all the contents of the given source HStoreFiles.
-   * We are merging multiple regions into a single new one.
+   * Merges the contents of the given source HStoreFiles into a single new one.
+   *
+   * @param srcFiles files to be merged
+   * @param fs file system
+   * @param conf configuration object
+   * @throws IOException
    */
    */
   void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs, 
   void mergeStoreFiles(Vector<HStoreFile> srcFiles, FileSystem fs, 
       Configuration conf) throws IOException {
       Configuration conf) throws IOException {
@@ -273,9 +324,9 @@ public class HStoreFile implements HConstants, WritableComparable {
       HStoreKey.class, ImmutableBytesWritable.class);
       HStoreKey.class, ImmutableBytesWritable.class);
     
     
     try {
     try {
-      for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
-        HStoreFile src = it.next();
-        MapFile.Reader in = new MapFile.Reader(fs, src.getMapFilePath().toString(), conf);
+      for(HStoreFile src: srcFiles) {
+        MapFile.Reader in =
+          new MapFile.Reader(fs, src.getMapFilePath().toString(), conf);
         
         
         try {
         try {
           HStoreKey readkey = new HStoreKey();
           HStoreKey readkey = new HStoreKey();
@@ -283,6 +334,7 @@ public class HStoreFile implements HConstants, WritableComparable {
           while(in.next(readkey, readval)) {
           while(in.next(readkey, readval)) {
             out.append(readkey, readval);
             out.append(readkey, readval);
           }
           }
+          
         } finally {
         } finally {
           in.close();
           in.close();
         }
         }
@@ -293,6 +345,7 @@ public class HStoreFile implements HConstants, WritableComparable {
     }
     }
 
 
     // Build a unified InfoFile from the source InfoFiles.
     // Build a unified InfoFile from the source InfoFiles.
+    
     long unifiedSeqId = -1;
     long unifiedSeqId = -1;
     for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
     for(Iterator<HStoreFile> it = srcFiles.iterator(); it.hasNext(); ) {
       HStoreFile hsf = it.next();
       HStoreFile hsf = it.next();
@@ -304,7 +357,13 @@ public class HStoreFile implements HConstants, WritableComparable {
     writeInfo(fs, unifiedSeqId);
     writeInfo(fs, unifiedSeqId);
   }
   }
 
 
-  /** Read in an info file, give it a unique ID. */
+  /** 
+   * Reads in an info file, and gives it a unique ID.
+   *
+   * @param fs file system
+   * @return new unique id
+   * @throws IOException
+   */
   long loadInfo(FileSystem fs) throws IOException {
   long loadInfo(FileSystem fs) throws IOException {
     Path p = getInfoFilePath();
     Path p = getInfoFilePath();
     DataInputStream in = new DataInputStream(fs.open(p));
     DataInputStream in = new DataInputStream(fs.open(p));
@@ -319,7 +378,13 @@ public class HStoreFile implements HConstants, WritableComparable {
     }
     }
   }
   }
   
   
-  /** Write the file-identifier to disk */
+  /**
+   * Writes the file-identifier to disk
+   * 
+   * @param fs file system
+   * @param infonum file id
+   * @throws IOException
+   */
   void writeInfo(FileSystem fs, long infonum) throws IOException {
   void writeInfo(FileSystem fs, long infonum) throws IOException {
     Path p = getInfoFilePath();
     Path p = getInfoFilePath();
     DataOutputStream out = new DataOutputStream(fs.create(p));
     DataOutputStream out = new DataOutputStream(fs.create(p));
@@ -333,11 +398,13 @@ public class HStoreFile implements HConstants, WritableComparable {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = this.dir.hashCode();
     int result = this.dir.hashCode();
@@ -347,13 +414,9 @@ public class HStoreFile implements HConstants, WritableComparable {
     return result;
     return result;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeUTF(dir.toString());
     out.writeUTF(dir.toString());
     regionName.write(out);
     regionName.write(out);
@@ -361,9 +424,7 @@ public class HStoreFile implements HConstants, WritableComparable {
     out.writeLong(fileId);
     out.writeLong(fileId);
   }
   }
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.dir = new Path(in.readUTF());
     this.dir = new Path(in.readUTF());
     this.regionName.readFields(in);
     this.regionName.readFields(in);
@@ -371,13 +432,9 @@ public class HStoreFile implements HConstants, WritableComparable {
     this.fileId = in.readLong();
     this.fileId = in.readLong();
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Comparable
   // Comparable
-  //////////////////////////////////////////////////////////////////////////////
 
 
-  /* (non-Javadoc)
-   * @see java.lang.Comparable#compareTo(java.lang.Object)
-   */
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     HStoreFile other = (HStoreFile) o;
     HStoreFile other = (HStoreFile) o;
     int result = this.dir.compareTo(other.dir);    
     int result = this.dir.compareTo(other.dir);    

+ 13 - 17
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HStoreKey.java

@@ -147,16 +147,14 @@ public class HStoreKey implements WritableComparable {
     this.timestamp = timestamp;
     this.timestamp = timestamp;
   }
   }
   
   
-  /**
-   * @return Approximate size in bytes of this key.
-   */
+  /** @return Approximate size in bytes of this key. */
   public long getSize() {
   public long getSize() {
     return this.row.getLength() + this.column.getLength() +
     return this.row.getLength() + this.column.getLength() +
       8 /* There is no sizeof in java. Presume long is 8 (64bit machine)*/;
       8 /* There is no sizeof in java. Presume long is 8 (64bit machine)*/;
   }
   }
   
   
   /**
   /**
-   * Construct a new HStoreKey from another
+   * Constructs a new HStoreKey from another
    * 
    * 
    * @param other the source key
    * @param other the source key
    */
    */
@@ -218,6 +216,7 @@ public class HStoreKey implements WritableComparable {
   }
   }
   
   
   /**
   /**
+   * Compares the row and column of two keys
    * @param other Key to compare against. Compares row and column.
    * @param other Key to compare against. Compares row and column.
    * @return True if same row and column.
    * @return True if same row and column.
    * @see #matchesWithoutColumn(HStoreKey)
    * @see #matchesWithoutColumn(HStoreKey)
@@ -229,6 +228,8 @@ public class HStoreKey implements WritableComparable {
   }
   }
   
   
   /**
   /**
+   * Compares the row and timestamp of two keys
+   * 
    * @param other Key to copmare against. Compares row and timestamp.
    * @param other Key to copmare against. Compares row and timestamp.
    * 
    * 
    * @return True if same row and timestamp is greater than <code>other</code>
    * @return True if same row and timestamp is greater than <code>other</code>
@@ -241,6 +242,8 @@ public class HStoreKey implements WritableComparable {
   }
   }
   
   
   /**
   /**
+   * Compares the row and column family of two keys
+   * 
    * @param that Key to compare against. Compares row and column family
    * @param that Key to compare against. Compares row and column family
    * 
    * 
    * @return true if same row and column family
    * @return true if same row and column family
@@ -255,16 +258,19 @@ public class HStoreKey implements WritableComparable {
         compareTo(extractFamily(that.getColumn())) == 0;
         compareTo(extractFamily(that.getColumn())) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     return row.toString() + "/" + column.toString() + "/" + timestamp;
     return row.toString() + "/" + column.toString() + "/" + timestamp;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     return compareTo(obj) == 0;
     return compareTo(obj) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = this.row.hashCode();
     int result = this.row.hashCode();
@@ -273,13 +279,9 @@ public class HStoreKey implements WritableComparable {
     return result;
     return result;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Comparable
   // Comparable
-  //////////////////////////////////////////////////////////////////////////////
 
 
-  /* (non-Javadoc)
-   * @see java.lang.Comparable#compareTo(java.lang.Object)
-   */
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     HStoreKey other = (HStoreKey) o;
     HStoreKey other = (HStoreKey) o;
     int result = this.row.compareTo(other.row);
     int result = this.row.compareTo(other.row);
@@ -296,22 +298,16 @@ public class HStoreKey implements WritableComparable {
     return result;
     return result;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     row.write(out);
     row.write(out);
     column.write(out);
     column.write(out);
     out.writeLong(timestamp);
     out.writeLong(timestamp);
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     row.readFields(in);
     row.readFields(in);
     column.readFields(in);
     column.readFields(in);

+ 9 - 6
src/contrib/hbase/src/java/org/apache/hadoop/hbase/HTableDescriptor.java

@@ -78,7 +78,7 @@ public class HTableDescriptor implements WritableComparable {
   }
   }
 
 
   /**
   /**
-   * Add a column family.
+   * Adds a column family.
    * @param family HColumnDescriptor of familyto add.
    * @param family HColumnDescriptor of familyto add.
    */
    */
   public void addFamily(HColumnDescriptor family) {
   public void addFamily(HColumnDescriptor family) {
@@ -95,7 +95,8 @@ public class HTableDescriptor implements WritableComparable {
     return families.containsKey(family);
     return families.containsKey(family);
   }
   }
 
 
-  /** All the column families in this table.
+  /** 
+   * All the column families in this table.
    * 
    * 
    *  TODO: What is this used for? Seems Dangerous to let people play with our
    *  TODO: What is this used for? Seems Dangerous to let people play with our
    *  private members.
    *  private members.
@@ -106,16 +107,19 @@ public class HTableDescriptor implements WritableComparable {
     return families;
     return families;
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "name: " + this.name.toString() + ", families: " + this.families;
     return "name: " + this.name.toString() + ", families: " + this.families;
       }
       }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object obj) {
   public boolean equals(Object obj) {
     return compareTo(obj) == 0;
     return compareTo(obj) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     // TODO: Cache.
     // TODO: Cache.
@@ -128,10 +132,9 @@ public class HTableDescriptor implements WritableComparable {
     return result;
     return result;
   }
   }
   
   
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     name.write(out);
     name.write(out);
     out.writeInt(families.size());
     out.writeInt(families.size());
@@ -141,6 +144,7 @@ public class HTableDescriptor implements WritableComparable {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.name.readFields(in);
     this.name.readFields(in);
     int numCols = in.readInt();
     int numCols = in.readInt();
@@ -152,10 +156,9 @@ public class HTableDescriptor implements WritableComparable {
     }
     }
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Comparable
   // Comparable
-  //////////////////////////////////////////////////////////////////////////////
 
 
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     HTableDescriptor other = (HTableDescriptor) o;
     HTableDescriptor other = (HTableDescriptor) o;
     int result = name.compareTo(other.name);
     int result = name.compareTo(other.name);

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/InvalidColumnNameException.java

@@ -21,12 +21,20 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown when an invalid column name is encountered
+ */
 public class InvalidColumnNameException extends IOException {
 public class InvalidColumnNameException extends IOException {
   private static final long serialVersionUID = 1L << 29 - 1L;
   private static final long serialVersionUID = 1L << 29 - 1L;
+  /** default constructor */
   public InvalidColumnNameException() {
   public InvalidColumnNameException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public InvalidColumnNameException(String s) {
   public InvalidColumnNameException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/LockException.java

@@ -21,12 +21,20 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown when a locking error is encountered
+ */
 public class LockException extends IOException {
 public class LockException extends IOException {
   private static final long serialVersionUID = 1L << 13 - 1L;
   private static final long serialVersionUID = 1L << 13 - 1L;
+  /** default constructor */
   public LockException() {
   public LockException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public LockException(String s) {
   public LockException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/MasterNotRunningException.java

@@ -21,12 +21,20 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown if the master is not running
+ */
 public class MasterNotRunningException extends IOException {
 public class MasterNotRunningException extends IOException {
   private static final long serialVersionUID = 1L << 23 - 1L;
   private static final long serialVersionUID = 1L << 23 - 1L;
+  /** default constructor */
   public MasterNotRunningException() {
   public MasterNotRunningException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public MasterNotRunningException(String s) {
   public MasterNotRunningException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/NoServerForRegionException.java

@@ -21,13 +21,21 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown when no region server can be found for a region
+ */
 public class NoServerForRegionException extends IOException {
 public class NoServerForRegionException extends IOException {
   private static final long serialVersionUID = 1L << 11 - 1L;
   private static final long serialVersionUID = 1L << 11 - 1L;
 
 
+  /** default constructor */
   public NoServerForRegionException() {
   public NoServerForRegionException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public NoServerForRegionException(String s) {
   public NoServerForRegionException(String s) {
     super(s);
     super(s);
   }
   }

+ 10 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/NotServingRegionException.java

@@ -21,12 +21,22 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown by a region server if it is sent a request for a region it is not
+ * serving.
+ */
 public class NotServingRegionException extends IOException {
 public class NotServingRegionException extends IOException {
   private static final long serialVersionUID = 1L << 17 - 1L;
   private static final long serialVersionUID = 1L << 17 - 1L;
+
+  /** default constructor */
   public NotServingRegionException() {
   public NotServingRegionException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public NotServingRegionException(String s) {
   public NotServingRegionException(String s) {
     super(s);
     super(s);
   }
   }

+ 10 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/TableExistsException.java

@@ -17,11 +17,21 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown when a table exists but should not
+ */
 public class TableExistsException extends IOException {
 public class TableExistsException extends IOException {
+  private static final long serialVersionUID = 1L << 7 - 1L;
+  /** default constructor */
   public TableExistsException() {
   public TableExistsException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * 
+   * @param s message
+   */
   public TableExistsException(String s) {
   public TableExistsException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/TableNotDisabledException.java

@@ -21,12 +21,20 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown if a table should be offline but is not
+ */
 public class TableNotDisabledException extends IOException {
 public class TableNotDisabledException extends IOException {
   private static final long serialVersionUID = 1L << 19 - 1L;
   private static final long serialVersionUID = 1L << 19 - 1L;
+  /** default constructor */
   public TableNotDisabledException() {
   public TableNotDisabledException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public TableNotDisabledException(String s) {
   public TableNotDisabledException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/UnknownScannerException.java

@@ -21,13 +21,21 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown if a region server is passed an unknown scanner id
+ */
 public class UnknownScannerException extends IOException {
 public class UnknownScannerException extends IOException {
   private static final long serialVersionUID = 993179627856392526L;
   private static final long serialVersionUID = 993179627856392526L;
 
 
+  /** constructor */
   public UnknownScannerException() {
   public UnknownScannerException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public UnknownScannerException(String s) {
   public UnknownScannerException(String s) {
     super(s);
     super(s);
   }
   }

+ 8 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/WrongRegionException.java

@@ -21,13 +21,21 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Thrown when a request contains a key which is not part of this region
+ */
 public class WrongRegionException extends IOException {
 public class WrongRegionException extends IOException {
   private static final long serialVersionUID = 993179627856392526L;
   private static final long serialVersionUID = 993179627856392526L;
 
 
+  /** constructor */
   public WrongRegionException() {
   public WrongRegionException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * Constructor
+   * @param s message
+   */
   public WrongRegionException(String s) {
   public WrongRegionException(String s) {
     super(s);
     super(s);
   }
   }

+ 6 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/InvalidRowFilterException.java

@@ -25,10 +25,16 @@ package org.apache.hadoop.hbase.filter;
 public class InvalidRowFilterException extends RuntimeException {
 public class InvalidRowFilterException extends RuntimeException {
   private static final long serialVersionUID = 2667894046345657865L;
   private static final long serialVersionUID = 2667894046345657865L;
 
 
+
+  /** constructor */
   public InvalidRowFilterException() {
   public InvalidRowFilterException() {
     super();
     super();
   }
   }
 
 
+  /**
+   * constructor
+   * @param s message
+   */
   public InvalidRowFilterException(String s) {
   public InvalidRowFilterException(String s) {
     super(s);
     super(s);
   }
   }

+ 3 - 5
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/PageRowFilter.java

@@ -81,11 +81,9 @@ public class PageRowFilter implements RowFilterInterface {
     rowsAccepted = 0;
     rowsAccepted = 0;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
-  public void rowProcessed(boolean filtered, Text rowKey) {
+  /** {@inheritDoc} */
+  public void rowProcessed(boolean filtered,
+      @SuppressWarnings("unused") Text rowKey) {
     if (!filtered) {
     if (!filtered) {
       this.rowsAccepted++;
       this.rowsAccepted++;
       if (LOG.isDebugEnabled()) {
       if (LOG.isDebugEnabled()) {

+ 3 - 8
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RegExpRowFilter.java

@@ -80,18 +80,13 @@ public class RegExpRowFilter implements RowFilterInterface {
     this.setColumnFilters(columnFilter);
     this.setColumnFilters(columnFilter);
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
+  @SuppressWarnings("unused")
   public void rowProcessed(boolean filtered, Text rowKey) {
   public void rowProcessed(boolean filtered, Text rowKey) {
     //doesn't care
     //doesn't care
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean processAlways() {
   public boolean processAlways() {
     return false;
     return false;
   }
   }

+ 1 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterInterface.java

@@ -45,6 +45,7 @@ public interface RowFilterInterface extends Writable {
    * RowFilterSet with an OR operator.
    * RowFilterSet with an OR operator.
    * 
    * 
    * @see RowFilterSet
    * @see RowFilterSet
+   * @param filtered
    * @param key
    * @param key
    */
    */
   void rowProcessed(boolean filtered, Text key);
   void rowProcessed(boolean filtered, Text key);

+ 15 - 41
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/RowFilterSet.java

@@ -38,8 +38,12 @@ import org.apache.hadoop.io.Text;
  */
  */
 public class RowFilterSet implements RowFilterInterface {
 public class RowFilterSet implements RowFilterInterface {
 
 
+  /** set operator */
   public static enum Operator {
   public static enum Operator {
-    MUST_PASS_ALL, MUST_PASS_ONE
+    /** !AND */
+    MUST_PASS_ALL,
+    /** !OR */
+    MUST_PASS_ONE
   }
   }
 
 
   private Operator operator = Operator.MUST_PASS_ALL;
   private Operator operator = Operator.MUST_PASS_ALL;
@@ -77,10 +81,7 @@ public class RowFilterSet implements RowFilterInterface {
     this.operator = operator;
     this.operator = operator;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void validate(final Text[] columns) {
   public void validate(final Text[] columns) {
     for (RowFilterInterface filter : filters) {
     for (RowFilterInterface filter : filters) {
       filter.validate(columns);
       filter.validate(columns);
@@ -91,10 +92,7 @@ public class RowFilterSet implements RowFilterInterface {
     }
     }
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void reset() {
   public void reset() {
     for (RowFilterInterface filter : filters) {
     for (RowFilterInterface filter : filters) {
       filter.reset();
       filter.reset();
@@ -105,10 +103,7 @@ public class RowFilterSet implements RowFilterInterface {
     }
     }
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void rowProcessed(boolean filtered, Text rowKey) {
   public void rowProcessed(boolean filtered, Text rowKey) {
     for (RowFilterInterface filter : filters) {
     for (RowFilterInterface filter : filters) {
       filter.rowProcessed(filtered, rowKey);
       filter.rowProcessed(filtered, rowKey);
@@ -119,10 +114,7 @@ public class RowFilterSet implements RowFilterInterface {
     }
     }
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean processAlways() {
   public boolean processAlways() {
     for (RowFilterInterface filter : filters) {
     for (RowFilterInterface filter : filters) {
       if (filter.processAlways()) {
       if (filter.processAlways()) {
@@ -136,10 +128,7 @@ public class RowFilterSet implements RowFilterInterface {
     return false;
     return false;
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filterAllRemaining() {
   public boolean filterAllRemaining() {
     boolean result = operator == Operator.MUST_PASS_ONE;
     boolean result = operator == Operator.MUST_PASS_ONE;
     for (RowFilterInterface filter : filters) {
     for (RowFilterInterface filter : filters) {
@@ -167,10 +156,7 @@ public class RowFilterSet implements RowFilterInterface {
     return result;
     return result;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filter(final Text rowKey) {
   public boolean filter(final Text rowKey) {
     boolean resultFound = false;
     boolean resultFound = false;
     boolean result = operator == Operator.MUST_PASS_ONE;
     boolean result = operator == Operator.MUST_PASS_ONE;
@@ -205,10 +191,7 @@ public class RowFilterSet implements RowFilterInterface {
     return result;
     return result;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filter(final Text rowKey, final Text colKey, 
   public boolean filter(final Text rowKey, final Text colKey, 
     final byte[] data) {
     final byte[] data) {
     boolean resultFound = false;
     boolean resultFound = false;
@@ -248,10 +231,7 @@ public class RowFilterSet implements RowFilterInterface {
     return result;
     return result;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
   public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
     boolean resultFound = false;
     boolean resultFound = false;
     boolean result = operator == Operator.MUST_PASS_ONE;
     boolean result = operator == Operator.MUST_PASS_ONE;
@@ -286,10 +266,7 @@ public class RowFilterSet implements RowFilterInterface {
     return result;
     return result;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void readFields(final DataInput in) throws IOException {
   public void readFields(final DataInput in) throws IOException {
     byte opByte = in.readByte();
     byte opByte = in.readByte();
     operator = Operator.values()[opByte];
     operator = Operator.values()[opByte];
@@ -323,10 +300,7 @@ public class RowFilterSet implements RowFilterInterface {
 
 
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void write(final DataOutput out) throws IOException {
   public void write(final DataOutput out) throws IOException {
     out.writeByte(operator.ordinal());
     out.writeByte(operator.ordinal());
     out.writeInt(filters.size());
     out.writeInt(filters.size());

+ 11 - 25
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/StopRowFilter.java

@@ -80,34 +80,23 @@ public class StopRowFilter implements RowFilterInterface {
     // Nothing to reset
     // Nothing to reset
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
+  @SuppressWarnings("unused")
   public void rowProcessed(boolean filtered, Text rowKey) {
   public void rowProcessed(boolean filtered, Text rowKey) {
     // Doesn't care
     // Doesn't care
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean processAlways() {
   public boolean processAlways() {
     return false;
     return false;
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filterAllRemaining() {
   public boolean filterAllRemaining() {
     return false;
     return false;
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filter(final Text rowKey) {
   public boolean filter(final Text rowKey) {
     boolean result = this.stopRowKey.compareTo(rowKey) <= 0;
     boolean result = this.stopRowKey.compareTo(rowKey) <= 0;
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
@@ -118,6 +107,8 @@ public class StopRowFilter implements RowFilterInterface {
   }
   }
 
 
   /**
   /**
+   * {@inheritDoc}
+   *
    * Because StopRowFilter does not examine column information, this method 
    * Because StopRowFilter does not examine column information, this method 
    * defaults to calling the rowKey-only version of filter.
    * defaults to calling the rowKey-only version of filter.
    */
    */
@@ -127,7 +118,8 @@ public class StopRowFilter implements RowFilterInterface {
     return filter(rowKey);
     return filter(rowKey);
   }
   }
 
 
-  /**
+  /** {@inheritDoc}
+   *
    * Because StopRowFilter does not examine column information, this method 
    * Because StopRowFilter does not examine column information, this method 
    * defaults to calling filterAllRemaining().
    * defaults to calling filterAllRemaining().
    * 
    * 
@@ -138,18 +130,12 @@ public class StopRowFilter implements RowFilterInterface {
     return filterAllRemaining();
     return filterAllRemaining();
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     stopRowKey = new Text(in.readUTF());
     stopRowKey = new Text(in.readUTF());
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeUTF(stopRowKey.toString());
     out.writeUTF(stopRowKey.toString());
   }
   }

+ 13 - 36
src/contrib/hbase/src/java/org/apache/hadoop/hbase/filter/WhileMatchRowFilter.java

@@ -50,6 +50,10 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     super();
     super();
   }
   }
   
   
+  /**
+   * Constructor
+   * @param filter
+   */
   public WhileMatchRowFilter(RowFilterInterface filter) {
   public WhileMatchRowFilter(RowFilterInterface filter) {
     this.filter = filter;
     this.filter = filter;
   }
   }
@@ -63,10 +67,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     return this.filter;
     return this.filter;
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void reset() {
   public void reset() {
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("Resetting.");
       LOG.debug("Resetting.");
@@ -75,10 +76,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     this.filter.reset();
     this.filter.reset();
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean processAlways() {
   public boolean processAlways() {
     return true;
     return true;
   }
   }
@@ -94,10 +92,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     return this.filterAllRemaining || this.filter.filterAllRemaining();
     return this.filterAllRemaining || this.filter.filterAllRemaining();
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filter(final Text rowKey) {
   public boolean filter(final Text rowKey) {
     changeFAR(this.filter.filter(rowKey));
     changeFAR(this.filter.filter(rowKey));
     boolean result = filterAllRemaining();
     boolean result = filterAllRemaining();
@@ -107,10 +102,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     return result;
     return result;
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filter(final Text rowKey, final Text colKey,
   public boolean filter(final Text rowKey, final Text colKey,
     final byte[] data) {
     final byte[] data) {
     changeFAR(this.filter.filter(rowKey, colKey, data));
     changeFAR(this.filter.filter(rowKey, colKey, data));
@@ -122,10 +114,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     return result;
     return result;
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
   public boolean filterNotNull(final TreeMap<Text, byte[]> columns) {
     changeFAR(this.filter.filterNotNull(columns));
     changeFAR(this.filter.filterNotNull(columns));
     boolean result = filterAllRemaining();
     boolean result = filterAllRemaining();
@@ -150,26 +139,17 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     }
     }
   }
   }
 
 
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void rowProcessed(boolean filtered, Text rowKey) {
   public void rowProcessed(boolean filtered, Text rowKey) {
     this.filter.rowProcessed(filtered, rowKey);
     this.filter.rowProcessed(filtered, rowKey);
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void validate(Text[] columns) {
   public void validate(Text[] columns) {
     this.filter.validate(columns);
     this.filter.validate(columns);
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     String className = in.readUTF();
     String className = in.readUTF();
     
     
@@ -193,10 +173,7 @@ public class WhileMatchRowFilter implements RowFilterInterface {
     }
     }
   }
   }
   
   
-  /**
-   * 
-   * {@inheritDoc}
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeUTF(this.filter.getClass().getName());
     out.writeUTF(this.filter.getClass().getName());
     this.filter.write(out);
     this.filter.write(out);

+ 19 - 13
src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/ImmutableBytesWritable.java

@@ -88,7 +88,7 @@ public class ImmutableBytesWritable implements WritableComparable {
   }
   }
   
   
   /**
   /**
-   * Get the current size of the buffer.
+   * @return the current size of the buffer.
    */
    */
   public int getSize() {
   public int getSize() {
     if (this.bytes == null) {
     if (this.bytes == null) {
@@ -99,13 +99,13 @@ public class ImmutableBytesWritable implements WritableComparable {
   }
   }
 
 
 
 
-  // inherit javadoc
+  /** {@inheritDoc} */
   public void readFields(final DataInput in) throws IOException {
   public void readFields(final DataInput in) throws IOException {
     this.bytes = new byte[in.readInt()];
     this.bytes = new byte[in.readInt()];
     in.readFully(this.bytes, 0, this.bytes.length);
     in.readFully(this.bytes, 0, this.bytes.length);
   }
   }
   
   
-  // inherit javadoc
+  /** {@inheritDoc} */
   public void write(final DataOutput out) throws IOException {
   public void write(final DataOutput out) throws IOException {
     out.writeInt(this.bytes.length);
     out.writeInt(this.bytes.length);
     out.write(this.bytes, 0, this.bytes.length);
     out.write(this.bytes, 0, this.bytes.length);
@@ -113,6 +113,8 @@ public class ImmutableBytesWritable implements WritableComparable {
   
   
   // Below methods copied from BytesWritable
   // Below methods copied from BytesWritable
   
   
+  /** {@inheritDoc} */
+  @Override
   public int hashCode() {
   public int hashCode() {
     return WritableComparator.hashBytes(bytes, this.bytes.length);
     return WritableComparator.hashBytes(bytes, this.bytes.length);
   }
   }
@@ -127,6 +129,12 @@ public class ImmutableBytesWritable implements WritableComparable {
     return compareTo(((ImmutableBytesWritable)right_obj).get());
     return compareTo(((ImmutableBytesWritable)right_obj).get());
   }
   }
   
   
+  /**
+   * Compares the bytes in this object to the specified byte array
+   * @param that
+   * @return Positive if left is bigger than right, 0 if they are equal, and
+   *         negative if left is smaller than right.
+   */
   public int compareTo(final byte [] that) {
   public int compareTo(final byte [] that) {
     int diff = this.bytes.length - that.length;
     int diff = this.bytes.length - that.length;
     return (diff != 0)?
     return (diff != 0)?
@@ -135,9 +143,8 @@ public class ImmutableBytesWritable implements WritableComparable {
         0, that.length);
         0, that.length);
   }
   }
   
   
-  /**
-   * Are the two byte sequences equal?
-   */
+  /** {@inheritDoc} */
+  @Override
   public boolean equals(Object right_obj) {
   public boolean equals(Object right_obj) {
     if (right_obj instanceof ImmutableBytesWritable) {
     if (right_obj instanceof ImmutableBytesWritable) {
       return compareTo(right_obj) == 0;
       return compareTo(right_obj) == 0;
@@ -145,9 +152,8 @@ public class ImmutableBytesWritable implements WritableComparable {
     return false;
     return false;
   }
   }
   
   
-  /**
-   * Generate the stream of bytes as hex pairs separated by ' '.
-   */
+  /** {@inheritDoc} */
+  @Override
   public String toString() { 
   public String toString() { 
     StringBuffer sb = new StringBuffer(3*this.bytes.length);
     StringBuffer sb = new StringBuffer(3*this.bytes.length);
     for (int idx = 0; idx < this.bytes.length; idx++) {
     for (int idx = 0; idx < this.bytes.length; idx++) {
@@ -170,14 +176,14 @@ public class ImmutableBytesWritable implements WritableComparable {
   public static class Comparator extends WritableComparator {
   public static class Comparator extends WritableComparator {
     private BytesWritable.Comparator comparator =
     private BytesWritable.Comparator comparator =
       new BytesWritable.Comparator();
       new BytesWritable.Comparator();
-    
+
+    /** constructor */
     public Comparator() {
     public Comparator() {
       super(ImmutableBytesWritable.class);
       super(ImmutableBytesWritable.class);
     }
     }
     
     
-    /**
-     * Compare the buffers in serialized form.
-     */
+    /** {@inheritDoc} */
+    @Override
     public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
     public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
       return comparator.compare(b1, s1, l1, b2, s2, l2);
       return comparator.compare(b1, s1, l1, b2, s2, l2);
     }
     }

+ 2 - 8
src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/KeyedData.java

@@ -56,22 +56,16 @@ public class KeyedData implements Writable {
     return data;
     return data;
   }
   }
 
 
-  //////////////////////////////////////////////////////////////////////////////
   // Writable
   // Writable
-  //////////////////////////////////////////////////////////////////////////////
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     key.write(out);
     key.write(out);
     out.writeInt(this.data.length);
     out.writeInt(this.data.length);
     out.write(this.data);
     out.write(this.data);
   }
   }
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     key.readFields(in);
     key.readFields(in);
     this.data = new byte[in.readInt()];
     this.data = new byte[in.readInt()];

+ 2 - 0
src/contrib/hbase/src/java/org/apache/hadoop/hbase/io/KeyedDataArrayWritable.java

@@ -61,6 +61,7 @@ public class KeyedDataArrayWritable implements Writable {
 
 
   // Writable
   // Writable
   
   
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     int len = in.readInt();
     int len = in.readInt();
     m_data = new KeyedData[len];
     m_data = new KeyedData[len];
@@ -70,6 +71,7 @@ public class KeyedDataArrayWritable implements Writable {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     int len = m_data.length;
     int len = m_data.length;
     out.writeInt(len);
     out.writeInt(len);

+ 1 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/GroupingTableMap.java

@@ -66,9 +66,7 @@ public class GroupingTableMap extends TableMap {
     job.set(GROUP_COLUMNS, groupColumns);
     job.set(GROUP_COLUMNS, groupColumns);
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.hbase.mapred.TableMap#configure(org.apache.hadoop.mapred.JobConf)
-   */
+  /** {@inheritDoc} */
   @Override
   @Override
   public void configure(JobConf job) {
   public void configure(JobConf job) {
     super.configure(job);
     super.configure(job);

+ 1 - 3
src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/TableMap.java

@@ -70,9 +70,7 @@ public abstract class TableMap extends MapReduceBase implements Mapper {
     job.set(TableInputFormat.COLUMN_LIST, columns);
     job.set(TableInputFormat.COLUMN_LIST, columns);
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.mapred.MapReduceBase#configure(org.apache.hadoop.mapred.JobConf)
-   */
+  /** {@inheritDoc} */
   @Override
   @Override
   public void configure(JobConf job) {
   public void configure(JobConf job) {
     super.configure(job);
     super.configure(job);

+ 5 - 12
src/contrib/hbase/src/java/org/apache/hadoop/hbase/mapred/TableSplit.java

@@ -69,40 +69,33 @@ public class TableSplit implements InputSplit {
     return m_endRow;
     return m_endRow;
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.mapred.InputSplit#getLength()
-   */
+  /** {@inheritDoc} */
   public long getLength() {
   public long getLength() {
     // Not clear how to obtain this... seems to be used only for sorting splits
     // Not clear how to obtain this... seems to be used only for sorting splits
     return 0;
     return 0;
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.mapred.InputSplit#getLocations()
-   */
+  /** {@inheritDoc} */
   public String[] getLocations() {
   public String[] getLocations() {
     // Return a random node from the cluster for now
     // Return a random node from the cluster for now
     return new String[] { };
     return new String[] { };
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     m_tableName.readFields(in);
     m_tableName.readFields(in);
     m_startRow.readFields(in);
     m_startRow.readFields(in);
     m_endRow.readFields(in);
     m_endRow.readFields(in);
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     m_tableName.write(out);
     m_tableName.write(out);
     m_startRow.write(out);
     m_startRow.write(out);
     m_endRow.write(out);
     m_endRow.write(out);
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString() {
   public String toString() {
     return m_tableName +"," + m_startRow + "," + m_endRow;
     return m_tableName +"," + m_startRow + "," + m_endRow;

+ 13 - 2
src/contrib/hbase/src/java/org/onelab/filter/BloomFilter.java

@@ -89,6 +89,7 @@ public class BloomFilter extends Filter {
     vector = new boolean[this.vectorSize];
     vector = new boolean[this.vectorSize];
   }//end constructor
   }//end constructor
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void add(Key key) {
   public void add(Key key) {
     if(key == null) {
     if(key == null) {
@@ -103,6 +104,7 @@ public class BloomFilter extends Filter {
     }
     }
   }//end add()
   }//end add()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void and(Filter filter){
   public void and(Filter filter){
     if(filter == null
     if(filter == null
@@ -119,6 +121,7 @@ public class BloomFilter extends Filter {
     }
     }
   }//end and()
   }//end and()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean membershipTest(Key key){
   public boolean membershipTest(Key key){
     if(key == null) {
     if(key == null) {
@@ -135,6 +138,7 @@ public class BloomFilter extends Filter {
     return true;
     return true;
   }//end memberhsipTest()
   }//end memberhsipTest()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void not(){
   public void not(){
     for(int i = 0; i < vectorSize; i++) {
     for(int i = 0; i < vectorSize; i++) {
@@ -142,6 +146,7 @@ public class BloomFilter extends Filter {
     }
     }
   }//end not()
   }//end not()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void or(Filter filter){
   public void or(Filter filter){
     if(filter == null
     if(filter == null
@@ -158,6 +163,7 @@ public class BloomFilter extends Filter {
     }
     }
   }//end or()
   }//end or()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void xor(Filter filter){
   public void xor(Filter filter){
     if(filter == null
     if(filter == null
@@ -175,7 +181,7 @@ public class BloomFilter extends Filter {
     }
     }
   }//and xor()
   }//and xor()
 
 
-  /** Returns a String representation of <i>this</i> Bloom filter. */
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString(){
   public String toString(){
     StringBuilder res = new StringBuilder();
     StringBuilder res = new StringBuilder();
@@ -186,7 +192,7 @@ public class BloomFilter extends Filter {
     return res.toString();
     return res.toString();
   }//end toString()
   }//end toString()
 
 
-  /** Returns a shallow copy of <i>this</i> Bloom filter. */
+  /** {@inheritDoc} */
   @Override
   @Override
   public Object clone(){
   public Object clone(){
     BloomFilter bf = new BloomFilter(vectorSize, nbHash);
     BloomFilter bf = new BloomFilter(vectorSize, nbHash);
@@ -194,11 +200,13 @@ public class BloomFilter extends Filter {
     return bf;
     return bf;
   }//end clone()
   }//end clone()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = super.hashCode();
     int result = super.hashCode();
@@ -210,6 +218,7 @@ public class BloomFilter extends Filter {
 
 
   // Writable
   // Writable
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
@@ -218,6 +227,7 @@ public class BloomFilter extends Filter {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
@@ -229,6 +239,7 @@ public class BloomFilter extends Filter {
 
 
   // Comparable
   // Comparable
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     int result = super.compareTo(o);
     int result = super.compareTo(o);

+ 13 - 0
src/contrib/hbase/src/java/org/onelab/filter/CountingBloomFilter.java

@@ -84,6 +84,7 @@ public final class CountingBloomFilter extends Filter {
     vector = new byte[vectorSize];
     vector = new byte[vectorSize];
   }//end constructor
   }//end constructor
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void add(Key key) {
   public void add(Key key) {
     if(key == null) {
     if(key == null) {
@@ -122,6 +123,7 @@ public final class CountingBloomFilter extends Filter {
     }
     }
   }//end delete
   }//end delete
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void and(Filter filter){
   public void and(Filter filter){
     if(filter == null
     if(filter == null
@@ -137,6 +139,7 @@ public final class CountingBloomFilter extends Filter {
     }
     }
   }//end and()
   }//end and()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean membershipTest(Key key){
   public boolean membershipTest(Key key){
     if(key == null) {
     if(key == null) {
@@ -155,12 +158,14 @@ public final class CountingBloomFilter extends Filter {
     return true;
     return true;
   }//end membershipTest()
   }//end membershipTest()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void not(){
   public void not(){
     throw new UnsupportedOperationException("not() is undefined for "
     throw new UnsupportedOperationException("not() is undefined for "
         + this.getClass().getName());
         + this.getClass().getName());
   }//end not()
   }//end not()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void or(Filter filter){
   public void or(Filter filter){
     if(filter == null
     if(filter == null
@@ -177,6 +182,7 @@ public final class CountingBloomFilter extends Filter {
     }
     }
   }//end or()
   }//end or()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   @SuppressWarnings("unused")
   @SuppressWarnings("unused")
   public void xor(Filter filter){
   public void xor(Filter filter){
@@ -184,6 +190,7 @@ public final class CountingBloomFilter extends Filter {
         + this.getClass().getName());
         + this.getClass().getName());
   }//end xor()
   }//end xor()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString(){
   public String toString(){
     StringBuilder res = new StringBuilder();
     StringBuilder res = new StringBuilder();
@@ -198,6 +205,7 @@ public final class CountingBloomFilter extends Filter {
     return res.toString();
     return res.toString();
   }//end toString()
   }//end toString()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public Object clone(){
   public Object clone(){
     CountingBloomFilter cbf = new CountingBloomFilter(vectorSize, nbHash);
     CountingBloomFilter cbf = new CountingBloomFilter(vectorSize, nbHash);
@@ -205,11 +213,13 @@ public final class CountingBloomFilter extends Filter {
     return cbf;
     return cbf;
   }//end clone()
   }//end clone()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = super.hashCode();
     int result = super.hashCode();
@@ -221,6 +231,7 @@ public final class CountingBloomFilter extends Filter {
 
 
   // Writable
   // Writable
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
@@ -229,6 +240,7 @@ public final class CountingBloomFilter extends Filter {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
@@ -240,6 +252,7 @@ public final class CountingBloomFilter extends Filter {
 
 
   // Comparable
   // Comparable
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     int result = super.compareTo(o);
     int result = super.compareTo(o);

+ 13 - 0
src/contrib/hbase/src/java/org/onelab/filter/DynamicBloomFilter.java

@@ -118,6 +118,7 @@ public class DynamicBloomFilter extends Filter {
     matrix[0] = new BloomFilter(this.vectorSize, this.nbHash);
     matrix[0] = new BloomFilter(this.vectorSize, this.nbHash);
   }//end constructor
   }//end constructor
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void add(Key key){
   public void add(Key key){
     if(key == null) {
     if(key == null) {
@@ -137,6 +138,7 @@ public class DynamicBloomFilter extends Filter {
     currentNbRecord++;
     currentNbRecord++;
   }//end add()
   }//end add()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void and(Filter filter) {
   public void and(Filter filter) {
     if(filter == null
     if(filter == null
@@ -157,6 +159,7 @@ public class DynamicBloomFilter extends Filter {
     }
     }
   }//end and()
   }//end and()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean membershipTest(Key key){
   public boolean membershipTest(Key key){
     if(key == null) {
     if(key == null) {
@@ -172,6 +175,7 @@ public class DynamicBloomFilter extends Filter {
     return false;
     return false;
   }//end membershipTest()
   }//end membershipTest()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void not(){
   public void not(){
     for(int i = 0; i < matrix.length; i++) {
     for(int i = 0; i < matrix.length; i++) {
@@ -179,6 +183,7 @@ public class DynamicBloomFilter extends Filter {
     }
     }
   }//end not()
   }//end not()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void or(Filter filter){
   public void or(Filter filter){
     if(filter == null
     if(filter == null
@@ -198,6 +203,7 @@ public class DynamicBloomFilter extends Filter {
     }
     }
   }//end or()
   }//end or()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void xor(Filter filter){
   public void xor(Filter filter){
     if(filter == null
     if(filter == null
@@ -217,6 +223,7 @@ public class DynamicBloomFilter extends Filter {
     }
     }
   }//end xor()
   }//end xor()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public String toString(){
   public String toString(){
     StringBuilder res = new StringBuilder();
     StringBuilder res = new StringBuilder();
@@ -228,6 +235,7 @@ public class DynamicBloomFilter extends Filter {
     return res.toString();
     return res.toString();
   }//end toString()
   }//end toString()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public Object clone(){
   public Object clone(){
     DynamicBloomFilter dbf = new DynamicBloomFilter(vectorSize, nbHash, nr);
     DynamicBloomFilter dbf = new DynamicBloomFilter(vectorSize, nbHash, nr);
@@ -239,11 +247,13 @@ public class DynamicBloomFilter extends Filter {
     return dbf;
     return dbf;
   }//end clone()
   }//end clone()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = super.hashCode();
     int result = super.hashCode();
@@ -255,6 +265,7 @@ public class DynamicBloomFilter extends Filter {
 
 
   // Writable
   // Writable
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
@@ -263,6 +274,7 @@ public class DynamicBloomFilter extends Filter {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
@@ -274,6 +286,7 @@ public class DynamicBloomFilter extends Filter {
 
 
   // Comparable
   // Comparable
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     int result = super.compareTo(o);
     int result = super.compareTo(o);

+ 4 - 9
src/contrib/hbase/src/java/org/onelab/filter/Filter.java

@@ -182,6 +182,7 @@ public abstract class Filter implements WritableComparable {
     }
     }
   }//end add()
   }//end add()
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = Integer.valueOf(this.nbHash).hashCode();
     int result = Integer.valueOf(this.nbHash).hashCode();
@@ -191,17 +192,13 @@ public abstract class Filter implements WritableComparable {
 
 
   // Writable interface
   // Writable interface
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeInt(this.nbHash);
     out.writeInt(this.nbHash);
     out.writeInt(this.vectorSize);
     out.writeInt(this.vectorSize);
   }
   }
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.nbHash = in.readInt();
     this.nbHash = in.readInt();
     this.vectorSize = in.readInt();
     this.vectorSize = in.readInt();
@@ -210,9 +207,7 @@ public abstract class Filter implements WritableComparable {
   
   
   // Comparable interface
   // Comparable interface
   
   
-  /* (non-Javadoc)
-   * @see java.lang.Comparable#compareTo(java.lang.Object)
-   */
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     Filter other = (Filter)o;
     Filter other = (Filter)o;
     int result = this.vectorSize - other.vectorSize;
     int result = this.vectorSize - other.vectorSize;

+ 5 - 9
src/contrib/hbase/src/java/org/onelab/filter/Key.java

@@ -121,11 +121,13 @@ public class Key implements WritableComparable {
     this.weight++;
     this.weight++;
   }//end incrementWeight()
   }//end incrementWeight()
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = 0;
     int result = 0;
@@ -138,18 +140,14 @@ public class Key implements WritableComparable {
 
 
   // Writable
   // Writable
 
 
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput)
-   */
+  /** {@inheritDoc} */
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     out.writeInt(bytes.length);
     out.writeInt(bytes.length);
     out.write(bytes);
     out.write(bytes);
     out.writeDouble(weight);
     out.writeDouble(weight);
   }
   }
   
   
-  /* (non-Javadoc)
-   * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput)
-   */
+  /** {@inheritDoc} */
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     this.bytes = new byte[in.readInt()];
     this.bytes = new byte[in.readInt()];
     in.readFully(this.bytes);
     in.readFully(this.bytes);
@@ -158,9 +156,7 @@ public class Key implements WritableComparable {
   
   
   // Comparable
   // Comparable
   
   
-  /* (non-Javadoc)
-   * @see java.lang.Comparable#compareTo(java.lang.Object)
-   */
+  /** {@inheritDoc} */
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     Key other = (Key)o;
     Key other = (Key)o;
 
 

+ 6 - 0
src/contrib/hbase/src/java/org/onelab/filter/RetouchedBloomFilter.java

@@ -105,6 +105,7 @@ implements RemoveScheme {
     createVector();
     createVector();
   }//end constructor
   }//end constructor
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void add(Key key){
   public void add(Key key){
     if(key == null) {
     if(key == null) {
@@ -392,11 +393,13 @@ implements RemoveScheme {
     }//end for -i
     }//end for -i
   }//end createVector()
   }//end createVector()
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     return this.compareTo(o) == 0;
     return this.compareTo(o) == 0;
   }
   }
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     int result = super.hashCode();
     int result = super.hashCode();
@@ -414,6 +417,7 @@ implements RemoveScheme {
 
 
   // Writable
   // Writable
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
@@ -436,6 +440,7 @@ implements RemoveScheme {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     super.readFields(in);
@@ -465,6 +470,7 @@ implements RemoveScheme {
 
 
   // Comparable
   // Comparable
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   public int compareTo(Object o) {
   public int compareTo(Object o) {
     int result = super.compareTo(o);
     int result = super.compareTo(o);

+ 15 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java

@@ -29,12 +29,26 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.apache.log4j.PatternLayout;
 import org.apache.log4j.PatternLayout;
 
 
+/**
+ * Initializes test environment
+ */
 public class StaticTestEnvironment {
 public class StaticTestEnvironment {
-  private StaticTestEnvironment() {};                   // Not instantiable
+  private StaticTestEnvironment() {}                    // Not instantiable
 
 
+  /** configuration parameter name for test directory */
   public static final String TEST_DIRECTORY_KEY = "test.build.data";
   public static final String TEST_DIRECTORY_KEY = "test.build.data";
+  
+  /** set to true if "DEBUGGING" is set in the environment */
   public static boolean debugging = false;
   public static boolean debugging = false;
 
 
+  /**
+   * Initializes parameters used in the test environment:
+   * 
+   * Sets the configuration parameter TEST_DIRECTORY_KEY if not already set.
+   * Sets the boolean debugging if "DEBUGGING" is set in the environment.
+   * If debugging is enabled, reconfigures loggin so that the root log level is
+   * set to WARN and the logging level for the package is set to DEBUG.
+   */
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public static void initialize() {
   public static void initialize() {
     String value = null;
     String value = null;

+ 1 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestCompare.java

@@ -26,6 +26,7 @@ import junit.framework.TestCase;
  * Test comparing HBase objects.
  * Test comparing HBase objects.
  */
  */
 public class TestCompare extends TestCase {
 public class TestCompare extends TestCase {
+  /** test case */
   public void testHRegionInfo() {
   public void testHRegionInfo() {
     HRegionInfo a = new HRegionInfo(1, new HTableDescriptor("a"), null, null);
     HRegionInfo a = new HRegionInfo(1, new HTableDescriptor("a"), null, null);
     HRegionInfo b = new HRegionInfo(2, new HTableDescriptor("b"), null, null);
     HRegionInfo b = new HRegionInfo(2, new HTableDescriptor("b"), null, null);

+ 2 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestHLog.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.io.SequenceFile.Reader;
 /** JUnit test case for HLog */
 /** JUnit test case for HLog */
 public class TestHLog extends HBaseTestCase implements HConstants {
 public class TestHLog extends HBaseTestCase implements HConstants {
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void setUp() throws Exception {
   public void setUp() throws Exception {
     super.setUp();
     super.setUp();
@@ -103,6 +104,7 @@ public class TestHLog extends HBaseTestCase implements HConstants {
     }
     }
   }
   }
 
 
+  /** {@inheritDoc} */
   @Override
   @Override
   public void tearDown() throws Exception {
   public void tearDown() throws Exception {
     super.tearDown();
     super.tearDown();

+ 3 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeMeta.java

@@ -22,6 +22,9 @@ package org.apache.hadoop.hbase;
 /** Tests region merging */
 /** Tests region merging */
 public class TestMergeMeta extends AbstractMergeTestBase {
 public class TestMergeMeta extends AbstractMergeTestBase {
   
   
+  /**
+   * test case
+   */
   public void testMergeMeta() {
   public void testMergeMeta() {
     try {
     try {
       HMerge.merge(conf, fs, HConstants.META_TABLE_NAME);
       HMerge.merge(conf, fs, HConstants.META_TABLE_NAME);

+ 7 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestMergeTable.java

@@ -21,8 +21,15 @@ package org.apache.hadoop.hbase;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+/**
+ * Tests merging a normal table's regions
+ */
 public class TestMergeTable extends AbstractMergeTestBase {
 public class TestMergeTable extends AbstractMergeTestBase {
 
 
+  /**
+   * Test case
+   * @throws IOException
+   */
   public void testMergeTable() throws IOException {
   public void testMergeTable() throws IOException {
     MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
     MiniHBaseCluster hCluster = new MiniHBaseCluster(conf, 1, dfsCluster);
     try {
     try {

+ 11 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/TestToString.java

@@ -22,7 +22,14 @@ import org.apache.hadoop.io.Text;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+/**
+ * Tests toString methods.
+ */
 public class TestToString extends TestCase {
 public class TestToString extends TestCase {
+  /**
+   * tests toString methods on HSeverAddress, HServerInfo
+   * @throws Exception
+   */
   public void testServerInfo() throws Exception {
   public void testServerInfo() throws Exception {
     final String hostport = "127.0.0.1:9999";
     final String hostport = "127.0.0.1:9999";
     HServerAddress address = new HServerAddress(hostport);
     HServerAddress address = new HServerAddress(hostport);
@@ -32,6 +39,10 @@ public class TestToString extends TestCase {
         "address: " + hostport + ", startcode: " + -1);
         "address: " + hostport + ", startcode: " + -1);
   }
   }
   
   
+  /**
+   * Tests toString method on HRegionInfo
+   * @throws Exception
+   */
   public void testHRegionInfo() throws Exception {
   public void testHRegionInfo() throws Exception {
     HTableDescriptor htd = new HTableDescriptor("hank");
     HTableDescriptor htd = new HTableDescriptor("hank");
     htd.addFamily(new HColumnDescriptor("hankfamily:"));
     htd.addFamily(new HColumnDescriptor("hankfamily:"));

+ 13 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestPageRowFilter.java

@@ -28,20 +28,33 @@ import org.apache.hadoop.io.Text;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+/**
+ * Tests for the page row filter
+ */
 public class TestPageRowFilter extends TestCase {
 public class TestPageRowFilter extends TestCase {
   
   
   RowFilterInterface mainFilter;
   RowFilterInterface mainFilter;
   final int ROW_LIMIT = 3;
   final int ROW_LIMIT = 3;
   
   
+  /** {@inheritDoc} */
+  @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
     mainFilter = new PageRowFilter(ROW_LIMIT);
     mainFilter = new PageRowFilter(ROW_LIMIT);
   }
   }
   
   
+  /**
+   * test page size filter
+   * @throws Exception
+   */
   public void testPageSize() throws Exception {
   public void testPageSize() throws Exception {
     pageSizeTests(mainFilter);
     pageSizeTests(mainFilter);
   }
   }
   
   
+  /**
+   * Test filter serialization
+   * @throws Exception
+   */
   public void testSerialization() throws Exception {
   public void testSerialization() throws Exception {
     // Decompose mainFilter to bytes.
     // Decompose mainFilter to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
     ByteArrayOutputStream stream = new ByteArrayOutputStream();

+ 20 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRegExpRowFilter.java

@@ -31,6 +31,9 @@ import junit.framework.TestCase;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Tests for regular expression row filter
+ */
 public class TestRegExpRowFilter extends TestCase {
 public class TestRegExpRowFilter extends TestCase {
   TreeMap<Text, byte []> colvalues;
   TreeMap<Text, byte []> colvalues;
   RowFilterInterface mainFilter;
   RowFilterInterface mainFilter;
@@ -39,6 +42,7 @@ public class TestRegExpRowFilter extends TestCase {
   byte [] GOOD_BYTES = "abc".getBytes();
   byte [] GOOD_BYTES = "abc".getBytes();
   final String HOST_PREFIX = "org.apache.site-";
   final String HOST_PREFIX = "org.apache.site-";
   
   
+  /** {@inheritDoc} */
   @Override
   @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
@@ -49,18 +53,34 @@ public class TestRegExpRowFilter extends TestCase {
     this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
     this.mainFilter = new RegExpRowFilter(HOST_PREFIX + ".*", colvalues);
   }
   }
   
   
+  /**
+   * Tests filtering using a regex on the row key
+   * @throws Exception
+   */
   public void testRegexOnRow() throws Exception {
   public void testRegexOnRow() throws Exception {
     regexRowTests(mainFilter);
     regexRowTests(mainFilter);
   }
   }
 
 
+  /**
+   * Tests filtering using a regex on row and colum
+   * @throws Exception
+   */
   public void testRegexOnRowAndColumn() throws Exception {
   public void testRegexOnRowAndColumn() throws Exception {
     regexRowColumnTests(mainFilter);
     regexRowColumnTests(mainFilter);
   }
   }
   
   
+  /**
+   * Only return values that are not null
+   * @throws Exception
+   */
   public void testFilterNotNull() throws Exception {
   public void testFilterNotNull() throws Exception {
     filterNotNullTests(mainFilter);
     filterNotNullTests(mainFilter);
   }
   }
   
   
+  /**
+   * Test serialization
+   * @throws Exception
+   */
   public void testSerialization() throws Exception {
   public void testSerialization() throws Exception {
     // Decompose mainFilter to bytes.
     // Decompose mainFilter to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
     ByteArrayOutputStream stream = new ByteArrayOutputStream();

+ 18 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestRowFilterSet.java

@@ -32,6 +32,9 @@ import org.apache.hadoop.io.Text;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+/**
+ * Tests filter sets
+ */
 public class TestRowFilterSet extends TestCase {
 public class TestRowFilterSet extends TestCase {
 
 
   RowFilterInterface filterMPALL;
   RowFilterInterface filterMPALL;
@@ -42,7 +45,9 @@ public class TestRowFilterSet extends TestCase {
   final byte[] GOOD_BYTES = "abc".getBytes();
   final byte[] GOOD_BYTES = "abc".getBytes();
   final byte[] BAD_BYTES = "def".getBytes();
   final byte[] BAD_BYTES = "def".getBytes();
   TreeMap<Text, byte[]> colvalues;
   TreeMap<Text, byte[]> colvalues;
-  
+
+  /** {@inheritDoc} */
+  @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
     
     
@@ -62,14 +67,26 @@ public class TestRowFilterSet extends TestCase {
       filters);
       filters);
   }
   }
   
   
+  /**
+   * Test "must pass one"
+   * @throws Exception
+   */
   public void testMPONE() throws Exception {
   public void testMPONE() throws Exception {
     MPONETests(filterMPONE);
     MPONETests(filterMPONE);
   }
   }
 
 
+  /**
+   * Test "must pass all"
+   * @throws Exception
+   */
   public void testMPALL() throws Exception {
   public void testMPALL() throws Exception {
     MPALLTests(filterMPALL);
     MPALLTests(filterMPALL);
   }
   }
   
   
+  /**
+   * Test serialization
+   * @throws Exception
+   */
   public void testSerialization() throws Exception {
   public void testSerialization() throws Exception {
     // Decompose filterMPALL to bytes.
     // Decompose filterMPALL to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
     ByteArrayOutputStream stream = new ByteArrayOutputStream();

+ 14 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestStopRowFilter.java

@@ -28,22 +28,35 @@ import org.apache.hadoop.io.Text;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+/**
+ * Tests the stop row filter
+ */
 public class TestStopRowFilter extends TestCase {
 public class TestStopRowFilter extends TestCase {
   private final Text STOP_ROW = new Text("stop_row");
   private final Text STOP_ROW = new Text("stop_row");
   private final Text GOOD_ROW = new Text("good_row");
   private final Text GOOD_ROW = new Text("good_row");
   private final Text PAST_STOP_ROW = new Text("zzzzzz");
   private final Text PAST_STOP_ROW = new Text("zzzzzz");
   
   
   RowFilterInterface mainFilter;
   RowFilterInterface mainFilter;
-  
+
+  /** {@inheritDoc} */
+  @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
     mainFilter = new StopRowFilter(STOP_ROW);
     mainFilter = new StopRowFilter(STOP_ROW);
   }
   }
   
   
+  /**
+   * Tests identification of the stop row
+   * @throws Exception
+   */
   public void testStopRowIdentification() throws Exception {
   public void testStopRowIdentification() throws Exception {
     stopRowTests(mainFilter);
     stopRowTests(mainFilter);
   }
   }
   
   
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
   public void testSerialization() throws Exception {
   public void testSerialization() throws Exception {
     // Decompose mainFilter to bytes.
     // Decompose mainFilter to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
     ByteArrayOutputStream stream = new ByteArrayOutputStream();

+ 18 - 1
src/contrib/hbase/src/test/org/apache/hadoop/hbase/filter/TestWhileMatchRowFilter.java

@@ -28,11 +28,16 @@ import junit.framework.TestCase;
 
 
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 
 
+/**
+ * Tests for the while-match filter
+ */
 public class TestWhileMatchRowFilter extends TestCase {
 public class TestWhileMatchRowFilter extends TestCase {
 
 
   WhileMatchRowFilter wmStopRowFilter;
   WhileMatchRowFilter wmStopRowFilter;
   WhileMatchRowFilter wmRegExpRowFilter;
   WhileMatchRowFilter wmRegExpRowFilter;
-  
+
+  /** {@inheritDoc} */
+  @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
     wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
     wmStopRowFilter = new WhileMatchRowFilter(new StopRowFilter(
@@ -41,14 +46,26 @@ public class TestWhileMatchRowFilter extends TestCase {
     ".*regex.*"));
     ".*regex.*"));
   }
   }
   
   
+  /**
+   * Tests while match stop row
+   * @throws Exception
+   */
   public void testWhileMatchStopRow() throws Exception {
   public void testWhileMatchStopRow() throws Exception {
     whileMatchStopRowTests(wmStopRowFilter);
     whileMatchStopRowTests(wmStopRowFilter);
   }
   }
   
   
+  /**
+   * Tests while match regex
+   * @throws Exception
+   */
   public void testWhileMatchRegExp() throws Exception {
   public void testWhileMatchRegExp() throws Exception {
     whileMatchRegExpTests(wmRegExpRowFilter);
     whileMatchRegExpTests(wmRegExpRowFilter);
   }
   }
   
   
+  /**
+   * Tests serialization
+   * @throws Exception
+   */
   public void testSerialization() throws Exception {
   public void testSerialization() throws Exception {
     // Decompose wmRegExpRowFilter to bytes.
     // Decompose wmRegExpRowFilter to bytes.
     ByteArrayOutputStream stream = new ByteArrayOutputStream();
     ByteArrayOutputStream stream = new ByteArrayOutputStream();

+ 4 - 4
src/contrib/hbase/src/test/org/apache/hadoop/hbase/shell/TestHBaseShell.java

@@ -21,13 +21,13 @@ package org.apache.hadoop.hbase.shell;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HClient;
-import org.apache.hadoop.hbase.shell.generated.ParseException;
 import org.apache.hadoop.hbase.shell.generated.Parser;
 import org.apache.hadoop.hbase.shell.generated.Parser;
 
 
+/**
+ * Tests for HBase shell
+ */
 public class TestHBaseShell extends TestCase {
 public class TestHBaseShell extends TestCase {
+  /** test parsing */
   public void testParse() {
   public void testParse() {
     String queryString1 = "SELECT test_table WHERE row='row_key' and " +
     String queryString1 = "SELECT test_table WHERE row='row_key' and " +
       "column='column_key';";
       "column='column_key';";

+ 11 - 0
src/contrib/hbase/src/test/org/apache/hadoop/hbase/util/TestKeying.java

@@ -21,16 +21,27 @@ package org.apache.hadoop.hbase.util;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 
 
+/**
+ * Tests url transformations
+ */
 public class TestKeying extends TestCase {
 public class TestKeying extends TestCase {
 
 
+  /** {@inheritDoc} */
+  @Override
   protected void setUp() throws Exception {
   protected void setUp() throws Exception {
     super.setUp();
     super.setUp();
   }
   }
 
 
+  /** {@inheritDoc} */
+  @Override
   protected void tearDown() throws Exception {
   protected void tearDown() throws Exception {
     super.tearDown();
     super.tearDown();
   }
   }
 
 
+  /**
+   * Test url transformations
+   * @throws Exception
+   */
   public void testURI() throws Exception {
   public void testURI() throws Exception {
     checkTransform("http://abc:bcd@www.example.com/index.html" +
     checkTransform("http://abc:bcd@www.example.com/index.html" +
       "?query=something#middle");
       "?query=something#middle");