hbase-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From st...@apache.org
Subject svn commit: r998555 - in /hbase/trunk: conf/ src/main/java/org/apache/hadoop/hbase/client/ src/main/java/org/apache/hadoop/hbase/master/ src/main/java/org/apache/hadoop/hbase/regionserver/ src/main/java/org/apache/hadoop/hbase/zookeeper/
Date Sat, 18 Sep 2010 21:56:37 GMT
Author: stack
Date: Sat Sep 18 21:56:37 2010
New Revision: 998555

URL: http://svn.apache.org/viewvc?rev=998555&view=rev
Log:
HBASE-3016  Refactor zk logging in trunk; do less and and identifier to log messages to help
debugging

Modified:
    hbase/trunk/conf/log4j.properties
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
    hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java

Modified: hbase/trunk/conf/log4j.properties
URL: http://svn.apache.org/viewvc/hbase/trunk/conf/log4j.properties?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/conf/log4j.properties (original)
+++ hbase/trunk/conf/log4j.properties Sat Sep 18 21:56:37 2010
@@ -44,3 +44,5 @@ log4j.logger.org.apache.zookeeper=INFO
 #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
 log4j.logger.org.apache.hadoop.hbase=DEBUG
 #log4j.logger.org.apache.hadoop.dfs=DEBUG
+# Set this class to log INFO only otherwise its OTT
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Sat Sep
18 21:56:37 2010
@@ -917,10 +917,7 @@ public class HConnectionManager {
         throws ZooKeeperConnectionException {
       if(zooKeeper == null) {
         try {
-          this.zooKeeper = new ZooKeeperWatcher(conf,
-            ZKUtil.getZooKeeperClusterKey(conf), this);
-          LOG.debug("zkw created, sessionid=0x" +
-            Long.toHexString(this.zooKeeper.getZooKeeper().getSessionId()));
+          this.zooKeeper = new ZooKeeperWatcher(conf, "hconnection", this);
         } catch (IOException e) {
           throw new ZooKeeperConnectionException(e);
         }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java Sat
Sep 18 21:56:37 2010
@@ -184,7 +184,7 @@ class ActiveMasterManager extends ZooKee
         ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
       }
     } catch (KeeperException e) {
-      watcher.error("Error deleting our own master address node", e);
+      LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
     }
   }
 }
\ No newline at end of file

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Sat
Sep 18 21:56:37 2010
@@ -1172,8 +1172,6 @@ public class HRegionServer implements HR
     // Start Server.  This service is like leases in that it internally runs
     // a thread.
     this.server.start();
-    LOG.info("HRegionServer started at: "
-        + this.serverInfo.getServerAddress().toString());
   }
 
   /*

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java Sat Sep 18 21:56:37
2010
@@ -19,6 +19,8 @@
  */
 package org.apache.hadoop.hbase.zookeeper;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.executor.RegionTransitionData;
 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
