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

svn commit: r1183167 - in /hbase/branches/0.92: ./ src/main/java/org/apache/hadoop/hbase/util/ src/main/java/org/apache/hadoop/hbase/zookeeper/ src/main/resources/hbase-webapps/master/

Author: nspiegelberg
Date: Thu Oct 13 23:51:31 2011
New Revision: 1183167

URL: http://svn.apache.org/viewvc?rev=1183167&view=rev
Log:
HBASE-4568  Make zk dump jsp response faster

Modified:
    hbase/branches/0.92/CHANGES.txt
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
    hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/branches/0.92/src/main/resources/hbase-webapps/master/zk.jsp

Modified: hbase/branches/0.92/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/CHANGES.txt?rev=1183167&r1=1183166&r2=1183167&view=diff
==============================================================================
--- hbase/branches/0.92/CHANGES.txt (original)
+++ hbase/branches/0.92/CHANGES.txt Thu Oct 13 23:51:31 2011
@@ -589,6 +589,7 @@ Release 0.92.0 - Unreleased
                CacheConf class (jgray)
    HBASE-4558  Refactor TestOpenedRegionHandler and TestOpenRegionHandler. (Ram)
    HBASE-4558  Addendum for TestMasterFailOver (Ram) - Breaks the build
+   HBASE-4568  Make zk dump jsp response faster
 
   TASKS
    HBASE-3559  Move report of split to master OFF the heartbeat channel

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java?rev=1183167&r1=1183166&r2=1183167&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/util/RetryCounter.java Thu Oct 13 23:51:31 2011
@@ -21,7 +21,11 @@ package org.apache.hadoop.hbase.util;
 
 import java.util.concurrent.TimeUnit;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 public class RetryCounter {
+  private static final Log LOG = LogFactory.getLog(RetryCounter.class);
   private final int maxRetries;
   private int retriesRemaining;
   private final int retryIntervalMillis;
@@ -39,8 +43,16 @@ public class RetryCounter {
     return maxRetries;
   }
 
+  /**
+   * Sleep for a exponentially back off time
+   * @throws InterruptedException
+   */
   public void sleepUntilNextRetry() throws InterruptedException {
-    timeUnit.sleep(retryIntervalMillis);
+    int attempts = getAttemptTimes();
+    long sleepTime = (long) (retryIntervalMillis * Math.pow(2, attempts));
+    LOG.info("The " + attempts + " times to retry  after sleeping " + sleepTime
+        + " ms");
+    timeUnit.sleep(sleepTime);
   }
 
   public boolean shouldRetry() {

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java?rev=1183167&r1=1183166&r2=1183167&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java Thu Oct 13 23:51:31 2011
@@ -132,8 +132,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
       isRetry = true;
@@ -170,8 +168,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -207,8 +203,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -244,8 +238,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -281,8 +273,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -319,9 +309,7 @@ public class RecoverableZooKeeper {
           default:
             throw e;
         }
-      } 
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
+      }
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -359,8 +347,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -419,8 +405,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }
@@ -509,8 +493,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
       isRetry = true;
@@ -550,8 +532,6 @@ public class RecoverableZooKeeper {
             throw e;
         }
       }
-      LOG.info("The "+retryCounter.getAttemptTimes()+" times to retry " +
-          "ZooKeeper after sleeping "+retryIntervalMillis+" ms");
       retryCounter.sleepUntilNextRetry();
       retryCounter.useRetry();
     }

Modified: hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=1183167&r1=1183166&r2=1183167&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/branches/0.92/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Thu Oct 13 23:51:31 2011
@@ -23,6 +23,7 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
+import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.util.ArrayList;
 import java.util.List;
