You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/10/25 08:51:31 UTC

svn commit: r1188537 [2/2] - in /hbase/trunk: ./ src/main/java/org/apache/hadoop/hbase/ src/main/java/org/apache/hadoop/hbase/catalog/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/client/coprocessor/ src/main/java...

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/BloomFilterFactory.java Tue Oct 25 06:51:28 2011
@@ -161,11 +161,11 @@ public final class BloomFilterFactory {
    * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
    *
    * @param conf
+   * @param cacheConf
    * @param bloomType
    * @param maxKeys an estimate of the number of keys we expect to insert.
    *        Irrelevant if compound Bloom filters are enabled.
    * @param writer the HFile writer
-   * @param bloomErrorRate
    * @return the new Bloom filter, or null in case Bloom filters are disabled
    *         or when failed to create one.
    */
@@ -231,10 +231,10 @@ public final class BloomFilterFactory {
    * Creates a new Delete Family Bloom filter at the time of
    * {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
    * @param conf
+   * @param cacheConf
    * @param maxKeys an estimate of the number of keys we expect to insert.
    *        Irrelevant if compound Bloom filters are enabled.
    * @param writer the HFile writer
-   * @param bloomErrorRate
    * @return the new Bloom filter, or null in case Bloom filters are disabled
    *         or when failed to create one.
    */

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/ByteBloomFilter.java Tue Oct 25 06:51:28 2011
@@ -164,8 +164,7 @@ public class ByteBloomFilter implements 
    * @param bitSize
    * @param errorRate
    * @return maximum number of keys that can be inserted into the Bloom filter
-   * @see {@link #computeMaxKeys(long, double, int)} for a more precise
-   *      estimate
+   * @see #computeMaxKeys(long, double, int) for a more precise estimate
    */
   public static long idealMaxKeys(long bitSize, double errorRate) {
     // The reason we need to use floor here is that otherwise we might put
@@ -227,7 +226,7 @@ public class ByteBloomFilter implements 
    *
    * @param bitSize
    * @param foldFactor
-   * @return
+   * @return Foldable byte size
    */
   public static int computeFoldableByteSize(long bitSize, int foldFactor) {
     long byteSizeLong = (bitSize + 7) / 8;

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Bytes.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Bytes.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Bytes.java Tue Oct 25 06:51:28 2011
@@ -759,7 +759,7 @@ public class Bytes {
    * This method will get a sequence of bytes from pos -> limit,
    * but will restore pos after.
    * @param buf
-   * @return
+   * @return byte array
    */
   public static byte[] getBytes(ByteBuffer buf) {
     int savedPos = buf.position();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/CompoundBloomFilterBase.java Tue Oct 25 06:51:28 2011
@@ -71,7 +71,7 @@ public class CompoundBloomFilterBase imp
 
   /**
    * Prepare an ordered pair of row and qualifier to be compared using
-   * {@link KeyValue.KeyComparator}. This is only used for row-column Bloom
+   * KeyValue.KeyComparator. This is only used for row-column Bloom
    * filters.
    */
   @Override

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java Tue Oct 25 06:51:28 2011
@@ -19,6 +19,17 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -38,23 +49,11 @@ import org.apache.hadoop.hbase.RemoteExc
 import org.apache.hadoop.hbase.master.HMaster;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 
-import java.io.DataInputStream;
-import java.io.EOFException;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Utility methods for interacting with the underlying file system.
  */
@@ -523,7 +522,7 @@ public abstract class FSUtils {
   /**
    * Compute HDFS blocks distribution of a given file, or a portion of the file
    * @param fs file system
-   * @param FileStatus file status of the file
+   * @param status file status of the file
    * @param start start position of the portion
    * @param length length of the portion 
    * @return The HDFS blocks distribution
@@ -1104,7 +1103,7 @@ public abstract class FSUtils {
   /**
    * Update table descriptor
    * @param fs
-   * @param conf
+   * @param rootdir
    * @param hTableDescriptor
    * @throws IOException
    */

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Objects.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Objects.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Objects.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/util/Objects.java Tue Oct 25 06:51:28 2011
@@ -21,14 +21,10 @@ package org.apache.hadoop.hbase.util;
 
 import java.lang.reflect.Array;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
-import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Action;
@@ -129,7 +125,7 @@ public class Objects {
    * Attempts to construct a text description of the given object, by
    * introspecting known classes and building a description of size.
    * @param obj
-   * @return
+   * @return Description
    */
   public static String describeQuantity(Object obj) {
     StringBuilder str = new StringBuilder();

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java Tue Oct 25 06:51:28 2011
@@ -59,7 +59,7 @@ import org.apache.zookeeper.data.Stat;
  * the create it will do a getChildren("/") and see "x-222-1", "x-542-30", 
  * "x-352-109", x-333-110". The process will know that the original create 
  * succeeded an the znode it created is "x-352-109".
- * @see http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling
+ * @see "http://wiki.apache.org/hadoop/ZooKeeper/ErrorHandling"
  */
 public class RecoverableZooKeeper {
   private static final Log LOG = LogFactory.getLog(RecoverableZooKeeper.class);
@@ -142,7 +142,7 @@ public class RecoverableZooKeeper {
    * exists is an idempotent operation. Retry before throw out exception
    * @param path
    * @param watcher
-   * @return
+   * @return A Stat instance
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -177,7 +177,7 @@ public class RecoverableZooKeeper {
    * exists is an idempotent operation. Retry before throw out exception
    * @param path
    * @param watch
-   * @return
+   * @return A Stat instance
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -212,7 +212,7 @@ public class RecoverableZooKeeper {
    * getChildren is an idempotent operation. Retry before throw out exception
    * @param path
    * @param watcher
-   * @return
+   * @return List of children znodes
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -247,7 +247,7 @@ public class RecoverableZooKeeper {
    * getChildren is an idempotent operation. Retry before throw out exception
    * @param path
    * @param watch
-   * @return
+   * @return List of children znodes
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -283,7 +283,7 @@ public class RecoverableZooKeeper {
    * @param path
    * @param watcher
    * @param stat
-   * @return
+   * @return Data
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -320,7 +320,7 @@ public class RecoverableZooKeeper {
    * @param path
    * @param watch
    * @param stat
-   * @return
+   * @return Data
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -359,7 +359,7 @@ public class RecoverableZooKeeper {
    * @param path
    * @param data
    * @param version
-   * @return 
+   * @return Stat instance
    * @throws KeeperException
    * @throws InterruptedException
    */
@@ -427,7 +427,7 @@ public class RecoverableZooKeeper {
    * @param data
    * @param acl
    * @param createMode
-   * @return
+   * @return Path
    * @throws KeeperException
    * @throws InterruptedException
    */

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/RegionServerTracker.java Tue Oct 25 06:51:28 2011
@@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hbase.Abortable;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.master.ServerManager;
-import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
 import org.apache.zookeeper.KeeperException;
 
 /**
@@ -41,7 +40,7 @@ import org.apache.zookeeper.KeeperExcept
  * listening for changes in the RS node list and watching each node.
  *
  * <p>If an RS node gets deleted, this automatically handles calling of
- * {@link ServerManager#expireServer(org.apache.hadoop.hbase.HServerInfo)}.
+ * {@link ServerManager#expireServer(ServerName)}
  */
 public class RegionServerTracker extends ZooKeeperListener {
   private static final Log LOG = LogFactory.getLog(RegionServerTracker.class);
@@ -121,7 +120,6 @@ public class RegionServerTracker extends
   /**
    * Gets the online servers.
    * @return list of online servers
-   * @throws KeeperException
    */
   public List<ServerName> getOnlineServers() {
     synchronized (this.regionServers) {

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java?rev=1188537&r1=1188536&r2=1188537&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperNodeTracker.java Tue Oct 25 06:51:28 2011
@@ -68,7 +68,7 @@ public abstract class ZooKeeperNodeTrack
    * Starts the tracking of the node in ZooKeeper.
    *
    * <p>Use {@link #blockUntilAvailable()} to block until the node is available
-   * or {@link #getData()} to get the data of the node if it is available.
+   * or {@link #getData(boolean)} to get the data of the node if it is available.
    */
   public synchronized void start() {
     this.watcher.registerListener(this);
@@ -143,7 +143,7 @@ public abstract class ZooKeeperNodeTrack
    * <p>If the node is currently available, the most up-to-date known version of
    * the data is returned.  If the node is not currently available, null is
    * returned.
-   * @param whether to refresh the data by calling ZK directly.
+   * @param refresh whether to refresh the data by calling ZK directly.
    * @return data of the node, null if unavailable
    */
   public synchronized byte [] getData(boolean refresh) {