@@ -83,6 +85,7 @@ import org.apache.zookeeper.data.Stat;
  * </ol>
  */
 public class ZKAssign {
+  private static final Log LOG = LogFactory.getLog(ZKAssign.class);
 
   /**
    * Gets the full path node name for the unassigned node for the specified
@@ -132,8 +135,8 @@ public class ZKAssign {
   public static void createNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
       String serverName, final EventType event)
   throws KeeperException, KeeperException.NodeExistsException {
-    zkw.debug("Creating an unassigned node for " + region.getEncodedName() +
-        " in an OFFLINE state");
+    LOG.debug(zkw.prefix("Creating an unassigned node for " +
+      region.getEncodedName() + " in an OFFLINE state"));
     RegionTransitionData data = new RegionTransitionData(event,
       region.getRegionName(), serverName);
     synchronized(zkw.getNodes()) {
@@ -164,8 +167,8 @@ public class ZKAssign {
   public static void forceNodeOffline(ZooKeeperWatcher zkw, HRegionInfo region,
       String serverName)
   throws KeeperException, KeeperException.NoNodeException {
-    zkw.debug("Forcing an existing unassigned node for " +
-        region.getEncodedName() + " to an OFFLINE state");
+    LOG.debug(zkw.prefix("Forcing an existing unassigned node for " +
+      region.getEncodedName() + " to an OFFLINE state"));
     RegionTransitionData data = new RegionTransitionData(
         EventType.M2ZK_REGION_OFFLINE, region.getRegionName(), serverName);
     synchronized(zkw.getNodes()) {
@@ -197,8 +200,8 @@ public class ZKAssign {
   public static boolean createOrForceNodeOffline(ZooKeeperWatcher zkw,
       HRegionInfo region, String serverName)
   throws KeeperException {
-    zkw.debug("Creating or updating an unassigned node for " +
-        region.getEncodedName() + " with an OFFLINE state");
+    LOG.debug(zkw.prefix("Creating or updating an unassigned node for " +
+      region.getEncodedName() + " with an OFFLINE state"));
     RegionTransitionData data = new RegionTransitionData(
         EventType.M2ZK_REGION_OFFLINE, region.getRegionName(), serverName);
     synchronized(zkw.getNodes()) {
@@ -316,8 +319,8 @@ public class ZKAssign {
   private static boolean deleteNode(ZooKeeperWatcher zkw, String regionName,
       EventType expectedState)
   throws KeeperException, KeeperException.NoNodeException {
-    zkw.debug("Deleting an existing unassigned node for " + regionName +
-        " that is in expected state " + expectedState);
+    LOG.debug(zkw.prefix("Deleting an existing unassigned " +
+      "node for " + regionName + " that is in expected state " + expectedState));
     String node = getNodeName(zkw, regionName);
     Stat stat = new Stat();
     byte [] bytes = ZKUtil.getDataNoWatch(zkw, node, stat);
@@ -326,17 +329,19 @@ public class ZKAssign {
     }
     RegionTransitionData data = RegionTransitionData.fromBytes(bytes);
     if(!data.getEventType().equals(expectedState)) {
-      zkw.warn("Attempting to delete an unassigned node in " + expectedState +
-          " state but node is in " + data.getEventType() + " state");
+      LOG.warn(zkw.prefix("Attempting to delete an unassigned " +
+        "node in " + expectedState +
+        " state but node is in " + data.getEventType() + " state"));
       return false;
     }
     synchronized(zkw.getNodes()) {
       // TODO: Does this go here or only if we successfully delete node?
       zkw.getNodes().remove(node);
       if(!ZKUtil.deleteNode(zkw, node, stat.getVersion())) {
-        zkw.warn("Attempting to delete an unassigned node in " + expectedState +
+        LOG.warn(zkw.prefix("Attempting to delete an " +
+          "unassigned node in " + expectedState +
             " state but " +
-            "after verifying it was in OPENED state, we got a version mismatch");
+            "after verifying it was in OPENED state, we got a version mismatch"));
         return false;
       }
       return true;
@@ -356,7 +361,7 @@ public class ZKAssign {
    */
   public static void deleteAllNodes(ZooKeeperWatcher zkw)
   throws KeeperException {
-    zkw.debug("Deleting any existing unassigned nodes");
+    LOG.debug(zkw.prefix("Deleting any existing unassigned nodes"));
     ZKUtil.deleteChildrenRecursively(zkw, zkw.assignmentZNode);
   }
 
@@ -387,8 +392,8 @@ public class ZKAssign {
   public static int createNodeClosing(ZooKeeperWatcher zkw, HRegionInfo region,
       String serverName)
   throws KeeperException, KeeperException.NodeExistsException {
-    zkw.debug("Creating an unassigned node for " + region.getEncodedName() +
-    " in a CLOSING state");
+    LOG.debug(zkw.prefix("Creating an unassigned node for " +
+      region.getEncodedName() + " in a CLOSING state"));
     RegionTransitionData data = new RegionTransitionData(
         EventType.RS2ZK_REGION_CLOSING, region.getRegionName(), serverName);
     synchronized(zkw.getNodes()) {
@@ -574,10 +579,10 @@ public class ZKAssign {
       int expectedVersion)
   throws KeeperException {
     String encoded = region.getEncodedName();
-    if(zkw.isDebugEnabled()) {
-      zkw.debug("Attempting to transition node " +
+    if(LOG.isDebugEnabled()) {
+      LOG.debug(zkw.prefix("Attempting to transition node " +
         HRegionInfo.prettyPrint(encoded) +
-        " from " + beginState.toString() + " to " + endState.toString());
+        " from " + beginState.toString() + " to " + endState.toString()));
     }
 
     String node = getNodeName(zkw, encoded);
@@ -591,18 +596,20 @@ public class ZKAssign {
 
     // Verify it is the expected version
     if(expectedVersion != -1 && stat.getVersion() != expectedVersion) {
-      zkw.warn("Attempt to transition the unassigned node for " + encoded +
-          " from " + beginState + " to " + endState + " failed, " +
-          "the node existed but was version " + stat.getVersion() +
-          " not the expected version " + expectedVersion);
+      LOG.warn(zkw.prefix("Attempt to transition the " +
+        "unassigned node for " + encoded +
+        " from " + beginState + " to " + endState + " failed, " +
+        "the node existed but was version " + stat.getVersion() +
+        " not the expected version " + expectedVersion));
         return -1;
     }
 
     // Verify it is in expected state
     if(!existingData.getEventType().equals(beginState)) {
-      zkw.warn("Attempt to transition the unassigned node for " + encoded +
+      LOG.warn(zkw.prefix("Attempt to transition the " +
+        "unassigned node for " + encoded +
         " from " + beginState + " to " + endState + " failed, " +
-        "the node existed but was in the state " + existingData.getEventType());
+        "the node existed but was in the state " + existingData.getEventType()));
       return -1;
     }
 
@@ -611,22 +618,24 @@ public class ZKAssign {
       RegionTransitionData data = new RegionTransitionData(endState,
           region.getRegionName(), serverName);
       if(!ZKUtil.setData(zkw, node, data.getBytes(), stat.getVersion())) {
-        zkw.warn("Attempt to transition the unassigned node for " + encoded +
+        LOG.warn(zkw.prefix("Attempt to transition the " +
+        "unassigned node for " + encoded +
         " from " + beginState + " to " + endState + " failed, " +
         "the node existed and was in the expected state but then when " +
-        "setting data we got a version mismatch");
+        "setting data we got a version mismatch"));
         return -1;
       }
-      if(zkw.isDebugEnabled()) {
-        zkw.debug("Successfully transitioned node " + encoded +
-          " from " + beginState + " to " + endState);
+      if(LOG.isDebugEnabled()) {
+        LOG.debug(zkw.prefix("Successfully transitioned node " + encoded +
+          " from " + beginState + " to " + endState));
       }
       return stat.getVersion() + 1;
     } catch (KeeperException.NoNodeException nne) {
-      zkw.warn("Attempt to transition the unassigned node for " + encoded +
+      LOG.warn(zkw.prefix("Attempt to transition the " +
+        "unassigned node for " + encoded +
         " from " + beginState + " to " + endState + " failed, " +
         "the node existed and was in the expected state but then when " +
-        "setting data it no longer existed");
+        "setting data it no longer existed"));
       return -1;
     }
   }

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java Sat Sep 18 21:56:37
2010
@@ -80,11 +80,18 @@ public class ZKUtil {
   public static ZooKeeper connect(Configuration conf, String quorum,
       Watcher watcher)
   throws IOException {
+    return connect(conf, quorum, watcher, "");
+  }
+
+  public static ZooKeeper connect(Configuration conf, String quorum,
+      Watcher watcher, final String descriptor)
+  throws IOException {
     if(quorum == null) {
       throw new IOException("Unable to determine ZooKeeper quorum");
     }
     int timeout = conf.getInt("zookeeper.session.timeout", 60 * 1000);
-    LOG.debug("Opening connection to ZooKeeper with quorum (" + quorum + ")");
+    LOG.info(descriptor + " opening connection to ZooKeeper with quorum (" +
+      quorum + ")");
     return new ZooKeeper(quorum, timeout, watcher);
   }
 
@@ -186,14 +193,14 @@ public class ZKUtil {
   throws KeeperException {
     try {
       Stat s = zkw.getZooKeeper().exists(znode, zkw);
-      zkw.debug("Set watcher on existing znode " + znode);
+      LOG.info(zkw.prefix("Set watcher on existing znode " + znode));
       return s != null ? true : false;
     } catch (KeeperException e) {
-      zkw.warn("Unable to set watcher on znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
       zkw.keeperException(e);
       return false;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to set watcher on znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
       zkw.interruptedException(e);
       return false;
     }
@@ -216,11 +223,11 @@ public class ZKUtil {
       Stat s = zkw.getZooKeeper().exists(znode, null);
       return s != null ? s.getVersion() : -1;
     } catch (KeeperException e) {
-      zkw.warn("Unable to set watcher on znode (" + znode + ")", e);
+      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
       zkw.keeperException(e);
       return -1;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to set watcher on znode (" + znode + ")", e);
+      LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
       zkw.interruptedException(e);
       return -1;
     }
@@ -253,15 +260,15 @@ public class ZKUtil {
       List<String> children = zkw.getZooKeeper().getChildren(znode, zkw);
       return children;
     } catch(KeeperException.NoNodeException ke) {
-      zkw.debug("Unable to list children of znode " + znode + " " +
-          "because node does not exist (not an error)");
+      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+          "because node does not exist (not an error)"));
       return null;
     } catch (KeeperException e) {
-      zkw.warn("Unable to list children of znode " + znode + " ", e);
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
       zkw.keeperException(e);
       return null;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to list children of znode " + znode + " ", e);
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
       zkw.interruptedException(e);
       return null;
     }
@@ -397,15 +404,15 @@ public class ZKUtil {
     try {
       return !zkw.getZooKeeper().getChildren(znode, null).isEmpty();
     } catch(KeeperException.NoNodeException ke) {
-      zkw.debug("Unable to list children of znode " + znode + " " +
-      "because node does not exist (not an error)");
+      LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
+      "because node does not exist (not an error)"));
       return false;
     } catch (KeeperException e) {
-      zkw.warn("Unable to list children of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
       zkw.keeperException(e);
       return false;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to list children of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
       zkw.interruptedException(e);
       return false;
     }
@@ -430,7 +437,7 @@ public class ZKUtil {
       Stat stat = zkw.getZooKeeper().exists(znode, null);
       return stat == null ? 0 : stat.getNumChildren();
     } catch(KeeperException e) {
-      zkw.warn("Unable to get children of node " + znode);
+      LOG.warn(zkw.prefix("Unable to get children of node " + znode));
       zkw.keeperException(e);
     } catch(InterruptedException e) {
       zkw.interruptedException(e);
@@ -450,18 +457,19 @@ public class ZKUtil {
   throws KeeperException {
     try {
       byte [] data = zkw.getZooKeeper().getData(znode, null, null);
-      zkw.debug("Retrieved " + data.length + " bytes of data from znode " + znode);
+      LOG.debug(zkw.prefix("Retrieved " + data.length +
+        " bytes of data from znode " + znode));
       return data;
     } catch (KeeperException.NoNodeException e) {
-      zkw.debug("Unable to get data of znode " + znode + " " +
-          "because node does not exist (not an error)");
+      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+        "because node does not exist (not an error)"));
       return null;
     } catch (KeeperException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.keeperException(e);
       return null;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.interruptedException(e);
       return null;
     }
@@ -482,19 +490,19 @@ public class ZKUtil {
   throws KeeperException {
     try {
       byte [] data = zkw.getZooKeeper().getData(znode, zkw, null);
-      zkw.debug("Retrieved " + data.length + " bytes of data from znode " +
-          znode + " and set a watcher");
+      LOG.debug(zkw.prefix("Retrieved " + data.length +
+        " bytes of data from znode " + znode + " and set a watcher"));
       return data;
     } catch (KeeperException.NoNodeException e) {
-      zkw.debug("Unable to get data of znode " + znode + " " +
-          "because node does not exist (not an error)");
+      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+        "because node does not exist (not an error)"));
       return null;
     } catch (KeeperException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.keeperException(e);
       return null;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.interruptedException(e);
       return null;
     }
@@ -520,18 +528,19 @@ public class ZKUtil {
   throws KeeperException {
     try {
       byte [] data = zkw.getZooKeeper().getData(znode, zkw, stat);
-      zkw.debug("Retrieved " + data.length + " bytes of data from znode " + znode);
+      LOG.debug(zkw.prefix("Retrieved " + data.length +
+        " bytes of data from znode " + znode));
       return data;
     } catch (KeeperException.NoNodeException e) {
-      zkw.debug("Unable to get data of znode " + znode + " " +
-          "because node does not exist (not necessarily an error)");
+      LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
+          "because node does not exist (not necessarily an error)"));
       return null;
     } catch (KeeperException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.keeperException(e);
       return null;
     } catch (InterruptedException e) {
-      zkw.warn("Unable to get data of znode " + znode, e);
+      LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
       zkw.interruptedException(e);
       return null;
     }
@@ -558,7 +567,8 @@ public class ZKUtil {
       return null;
     }
     String addrString = Bytes.toString(data);
-    zkw.debug("Read server address from znode " + znode + ": " + addrString);
+    LOG.debug(zkw.prefix("Read server address from znode " + znode + ": " +
+      addrString));
     return new HServerAddress(addrString);
   }
 
@@ -705,6 +715,7 @@ public class ZKUtil {
       return false;
     } catch (InterruptedException e) {
       LOG.info("Interrupted", e);
+      Thread.currentThread().interrupt();
     }
     return true;
   }
@@ -996,4 +1007,4 @@ public class ZKUtil {
     socket.close();
     return res.toArray(new String[res.size()]);
   }
-}
\ No newline at end of file
+}

Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java?rev=998555&r1=998554&r2=998555&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java Sat
Sep 18 21:56:37 2010
@@ -42,14 +42,15 @@ import org.apache.zookeeper.ZooKeeper;
  * classes which need to be notified of ZooKeeper events must register with
  * the local instance of this watcher via {@link #registerListener}.
  *
- * <p>This class also holds and manages the connection to ZooKeeper.  Code to deal
- * with connection related events and exceptions are handled here.
+ * <p>This class also holds and manages the connection to ZooKeeper.  Code to
+ * deal with connection related events and exceptions are handled here.
  */
 public class ZooKeeperWatcher implements Watcher {
   private static final Log LOG = LogFactory.getLog(ZooKeeperWatcher.class);
 
-  // name of this watcher (for logging only)
-  private String name;
+  // Identifiier for this watcher (for logging only).  Its made of the prefix
+  // passed on construction and the zookeeper sessionid.
+  private String identifier;
 
   // zookeeper quorum
   private String quorum;
@@ -86,17 +87,19 @@ public class ZooKeeperWatcher implements
 
   /**
    * Instantiate a ZooKeeper connection and watcher.
-   * @param name name of this watcher, for logging/debug purposes only
+   * @param descriptor Descriptive string that is added to zookeeper sessionid
+   * and used as identifier for this instance.
    * @throws IOException
    */
-  public ZooKeeperWatcher(Configuration conf, String name,
+  public ZooKeeperWatcher(Configuration conf, String descriptor,
       Abortable abortable)
   throws IOException {
-    this.name = name;
     this.quorum = ZKConfig.getZKQuorumServersString(conf);
-    this.zooKeeper = ZKUtil.connect(conf, quorum, this);
+    this.zooKeeper = ZKUtil.connect(conf, quorum, this, descriptor);
+    // Identifier will get the sessionid appended later below down when we
+    // handle the syncconnect event.
+    this.identifier = descriptor;
     this.abortable = abortable;
-    info("Connected to ZooKeeper");
     setNodeNames(conf);
     try {
       // Create all the necessary "directories" of znodes
@@ -106,12 +109,26 @@ public class ZooKeeperWatcher implements
       ZKUtil.createAndFailSilent(this, rsZNode);
       ZKUtil.createAndFailSilent(this, tableZNode);
     } catch (KeeperException e) {
-      error("Unexpected KeeperException creating base node", e);
-      error("Message: " + e.getMessage());
+      LOG.error(prefix("Unexpected KeeperException creating base node"), e);
       throw new IOException(e);
     }
   }
 
+  @Override
+  public String toString() {
+    return this.identifier;
+  }
+
+  /**
+   * Adds this instance's identifier as a prefix to the passed <code>str</code>
+   * @param str String to amend.
+   * @return A new string with this instance's identifier as prefix: e.g.
+   * if passed 'hello world', the returned string could be
+   */
+  public String prefix(final String str) {
+    return this.toString() + " " + str;
+  }
+
   /**
    * Set the local variable node names using the specified configuration.
    */
@@ -164,10 +181,10 @@ public class ZooKeeperWatcher implements
    */
   @Override
   public void process(WatchedEvent event) {
-    LOG.debug("<" + name + "> Received ZooKeeper Event, " +
+    LOG.debug(prefix("Received ZooKeeper Event, " +
         "type=" + event.getType() + ", " +
         "state=" + event.getState() + ", " +
-        "path=" + event.getPath());
+        "path=" + event.getPath()));
 
     switch(event.getType()) {
 
@@ -220,20 +237,22 @@ public class ZooKeeperWatcher implements
    */
   private void connectionEvent(WatchedEvent event) {
     switch(event.getState()) {
-      // SyncConnected is normal, ignore
       case SyncConnected:
+        // Update our identifier.  Otherwise ignore.
+        this.identifier = this.identifier + "-0x" +
+          Long.toHexString(this.zooKeeper.getSessionId());
+        LOG.info(this.identifier + " connected");
         break;
 
       // Abort the server if Disconnected or Expired
       // TODO: Åny reason to handle these two differently?
       case Disconnected:
-        info("Received Disconnected from ZooKeeper, ignoring");
+        LOG.info(prefix("Received Disconnected from ZooKeeper, ignoring"));
         break;
       case Expired:
-        error("Received Expired from ZooKeeper, aborting server");
-        if(abortable != null) {
-          abortable.abort("Received Expired from ZooKeeper, aborting server", null);
-        }
+        String msg = prefix("Received Expired from ZooKeeper, aborting server");
+        LOG.error(msg);
+        if (abortable != null) abortable.abort(msg, null);
         break;
     }
   }
@@ -258,7 +277,7 @@ public class ZooKeeperWatcher implements
    */
   public void keeperException(KeeperException ke)
   throws KeeperException {
-    error("Received unexpected KeeperException, re-throwing exception", ke);
+    LOG.error(prefix("Received unexpected KeeperException, re-throwing exception"), ke);
     throw ke;
   }
 
@@ -274,74 +293,12 @@ public class ZooKeeperWatcher implements
    * @param ie
    */
   public void interruptedException(InterruptedException ie) {
-    debug("Received InterruptedException, doing nothing here", ie);
+    LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie);
+    // At least preserver interrupt.
+    Thread.currentThread().interrupt();
     // no-op
   }
 
-  // Logging methods
-
-  /**
-   * Exposed info logging method so our zookeeper output is named.
-   * @param string log line
-   */
-  public void info(String string) {
-    LOG.info("<" + name + "> " + string);
-  }
-
-  /**
-   * Exposed debug logging method so our zookeeper output is named.
-   * @param string log line
-   */
-  public void debug(String string) {
-    LOG.debug("<" + name + "> " + string);
-  }
-
-  /**
-   * Exposed debug logging method so our zookeeper output is named.
-   * @param string log line
-   */
-  public void debug(String string, Throwable t) {
-    LOG.debug("<" + name + "> " + string, t);
-  }
-
-  /**
-   * Exposed warn logging method so our zookeeper output is named.
-   * @param string log line
-   */
-  public void warn(String string) {
-    LOG.warn("<" + name + "> " + string);
-  }
-
-  /**
-   * Exposed warn logging method so our zookeeper output is named.
-   * @param string log line
-   * @param t exception
-   */
-  public void warn(String string, Throwable t) {
-    LOG.warn("<" + name + "> " + string, t);
-  }
-
-  /**
-   * Exposed error logging method so our zookeeper output is named.
-   * @param string log line
-   */
-  public void error(String string) {
-    LOG.error("<" + name + "> " + string);
-  }
-
-  /**
-   * Exposed error logging method so our zookeeper output is named.
-   * @param string log line
-   * @param t exception
-   */
-  public void error(String string, Throwable t) {
-    LOG.error("<" + name + "> " + string, t);
-  }
-
-  public boolean isDebugEnabled() {
-    return LOG.isDebugEnabled();
-  }
-
   /**
    * Close the connection to ZooKeeper.
    * @throws InterruptedException
@@ -355,4 +312,4 @@ public class ZooKeeperWatcher implements
     } catch (InterruptedException e) {
     }
   }
-}
\ No newline at end of file
+}



Mime
View raw message