This is an automated email from the ASF dual-hosted git repository.
ddanielr pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push:
new cd07dc6036 improves trace logging in zoocache (#5133)
cd07dc6036 is described below
commit cd07dc60360b3c14009380b0b0d96f2197757b26
Author: Keith Turner <[email protected]>
AuthorDate: Wed Dec 4 13:34:02 2024 -0500
improves trace logging in zoocache (#5133)
* improves trace logging in zoocache
Noticed a lot of tsever lock checking was blocking on zoocache in
jstacks. Was not sure why this was happening. Added more detailed
logging to zoocache inorder to know what paths are missing in the cache
and when a path is removed from the cache.
* Update
core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
Co-authored-by: Daniel Roberts <[email protected]>
---------
Co-authored-by: Daniel Roberts <[email protected]>
---
.../accumulo/core/fate/zookeeper/ZooCache.java | 45 ++++++++++++++++------
1 file changed, 34 insertions(+), 11 deletions(-)
diff --git
a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
index e0f00994d5..86b869fa15 100644
--- a/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
+++ b/core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java
@@ -53,6 +53,9 @@ public class ZooCache {
private final ZCacheWatcher watcher = new ZCacheWatcher();
private final Watcher externalWatcher;
+ private static final AtomicLong nextCacheId = new AtomicLong(0);
+ private final String cacheId = "ZC" + nextCacheId.incrementAndGet();
+
private static class ZcNode {
final byte[] data;
final ZcStat stat;
@@ -150,7 +153,7 @@ public class ZooCache {
@Override
public void process(WatchedEvent event) {
if (log.isTraceEnabled()) {
- log.trace("{}", event);
+ log.trace("{}: {}", cacheId, event);
}
switch (event.getType()) {
@@ -167,26 +170,26 @@ public class ZooCache {
// These are ignored, because they are generated by
SingletonManager closing
// ZooKeepers for ZooSession, and SingletonManager is already
responsible for clearing
// caches via the registered ZooCacheFactory singleton
- log.trace("ZooKeeper connection closed, ignoring; {}", event);
+ log.trace("{} ZooKeeper connection closed, ignoring; {}",
cacheId, event);
break;
case Disconnected:
- log.trace("ZooKeeper connection disconnected, clearing cache;
{}", event);
+ log.trace("{} ZooKeeper connection disconnected, clearing cache;
{}", cacheId, event);
clear();
break;
case SyncConnected:
- log.trace("ZooKeeper connection established, ignoring; {}",
event);
+ log.trace("{} ZooKeeper connection established, ignoring; {}",
cacheId, event);
break;
case Expired:
- log.trace("ZooKeeper connection expired, clearing cache; {}",
event);
+ log.trace("{} ZooKeeper connection expired, clearing cache; {}",
cacheId, event);
clear();
break;
default:
- log.warn("Unhandled {}", event);
+ log.warn("{} Unhandled {}", cacheId, event);
break;
}
break;
default:
- log.warn("Unhandled {}", event);
+ log.warn("{} Unhandled {}", cacheId, event);
break;
}
@@ -206,6 +209,7 @@ public class ZooCache {
this.zReader = reader;
nodeCache = new ConcurrentHashMap<>();
this.externalWatcher = watcher;
+ log.trace("{} created new cache", cacheId, new Exception());
}
private abstract static class ZooRunnable<T> {
@@ -316,6 +320,8 @@ public class ZooCache {
return zcNode.getChildren();
}
+ log.trace("{} {} was not in children cache, looking up in zookeeper",
cacheId, zPath);
+
try {
zcNode = nodeCache.compute(zPath, (zp, zcn) -> {
// recheck the children now that lock is held on key
@@ -387,6 +393,8 @@ public class ZooCache {
return zcNode.getData();
}
+ log.trace("{} {} was not in data cache, looking up in zookeeper",
cacheId, zPath);
+
zcNode = nodeCache.compute(zPath, (zp, zcn) -> {
// recheck the now that lock is held on key, it may be present now.
Could have been
// computed while waiting for lock.
@@ -408,7 +416,7 @@ public class ZooCache {
ZcStat zstat = null;
if (stat == null) {
if (log.isTraceEnabled()) {
- log.trace("zookeeper did not contain {}", zPath);
+ log.trace("{} zookeeper did not contain {}", cacheId, zPath);
}
} else {
try {
@@ -420,7 +428,7 @@ public class ZooCache {
throw new ZcInterruptedException(e);
}
if (log.isTraceEnabled()) {
- log.trace("zookeeper contained {} {}", zPath,
+ log.trace("{} zookeeper contained {} {}", cacheId, zPath,
(data == null ? null : new String(data, UTF_8)));
}
}
@@ -460,6 +468,7 @@ public class ZooCache {
private void remove(String zPath) {
nodeCache.remove(zPath);
+ log.trace("{} removed {} from cache", cacheId, zPath);
updateCount.incrementAndGet();
}
@@ -470,6 +479,7 @@ public class ZooCache {
Preconditions.checkState(!closed);
nodeCache.clear();
updateCount.incrementAndGet();
+ log.trace("{} cleared all from cache", cacheId);
}
public void close() {
@@ -514,7 +524,20 @@ public class ZooCache {
*/
public void clear(Predicate<String> pathPredicate) {
Preconditions.checkState(!closed);
- nodeCache.keySet().removeIf(pathPredicate);
+
+ Predicate<String> pathPredicateToUse;
+ if (log.isTraceEnabled()) {
+ pathPredicateToUse = path -> {
+ boolean testResult = pathPredicate.test(path);
+ if (testResult) {
+ log.trace("{} removing {} from cache", cacheId, path);
+ }
+ return testResult;
+ };
+ } else {
+ pathPredicateToUse = pathPredicate;
+ }
+ nodeCache.keySet().removeIf(pathPredicateToUse);
updateCount.incrementAndGet();
}
@@ -536,7 +559,7 @@ public class ZooCache {
byte[] lockData = get(path + "/" + lockNode);
if (log.isTraceEnabled()) {
- log.trace("Data from lockNode {} is {}", lockNode, new String(lockData,
UTF_8));
+ log.trace("{} Data from lockNode {} is {}", cacheId, lockNode, new
String(lockData, UTF_8));
}
if (lockData == null) {
lockData = new byte[0];