ctubbsii commented on code in PR #5256:
URL: https://github.com/apache/accumulo/pull/5256#discussion_r1920278941


##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -1062,7 +1064,9 @@ public synchronized ZookeeperLockChecker 
getTServerLockChecker() {
       // because that client could be closed, and its ZooSession also closed
       // this needs to be fixed; TODO 
https://github.com/apache/accumulo/issues/2301
       var zk = 
info.getZooKeeperSupplier(ZookeeperLockChecker.class.getSimpleName()).get();
-      this.zkLockChecker = new ZookeeperLockChecker(new ZooCache(zk), 
getZooKeeperRoot());
+      String zkRoot = getZooKeeperRoot();
+      this.zkLockChecker =
+          new ZookeeperLockChecker(new ZooCache(zk, List.of(zkRoot + 
Constants.ZTSERVERS)), zkRoot);

Review Comment:
   This one works more the way I was thinking.



##########
core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java:
##########
@@ -152,55 +193,86 @@ public void process(WatchedEvent event) {
           break;
       }
 
-      externalWatcher.ifPresent(w -> w.accept(event));
+      externalWatchers.forEach(ew -> ew.accept(event));
     }
   }
 
   /**
-   * Creates a new cache without an external watcher.
+   * Creates a ZooCache instance that uses the supplied ZooSession for 
communicating with the
+   * instance's ZooKeeper servers. The ZooCache will create persistent 
watchers at the given
+   * pathsToWatch, if any, to be updated when changes are made in ZooKeeper 
for nodes at or below in
+   * the tree. If ZooCacheWatcher's are added via {@code addZooCacheWatcher}, 
then they will be
+   * notified when this object is notified of changes via the 
PersistentWatcher callback.
    *
-   * @param zk the ZooKeeper instance
-   * @throws NullPointerException if zk is {@code null}
+   * @param zk ZooSession for this instance
+   * @param pathsToWatch Paths in ZooKeeper to watch
    */
-  public ZooCache(ZooSession zk) {
-    this(zk, Optional.empty(), Duration.ofMinutes(3));
-  }
+  public ZooCache(ZooSession zk, List<String> pathsToWatch) {
+    this.zk = requireNonNull(zk);

Review Comment:
   This could just be:
   
   ```java
       this(zk, pathsToWatch, Ticker.systemTicker());
   ```



##########
core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java:
##########
@@ -235,7 +236,8 @@ public ClientContext(SingletonReservation reservation, 
ClientInfo info,
       return zk;
     });
 
-    this.zooCache = memoize(() -> new ZooCache(getZooSession()));
+    this.zooCache = memoize(() -> new ZooCache(getZooSession(),
+        
ZooCache.createPersistentWatcherPaths(ZooUtil.getRoot(getInstanceID()))));

Review Comment:
   When I previously suggested that these paths be provided to ZooCache, rather 
than built in, I was kind of thinking that the caller was responsible for 
determining what they were. So, basically, this static method should be in the 
context, rather than ZooCache. I'm also wondering if it might make sense to 
just pass a single path at `ZooUtil.getRoot(getInstanceID())` instead of 
several separately named ones.



##########
server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java:
##########
@@ -468,22 +456,19 @@ private void createTablePerm(String user, TableId table, 
Set<TablePermission> pe
    */
   private void createNamespacePerm(String user, NamespaceId namespace,
       Set<NamespacePermission> perms) throws KeeperException, 
InterruptedException {
-    synchronized (zooCache) {

Review Comment:
   Do you happen to know why all these previous callers were synchronizing here 
and why it's safe to remove it? It seems like it might have something to do 
with the fact that this code is updating a path that should be cached, so maybe 
we want to make sure it's cleared after the update is done?



##########
core/src/main/java/org/apache/accumulo/core/util/tables/TableZooHelper.java:
##########
@@ -173,7 +173,7 @@ public TableState getTableState(TableId tableId, boolean 
clearCachedState) {
     String statePath = context.getZooKeeperRoot() + Constants.ZTABLES + "/" + 
tableId.canonical()
         + Constants.ZTABLE_STATE;
     if (clearCachedState) {
-      context.getZooCache().clear(context.getZooKeeperRoot() + statePath);
+      context.getZooCache().clear(statePath);

Review Comment:
   It looks like this is a preexisting bug. I'm not sure how many branches this 
affects, but this should be backported to any affected branches that it might 
apply to.



##########
core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java:
##########
@@ -59,12 +65,33 @@ public interface ZooCacheWatcher extends 
Consumer<WatchedEvent> {}
 
   private static final Logger log = LoggerFactory.getLogger(ZooCache.class);
 
-  private final ZCacheWatcher watcher = new ZCacheWatcher();
-  private final Optional<ZooCacheWatcher> externalWatcher;
+  public static List<String> createPersistentWatcherPaths(String zkRoot) {
+    List<String> pathsToWatch = new ArrayList<>();
+    for (String path : Set.of(Constants.ZCOMPACTORS, Constants.ZDEADTSERVERS, 
Constants.ZGC_LOCK,
+        Constants.ZMANAGER_LOCK, Constants.ZMINI_LOCK, Constants.ZMONITOR_LOCK,
+        Constants.ZNAMESPACES, Constants.ZRECOVERY, Constants.ZSSERVERS, 
Constants.ZTABLES,
+        Constants.ZTSERVERS, Constants.ZUSERS, RootTable.ZROOT_TABLET)) {
+      pathsToWatch.add(zkRoot + path);
+    }
+    return pathsToWatch;

Review Comment:
   It doesn't seem worth taking a set and converting it to a list. Either the 
constructor should accept a set (or a collection), in which case, we can pass a 
set, or you can just do `List.of(...)` here. If you really want the List over 
the Set, for some reason, you should be able to do 
`pathsToWatch.addAll(Set.of(...))` instead of the for loop here.



##########
core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java:
##########
@@ -514,18 +583,15 @@ public boolean childrenCached(String zPath) {
    */
   public void clear(Predicate<String> pathPredicate) {
     Preconditions.checkState(!closed);
-
-    Predicate<String> pathPredicateToUse;
-    if (log.isTraceEnabled()) {
-      pathPredicateToUse = pathPredicate.and(path -> {
-        log.trace("removing {} from cache", path);
-        return true;
-      });
-    } else {
-      pathPredicateToUse = pathPredicate;
-    }
-    nodeCache.keySet().removeIf(pathPredicateToUse);
-    updateCount.incrementAndGet();
+    Predicate<String> pathPredicateWrapper = path -> {
+      boolean testResult = isWatchedPath(path) && pathPredicate.test(path);

Review Comment:
   I'm not sure you need the isWatchedPath(path) here. The paths that can be 
removed are ones that are in ZooCache already... and they can only get there if 
they previously passed the isWatchedPath check when we called getChildren or 
getData, so isWatchedPath(path) here will always return true. If for some 
reason it were to be false (I don't see how), then it seems like this check 
would prevent paths from being removed that should never have been added in the 
first place.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to