summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authortison <wander4096@gmail.com>2023-02-27 14:28:59 +0800
committerGitHub <noreply@github.com>2023-02-27 14:28:59 +0800
commita9517d88605ed61fed8f3dfdc2f5a04dfbcee1f8 (patch)
tree85ec47158b0125d58e80257e0cf561b481057f2f
parentee8014adc748901b642e66c5734082d50b077a84 (diff)
downloadzookeeper-a9517d88605ed61fed8f3dfdc2f5a04dfbcee1f8.tar.gz
[ZOOKEEPER-4677] Cleanup code in the DataTree class (#1968)
Signed-off-by: tison <wander4096@gmail.com>
-rw-r--r--zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java273
1 files changed, 122 insertions, 151 deletions
diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
index d60528224..a6f605390 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java
+++ b/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;
}
}