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


##########
test/src/main/java/org/apache/accumulo/test/zookeeper/ZooCacheIT.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.accumulo.test.zookeeper;
+
+import static 
org.apache.accumulo.harness.AccumuloITBase.ZOOKEEPER_TESTING_SERVER;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.File;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.accumulo.core.fate.zookeeper.ZooCache;
+import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
+import org.apache.accumulo.test.util.Wait;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+@Tag(ZOOKEEPER_TESTING_SERVER)
+public class ZooCacheIT {
+
+  private ZooKeeperTestingServer szk = null;
+  private ZooReaderWriter zk = null;
+
+  @TempDir
+  private File tempDir;
+
+  @BeforeEach
+  public void setup() throws Exception {
+    szk = new ZooKeeperTestingServer(tempDir);
+    zk = szk.getZooReaderWriter();
+  }
+
+  @AfterEach
+  public void teardown() throws Exception {
+    szk.close();
+  }
+
+  @Test
+  public void testGetChildren() throws Exception {
+    ZooCache zooCache = new ZooCache(zk, null);
+
+    zk.mkdirs("/test2");
+    zk.mkdirs("/test3");
+    zk.mkdirs("/test3/c1");
+    zk.mkdirs("/test3/c2");

Review Comment:
   mkdirs on the parent before the child is redundant. The purpose of the 
method is to create each node in the hierarchy. The purpose of the method is 
analogous to `mkdir -p`
   
   ```suggestion
       zk.mkdirs("/test2");
       zk.mkdirs("/test3/c1");
       zk.mkdirs("/test3/c2");
   ```



##########
core/src/main/java/org/apache/accumulo/core/fate/zookeeper/ZooCache.java:
##########
@@ -316,43 +277,46 @@ public List<String> getChildren(final String zPath) {
       public List<String> run() throws KeeperException, InterruptedException {
 
         var zcNode = nodeCache.get(zPath);
-        if (zcNode != null && zcNode.childrenSet) {
+        if (zcNode != null && zcNode.cachedChildren()) {
           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
-            if (zcn != null && zcn.childrenSet) {
-              return zcn;
-            }
+        zcNode = nodeCache.compute(zPath, (zp, zcn) -> {
+          // recheck the children now that lock is held on key
+          if (zcn != null && zcn.cachedChildren()) {
+            return zcn;
+          }
 
-            try {
-              final ZooKeeper zooKeeper = getZooKeeper();
-              List<String> children;
-              children = zooKeeper.getChildren(zPath, watcher);
-              if (children != null) {
-                children = List.copyOf(children);
-              }
-              return new ZcNode(zcn, children);
-            } catch (KeeperException e) {
-              throw new ZcException(e);
-            } catch (InterruptedException e) {
-              throw new ZcInterruptedException(e);
+          try {
+            final ZooKeeper zooKeeper = getZooKeeper();
+            // Register a watcher on the node to monitor creation/deletion 
events for the node. It
+            // is possible that an event from this watch could trigger prior 
to calling getChildren.
+            // That is ok because the compute() call on the map has a lock and 
processing the event
+            // will block until compute() returns. After compute() returns the 
event processing
+            // would clear the map entry.
+            Stat stat = zooKeeper.exists(zPath, watcher);

Review Comment:
   I'm wondering if this is going to cause an increase in the number of watches 
that we cannot clean up (or have a hard time doing so). If the node doesn't 
exist, and never comes into existence, the watch will be registered forever. 
This was one of the problems that motivated moving the system/table/namespace 
configs to a single node, because we were watching for exists on property keys 
for properties that were never stored in ZK, or for tables that had already 
been deleted, and so the properties would never come into being.
   
   So, caching non-existence seems risky... prone to the same sort of memory 
leak, where the ZK client memory keeps growing and growing due to an unbounded 
number of watches on nodes that don't exist and may never exist.



-- 
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