浏览代码

ZOOKEEPER-4799: Refactor ACL check in 'addWatch' command

As of today, it is impossible to diagnose which watch events are dropped
because of ACLs.  Let's centralize, systematize, and log the checks at
the 'process()' site in the Netty and NIO connections.

(These 'process()' methods contain some duplicated code, and should also
be refactored at some point.  This series does not change them.)

This patch also adds a substantial number of tests in order to avoid
unexpected regressions.

Co-authored-by: Patrick Hunt <phunt@apache.org>
Co-authored-by: Damien Diederen <ddiederen@apache.org>
Andor Molnar 1 年之前
父节点
当前提交
d12aba5992

+ 3 - 3
zookeeper-it/src/main/java/org/apache/zookeeper/server/watch/WatchBench.java

@@ -191,7 +191,7 @@ public class WatchBench {
     @Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
     @Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
     public void testTriggerConcentrateWatch(InvocationState state) throws Exception {
     public void testTriggerConcentrateWatch(InvocationState state) throws Exception {
         for (String path : state.paths) {
         for (String path : state.paths) {
-            state.watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID);
+            state.watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID, null);
         }
         }
     }
     }
 
 
@@ -225,7 +225,7 @@ public class WatchBench {
 
 
             // clear all the watches
             // clear all the watches
             for (String path : paths) {
             for (String path : paths) {
-                watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID);
+                watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID, null);
             }
             }
         }
         }
     }
     }
@@ -294,7 +294,7 @@ public class WatchBench {
     @Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
     @Measurement(iterations = 3, time = 10, timeUnit = TimeUnit.SECONDS)
     public void testTriggerSparseWatch(TriggerSparseWatchState state) throws Exception {
     public void testTriggerSparseWatch(TriggerSparseWatchState state) throws Exception {
         for (String path : state.paths) {
         for (String path : state.paths) {
-            state.watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID);
+            state.watchManager.triggerWatch(path, event, WatchedEvent.NO_ZXID, null);
         }
         }
     }
     }
 }
 }

+ 17 - 6
zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java

