瀏覽代碼

[ZOOKEEPER-4677] Cleanup code in the DataTree class (#1968)

Signed-off-by: tison <wander4096@gmail.com>
tison 2 年之前
父節點
當前提交
a9517d8860
共有 1 個文件被更改,包括 122 次插入151 次删除
  1. 122 151
      zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java

+ 122 - 151
zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java

@@ -86,10 +86,9 @@ import org.slf4j.LoggerFactory;
 
 /**
  * This class maintains the tree data structure. It doesn't have any networking
- * or client connection code in it so that it can be tested in a stand alone
- * way.
- * <p>
- * The tree maintains two parallel data structures: a hashtable that maps from
+ * or client connection code in it so that it can be tested in a standalone way.
+ *
+ * <p>The tree maintains two parallel data structures: a hashtable that maps from
  * full paths to DataNodes and a tree of DataNodes. All accesses to a path is
  * through the hashtable. The tree is traversed only when serializing to disk.
  */
@@ -100,7 +99,7 @@ public class DataTree {
     private final RateLogger RATE_LOGGER = new RateLogger(LOG, 15 * 60 * 1000);
 
     /**
-     * This map provides a fast lookup to the datanodes. The tree is the
+     * This map provides a fast lookup to the data nodes. The tree is the
      * source of truth and is where all the locking occurs
      */
     private final NodeHashMap nodes;
@@ -118,7 +117,7 @@ public class DataTree {
     /** the zookeeper nodes that acts as the management and status node **/
     private static final String procZookeeper = Quotas.procZookeeper;
 
-    /** this will be the string thats stored as a child of root */
+    /** this will be the string that's stored as a child of root */
     private static final String procChildZookeeper = procZookeeper.substring(1);
 
     /**
@@ -127,7 +126,7 @@ public class DataTree {
      */
     private static final String quotaZookeeper = Quotas.quotaZookeeper;
 
-    /** this will be the string thats stored as a child of /zookeeper */
+    /** this will be the string that's stored as a child of /zookeeper */
     private static final String quotaChildZookeeper = quotaZookeeper.substring(procZookeeper.length() + 1);
 
     /**
@@ -136,7 +135,7 @@ public class DataTree {
      */
     private static final String configZookeeper = ZooDefs.CONFIG_NODE;
 
-    /** this will be the string thats stored as a child of /zookeeper */
+    /** this will be the string that's stored as a child of /zookeeper */
     private static final String configChildZookeeper = configZookeeper.substring(procZookeeper.length() + 1);
 
     /**
@@ -145,7 +144,7 @@ public class DataTree {
     private final PathTrie pTrie = new PathTrie();
 
     /**
-     * over-the-wire size of znode's stat. Counting the fields of Stat class
+     * over-the-wire size of znode stat. Counting the fields of Stat class
      */
     public static final int STAT_OVERHEAD_BYTES = (6 * 8) + (5 * 4);
 
@@ -186,21 +185,19 @@ public class DataTree {
     private final List<DigestWatcher> digestWatchers = new ArrayList<>();
 
     // The historical digests list.
-    private LinkedList<ZxidDigest> digestLog = new LinkedList<>();
+    private final LinkedList<ZxidDigest> digestLog = new LinkedList<>();
 
     private final DigestCalculator digestCalculator;
 
     @SuppressWarnings("unchecked")
     public Set<String> getEphemerals(long sessionId) {
-        HashSet<String> retv = ephemerals.get(sessionId);
-        if (retv == null) {
+        HashSet<String> ret = ephemerals.get(sessionId);
+        if (ret == null) {
             return new HashSet<>();
         }
-        Set<String> cloned = null;
-        synchronized (retv) {
-            cloned = (HashSet<String>) retv.clone();
+        synchronized (ret) {
+            return (HashSet<String>) ret.clone();
         }
-        return cloned;
     }
 
     public Set<String> getContainers() {
@@ -287,11 +284,11 @@ public class DataTree {
         this.digestCalculator = digestCalculator;
         nodes = new NodeHashMapImpl(digestCalculator);
 
-        /* Rather than fight it, let root have an alias */
+        // rather than fight it, let root have an alias
         nodes.put("", root);
         nodes.putWithoutDigest(rootZookeeper, root);
 
-        /** add the proc node and quota node */
+        // add the proc node and quota node
         root.addChild(procChildZookeeper);
         nodes.put(procZookeeper, procDataNode);
 
@@ -326,7 +323,7 @@ public class DataTree {
         try {
             // Reconfig node is access controlled by default (ZOOKEEPER-2014).
             setACL(configZookeeper, ZooDefs.Ids.READ_ACL_UNSAFE, -1);
-        } catch (KeeperException.NoNodeException e) {
+        } catch (NoNodeException e) {
             assert false : "There's no " + configZookeeper + " znode - this should never happen.";
         }
     }
@@ -382,21 +379,19 @@ public class DataTree {
      *            the diff to be added to the count
      */
     public void updateQuotaStat(String lastPrefix, long bytesDiff, int countDiff) {
-
         String statNodePath = Quotas.statPath(lastPrefix);
         DataNode statNode = nodes.get(statNodePath);
 
-        StatsTrack updatedStat;
         if (statNode == null) {
             // should not happen
             LOG.error("Missing node for stat {}", statNodePath);
             return;
         }
+
         synchronized (statNode) {
-            updatedStat = new StatsTrack(statNode.data);
+            StatsTrack updatedStat = new StatsTrack(statNode.data);
             updatedStat.setCount(updatedStat.getCount() + countDiff);
             updatedStat.setBytes(updatedStat.getBytes() + bytesDiff);
-
             statNode.data = updatedStat.getStatsBytes();
         }
     }
@@ -441,32 +436,32 @@ public class DataTree {
      * @throws NodeExistsException
      * @throws NoNodeException
      */
-    public void createNode(final String path, byte[] data, List<ACL> acl, long ephemeralOwner, int parentCVersion, long zxid, long time, Stat outputStat) throws KeeperException.NoNodeException, KeeperException.NodeExistsException {
+    public void createNode(final String path, byte[] data, List<ACL> acl, long ephemeralOwner, int parentCVersion, long zxid, long time, Stat outputStat) throws NoNodeException, NodeExistsException {
         int lastSlash = path.lastIndexOf('/');
         String parentName = path.substring(0, lastSlash);
         String childName = path.substring(lastSlash + 1);
         StatPersisted stat = createStat(zxid, time, ephemeralOwner);
         DataNode parent = nodes.get(parentName);
         if (parent == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         synchronized (parent) {
             // Add the ACL to ACL cache first, to avoid the ACL not being
             // created race condition during fuzzy snapshot sync.
             //
             // This is the simplest fix, which may add ACL reference count
-            // again if it's already counted in in the ACL map of fuzzy
+            // again if it's already counted in the ACL map of fuzzy
             // snapshot, which might also happen for deleteNode txn, but
             // at least it won't cause the ACL not exist issue.
             //
             // Later we can audit and delete all non-referenced ACLs from
             // ACL map when loading the snapshot/txns from disk, like what
             // we did for the global sessions.
-            Long longval = aclCache.convertAcls(acl);
+            Long acls = aclCache.convertAcls(acl);
 
             Set<String> children = parent.getChildren();
             if (children.contains(childName)) {
-                throw new KeeperException.NodeExistsException();
+                throw new NodeExistsException();
             }
 
             nodes.preChange(parentName, parent);
@@ -483,7 +478,7 @@ public class DataTree {
                 parent.stat.setCversion(parentCVersion);
                 parent.stat.setPzxid(zxid);
             }
-            DataNode child = new DataNode(data, longval, stat);
+            DataNode child = new DataNode(data, acls, stat);
             parent.addChild(childName);
             nodes.postChange(parentName, parent);
             nodeDataSize.addAndGet(getNodeSize(path, child.data));
@@ -494,11 +489,7 @@ public class DataTree {
             } else if (ephemeralType == EphemeralType.TTL) {
                 ttls.add(path);
             } else if (ephemeralOwner != 0) {
-                HashSet<String> list = ephemerals.get(ephemeralOwner);
-                if (list == null) {
-                    list = new HashSet<>();
-                    ephemerals.put(ephemeralOwner, list);
-                }
+                HashSet<String> list = ephemerals.computeIfAbsent(ephemeralOwner, k -> new HashSet<>());
                 synchronized (list) {
                     list.add(path);
                 }
@@ -509,7 +500,7 @@ public class DataTree {
         }
         // now check if its one of the zookeeper node child
         if (parentName.startsWith(quotaZookeeper)) {
-            // now check if its the limit node
+            // now check if it's the limit node
             if (Quotas.limitNode.equals(childName)) {
                 // this is the limit node
                 // get the parent and add it to the trie
@@ -538,9 +529,9 @@ public class DataTree {
      *            the path to of the node to be deleted
      * @param zxid
      *            the current zxid
-     * @throws KeeperException.NoNodeException
+     * @throws NoNodeException
      */
-    public void deleteNode(String path, long zxid) throws KeeperException.NoNodeException {
+    public void deleteNode(String path, long zxid) throws NoNodeException {
         int lastSlash = path.lastIndexOf('/');
         String parentName = path.substring(0, lastSlash);
         String childName = path.substring(lastSlash + 1);
@@ -550,14 +541,14 @@ public class DataTree {
         // for no such child
         DataNode parent = nodes.get(parentName);
         if (parent == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         synchronized (parent) {
             nodes.preChange(parentName, parent);
             parent.removeChild(childName);
             // Only update pzxid when the zxid is larger than the current pzxid,
-            // otherwise we might override some higher pzxid set by a create
-            // Txn, which could cause the cversion and pzxid inconsistent
+            // otherwise we might override some higher pzxid set by a CreateTxn,
+            // which could cause the cversion and pzxid inconsistent
             if (zxid > parent.stat.getPzxid()) {
                 parent.stat.setPzxid(zxid);
             }
@@ -566,7 +557,7 @@ public class DataTree {
 
         DataNode node = nodes.get(path);
         if (node == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         nodes.remove(path);
         synchronized (node) {
@@ -578,14 +569,14 @@ public class DataTree {
         // only need to sync on containers and ttls, will update it in a
         // separate patch.
         synchronized (parent) {
-            long eowner = node.stat.getEphemeralOwner();
-            EphemeralType ephemeralType = EphemeralType.get(eowner);
+            long owner = node.stat.getEphemeralOwner();
+            EphemeralType ephemeralType = EphemeralType.get(owner);
             if (ephemeralType == EphemeralType.CONTAINER) {
                 containers.remove(path);
             } else if (ephemeralType == EphemeralType.TTL) {
                 ttls.remove(path);
-            } else if (eowner != 0) {
-                Set<String> nodes = ephemerals.get(eowner);
+            } else if (owner != 0) {
+                Set<String> nodes = ephemerals.get(owner);
                 if (nodes != null) {
                     synchronized (nodes) {
                         nodes.remove(path);
@@ -604,7 +595,7 @@ public class DataTree {
         String lastPrefix = getMaxPrefixWithQuota(path);
         if (lastPrefix != null) {
             // ok we have some match and need to update
-            long bytes = 0;
+            long bytes;
             synchronized (node) {
                 bytes = (node.data == null ? 0 : -(node.data.length));
             }
@@ -629,15 +620,15 @@ public class DataTree {
         childWatches.triggerWatch("".equals(parentName) ? "/" : parentName, EventType.NodeChildrenChanged);
     }
 
-    public Stat setData(String path, byte[] data, int version, long zxid, long time) throws KeeperException.NoNodeException {
+    public Stat setData(String path, byte[] data, int version, long zxid, long time) throws NoNodeException {
         Stat s = new Stat();
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
-        byte[] lastdata = null;
+        byte[] lastData;
         synchronized (n) {
-            lastdata = n.data;
+            lastData = n.data;
             nodes.preChange(path, n);
             n.data = data;
             n.stat.setMtime(time);
@@ -649,13 +640,13 @@ public class DataTree {
 
         // first do a quota check if the path is in a quota subtree.
         String lastPrefix = getMaxPrefixWithQuota(path);
-        long bytesDiff = (data == null ? 0 : data.length) - (lastdata == null ? 0 : lastdata.length);
+        long bytesDiff = (data == null ? 0 : data.length) - (lastData == null ? 0 : lastData.length);
         // now update if the path is in a quota subtree.
         long dataBytes = data == null ? 0 : data.length;
         if (lastPrefix != null) {
             updateQuotaStat(lastPrefix, bytesDiff, 0);
         }
-        nodeDataSize.addAndGet(getNodeSize(path, data) - getNodeSize(path, lastdata));
+        nodeDataSize.addAndGet(getNodeSize(path, data) - getNodeSize(path, lastData));
 
         updateWriteStat(path, dataBytes);
         dataWatches.triggerWatch(path, EventType.NodeDataChanged);
@@ -687,12 +678,12 @@ public class DataTree {
         childWatches.addWatch(basePath, watcher, watcherMode);
     }
 
-    public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException {
+    public byte[] getData(String path, Stat stat, Watcher watcher) throws NoNodeException {
         DataNode n = nodes.get(path);
-        byte[] data = null;
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
+        byte[] data;
         synchronized (n) {
             n.copyStat(stat);
             if (watcher != null) {
@@ -704,15 +695,15 @@ public class DataTree {
         return data;
     }
 
-    public Stat statNode(String path, Watcher watcher) throws KeeperException.NoNodeException {
-        Stat stat = new Stat();
-        DataNode n = nodes.get(path);
+    public Stat statNode(String path, Watcher watcher) throws NoNodeException {
         if (watcher != null) {
             dataWatches.addWatch(path, watcher);
         }
+        DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
+        Stat stat = new Stat();
         synchronized (n) {
             n.copyStat(stat);
         }
@@ -720,10 +711,10 @@ public class DataTree {
         return stat;
     }
 
-    public List<String> getChildren(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException {
+    public List<String> getChildren(String path, Stat stat, Watcher watcher) throws NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         List<String> children;
         synchronized (n) {
@@ -747,7 +738,7 @@ public class DataTree {
     }
 
     public int getAllChildrenNumber(String path) {
-        //cull out these two keys:"", "/"
+        // cull out these two keys:"", "/"
         if ("/".equals(path)) {
             return nodes.size() - 2;
         }
@@ -755,13 +746,13 @@ public class DataTree {
         return (int) nodes.entrySet().parallelStream().filter(entry -> entry.getKey().startsWith(path + "/")).count();
     }
 
-    public Stat setACL(String path, List<ACL> acl, int version) throws KeeperException.NoNodeException {
-        Stat stat = new Stat();
+    public Stat setACL(String path, List<ACL> acl, int version) throws NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         synchronized (n) {
+            Stat stat = new Stat();
             aclCache.removeUsage(n.acl);
             nodes.preChange(path, n);
             n.stat.setAversion(version);
@@ -772,10 +763,10 @@ public class DataTree {
         }
     }
 
-    public List<ACL> getACL(String path, Stat stat) throws KeeperException.NoNodeException {
+    public List<ACL> getACL(String path, Stat stat) throws NoNodeException {
         DataNode n = nodes.get(path);
         if (n == null) {
-            throw new KeeperException.NoNodeException();
+            throw new NoNodeException();
         }
         synchronized (n) {
             if (stat != null) {
@@ -829,7 +820,7 @@ public class DataTree {
         }
 
         /**
-         * See equals() to find the rational for how this hashcode is generated.
+         * See equals() to find the rationale for how this hashcode is generated.
          *
          * @see ProcessTxnResult#equals(Object)
          * @see java.lang.Object#hashCode()
@@ -977,7 +968,7 @@ public class DataTree {
                 boolean post_failed = false;
                 for (Txn subtxn : txns) {
                     ByteBuffer bb = ByteBuffer.wrap(subtxn.getData());
-                    Record record = null;
+                    Record record;
                     switch (subtxn.getType()) {
                     case OpCode.create:
                         record = new CreateTxn();
@@ -1005,7 +996,8 @@ public class DataTree {
                     default:
                         throw new IOException("Invalid type of op: " + subtxn.getType());
                     }
-                    assert (record != null);
+
+                    assert record != null;
 
                     ByteBufferInputStream.byteBuffer2Record(bb, record);
 
@@ -1049,7 +1041,7 @@ public class DataTree {
          * But there is a tricky scenario that if the child is deleted due
          * to session close and re-created in a different global session
          * after that the parent is serialized, then when replay the txn
-         * because the node is belonging to a different session, replay the
+         * because the node belongs to a different session, replay the
          * closeSession txn won't delete it anymore, and we'll get NODEEXISTS
          * error when replay the createNode txn. In this case, we need to
          * update the cversion and pzxid to the new value.
@@ -1064,7 +1056,7 @@ public class DataTree {
             CreateTxn cTxn = (CreateTxn) txn;
             try {
                 setCversionPzxid(parentName, cTxn.getParentCVersion(), header.getZxid());
-            } catch (KeeperException.NoNodeException e) {
+            } catch (NoNodeException e) {
                 LOG.error("Failed to set parent cversion for: {}", parentName, e);
                 rc.err = e.code().intValue();
             }
@@ -1118,7 +1110,7 @@ public class DataTree {
     }
 
     void killSession(long session, long zxid) {
-        // the list is already removed from the ephemerals
+        // The list is already removed from the ephemerals,
         // so we do not have to worry about synchronizing on
         // the list. This is only called from FinalRequestProcessor
         // so there is no need for synchronization. The list is not
@@ -1182,33 +1174,28 @@ public class DataTree {
     }
 
     /**
-     * a encapsultaing class for return value
+     * An encapsulation class for return value
      */
     private static class Counts {
-
         long bytes;
         int count;
-
     }
 
     /**
      * this method gets the count of nodes and the bytes under a subtree
      *
-     * @param path
-     *            the path to be used
-     * @param counts
-     *            the int count
+     * @param path the path to be used
+     * @param counts the int count
      */
     private void getCounts(String path, Counts counts) {
         DataNode node = getNode(path);
         if (node == null) {
             return;
         }
-        String[] children = null;
-        int len = 0;
+        String[] children;
+        int len;
         synchronized (node) {
-            Set<String> childs = node.getChildren();
-            children = childs.toArray(new String[childs.size()]);
+            children = node.getChildren().toArray(new String[0]);
             len = (node.data == null ? 0 : node.data.length);
         }
         // add itself
@@ -1222,15 +1209,14 @@ public class DataTree {
     /**
      * update the quota for the given path
      *
-     * @param path
-     *            the path to be used
+     * @param path the path to be used
      */
     private void updateQuotaForPath(String path) {
         Counts c = new Counts();
         getCounts(path, c);
-        StatsTrack strack = new StatsTrack();
-        strack.setBytes(c.bytes);
-        strack.setCount(c.count);
+        StatsTrack statsTrack = new StatsTrack();
+        statsTrack.setBytes(c.bytes);
+        statsTrack.setCount(c.count);
         String statPath = Quotas.statPath(path);
         DataNode node = getNode(statPath);
         // it should exist
@@ -1240,7 +1226,7 @@ public class DataTree {
         }
         synchronized (node) {
             nodes.preChange(statPath, node);
-            node.data = strack.getStatsBytes();
+            node.data = statsTrack.getStatsBytes();
             nodes.postChange(statPath, node);
         }
     }
@@ -1248,19 +1234,18 @@ public class DataTree {
     /**
      * this method traverses the quota path and update the path trie and sets
      *
-     * @param path
+     * @param path the path to be used
      */
     private void traverseNode(String path) {
         DataNode node = getNode(path);
-        String[] children = null;
+        String[] children;
         synchronized (node) {
-            Set<String> childs = node.getChildren();
-            children = childs.toArray(new String[childs.size()]);
+            children = node.getChildren().toArray(new String[0]);
         }
         if (children.length == 0) {
             // this node does not have a child
             // is the leaf node
-            // check if its the leaf node
+            // check if it's the leaf node
             String endString = "/" + Quotas.limitNode;
             if (path.endsWith(endString)) {
                 // ok this is the limit node
@@ -1293,10 +1278,8 @@ public class DataTree {
      * this method uses a stringbuilder to create a new path for children. This
      * is faster than string appends ( str1 + str2).
      *
-     * @param oa
-     *            OutputArchive to write to.
-     * @param path
-     *            a string builder.
+     * @param oa OutputArchive to write to.
+     * @param path a string builder.
      * @throws IOException
      */
     void serializeNode(OutputArchive oa, StringBuilder path) throws IOException {
@@ -1305,7 +1288,7 @@ public class DataTree {
         if (node == null) {
             return;
         }
-        String[] children = null;
+        String[] children;
         DataNode nodeCopy;
         synchronized (node) {
             StatPersisted statCopy = new StatPersisted();
@@ -1313,23 +1296,20 @@ public class DataTree {
             //we do not need to make a copy of node.data because the contents
             //are never changed
             nodeCopy = new DataNode(node.data, node.acl, statCopy);
-            Set<String> childs = node.getChildren();
-            children = childs.toArray(new String[childs.size()]);
+            children = node.getChildren().toArray(new String[0]);
         }
         serializeNodeData(oa, pathString, nodeCopy);
         path.append('/');
         int off = path.length();
         for (String child : children) {
-            // since this is single buffer being resused
-            // we need
-            // to truncate the previous bytes of string.
+            // Since this is single buffer being reused, we need to truncate the previous bytes of string.
             path.delete(off, Integer.MAX_VALUE);
             path.append(child);
             serializeNode(oa, path);
         }
     }
 
-    // visiable for test
+    // visible for test
     public void serializeNodeData(OutputArchive oa, String path, DataNode node) throws IOException {
         oa.writeString(path, "path");
         oa.writeRecord(node, "node");
@@ -1373,25 +1353,18 @@ public class DataTree {
                 String parentPath = path.substring(0, lastSlash);
                 DataNode parent = nodes.get(parentPath);
                 if (parent == null) {
-                    throw new IOException("Invalid Datatree, unable to find "
-                                          + "parent "
-                                          + parentPath
-                                          + " of path "
-                                          + path);
+                    throw new IOException(
+                            "Invalid Datatree, unable to find parent " + parentPath + " of path " + path);
                 }
                 parent.addChild(path.substring(lastSlash + 1));
-                long eowner = node.stat.getEphemeralOwner();
-                EphemeralType ephemeralType = EphemeralType.get(eowner);
+                long owner = node.stat.getEphemeralOwner();
+                EphemeralType ephemeralType = EphemeralType.get(owner);
                 if (ephemeralType == EphemeralType.CONTAINER) {
                     containers.add(path);
                 } else if (ephemeralType == EphemeralType.TTL) {
                     ttls.add(path);
-                } else if (eowner != 0) {
-                    HashSet<String> list = ephemerals.get(eowner);
-                    if (list == null) {
-                        list = new HashSet<>();
-                        ephemerals.put(eowner, list);
-                    }
+                } else if (owner != 0) {
+                    HashSet<String> list = ephemerals.computeIfAbsent(owner, k -> new HashSet<>());
                     list.add(path);
                 }
             }
@@ -1403,8 +1376,7 @@ public class DataTree {
 
         nodeDataSize.set(approximateDataSize());
 
-        // we are done with deserializing the
-        // the datatree
+        // we are done with deserializing the datatree
         // update the quotas - create path trie
         // and also update the stat nodes
         setupQuota();
@@ -1414,19 +1386,19 @@ public class DataTree {
 
     /**
      * Summary of the watches on the datatree.
-     * @param pwriter the output to write to
+     * @param writer the output to write to
      */
-    public synchronized void dumpWatchesSummary(PrintWriter pwriter) {
-        pwriter.print(dataWatches.toString());
+    public synchronized void dumpWatchesSummary(PrintWriter writer) {
+        writer.print(dataWatches.toString());
     }
 
     /**
      * Write a text dump of all the watches on the datatree.
      * Warning, this is expensive, use sparingly!
-     * @param pwriter the output to write to
+     * @param writer the output to write to
      */
-    public synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) {
-        dataWatches.dumpWatches(pwriter, byPath);
+    public synchronized void dumpWatches(PrintWriter writer, boolean byPath) {
+        dataWatches.dumpWatches(writer, byPath);
     }
 
     /**
@@ -1461,18 +1433,18 @@ public class DataTree {
 
     /**
      * Write a text dump of all the ephemerals in the datatree.
-     * @param pwriter the output to write to
+     * @param writer the output to write to
      */
-    public void dumpEphemerals(PrintWriter pwriter) {
-        pwriter.println("Sessions with Ephemerals (" + ephemerals.keySet().size() + "):");
+    public void dumpEphemerals(PrintWriter writer) {
+        writer.println("Sessions with Ephemerals (" + ephemerals.keySet().size() + "):");
         for (Entry<Long, HashSet<String>> entry : ephemerals.entrySet()) {
-            pwriter.print("0x" + Long.toHexString(entry.getKey()));
-            pwriter.println(":");
+            writer.print("0x" + Long.toHexString(entry.getKey()));
+            writer.println(":");
             Set<String> tmp = entry.getValue();
             if (tmp != null) {
                 synchronized (tmp) {
                     for (String path : tmp) {
-                        pwriter.println("\t" + path);
+                        writer.println("\t" + path);
                     }
                 }
             }
@@ -1508,7 +1480,6 @@ public class DataTree {
                            List<String> persistentWatches, List<String> persistentRecursiveWatches, Watcher watcher) {
         for (String path : dataWatches) {
             DataNode node = getNode(path);
-            WatchedEvent e = null;
             if (node == null) {
                 watcher.process(new WatchedEvent(EventType.NodeDeleted, KeeperState.SyncConnected, path));
             } else if (node.stat.getMzxid() > relativeZxid) {
@@ -1558,16 +1529,16 @@ public class DataTree {
      *     Value to be assigned to Cversion
      * @param zxid
      *     Value to be assigned to Pzxid
-     * @throws KeeperException.NoNodeException
+     * @throws NoNodeException
      *     If znode not found.
      **/
-    public void setCversionPzxid(String path, int newCversion, long zxid) throws KeeperException.NoNodeException {
+    public void setCversionPzxid(String path, int newCversion, long zxid) throws NoNodeException {
         if (path.endsWith("/")) {
             path = path.substring(0, path.length() - 1);
         }
         DataNode node = nodes.get(path);
         if (node == null) {
-            throw new KeeperException.NoNodeException(path);
+            throw new NoNodeException(path);
         }
         synchronized (node) {
             if (newCversion == -1) {
@@ -1664,7 +1635,7 @@ public class DataTree {
 
     /**
      * Serializing the digest to snapshot, this is done after the data tree
-     * is being serialized, so when we replay the txns and it hits this zxid
+     * is being serialized, so when we replay the txns, and it hits this zxid
      * we know we should be in a non-fuzzy state, and have the same digest.
      *
      * @param oa the output stream to write to
@@ -1721,12 +1692,12 @@ public class DataTree {
             // epoch + 0, which is not mapping to any txn, and it uses
             // this to take snapshot, which is possible if we don't
             // clean database before switching to LOOKING. In this case
-            // the currentZxidDigest will be the zxid of last epoch and
+            // the currentZxidDigest will be the zxid of last epoch, and
             // it's smaller than the zxid of the snapshot file.
             //
             // It's safe to reset the targetZxidDigest to null and start
             // to compare digest when replaying the first txn, since it's
-            // a non fuzzy snapshot.
+            // a non-fuzzy snapshot.
             if (digestFromLoadedSnapshot != null && digestFromLoadedSnapshot.zxid < startZxidOfSnapshot) {
                 LOG.info("The zxid of snapshot digest 0x{} is smaller "
                         + "than the known snapshot highest zxid, the snapshot "
@@ -1782,7 +1753,7 @@ public class DataTree {
 
     /**
      * Compares the actual tree's digest with that in the snapshot.
-     * Resets digestFromLoadedSnapshot after comparision.
+     * Resets digestFromLoadedSnapshot after comparison.
      *
      * @param zxid zxid
      */
@@ -1801,8 +1772,9 @@ public class DataTree {
             }
             digestFromLoadedSnapshot = null;
         } else if (digestFromLoadedSnapshot.zxid != 0 && zxid > digestFromLoadedSnapshot.zxid) {
-            RATE_LOGGER.rateLimitLog("The txn 0x{} of snapshot digest does not "
-                    + "exist.", Long.toHexString(digestFromLoadedSnapshot.zxid));
+            RATE_LOGGER.rateLimitLog(
+                    "The txn 0x{} of snapshot digest does not exist.",
+                    Long.toHexString(digestFromLoadedSnapshot.zxid));
         }
     }
 
@@ -1814,8 +1786,7 @@ public class DataTree {
      * @param txn    transaction
      * @param digest transaction digest
      *
-     * @return false if digest in the txn doesn't match what we have now in
-     *               the data tree
+     * @return false if digest in the txn doesn't match what we have now in the data tree
      */
     public boolean compareDigest(TxnHeader header, Record txn, TxnDigest digest) {
         long zxid = header.getZxid();
@@ -1829,8 +1800,7 @@ public class DataTree {
         }
         // do not compare digest if there is digest version change
         if (digestCalculator.getDigestVersion() != digest.getVersion()) {
-            RATE_LOGGER.rateLimitLog("Digest version not the same on zxid.",
-                    String.valueOf(zxid));
+            RATE_LOGGER.rateLimitLog("Digest version not the same on zxid.", String.valueOf(zxid));
             return true;
         }
 
@@ -1840,16 +1810,17 @@ public class DataTree {
             reportDigestMismatch(zxid);
             LOG.debug("Digest in log: {}, actual tree: {}", logDigest, actualDigest);
             if (firstMismatchTxn) {
-                LOG.error("First digest mismatch on txn: {}, {}, "
-                        + "expected digest is {}, actual digest is {}, ",
+                LOG.error(
+                        "First digest mismatch on txn: {}, {}, expected digest is {}, actual digest is {}, ",
                         header, txn, digest, actualDigest);
                 firstMismatchTxn = false;
             }
             return false;
         } else {
             RATE_LOGGER.flush();
-            LOG.debug("Digests are matching for Zxid: {}, Digest in log "
-                    + "and actual tree: {}", Long.toHexString(zxid), logDigest);
+            LOG.debug(
+                    "Digests are matching for Zxid: {}, Digest in log and actual tree: {}",
+                    Long.toHexString(zxid), logDigest);
             return true;
         }
     }