diff --git a/conf/log4j.properties b/conf/log4j.properties index bf97dbaccd2..ec341410a9c 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -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 diff --git a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index be145460e03..fbdec0b375c 100644 --- a/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -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); } diff --git a/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java b/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java index 6bf49b7a2ec..5a359f475c1 100644 --- a/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java +++ b/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java @@ -184,7 +184,7 @@ class ActiveMasterManager extends ZooKeeperListener { 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 diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index da536dda3bb..f2e4e7cbae3 100644 --- a/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -1172,8 +1172,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler, // 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()); } /* diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java index c88a38fac4c..a26ff65eefc 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKAssign.java @@ -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; * */ 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; } } diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 1595d234b97..30ed9b30574 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -79,12 +79,19 @@ 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 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 +} diff --git a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index aaa88a0c41b..a8d08282fcd 100644 --- a/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -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}. * - *

This class also holds and manages the connection to ZooKeeper. Code to deal - * with connection related events and exceptions are handled here. + *

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 Watcher { /** * 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 Watcher { 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 str + * @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 Watcher { */ @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 Watcher { */ 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 Watcher { */ 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 Watcher { * @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 Watcher { } catch (InterruptedException e) { } } -} \ No newline at end of file +}