@@ -445,7 +445,10 @@ public class DataTree {
         if (parent == null) {
         if (parent == null) {
             throw new NoNodeException();
             throw new NoNodeException();
         }
         }
+        List<ACL> parentAcl;
         synchronized (parent) {
         synchronized (parent) {
+            parentAcl = getACL(parent);
+
             // Add the ACL to ACL cache first, to avoid the ACL not being
             // Add the ACL to ACL cache first, to avoid the ACL not being
             // created race condition during fuzzy snapshot sync.
             // created race condition during fuzzy snapshot sync.
             //
             //
@@ -518,8 +521,9 @@ public class DataTree {
             updateQuotaStat(lastPrefix, bytes, 1);
             updateQuotaStat(lastPrefix, bytes, 1);
         }
         }
         updateWriteStat(path, bytes);
         updateWriteStat(path, bytes);
-        dataWatches.triggerWatch(path, Event.EventType.NodeCreated, zxid);
-        childWatches.triggerWatch(parentName.equals("") ? "/" : parentName, Event.EventType.NodeChildrenChanged, zxid);
+        dataWatches.triggerWatch(path, Event.EventType.NodeCreated, zxid, acl);
+        childWatches.triggerWatch(parentName.equals("") ? "/" : parentName,
+            Event.EventType.NodeChildrenChanged, zxid, parentAcl);
     }
     }
 
 
     /**
     /**
@@ -559,8 +563,10 @@ public class DataTree {
         if (node == null) {
         if (node == null) {
             throw new NoNodeException();
             throw new NoNodeException();
         }
         }
+        List<ACL> acl;
         nodes.remove(path);
         nodes.remove(path);
         synchronized (node) {
         synchronized (node) {
+            acl = getACL(node);
             aclCache.removeUsage(node.acl);
             aclCache.removeUsage(node.acl);
             nodeDataSize.addAndGet(-getNodeSize(path, node.data));
             nodeDataSize.addAndGet(-getNodeSize(path, node.data));
         }
         }
@@ -568,7 +574,9 @@ public class DataTree {
         // Synchronized to sync the containers and ttls change, probably
         // Synchronized to sync the containers and ttls change, probably
         // only need to sync on containers and ttls, will update it in a
         // only need to sync on containers and ttls, will update it in a
         // separate patch.
         // separate patch.
+        List<ACL> parentAcl;
         synchronized (parent) {
         synchronized (parent) {
+            parentAcl = getACL(parent);
             long owner = node.stat.getEphemeralOwner();
             long owner = node.stat.getEphemeralOwner();
             EphemeralType ephemeralType = EphemeralType.get(owner);
             EphemeralType ephemeralType = EphemeralType.get(owner);
             if (ephemeralType == EphemeralType.CONTAINER) {
             if (ephemeralType == EphemeralType.CONTAINER) {
@@ -615,9 +623,10 @@ public class DataTree {
                 "childWatches.triggerWatch " + parentName);
                 "childWatches.triggerWatch " + parentName);
         }
         }
 
 
-        WatcherOrBitSet processed = dataWatches.triggerWatch(path, EventType.NodeDeleted, zxid);
-        childWatches.triggerWatch(path, EventType.NodeDeleted, zxid, processed);
-        childWatches.triggerWatch("".equals(parentName) ? "/" : parentName, EventType.NodeChildrenChanged, zxid);
+        WatcherOrBitSet processed = dataWatches.triggerWatch(path, EventType.NodeDeleted, zxid, acl);
+        childWatches.triggerWatch(path, EventType.NodeDeleted, zxid, acl, processed);
+        childWatches.triggerWatch("".equals(parentName) ? "/" : parentName,
+            EventType.NodeChildrenChanged, zxid, parentAcl);
     }
     }
 
 
     public Stat setData(String path, byte[] data, int version, long zxid, long time) throws NoNodeException {
     public Stat setData(String path, byte[] data, int version, long zxid, long time) throws NoNodeException {
@@ -626,8 +635,10 @@ public class DataTree {
         if (n == null) {
         if (n == null) {
             throw new NoNodeException();
             throw new NoNodeException();
         }
         }
+        List<ACL> acl;
         byte[] lastData;
         byte[] lastData;
         synchronized (n) {
         synchronized (n) {
+            acl = getACL(n);
             lastData = n.data;
             lastData = n.data;
             nodes.preChange(path, n);
             nodes.preChange(path, n);
             n.data = data;
             n.data = data;
@@ -649,7 +660,7 @@ public class DataTree {
         nodeDataSize.addAndGet(getNodeSize(path, data) - getNodeSize(path, lastData));
         nodeDataSize.addAndGet(getNodeSize(path, data) - getNodeSize(path, lastData));
 
 
         updateWriteStat(path, dataBytes);
         updateWriteStat(path, dataBytes);
-        dataWatches.triggerWatch(path, EventType.NodeDataChanged, zxid);
+        dataWatches.triggerWatch(path, EventType.NodeDataChanged, zxid, acl);
         return s;
         return s;
     }
     }
 
 

+ 3 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/DumbWatcher.java

@@ -22,8 +22,10 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.security.cert.Certificate;
 import java.security.cert.Certificate;
+import java.util.List;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.ReplyHeader;
 
 
@@ -51,7 +53,7 @@ public class DumbWatcher extends ServerCnxn {
     }
     }
 
 
     @Override
     @Override
-    public void process(WatchedEvent event) {
+    public void process(WatchedEvent event, List<ACL> znodeAcl) {
         mostRecentEventType = event.getType();
         mostRecentEventType = event.getType();
         mostRecentZxid = event.getZxid();
         mostRecentZxid = event.getZxid();
         mostRecentPath = event.getPath();
         mostRecentPath = event.getPath();

+ 15 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/NIOServerCnxn.java

@@ -30,14 +30,17 @@ import java.nio.channels.CancelledKeyException;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SocketChannel;
 import java.nio.channels.SocketChannel;
 import java.security.cert.Certificate;
 import java.security.cert.Certificate;
+import java.util.List;
 import java.util.Queue;
 import java.util.Queue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.ClientCnxn;
 import org.apache.zookeeper.ClientCnxn;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.ConnectRequest;
 import org.apache.zookeeper.proto.ConnectRequest;
@@ -704,7 +707,18 @@ public class NIOServerCnxn extends ServerCnxn {
      * @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
      * @see org.apache.zookeeper.server.ServerCnxnIface#process(org.apache.zookeeper.proto.WatcherEvent)
      */
      */
     @Override
     @Override
