You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by sy...@apache.org on 2022/07/26 07:39:56 UTC

[zookeeper] branch master updated: ZOOKEEPER-4565: Refine chroot strip to accommodate /zookeeper/ subtree

This is an automated email from the ASF dual-hosted git repository.

symat pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/zookeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 2cd0c2345 ZOOKEEPER-4565: Refine chroot strip to accommodate /zookeeper/ subtree
2cd0c2345 is described below

commit 2cd0c23454071faf0a16c58edb2414591ae5c5c3
Author: Kezhu Wang <ke...@gmail.com>
AuthorDate: Tue Jul 26 09:39:20 2022 +0200

    ZOOKEEPER-4565: Refine chroot strip to accommodate /zookeeper/ subtree
    
    ```java
    if (serverPath.length() > chrootPath.length()) {
        event.setPath(serverPath.substring(chrootPath.length()));
    }
    ```
    
    Currently, chroot strip code listed above could result in illegal path
    (aka. path not start with "/"). This will disconnect zookeeper client
    due to `StringIndexOutOfBoundsException` from `PathParentIterator.next`
    in event handling.
    
    Author: Kezhu Wang <ke...@gmail.com>
    
    Reviewers: Enrico Olivelli <eo...@apache.org>, Mate Szalay-Beko <sy...@apache.org>
    
    Closes #1899 from kezhuw/ZOOKEEPER-4565-refine-chroot-strip
---
 .../main/java/org/apache/zookeeper/ClientCnxn.java | 22 ++++++++++++++--------
 .../java/org/apache/zookeeper/test/ChrootTest.java | 22 ++++++++++++++++++++++
 2 files changed, 36 insertions(+), 8 deletions(-)

diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
index 837c12d5a..96ff8eb08 100644
--- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
+++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java
@@ -855,6 +855,18 @@ public class ClientCnxn {
         private boolean isFirstConnect = true;
         private volatile ZooKeeperSaslClient zooKeeperSaslClient;
 
+        private String stripChroot(String serverPath) {
+            if (serverPath.startsWith(chrootPath)) {
+                if (serverPath.length() == chrootPath.length()) {
+                    return "/";
+                }
+                return serverPath.substring(chrootPath.length());
+            } else if (serverPath.startsWith(ZooDefs.ZOOKEEPER_NODE_SUBTREE)) {
+                return serverPath;
+            }
+            LOG.warn("Got server path {} which is not descendant of chroot path {}.", serverPath, chrootPath);
+            return serverPath;
+        }
 
         void readResponse(ByteBuffer incomingBuffer) throws IOException {
             ByteBufferInputStream bbis = new ByteBufferInputStream(incomingBuffer);
@@ -886,14 +898,8 @@ public class ClientCnxn {
                 // convert from a server path to a client path
                 if (chrootPath != null) {
                     String serverPath = event.getPath();
-                    if (serverPath.compareTo(chrootPath) == 0) {
-                        event.setPath("/");
-                    } else if (serverPath.length() > chrootPath.length()) {
-                        event.setPath(serverPath.substring(chrootPath.length()));
-                     } else {
-                         LOG.warn("Got server path {} which is too short for chroot path {}.",
-                             event.getPath(), chrootPath);
-                     }
+                    String clientPath = stripChroot(serverPath);
+                    event.setPath(clientPath);
                 }
 
                 WatchedEvent we = new WatchedEvent(event);
diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
index 0b156156d..fca48c751 100644
--- a/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
+++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/ChrootTest.java
@@ -25,12 +25,15 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.ZooKeeper;
 import org.junit.jupiter.api.Test;
@@ -60,6 +63,25 @@ public class ChrootTest extends ClientBase {
 
     }
 
+    @Test
+    public void testChrootWithZooKeeperPathWatcher() throws Exception {
+        ZooKeeper zk1 = createClient(hostPort + "/chroot");
+        BlockingQueue<WatchedEvent> events = new LinkedBlockingQueue<>();
+        byte[] config = zk1.getConfig(events::add, null);
+
+        ZooKeeper zk2 = createClient();
+        zk2.addAuthInfo("digest", "super:test".getBytes());
+        zk2.setData(ZooDefs.CONFIG_NODE, config, -1);
+
+        waitFor("config watcher receive no event", () -> !events.isEmpty(), 10);
+
+        WatchedEvent event = events.poll();
+        assertNotNull(event);
+        assertEquals(Watcher.Event.KeeperState.SyncConnected, event.getState());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, event.getType());
+        assertEquals(ZooDefs.CONFIG_NODE, event.getPath());
+    }
+
     @Test
     public void testChrootSynchronous() throws IOException, InterruptedException, KeeperException {
         ZooKeeper zk1 = createClient();