@@ -58,6 +59,7 @@ public class ZKUtil {
 
   // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
   private static final char ZNODE_PATH_SEPARATOR = '/';
+  private static int zkDumpConnectionTimeOut;
 
   /**
    * Creates a new connection to ZooKeeper, pulling settings and ensemble config
@@ -93,9 +95,11 @@ public class ZKUtil {
     LOG.debug(descriptor + " opening connection to ZooKeeper with ensemble (" +
         ensemble + ")");
     int retry = conf.getInt("zookeeper.recovery.retry", 3);
-    int retryIntervalMillis = 
+    int retryIntervalMillis =
       conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
-    return new RecoverableZooKeeper(ensemble, timeout, watcher, 
+    zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
+        1000);
+    return new RecoverableZooKeeper(ensemble, timeout, watcher,
         retry, retryIntervalMillis);
   }
 
@@ -223,7 +227,7 @@ public class ZKUtil {
       if (exists) {
         LOG.debug(zkw.prefix("Set watcher on existing znode " + znode));
       } else {
-        LOG.debug(zkw.prefix(znode+" does not exist. Watcher is set."));        
+        LOG.debug(zkw.prefix(znode+" does not exist. Watcher is set."));
       }
       return exists;
     } catch (KeeperException e) {
@@ -949,11 +953,11 @@ public class ZKUtil {
     try {
       sb.append("HBase is rooted at ").append(zkw.baseZNode);
       sb.append("\nMaster address: ").append(
-        Bytes.toStringBinary(getData(zkw, zkw.masterAddressZNode)));
+          Bytes.toStringBinary(getData(zkw, zkw.masterAddressZNode)));
       sb.append("\nRegion server holding ROOT: ").append(
-        Bytes.toStringBinary(getData(zkw, zkw.rootServerZNode)));
+          Bytes.toStringBinary(getData(zkw, zkw.rootServerZNode)));
       sb.append("\nRegion servers:");
-      for (String child: listChildrenNoWatch(zkw, zkw.rsZNode)) {
+      for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
         sb.append("\n ").append(child);
       }
       sb.append("\nQuorum Server Statistics:");
@@ -961,7 +965,13 @@ public class ZKUtil {
       for (String server : servers) {
         sb.append("\n ").append(server);
         try {
-          String[] stat = getServerStats(server);
+          String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
+
+          if (stat == null) {
+            sb.append("[Error] invalid quorum server: " + server);
+            break;
+          }
+
           for (String s : stat) {
             sb.append("\n  ").append(s);
           }
@@ -969,7 +979,7 @@ public class ZKUtil {
           sb.append("\n  ERROR: ").append(e.getMessage());
         }
       }
-    } catch(KeeperException ke) {
+    } catch (KeeperException ke) {
       sb.append("\nFATAL ZooKeeper Exception!\n");
       sb.append("\n" + ke.getMessage());
     }
@@ -977,18 +987,6 @@ public class ZKUtil {
   }
 
   /**
-   * Gets the statistics from the given server. Uses a 1 minute timeout.
-   *
-   * @param server  The server to get the statistics from.
-   * @return The array of response strings.
-   * @throws IOException When the socket communication fails.
-   */
-  public static String[] getServerStats(String server)
-  throws IOException {
-    return getServerStats(server, 60 * 1000);
-  }
-
-  /**
    * Gets the statistics from the given server.
    *
    * @param server  The server to get the statistics from.
@@ -999,8 +997,18 @@ public class ZKUtil {
   public static String[] getServerStats(String server, int timeout)
   throws IOException {
     String[] sp = server.split(":");
-    Socket socket = new Socket(sp[0],
-      sp.length > 1 ? Integer.parseInt(sp[1]) : 2181);
+    if (sp == null || sp.length == 0) {
+      return null;
+    }
+
+    String host = sp[0];
+    int port = sp.length > 1 ? Integer.parseInt(sp[1])
+        : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
+
+    Socket socket = new Socket();
+    InetSocketAddress sockAddr = new InetSocketAddress(host, port);
+    socket.connect(sockAddr, timeout);
+
     socket.setSoTimeout(timeout);
     PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
     BufferedReader in = new BufferedReader(new InputStreamReader(

Modified: hbase/branches/0.92/src/main/resources/hbase-webapps/master/zk.jsp
URL: http://svn.apache.org/viewvc/hbase/branches/0.92/src/main/resources/hbase-webapps/master/zk.jsp?rev=1183167&r1=1183166&r2=1183167&view=diff
==============================================================================
--- hbase/branches/0.92/src/main/resources/hbase-webapps/master/zk.jsp (original)
+++ hbase/branches/0.92/src/main/resources/hbase-webapps/master/zk.jsp Thu Oct 13 23:51:31 2011
@@ -8,18 +8,15 @@
   import="org.apache.hadoop.hbase.zookeeper.ZKUtil"
   import="org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher"
   import="org.apache.hadoop.hbase.HBaseConfiguration"
-  import="org.apache.hadoop.hbase.master.HMaster" 
+  import="org.apache.hadoop.hbase.master.HMaster"
   import="org.apache.hadoop.hbase.HConstants"%><%
   HMaster master = (HMaster)getServletContext().getAttribute(HMaster.MASTER);
-  Configuration conf = master.getConfiguration();
-  HBaseAdmin hbadmin = new HBaseAdmin(conf);
-  HConnection connection = hbadmin.getConnection();
-  ZooKeeperWatcher watcher = connection.getZooKeeperWatcher();
+  ZooKeeperWatcher watcher = master.getZooKeeperWatcher();
 %>
 
 <?xml version="1.0" encoding="UTF-8" ?>
-<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN" 
-  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd"> 
+<!DOCTYPE html PUBLIC "-//W3C//DTD XHTML 1.0 Transitional//EN"
+  "http://www.w3.org/TR/xhtml1/DTD/xhtml1-transitional.dtd">
 <html xmlns="http://www.w3.org/1999/xhtml">
 <head><meta http-equiv="Content-Type" content="text/html;charset=UTF-8"/>
 <title>ZooKeeper Dump</title>
@@ -32,7 +29,6 @@
 <hr id="head_rule" />
 <pre>
 <%= ZKUtil.dump(watcher) %>
-<% HConnectionManager.deleteConnection(hbadmin.getConfiguration(), false); %>
 </pre>
 
 </body>