-    public void process(WatchedEvent event) {
+    public void process(WatchedEvent event, List<ACL> znodeAcl) {
+        try {
+            zkServer.checkACL(this, znodeAcl, ZooDefs.Perms.READ, getAuthInfo(), event.getPath(), null);
+        } catch (KeeperException.NoAuthException e) {
+            if (LOG.isTraceEnabled()) {
+                ZooTrace.logTraceMessage(
+                    LOG,
+                    ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+                    "Not delivering event " + event + " to 0x" + Long.toHexString(this.sessionId) + " (filtered by ACL)");
+            }
+            return;
+        }
         ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, event.getZxid(), 0);
         ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, event.getZxid(), 0);
         if (LOG.isTraceEnabled()) {
         if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(
             ZooTrace.logTraceMessage(

+ 16 - 1
zookeeper-server/src/main/java/org/apache/zookeeper/server/NettyServerCnxn.java

@@ -38,11 +38,15 @@ import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
 import java.nio.channels.SelectionKey;
 import java.security.cert.Certificate;
 import java.security.cert.Certificate;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.BinaryInputArchive;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.ClientCnxn;
 import org.apache.zookeeper.ClientCnxn;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.ConnectRequest;
 import org.apache.zookeeper.proto.ConnectRequest;
@@ -161,7 +165,18 @@ public class NettyServerCnxn extends ServerCnxn {
     }
     }
 
 
     @Override
     @Override
-    public void process(WatchedEvent event) {
+    public void process(WatchedEvent event, List<ACL> znodeAcl) {
+        try {
+            zkServer.checkACL(this, znodeAcl, ZooDefs.Perms.READ, getAuthInfo(), event.getPath(), null);
+        } catch (KeeperException.NoAuthException e) {
+            if (LOG.isTraceEnabled()) {
+                ZooTrace.logTraceMessage(
+                    LOG,
+                    ZooTrace.EVENT_DELIVERY_TRACE_MASK,
+                    "Not delivering event " + event + " to 0x" + Long.toHexString(this.sessionId) + " (filtered by ACL)");
+            }
+            return;
+        }
         ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, event.getZxid(), 0);
         ReplyHeader h = new ReplyHeader(ClientCnxn.NOTIFICATION_XID, event.getZxid(), 0);
         if (LOG.isTraceEnabled()) {
         if (LOG.isTraceEnabled()) {
             ZooTrace.logTraceMessage(
             ZooTrace.logTraceMessage(

+ 7 - 3
zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java

@@ -37,9 +37,9 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.Quotas;
 import org.apache.zookeeper.Quotas;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.ZooDefs.OpCode;
 import org.apache.zookeeper.compat.ProtocolManager;
 import org.apache.zookeeper.compat.ProtocolManager;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.metrics.Counter;
 import org.apache.zookeeper.metrics.Counter;
@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  * Interface to a Server connection - represents a connection from a client
  * Interface to a Server connection - represents a connection from a client
  * to the server.
  * to the server.
  */
  */
-public abstract class ServerCnxn implements Stats, Watcher {
+public abstract class ServerCnxn implements Stats, ServerWatcher {
 
 
     // This is just an arbitrary object to represent requests issued by
     // This is just an arbitrary object to represent requests issued by
     // (aka owned by) this class
     // (aka owned by) this class
@@ -254,7 +254,11 @@ public abstract class ServerCnxn implements Stats, Watcher {
     /* notify the client the session is closing and close/cleanup socket */
     /* notify the client the session is closing and close/cleanup socket */
     public abstract void sendCloseSession();
     public abstract void sendCloseSession();
 
 
-    public abstract void process(WatchedEvent event);
+    public void process(WatchedEvent event) {
+        process(event, null);
+    }
+
+    public abstract void process(WatchedEvent event, List<ACL> znodeAcl);
 
 
     public abstract long getSessionId();
     public abstract long getSessionId();
 
 

+ 29 - 0
zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerWatcher.java

@@ -0,0 +1,29 @@
+/*
+ * 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
+ *
+ *     http://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.zookeeper.server;
+
+import java.util.List;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.ACL;
+
+public interface ServerWatcher extends Watcher {
+
+  void process(WatchedEvent event, List<ACL> znodeAcl);
+
+}

+ 5 - 2
zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java

@@ -19,9 +19,11 @@
 package org.apache.zookeeper.server.watch;
 package org.apache.zookeeper.server.watch;
 
 
 import java.io.PrintWriter;
 import java.io.PrintWriter;
+import java.util.List;
 import javax.annotation.Nullable;
 import javax.annotation.Nullable;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.data.ACL;
 
 
 public interface IWatchManager {
 public interface IWatchManager {
 
 
@@ -114,10 +116,11 @@ public interface IWatchManager {
      * @param path znode path
      * @param path znode path
      * @param type the watch event type
      * @param type the watch event type
      * @param zxid the zxid for the corresponding change that triggered this event
      * @param zxid the zxid for the corresponding change that triggered this event
+     * @param acl ACL of the znode in path
      *
      *
      * @return the watchers have been notified
      * @return the watchers have been notified
      */
      */
-    WatcherOrBitSet triggerWatch(String path, EventType type, long zxid);
+    WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl);
 
 
     /**
     /**
      * Distribute the watch event for the given path, but ignore those
      * Distribute the watch event for the given path, but ignore those
@@ -130,7 +133,7 @@ public interface IWatchManager {
      *
      *
      * @return the watchers have been notified
      * @return the watchers have been notified
      */
      */
-    WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, WatcherOrBitSet suppress);
+    WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl, WatcherOrBitSet suppress);
 
 
     /**
     /**
      * Get the size of watchers.
      * Get the size of watchers.

+ 11 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java

@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Set;
@@ -30,8 +31,10 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerMetrics;
 import org.apache.zookeeper.server.ServerMetrics;
+import org.apache.zookeeper.server.ServerWatcher;
 import org.apache.zookeeper.server.ZooTrace;
 import org.apache.zookeeper.server.ZooTrace;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -129,12 +132,12 @@ public class WatchManager implements IWatchManager {
     }
     }
 
 
     @Override
     @Override
-    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid) {
-        return triggerWatch(path, type, zxid, null);
+    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl) {
+        return triggerWatch(path, type, zxid, acl, null);
     }
     }
 
 
     @Override
     @Override
-    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, WatcherOrBitSet supress) {
+    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl, WatcherOrBitSet supress) {
         WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path, zxid);
         WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path, zxid);
         Set<Watcher> watchers = new HashSet<>();
         Set<Watcher> watchers = new HashSet<>();
         synchronized (this) {
         synchronized (this) {
@@ -182,7 +185,11 @@ public class WatchManager implements IWatchManager {
             if (supress != null && supress.contains(w)) {
             if (supress != null && supress.contains(w)) {
                 continue;
                 continue;
             }
             }
-            w.process(e);
+            if (w instanceof ServerWatcher) {
+                ((ServerWatcher) w).process(e, acl);
+            } else {
+                w.process(e);
+            }
         }
         }
 
 
         switch (type) {
         switch (type) {

+ 11 - 4
zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManagerOptimized.java

@@ -22,6 +22,7 @@ import java.io.PrintWriter;
 import java.util.BitSet;
 import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.Set;
@@ -31,8 +32,10 @@ import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerCnxn;
 import org.apache.zookeeper.server.ServerMetrics;
 import org.apache.zookeeper.server.ServerMetrics;
+import org.apache.zookeeper.server.ServerWatcher;
 import org.apache.zookeeper.server.util.BitHashSet;
 import org.apache.zookeeper.server.util.BitHashSet;
 import org.apache.zookeeper.server.util.BitMap;
 import org.apache.zookeeper.server.util.BitMap;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -202,12 +205,12 @@ public class WatchManagerOptimized implements IWatchManager, IDeadWatcherListene
     }
     }
 
 
     @Override
     @Override
-    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid) {
-        return triggerWatch(path, type, zxid, null);
+    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl) {
+        return triggerWatch(path, type, zxid, acl, null);
     }
     }
 
 
     @Override
     @Override
-    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, WatcherOrBitSet suppress) {
+    public WatcherOrBitSet triggerWatch(String path, EventType type, long zxid, List<ACL> acl, WatcherOrBitSet suppress) {
         WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path, zxid);
         WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path, zxid);
 
 
         BitHashSet watchers = remove(path);
         BitHashSet watchers = remove(path);
@@ -232,7 +235,11 @@ public class WatchManagerOptimized implements IWatchManager, IDeadWatcherListene
                     continue;
                     continue;
                 }
                 }
 
 
-                w.process(e);
+                if (w instanceof ServerWatcher) {
+                    ((ServerWatcher) w).process(e, acl);
+                } else {
+                    w.process(e);
+                }
                 triggeredWatches++;
                 triggeredWatches++;
             }
             }
         }
         }

+ 3 - 1
zookeeper-server/src/test/java/org/apache/zookeeper/server/MockServerCnxn.java

@@ -22,8 +22,10 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.security.cert.Certificate;
 import java.security.cert.Certificate;
+import java.util.List;
 import org.apache.jute.Record;
 import org.apache.jute.Record;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.data.Stat;
 import org.apache.zookeeper.proto.ReplyHeader;
 import org.apache.zookeeper.proto.ReplyHeader;
 
 
@@ -56,7 +58,7 @@ public class MockServerCnxn extends ServerCnxn {
     }
     }
 
 
     @Override
     @Override
-    public void process(WatchedEvent event) {
+    public void process(WatchedEvent event, List<ACL> acl) {
     }
     }
 
 
     @Override
     @Override

+ 7 - 7
zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java

@@ -133,7 +133,7 @@ public class WatchManagerTest extends ZKTestCase {
         public void run() {
         public void run() {
             while (!stopped) {
             while (!stopped) {
                 String path = PATH_PREFIX + r.nextInt(paths);
                 String path = PATH_PREFIX + r.nextInt(paths);
-                WatcherOrBitSet s = manager.triggerWatch(path, EventType.NodeDeleted, -1);
+                WatcherOrBitSet s = manager.triggerWatch(path, EventType.NodeDeleted, -1, null);
                 if (s != null) {
                 if (s != null) {
                     triggeredCount.addAndGet(s.size());
                     triggeredCount.addAndGet(s.size());
                 }
                 }
@@ -756,7 +756,7 @@ public class WatchManagerTest extends ZKTestCase {
         //path2 is watched by watcher1
         //path2 is watched by watcher1
         manager.addWatch(path2, watcher1);
         manager.addWatch(path2, watcher1);
 
 
-        manager.triggerWatch(path3, EventType.NodeCreated, 1);
+        manager.triggerWatch(path3, EventType.NodeCreated, 1, null);
         //path3 is not being watched so metric is 0
         //path3 is not being watched so metric is 0
         checkMetrics("node_created_watch_count", 0L, 0L, 0D, 0L, 0L);
         checkMetrics("node_created_watch_count", 0L, 0L, 0D, 0L, 0L);
         // Watchers shouldn't have received any events yet so the zxid should be -1.
         // Watchers shouldn't have received any events yet so the zxid should be -1.
@@ -764,19 +764,19 @@ public class WatchManagerTest extends ZKTestCase {
         checkMostRecentWatchedEvent(watcher2, null, null, -1);
         checkMostRecentWatchedEvent(watcher2, null, null, -1);
 
 
         //path1 is watched by two watchers so two fired
         //path1 is watched by two watchers so two fired
-        manager.triggerWatch(path1, EventType.NodeCreated, 2);
+        manager.triggerWatch(path1, EventType.NodeCreated, 2, null);
         checkMetrics("node_created_watch_count", 2L, 2L, 2D, 1L, 2L);
         checkMetrics("node_created_watch_count", 2L, 2L, 2D, 1L, 2L);
         checkMostRecentWatchedEvent(watcher1, path1, EventType.NodeCreated, 2);
         checkMostRecentWatchedEvent(watcher1, path1, EventType.NodeCreated, 2);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
 
 
         //path2 is watched by one watcher so one fired now total is 3
         //path2 is watched by one watcher so one fired now total is 3
-        manager.triggerWatch(path2, EventType.NodeCreated, 3);
+        manager.triggerWatch(path2, EventType.NodeCreated, 3, null);
         checkMetrics("node_created_watch_count", 1L, 2L, 1.5D, 2L, 3L);
         checkMetrics("node_created_watch_count", 1L, 2L, 1.5D, 2L, 3L);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeCreated, 3);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeCreated, 3);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
 
 
         //watches on path1 are no longer there so zero fired
         //watches on path1 are no longer there so zero fired
-        manager.triggerWatch(path1, EventType.NodeDataChanged, 4);
+        manager.triggerWatch(path1, EventType.NodeDataChanged, 4, null);
         checkMetrics("node_changed_watch_count", 0L, 0L, 0D, 0L, 0L);
         checkMetrics("node_changed_watch_count", 0L, 0L, 0D, 0L, 0L);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeCreated, 3);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeCreated, 3);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeCreated, 2);
@@ -788,12 +788,12 @@ public class WatchManagerTest extends ZKTestCase {
         //path2 is watched by watcher1
         //path2 is watched by watcher1
         manager.addWatch(path2, watcher1);
         manager.addWatch(path2, watcher1);
 
 
-        manager.triggerWatch(path1, EventType.NodeDataChanged, 5);
+        manager.triggerWatch(path1, EventType.NodeDataChanged, 5, null);
         checkMetrics("node_changed_watch_count", 2L, 2L, 2D, 1L, 2L);
         checkMetrics("node_changed_watch_count", 2L, 2L, 2D, 1L, 2L);
         checkMostRecentWatchedEvent(watcher1, path1, EventType.NodeDataChanged, 5);
         checkMostRecentWatchedEvent(watcher1, path1, EventType.NodeDataChanged, 5);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeDataChanged, 5);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeDataChanged, 5);
 
 
-        manager.triggerWatch(path2, EventType.NodeDeleted, 6);
+        manager.triggerWatch(path2, EventType.NodeDeleted, 6, null);
         checkMetrics("node_deleted_watch_count", 1L, 1L, 1D, 1L, 1L);
         checkMetrics("node_deleted_watch_count", 1L, 1L, 1D, 1L, 1L);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeDeleted, 6);
         checkMostRecentWatchedEvent(watcher1, path2, EventType.NodeDeleted, 6);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeDataChanged, 5);
         checkMostRecentWatchedEvent(watcher2, path1, EventType.NodeDataChanged, 5);

+ 629 - 0
zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherACLTest.java

@@ -0,0 +1,629 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.zookeeper.test;
+
+import static org.apache.zookeeper.AddWatchMode.PERSISTENT;
+import static org.apache.zookeeper.AddWatchMode.PERSISTENT_RECURSIVE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.fail;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class encodes a set of tests corresponding to a "truth table"
+ * of interactions between persistent watchers and znode ACLs:
+ *
+ * <a href="https://docs.google.com/spreadsheets/d/1eMH2aimrrMc_b6McU8CHm2yCj2X-w30Fy4fCBOHn7NA/edit#gid=0">https://docs.google.com/spreadsheets/d/1eMH2aimrrMc_b6McU8CHm2yCj2X-w30Fy4fCBOHn7NA/edit#gid=0</a>
+ */
+public class PersistentWatcherACLTest extends ClientBase {
+    private static final Logger LOG = LoggerFactory.getLogger(PersistentWatcherACLTest.class);
+    /** An ACL denying READ. */
+    private static final List<ACL> ACL_NO_READ = Collections.singletonList(new ACL(ZooDefs.Perms.ALL & ~ZooDefs.Perms.READ, ZooDefs.Ids.ANYONE_ID_UNSAFE));
+    private BlockingQueue<WatchedEvent> events;
+    private Watcher persistentWatcher;
+
+    @Override
+    @BeforeEach
+    public void setUp() throws Exception {
+        super.setUp();
+
+        events = new LinkedBlockingQueue<>();
+        persistentWatcher = event -> {
+            events.add(event);
+            LOG.info("Added event: {}; total: {}", event, events.size());
+        };
+    }
+
+    /**
+     * This Step class, with the Round class below, is used to encode
+     * the contents of the truth table.
+     *
+     * (These should become Records once we target JDK 14+.)
+     */
+    private static class Step {
+        Step(int opCode, String target) {
+            this(opCode, target, null, null);
+        }
+        Step(int opCode, String target, EventType eventType, String eventPath) {
+            this.opCode = opCode;
+            this.target = target;
+            this.eventType = eventType;
+            this.eventPath = eventPath;
+        }
+        /** Action: create, setData or delete */
+        final int opCode;
+        /** Target path */
+        final String target;
+        /** Expected event type, {@code null} if no event is expected */
+        final EventType eventType;
+        /** Expected event path, {@code null} if no event is expected */
+        final String eventPath;
+    }
+
+    /**
+     * This Round class, with the Step class above, is used to encode
+     * the contents of the truth table.
+     *
+     * (These should become Records once we target JDK 14+.)
+     */
+    private static class Round {
+        Round(String summary, Boolean allowA, Boolean allowB, Boolean allowC, String watchTarget, AddWatchMode watchMode, Step[] steps) {
+            this.summary = summary;
+            this.allowA = allowA;
+            this.allowB = allowB;
+            this.allowC = allowC;
+            this.watchTarget = watchTarget;
+            this.watchMode = watchMode;
+            this.steps = steps;
+        }
+        /** Notes/summary */
+        final String summary;
+        /** Should /a's ACL leave it readable? */
+        final Boolean allowA;
+        /** Should /a/b's ACL leave it readable? */
+        final Boolean allowB;
+        /** Should /a/b/c's ACL leave it readable? */
+        final Boolean allowC;
+        /** Watch path */
+        final String watchTarget;
+        /** Watch mode */
+        final AddWatchMode watchMode;
+        /** Actions and expected events */
+        final Step[] steps;
+    }
+
+    /**
+     * A "round" of tests from the table encoded as Java objects.
+     *
+     * Note that the set of rounds is collected in a {@code ROUNDS}
+     * array below, and that this test class includes a {@code main}
+     * method which produces a "CSV" rendition of the table, for ease
+     * of comparison with the original.
+     *
+     * @see #ROUNDS
+     */
+    private static final Round roundNothingAsAIsWatchedButDeniedBIsNotWatched =
+        new Round(
+            "Nothing as a is watched but denied. b is not watched",
+            false, true, null, "/a", PERSISTENT, new Step[] {
+                new Step(ZooDefs.OpCode.setData, "/a"),
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundNothingAsBothAAndBDenied =
+        new Round(
+            "Nothing as both a and b denied",
+            false, false, null, "/a", PERSISTENT, new Step[] {
+                new Step(ZooDefs.OpCode.setData, "/a"),
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundAChangesInclChildrenAreSeen =
+        new Round(
+            "a changes, incl children, are seen",
+            true, false, null, "/a", PERSISTENT, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a", EventType.NodeCreated, "/a"),
+                new Step(ZooDefs.OpCode.setData, "/a", EventType.NodeDataChanged, "/a"),
+                new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeChildrenChanged, "/a"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeChildrenChanged, "/a"),
+                new Step(ZooDefs.OpCode.delete, "/a", EventType.NodeDeleted, "/a"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundNothingForAAsItSDeniedBChangesSeen =
+        new Round(
+            "Nothing for a as it's denied, b changes allowed/seen",
+            false, true, null, "/a", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.setData, "/a"),
+                new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundNothingBothDenied =
+        new Round(
+            "Nothing - both denied",
+            false, false, null, "/a", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.setData, "/a"),
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+                new Step(ZooDefs.OpCode.delete, "/a"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundNothingAllDenied =
+        new Round(
+            "Nothing - all denied",
+            false, false, false, "/a", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.create, "/a/b/c"),
+                new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundADeniesSeeAllChangesForBAndCIncludingBChildren =
+        new Round(
+            "a denies, see all changes for b and c, including b's children",
+            false, true, true, "/a", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+                new Step(ZooDefs.OpCode.create, "/a/b/c", EventType.NodeCreated, "/a/b/c"),
+                new Step(ZooDefs.OpCode.setData, "/a/b/c", EventType.NodeDataChanged, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b/c", EventType.NodeDeleted, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundADeniesSeeAllBChangesAndBChildrenNothingForC =
+        new Round(
+            "a denies, see all b changes and b's children, nothing for c",
+            false, true, false, "/a", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a/b", EventType.NodeCreated, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b", EventType.NodeDataChanged, "/a/b"),
+                new Step(ZooDefs.OpCode.create, "/a/b/c"),
+                new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b", EventType.NodeDeleted, "/a/b"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundNothingTheWatchIsOnC =
+        new Round(
+            "Nothing - the watch is on c",
+            false, true, false, "/a/b/c", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.create, "/a/b/c"),
+                new Step(ZooDefs.OpCode.setData, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+            }
+        );
+
+    /**
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round roundTheWatchIsOnlyOnCBAndCAllowed =
+        new Round(
+            "The watch is only on c (b and c allowed)",
+            false, true, true, "/a/b/c", PERSISTENT_RECURSIVE, new Step[] {
+                new Step(ZooDefs.OpCode.create, "/a/b"),
+                new Step(ZooDefs.OpCode.setData, "/a/b"),
+                new Step(ZooDefs.OpCode.create, "/a/b/c", EventType.NodeCreated, "/a/b/c"),
+                new Step(ZooDefs.OpCode.setData, "/a/b/c", EventType.NodeDataChanged, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b/c", EventType.NodeDeleted, "/a/b/c"),
+                new Step(ZooDefs.OpCode.delete, "/a/b"),
+            }
+        );
+
+    /**
+     * Transform the "tristate" {@code allow} property to a concrete
+     * ACL which can be passed to the ZooKeeper API.
+     *
+     * @param allow "tristate" value: {@code null}/don't care, {@code
+     * true}, {@code false}
+     * @return the ACL
+     */
+    private static List<ACL> selectAcl(Boolean allow) {
+        if (allow == null) {
+            return null;
+        } else if (!allow) {
+            return ACL_NO_READ;
+        } else {
+            return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+        }
+    }
+
+    /**
+     * Executes one "round" of tests from the Java object encoding of
+     * the table.
+     *
+     * @param round the "round"
+     *
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see PersistentWatcherACLTest.Round
+     * @see PersistentWatcherACLTest.Step
+     */
+    private void execRound(Round round)
+        throws IOException, InterruptedException, KeeperException {
+        try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) {
+            List<ACL> aclForA = selectAcl(round.allowA);
+            List<ACL> aclForB = selectAcl(round.allowB);
+            List<ACL> aclForC = selectAcl(round.allowC);
+
+            boolean firstStepCreatesA = round.steps.length > 0
+                && round.steps[0].opCode == ZooDefs.OpCode.create
+                && round.steps[0].target.equals("/a");
+
+            // Assume /a always exists (except if it's about to be created)
+            if (!firstStepCreatesA) {
+                zk.create("/a", new byte[0], aclForA, CreateMode.PERSISTENT);
+            }
+
+            zk.addWatch(round.watchTarget, persistentWatcher, round.watchMode);
+
+            for (int i = 0; i < round.steps.length; i++) {
+                Step step = round.steps[i];
+
+                switch (step.opCode) {
+                case ZooDefs.OpCode.create:
+                    List<ACL> acl = step.target.endsWith("/c")
+                        ? aclForC
+                        : step.target.endsWith("/b")
+                        ? aclForB
+                        : aclForA;
+                    zk.create(step.target, new byte[0], acl, CreateMode.PERSISTENT);
+                    break;
+                case ZooDefs.OpCode.delete:
+                    zk.delete(step.target, -1);
+                    break;
+                case ZooDefs.OpCode.setData:
+                    zk.setData(step.target, new byte[0], -1);
+                    break;
+                default:
+                    fail("Unexpected opCode " + step.opCode + " in step " + i);
+                    break;
+                }
+
+                WatchedEvent actualEvent = events.poll(500, TimeUnit.MILLISECONDS);
+                if (step.eventType == null) {
+                    assertNull(actualEvent, "Unexpected event " + actualEvent + " at step " + i);
+                } else {
+                    String m = "In event " + actualEvent + " at step " + i;
+                    assertNotNull(actualEvent, m);
+                    assertEquals(step.eventType,  actualEvent.getType(), m);
+                    assertEquals(step.eventPath, actualEvent.getPath(), m);
+                }
+            }
+        }
+    }
+
+    /**
+     * A test method, wrapping the definition of a "round."  This
+     * should really use JUnit 5's runtime test case generation
+     * facilities, but that would prevent backporting this suite to
+     * JUnit 4.
+     *
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see <a href="https://junit.org/junit5/docs/5.0.2/api/org/junit/jupiter/api/DynamicTest.html">JUnit 5 runtime test case generation</a>
+     */
+    @Test
+    public void testNothingAsAIsWatchedButDeniedBIsNotWatched()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingAsAIsWatchedButDeniedBIsNotWatched);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundNothingAsBothAAndBDenied
+     */
+    @Test
+    public void testNothingAsBothAAndBDenied()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingAsBothAAndBDenied);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundAChangesInclChildrenAreSeen
+     */
+    @Test
+    public void testAChangesInclChildrenAreSeen()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundAChangesInclChildrenAreSeen);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundNothingForAAsItSDeniedBChangesSeen
+     */
+    @Test
+    public void testNothingForAAsItSDeniedBChangesSeen()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingForAAsItSDeniedBChangesSeen);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundNothingBothDenied
+     */
+    @Test
+    public void testNothingBothDenied()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingBothDenied);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundNothingAllDenied
+     */
+    @Test
+    public void testNothingAllDenied()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingAllDenied);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundADeniesSeeAllChangesForBAndCIncludingBChildren
+     */
+    @Test
+    public void testADeniesSeeAllChangesForBAndCIncludingBChildren()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundADeniesSeeAllChangesForBAndCIncludingBChildren);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundADeniesSeeAllBChangesAndBChildrenNothingForC
+     */
+    @Test
+    public void testADeniesSeeAllBChangesAndBChildrenNothingForC()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundADeniesSeeAllBChangesAndBChildrenNothingForC);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundNothingTheWatchIsOnC
+     */
+    @Test
+    public void testNothingTheWatchIsOnC()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundNothingTheWatchIsOnC);
+    }
+
+    /**
+     * @see #testNothingAsAIsWatchedButDeniedBIsNotWatched
+     * @see #roundTheWatchIsOnlyOnCBAndCAllowed
+     */
+    @Test
+    public void testTheWatchIsOnlyOnCBAndCAllowed()
+        throws IOException, InterruptedException, KeeperException {
+        execRound(roundTheWatchIsOnlyOnCBAndCAllowed);
+    }
+
+    // The rest of this class is the world's lamest "CSV" encoder.
+
+    /**
+     * The set of rounds.  This array includes one entry for each
+     * {@code private static final Round round*} member variable
+     * defined above.
+     *
+     * @see #roundNothingAsAIsWatchedButDeniedBIsNotWatched
+     */
+    private static final Round[] ROUNDS = new Round[] {
+        roundNothingAsAIsWatchedButDeniedBIsNotWatched,
+        roundNothingAsBothAAndBDenied,
+        roundAChangesInclChildrenAreSeen,
+        roundNothingForAAsItSDeniedBChangesSeen,
+        roundNothingBothDenied,
+        roundNothingAllDenied,
+        roundADeniesSeeAllChangesForBAndCIncludingBChildren,
+        roundADeniesSeeAllBChangesAndBChildrenNothingForC,
+        roundNothingTheWatchIsOnC,
+        roundTheWatchIsOnlyOnCBAndCAllowed,
+    };
+
+    private static String allowString(String prefix, Boolean allow) {
+        if (allow == null) {
+            return "";
+        } else {
+            return prefix + (allow ? "allow" : "deny");
+        }
+    }
+
+    private static String watchModeString(AddWatchMode watchMode) {
+        switch (watchMode) {
+        case PERSISTENT:
+            return "PERSISTENT";
+        case PERSISTENT_RECURSIVE:
+            return "PRECURSIVE";
+        default:
+            return "?";
+        }
+    }
+
+    private static String actionString(int opCode) {
+        switch (opCode) {
+        case ZooDefs.OpCode.create:
+            return "create";
+        case ZooDefs.OpCode.delete:
+            return "delete";
+        case ZooDefs.OpCode.setData:
+            return "modify";
+        default:
+            return "?";
+        }
+    }
+
+    private static String eventPathString(String eventPath) {
+        if (eventPath == null) {
+            return "?";
+        } else if (eventPath.length() <= 1) {
+            return eventPath;
+        } else {
+            return eventPath.substring(eventPath.lastIndexOf('/') + 1);
+        }
+    }
+
+    /**
+     * Generates a "CSV" rendition of the table in sb.
+     *
+     * @param sb the target string builder
+     */
+    private static void genCsv(StringBuilder sb) {
+        sb.append("Initial State,")
+            .append("Action,")
+            .append("NodeCreated,")
+            .append("NodeDeleted,")
+            .append("NodeDataChanged,")
+            .append("NodeChildrenChanged,")
+            .append("Notes/summary\n");
+        sb.append("Assume /a always exists\n\n");
+
+        for (Round round : ROUNDS) {
+            sb.append("\"ACL")
+                .append(allowString(": a ", round.allowA))
+                .append(allowString(", b ", round.allowB))
+                .append(allowString(", c ", round.allowC))
+                .append("\"")
+                .append(",,,,,,\"")
+                .append(round.summary)
+                .append("\"\n");
+            for (int i = 0; i < round.steps.length; i++) {
+                Step step = round.steps[i];
+
+                if (i == 0) {
+                    sb.append("\"addWatch(")
+                        .append(round.watchTarget)
+                        .append(", ")
+                        .append(watchModeString(round.watchMode))
+                        .append(")\"");
+                }
+
+                sb.append(",")
+                    .append(actionString(step.opCode))
+                    .append(" ")
+                    .append(step.target)
+                    .append(",");
+
+                if (step.eventType == EventType.NodeCreated) {
+                    sb.append("y - ")
+                        .append(eventPathString(step.eventPath));
+                }
+
+                sb.append(",");
+
+                if (step.eventType == EventType.NodeDeleted) {
+                    sb.append("y - ")
+                        .append(eventPathString(step.eventPath));
+                }
+
+                sb.append(",");
+
+                if (step.eventType == EventType.NodeDataChanged) {
+                    sb.append("y - ")
+                        .append(eventPathString(step.eventPath));
+                }
+
+                sb.append(",");
+
+                if (round.watchMode == PERSISTENT_RECURSIVE) {
+                    sb.append("n");
+                } else if (step.eventType == EventType.NodeChildrenChanged) {
+                    sb.append("y - ")
+                        .append(eventPathString(step.eventPath));
+                }
+
+                sb.append("\n");
+            }
+
+            sb.append("\n");
+        }
+    }
+
+    /**
+     * Generates a "CSV" rendition of the table to standard output.
+     *
+     * @see #ROUNDS
+     */
+    public static void main(String[] args) {
+        StringBuilder sb = new StringBuilder();
+        genCsv(sb);
+        System.out.println(sb);
+    }
+}

+ 7 - 2
zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java

@@ -21,10 +21,14 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.PrintWriter;
 import java.util.Collections;
 import java.util.Collections;
+import java.util.List;
 import org.apache.zookeeper.AddWatchMode;
 import org.apache.zookeeper.AddWatchMode;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.server.watch.IWatchManager;
 import org.apache.zookeeper.server.watch.IWatchManager;
 import org.apache.zookeeper.server.watch.WatchManagerFactory;
 import org.apache.zookeeper.server.watch.WatchManagerFactory;
 import org.apache.zookeeper.server.watch.WatcherOrBitSet;
 import org.apache.zookeeper.server.watch.WatcherOrBitSet;
@@ -59,12 +63,12 @@ public class UnsupportedAddWatcherTest extends ClientBase {
         }
         }
 
 
         @Override
         @Override
-        public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, long zxid) {
+        public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, long zxid, List<ACL> acl) {
             return new WatcherOrBitSet(Collections.emptySet());
             return new WatcherOrBitSet(Collections.emptySet());
         }
         }
 
 
         @Override
         @Override
-        public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, long zxid, WatcherOrBitSet suppress) {
+        public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, long zxid, List<ACL> acl, WatcherOrBitSet suppress) {
             return new WatcherOrBitSet(Collections.emptySet());
             return new WatcherOrBitSet(Collections.emptySet());
         }
         }
 
 
@@ -120,6 +124,7 @@ public class UnsupportedAddWatcherTest extends ClientBase {
             try (ZooKeeper zk = createClient(hostPort)) {
             try (ZooKeeper zk = createClient(hostPort)) {
                 // the server will generate an exception as our custom watch manager doesn't implement
                 // the server will generate an exception as our custom watch manager doesn't implement
                 // the new version of addWatch()
                 // the new version of addWatch()
+                zk.create("/foo", null, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
                 zk.addWatch("/foo", event -> {
                 zk.addWatch("/foo", event -> {
                 }, AddWatchMode.PERSISTENT_RECURSIVE);
                 }, AddWatchMode.PERSISTENT_RECURSIVE);
             }
             }