You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@zookeeper.apache.org by GitBox <gi...@apache.org> on 2019/07/10 11:27:52 UTC

[GitHub] [zookeeper] anmolnar commented on a change in pull request #888: ZOOKEEPER-3347: Improve PathTrie Consistency

anmolnar commented on a change in pull request #888: ZOOKEEPER-3347: Improve PathTrie Consistency
URL: https://github.com/apache/zookeeper/pull/888#discussion_r302015551
 
 

 ##########
 File path: zookeeper-server/src/main/java/org/apache/zookeeper/common/PathTrie.java
 ##########
 @@ -40,254 +46,275 @@
  *      (bc)
  *   cf/
  *   (cf)
- */    
+ */
 public class PathTrie {
-    /**
-     * the logger for this class
-     */
+
+    /** Logger for this class */
     private static final Logger LOG = LoggerFactory.getLogger(PathTrie.class);
-    
-    /**
-     * the root node of PathTrie
-     */
-    private final TrieNode rootNode ;
-    
+
+    /** Root node of PathTrie */
+    private final TrieNode rootNode;
+
+    private final ReadWriteLock lock = new ReentrantReadWriteLock(true);
+
+    private final Lock readLock = lock.readLock();
+
+    private final Lock writeLock = lock.writeLock();
+
     static class TrieNode {
-        boolean property = false;
+        final String value;
         final Map<String, TrieNode> children;
-        TrieNode parent = null;
+        boolean property;
+        TrieNode parent;
+
         /**
-         * create a trienode with parent
-         * as parameter
-         * @param parent the parent of this trienode
+         * Create a trie node with parent as parameter.
+         *
+         * @param parent the parent of this node
+         * @param value the value stored in this node
          */
-        private TrieNode(TrieNode parent) {
-            children = new HashMap<String, TrieNode>();
+        private TrieNode(TrieNode parent, String value) {
+            this.value = value;
             this.parent = parent;
+            this.property = false;
+            this.children = new HashMap<>(4);
         }
-        
+
         /**
-         * get the parent of this node
+         * Get the parent of this node.
+         *
          * @return the parent node
          */
         TrieNode getParent() {
             return this.parent;
         }
-        
+
         /**
-         * set the parent of this node
+         * set the parent of this node.
+         *
          * @param parent the parent to set to
          */
         void setParent(TrieNode parent) {
             this.parent = parent;
         }
-        
+
         /**
-         * a property that is set 
-         * for a node - making it 
-         * special.
+         * A property that is set for a node - making it special.
          */
         void setProperty(boolean prop) {
             this.property = prop;
         }
-        
-        /** the property of this
-         * node 
-         * @return the property for this
-         * node
+
+        /**
+         * The property of this node.
+         *
+         * @return the property for this node
          */
-        boolean getProperty() {
+        boolean hasProperty() {
             return this.property;
         }
+
+        /**
+         * The value stored in this node.
+         *
+         * @return the value stored in this node
+         */
+        public String getValue() {
+          return this.value;
+        }
+
         /**
-         * add a child to the existing node
+         * Add a child to the existing node.
+         *
          * @param childName the string name of the child
          * @param node the node that is the child
          */
         void addChild(String childName, TrieNode node) {
-            synchronized(children) {
-                if (children.containsKey(childName)) {
-                    return;
-                }
-                children.put(childName, node);
-            }
+            this.children.putIfAbsent(childName, node);
         }
-     
+
         /**
-         * delete child from this node
-         * @param childName the string name of the child to 
-         * be deleted
+         * Delete child from this node.
+         *
+         * @param childName the name of the child to be deleted
          */
         void deleteChild(String childName) {
-            synchronized(children) {
-                if (!children.containsKey(childName)) {
-                    return;
-                }
-                TrieNode childNode = children.get(childName);
-                // this is the only child node.
-                if (childNode.getChildren().length == 1) { 
+            this.children.computeIfPresent(childName, (key, childNode) -> {
+                // Node no longer has an external property associated
+                childNode.setProperty(false);
+
+                // Delete it if it has no children (is a leaf node)
+                if (childNode.isLeafNode()) {
                     childNode.setParent(null);
-                    children.remove(childName);
+                    return null;
                 }
-                else {
-                    // their are more child nodes
-                    // so just reset property.
-                    childNode.setProperty(false);
-                }
-            }
+
+                return childNode;
+            });
         }
-        
+
         /**
-         * return the child of a node mapping
-         * to the input childname
+         * Return the child of a node mapping to the input child name.
+         *
          * @param childName the name of the child
          * @return the child of a node
          */
         TrieNode getChild(String childName) {
-            synchronized(children) {
-               if (!children.containsKey(childName)) {
-                   return null;
-               }
-               else {
-                   return children.get(childName);
-               }
-            }
+            return this.children.get(childName);
         }
 
         /**
-         * get the list of children of this 
-         * trienode.
-         * @return the string list of its children
+         * Get the list of children of this trienode.
+         *
+         * @return A collection containing the node's children
          */
-        String[] getChildren() {
-           synchronized(children) {
-               return children.keySet().toArray(new String[0]);
-           }
+        Collection<String> getChildren() {
+            return children.keySet();
         }
-        
+
         /**
-         * get the string representation
-         * for this node
+         * Determine if this node is a leaf (has no children).
+         *
+         * @return true if this node is a lead node; otherwise false
          */
+        boolean isLeafNode() {
+            return children.isEmpty();
+        }
+
+        @Override
         public String toString() {
-            StringBuilder sb = new StringBuilder();
-            sb.append("Children of trienode: ");
-            synchronized(children) {
-                for (String str: children.keySet()) {
-                    sb.append(" " + str);
-                }
-            }
-            return sb.toString();
+          return "TrieNode [name=" + value + ", property=" + property
+              + ", children=" + children.keySet() + "]";
         }
     }
-    
+
     /**
-     * construct a new PathTrie with
-     * a root node of /
+     * Construct a new PathTrie with a root node.
      */
     public PathTrie() {
-        this.rootNode = new TrieNode(null);
+        this.rootNode = new TrieNode(null, "");
     }
-    
+
     /**
-     * add a path to the path trie 
-     * @param path
+     * Add a path to the path trie.
+     *
+     * @param path the path to add to the trie
      */
-    public void addPath(String path) {
-        if (path == null) {
-            return;
-        }
-        String[] pathComponents = path.split("/");
-        TrieNode parent = rootNode;
-        String part = null;
-        if (pathComponents.length <= 1) {
-            throw new IllegalArgumentException("Invalid path " + path);
+    public void addPath(final String path) {
+        Objects.requireNonNull(path, "Path cannot be null");
+
+        final String[] pathComponents = StringUtils.split(path, '/');
+        if (pathComponents.length == 0) {
+            throw new IllegalArgumentException("Invalid path: " + path);
         }
-        for (int i=1; i<pathComponents.length; i++) {
-            part = pathComponents[i];
-            if (parent.getChild(part) == null) {
-                parent.addChild(part, new TrieNode(parent));
+
+        writeLock.lock();
+        try {
+            TrieNode parent = rootNode;
+            for (final String part : pathComponents) {
+                TrieNode child = parent.getChild(part);
+                if (child == null) {
+                    child = new TrieNode(parent, part);
+                    parent.addChild(part, child);
 
 Review comment:
   Let's take `addPath()` as an example.
   
   In the previous version of `TrieNode` the class was already consistent, because methods were protected with synchronized and both getChild() and addChild() validated if the child was already added to `children`.
   For instance, if a thread gets interrupted between getChild() and addChild() modifying the trie by adding the same child on another thread, the addChild() will be a no-op.
   
   In your version `TrieNode` is no longer thread-safe, because you removed synchronized from all of its methods, but the entire tree is guarded by Read/Write lock in `PathTrie` class.
   
   This seems to me heavier, because if multiple threads are manipulating different parts of the Trie, locking the entire object is not needed, because those threads are not conflicting. The previous version seems to me more accurate in certain workloads.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services