From e41cac853de416026e0c2011e0429e858e5823b8 Mon Sep 17 00:00:00 2001 From: randgalt Date: Wed, 6 Nov 2019 15:21:19 +0100 Subject: [PATCH 1/4] ZOOKEEPER-3605: connThrottle needs to be assigned in alternate consructor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit `connThrottle` needs to be assigned in alternate consructor to avoid NPEs Author: randgalt Reviewers: Enrico Olivelli, Andor Molnár Closes #1132 from Randgalt/ZOOKEEPER-3605 --- .../zookeeper/server/ZooKeeperServer.java | 4 +- .../server/ZooKeeperServerCreationTest.java | 61 +++++++++++++++++++ 2 files changed, 62 insertions(+), 3 deletions(-) create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerCreationTest.java diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java index e78cc38cb09..05bf82e08d9 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServer.java @@ -236,7 +236,7 @@ protected enum State { } // Connection throttling - private BlueThrottle connThrottle; + private BlueThrottle connThrottle = new BlueThrottle(); @SuppressFBWarnings(value = "IS2_INCONSISTENT_SYNC", justification = "Internally the throttler has a BlockingQueue so " @@ -308,8 +308,6 @@ public ZooKeeperServer(FileTxnSnapLog txnLogFactory, int tickTime, int minSessio readResponseCache = new ResponseCache(); - connThrottle = new BlueThrottle(); - this.initialConfig = initialConfig; this.requestPathMetricsCollector = new RequestPathMetricsCollector(); diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerCreationTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerCreationTest.java new file mode 100644 index 00000000000..670feaa4832 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/ZooKeeperServerCreationTest.java @@ -0,0 +1,61 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.File; +import java.nio.ByteBuffer; +import org.apache.jute.BinaryOutputArchive; +import org.apache.zookeeper.proto.ConnectRequest; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; +import org.apache.zookeeper.test.ClientBase; +import org.junit.Test; + +public class ZooKeeperServerCreationTest { + + /** + * Test the default ZooKeeperServer and call processConnectRequest() to make sure + * that all needed fields are initialized properly, etc. + */ + @Test + public void testDefaultConstructor() throws Exception { + File tmpDir = ClientBase.createEmptyTestDir(); + FileTxnSnapLog fileTxnSnapLog = new FileTxnSnapLog(new File(tmpDir, "data"), new File(tmpDir, "data_txnlog")); + + ZooKeeperServer zks = new ZooKeeperServer() { + @Override + public void submitRequest(Request si) { + // NOP + } + }; + zks.setTxnLogFactory(fileTxnSnapLog); + zks.setZKDatabase(new ZKDatabase(fileTxnSnapLog)); + zks.createSessionTracker(); + + ServerCnxnFactory cnxnFactory = ServerCnxnFactory.createFactory(); + ServerCnxn cnxn = new MockServerCnxn(); + + ConnectRequest connReq = new ConnectRequest(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + BinaryOutputArchive boa = BinaryOutputArchive.getArchive(baos); + connReq.serialize(boa, "connect"); + zks.processConnectRequest(cnxn, ByteBuffer.wrap(baos.toByteArray())); + } + +} From 553639378d5cf86c2488afff4586e5e4cce38061 Mon Sep 17 00:00:00 2001 From: randgalt Date: Fri, 8 Nov 2019 17:30:25 +0100 Subject: [PATCH 2/4] ZOOKEEPER-1416 - Persistent, recursive watchers MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Background Note: this is a port of https://github.com/apache/zookeeper/pull/136 Implementation for a persistent, recursive watch addition for ZK. These watches are set via a new method, addPersistentWatch() and are removed via the existing watcher removal methods. Persistent, recursive watches have these characteristics: a) Once set, they do not auto-remove when triggered; b) they trigger for all event types (child, data, etc.) on the node they are registered for and any child znode recursively; c) they are efficiently implemented by using the existing watch internals. A new class PathIterator walks up the path parent-by-parent when checking if a watcher applies. ### Implementation Details - A new enum manages the different "modes" for watchers: `WatcherMode`. - For traditional, "standard" watchers, the code path is almost exactly the same. There is very little overhead other than a bit of extra checks in `WatchManager` - Given how this is implemented it was difficult to add support when `WatchManagerOptimized` is used. I'm open to adding it for that version but it will take work. We should consider not supporting persistent/recursive watchers when WatchManagerOptimized is used. I notice that `WatchManagerOptimized` is not even mentioned in the docs. - The mode for a given watcher/path pair is held in a map inside of `WatcherModeManager`. The absence of an entry means Standard. This way, there's no overhead for old, standard watchers. - `PathParentIterator` is the "meat" of the implementation. Rather than set watchers on every ZNode implied by a recursive watcher. WatchManager passes any paths it processes through PathParentIterator which iterates up each parent znode looking for watchers. - The remainder of the changes are scaffolding to match how other watchers are used as well as Jute/API changes to set persistent/recursive watchers ### Testing The tests were written years ago. I think they're comprehensive but reviewers should pay attention to anything that was missed. There is much ZooKeeper knowledge that's only in the heads of ZK committers. - `PersistentWatcherTest` - tests persistent, non-recursive watchers - `PersistentRecursiveWatcherTest` - tests persistent, recursive watchers - `PathParentIteratorTest`- exercises edges of PathParentIterator Author: randgalt Reviewers: Enrico Olivelli , Norbert Kalmar , Andor Molnár , Justin Mao Ling Closes #1106 from Randgalt/ZOOKEEPER-1416 --- .../main/resources/markdown/zookeeperOver.md | 5 + .../markdown/zookeeperProgrammers.md | 27 ++- .../src/main/resources/zookeeper.jute | 12 + .../org/apache/zookeeper/AddWatchMode.java | 67 ++++++ .../java/org/apache/zookeeper/ClientCnxn.java | 36 ++- .../java/org/apache/zookeeper/Watcher.java | 5 +- .../java/org/apache/zookeeper/ZooDefs.java | 11 + .../java/org/apache/zookeeper/ZooKeeper.java | 204 +++++++++++++++++ .../org/apache/zookeeper/ZooKeeperMain.java | 2 + .../apache/zookeeper/cli/AddWatchCommand.java | 87 ++++++++ .../org/apache/zookeeper/server/DataTree.java | 18 +- .../server/FinalRequestProcessor.java | 33 ++- .../server/PrepRequestProcessor.java | 2 + .../org/apache/zookeeper/server/Request.java | 5 + .../apache/zookeeper/server/ZKDatabase.java | 21 +- .../util/RequestPathMetricsCollector.java | 2 + .../zookeeper/server/watch/IWatchManager.java | 24 ++ .../server/watch/PathParentIterator.java | 106 +++++++++ .../zookeeper/server/watch/WatchManager.java | 142 ++++++++---- .../zookeeper/server/watch/WatcherMode.java | 56 +++++ .../server/watch/WatcherModeManager.java | 96 ++++++++ .../server/watch/PathParentIteratorTest.java | 84 +++++++ .../server/watch/RecursiveWatchQtyTest.java | 197 ++++++++++++++++ .../server/watch/WatchManagerTest.java | 2 +- .../test/PersistentRecursiveWatcherTest.java | 174 +++++++++++++++ .../zookeeper/test/PersistentWatcherTest.java | 211 ++++++++++++++++++ .../test/UnsupportedAddWatcherTest.java | 124 ++++++++++ 27 files changed, 1699 insertions(+), 54 deletions(-) create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/AddWatchMode.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/cli/AddWatchCommand.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/PathParentIterator.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherMode.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherModeManager.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/PathParentIteratorTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/RecursiveWatchQtyTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperOver.md b/zookeeper-docs/src/main/resources/markdown/zookeeperOver.md index 1c00609e6d4..4c60a3de7e2 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperOver.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperOver.md @@ -146,6 +146,11 @@ receives a packet saying that the znode has changed. If the connection between the client and one of the ZooKeeper servers is broken, the client will receive a local notification. +**New in 3.6.0:** Clients can also set +permanent, recursive watches on a znode that are not removed when triggered +and that trigger for changes on the registered znode as well as any children +znodes recursively. + ### Guarantees diff --git a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md index 03e83c092f9..08d30c8158a 100644 --- a/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md +++ b/zookeeper-docs/src/main/resources/markdown/zookeeperProgrammers.md @@ -32,6 +32,7 @@ limitations under the License. * [ZooKeeper Sessions](#ch_zkSessions) * [ZooKeeper Watches](#ch_zkWatches) * [Semantics of Watches](#sc_WatchSemantics) + * [Persistent, Recursive Watches](#sc_WatchPersistentRecursive) * [Remove Watches](#sc_WatchRemoval) * [What ZooKeeper Guarantees about Watches](#sc_WatchGuarantees) * [Things to Remember about Watches](#sc_WatchRememberThese) @@ -640,6 +641,11 @@ general this all occurs transparently. There is one case where a watch may be missed: a watch for the existence of a znode not yet created will be missed if the znode is created and deleted while disconnected. +**New in 3.6.0:** Clients can also set +permanent, recursive watches on a znode that are not removed when triggered +and that trigger for changes on the registered znode as well as any children +znodes recursively. + ### Semantics of Watches @@ -657,6 +663,21 @@ the events that a watch can trigger and the calls that enable them: * **Child event:** Enabled with a call to getChildren. + + +### Persistent, Recursive Watches + +**New in 3.6.0:** There is now a variation on the standard +watch described above whereby you can set a watch that does not get removed when triggered. +Additionally, these watches trigger the event types *NodeCreated*, *NodeDeleted*, and *NodeDataChanged* +and, optionally, recursively for all znodes starting at the znode that the watch is registered for. Note +that *NodeChildrenChanged* events are not triggered for persistent recursive watches as it would be redundant. + +Persistent watches are set using the method *addWatch()*. The triggering semantics and guarantees +(other than one-time triggering) are the same as standard watches. The only exception regarding events is that +recursive persistent watchers never trigger child changed events as they are redundant. +Persistent watches are removed using *removeWatches()* with watcher type *WatcherType.Any*. + ### Remove Watches @@ -671,6 +692,8 @@ successful watch removal. Watcher which was added with a call to getChildren. * **Data Remove event:** Watcher which was added with a call to exists or getData. +* **Persistent Remove event:** + Watcher which was added with a call to add a persistent watch. @@ -693,11 +716,11 @@ guarantees: ### Things to Remember about Watches -* Watches are one time triggers; if you get a watch event and +* Standard watches are one time triggers; if you get a watch event and you want to get notified of future changes, you must set another watch. -* Because watches are one time triggers and there is latency +* Because standard watches are one time triggers and there is latency between getting the event and sending a new request to get a watch you cannot reliably see every change that happens to a node in ZooKeeper. Be prepared to handle the case where the znode changes diff --git a/zookeeper-jute/src/main/resources/zookeeper.jute b/zookeeper-jute/src/main/resources/zookeeper.jute index 8310664a6e5..6d553657540 100644 --- a/zookeeper-jute/src/main/resources/zookeeper.jute +++ b/zookeeper-jute/src/main/resources/zookeeper.jute @@ -73,6 +73,14 @@ module org.apache.zookeeper.proto { vectorexistWatches; vectorchildWatches; } + class SetWatches2 { + long relativeZxid; + vectordataWatches; + vectorexistWatches; + vectorchildWatches; + vectorpersistentWatches; + vectorpersistentRecursiveWatches; + } class RequestHeader { int xid; int type; @@ -180,6 +188,10 @@ module org.apache.zookeeper.proto { class SetACLResponse { org.apache.zookeeper.data.Stat stat; } + class AddWatchRequest { + ustring path; + int mode; + } class WatcherEvent { int type; // event type int state; // state of the Keeper client runtime diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/AddWatchMode.java b/zookeeper-server/src/main/java/org/apache/zookeeper/AddWatchMode.java new file mode 100644 index 00000000000..0f339c1197a --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/AddWatchMode.java @@ -0,0 +1,67 @@ +/* + * 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; + +/** + * Modes available to {@link ZooKeeper#addWatch(String, Watcher, AddWatchMode)} + */ +public enum AddWatchMode { + /** + *

+ * Set a watcher on the given path that does not get removed when triggered (i.e. it stays active + * until it is removed). This watcher + * is triggered for both data and child events. To remove the watcher, use + * removeWatches() with WatcherType.Any. The watcher behaves as if you placed an exists() watch and + * a getData() watch on the ZNode at the given path. + *

+ */ + PERSISTENT(ZooDefs.AddWatchModes.persistent), + + /** + *

+ * Set a watcher on the given path that: a) does not get removed when triggered (i.e. it stays active + * until it is removed); b) applies not only to the registered path but all child paths recursively. This watcher + * is triggered for both data and child events. To remove the watcher, use + * removeWatches() with WatcherType.Any + *

+ * + *

+ * The watcher behaves as if you placed an exists() watch and + * a getData() watch on the ZNode at the given path and any ZNodes that are children + * of the given path including children added later. + *

+ * + *

+ * NOTE: when there are active recursive watches there is a small performance decrease as all segments + * of ZNode paths must be checked for watch triggering. + *

+ */ + PERSISTENT_RECURSIVE(ZooDefs.AddWatchModes.persistentRecursive) + ; + + public int getMode() { + return mode; + } + + private final int mode; + + AddWatchMode(int mode) { + this.mode = mode; + } +} 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 b0f7b0710e1..3713646d631 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ClientCnxn.java @@ -86,6 +86,7 @@ import org.apache.zookeeper.proto.SetACLResponse; import org.apache.zookeeper.proto.SetDataResponse; import org.apache.zookeeper.proto.SetWatches; +import org.apache.zookeeper.proto.SetWatches2; import org.apache.zookeeper.proto.WatcherEvent; import org.apache.zookeeper.server.ByteBufferInputStream; import org.apache.zookeeper.server.ZooKeeperThread; @@ -990,16 +991,24 @@ void primeConnection() throws IOException { List dataWatches = zooKeeper.getDataWatches(); List existWatches = zooKeeper.getExistWatches(); List childWatches = zooKeeper.getChildWatches(); - if (!dataWatches.isEmpty() || !existWatches.isEmpty() || !childWatches.isEmpty()) { + List persistentWatches = zooKeeper.getPersistentWatches(); + List persistentRecursiveWatches = zooKeeper.getPersistentRecursiveWatches(); + if (!dataWatches.isEmpty() || !existWatches.isEmpty() || !childWatches.isEmpty() + || !persistentWatches.isEmpty() || !persistentRecursiveWatches.isEmpty()) { Iterator dataWatchesIter = prependChroot(dataWatches).iterator(); Iterator existWatchesIter = prependChroot(existWatches).iterator(); Iterator childWatchesIter = prependChroot(childWatches).iterator(); + Iterator persistentWatchesIter = prependChroot(persistentWatches).iterator(); + Iterator persistentRecursiveWatchesIter = prependChroot(persistentRecursiveWatches).iterator(); long setWatchesLastZxid = lastZxid; - while (dataWatchesIter.hasNext() || existWatchesIter.hasNext() || childWatchesIter.hasNext()) { + while (dataWatchesIter.hasNext() || existWatchesIter.hasNext() || childWatchesIter.hasNext() + || persistentWatchesIter.hasNext() || persistentRecursiveWatchesIter.hasNext()) { List dataWatchesBatch = new ArrayList(); List existWatchesBatch = new ArrayList(); List childWatchesBatch = new ArrayList(); + List persistentWatchesBatch = new ArrayList(); + List persistentRecursiveWatchesBatch = new ArrayList(); int batchLength = 0; // Note, we may exceed our max length by a bit when we add the last @@ -1015,15 +1024,32 @@ void primeConnection() throws IOException { } else if (childWatchesIter.hasNext()) { watch = childWatchesIter.next(); childWatchesBatch.add(watch); + } else if (persistentWatchesIter.hasNext()) { + watch = persistentWatchesIter.next(); + persistentWatchesBatch.add(watch); + } else if (persistentRecursiveWatchesIter.hasNext()) { + watch = persistentRecursiveWatchesIter.next(); + persistentRecursiveWatchesBatch.add(watch); } else { break; } batchLength += watch.length(); } - SetWatches sw = new SetWatches(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch, childWatchesBatch); - RequestHeader header = new RequestHeader(-8, OpCode.setWatches); - Packet packet = new Packet(header, new ReplyHeader(), sw, null, null); + Record record; + int opcode; + if (persistentWatchesBatch.isEmpty() && persistentRecursiveWatchesBatch.isEmpty()) { + // maintain compatibility with older servers - if no persistent/recursive watchers + // are used, use the old version of SetWatches + record = new SetWatches(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch, childWatchesBatch); + opcode = OpCode.setWatches; + } else { + record = new SetWatches2(setWatchesLastZxid, dataWatchesBatch, existWatchesBatch, + childWatchesBatch, persistentWatchesBatch, persistentRecursiveWatchesBatch); + opcode = OpCode.setWatches2; + } + RequestHeader header = new RequestHeader(-8, opcode); + Packet packet = new Packet(header, new ReplyHeader(), record, null, null); outgoingQueue.addFirst(packet); } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Watcher.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Watcher.java index db81fdf2668..ab4b654880e 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/Watcher.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Watcher.java @@ -143,7 +143,8 @@ enum EventType { NodeDataChanged(3), NodeChildrenChanged(4), DataWatchRemoved(5), - ChildWatchRemoved(6); + ChildWatchRemoved(6), + PersistentWatchRemoved (7); private final int intValue; // Integer representation of value // for sending over wire @@ -172,6 +173,8 @@ public static EventType fromInt(int intValue) { return EventType.DataWatchRemoved; case 6: return EventType.ChildWatchRemoved; + case 7: + return EventType.PersistentWatchRemoved; default: throw new RuntimeException("Invalid integer value for conversion to EventType"); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java index db176f5b76e..a12e5803c27 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooDefs.java @@ -89,6 +89,10 @@ public interface OpCode { int getAllChildrenNumber = 104; + int setWatches2 = 105; + + int addWatch = 106; + int createSession = -10; int closeSession = -11; @@ -148,6 +152,13 @@ public interface Ids { } + @InterfaceAudience.Public + public interface AddWatchModes { + int persistent = 0; // matches AddWatchMode.PERSISTENT + + int persistentRecursive = 1; // matches AddWatchMode.PERSISTENT_RECURSIVE + } + public static final String[] opNames = {"notification", "create", "delete", "exists", "getData", "setData", "getACL", "setACL", "getChildren", "getChildren2", "getMaxChildren", "setMaxChildren", "ping", "reconfig", "getConfig"}; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java index 7f3b8483572..f6f165deaf4 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeper.java @@ -53,12 +53,14 @@ import org.apache.zookeeper.common.PathUtils; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.AddWatchRequest; import org.apache.zookeeper.proto.CheckWatchesRequest; import org.apache.zookeeper.proto.Create2Response; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.CreateResponse; import org.apache.zookeeper.proto.CreateTTLRequest; import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.ErrorResponse; import org.apache.zookeeper.proto.ExistsRequest; import org.apache.zookeeper.proto.GetACLRequest; import org.apache.zookeeper.proto.GetACLResponse; @@ -83,6 +85,7 @@ import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree; import org.apache.zookeeper.server.EphemeralType; +import org.apache.zookeeper.server.watch.PathParentIterator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -254,6 +257,18 @@ protected List getChildWatches() { return rc; } } + protected List getPersistentWatches() { + synchronized (watchManager.persistentWatches) { + List rc = new ArrayList(watchManager.persistentWatches.keySet()); + return rc; + } + } + protected List getPersistentRecursiveWatches() { + synchronized (watchManager.persistentRecursiveWatches) { + List rc = new ArrayList(watchManager.persistentRecursiveWatches.keySet()); + return rc; + } + } /** * Manage watchers and handle events generated by the ClientCnxn object. @@ -267,6 +282,8 @@ static class ZKWatchManager implements ClientWatchManager { private final Map> dataWatches = new HashMap>(); private final Map> existWatches = new HashMap>(); private final Map> childWatches = new HashMap>(); + private final Map> persistentWatches = new HashMap>(); + private final Map> persistentRecursiveWatches = new HashMap>(); private boolean disableAutoWatchReset; ZKWatchManager(boolean disableAutoWatchReset) { @@ -296,6 +313,8 @@ public Map> removeWatcher( removedWatchers.put(EventType.ChildWatchRemoved, childWatchersToRem); HashSet dataWatchersToRem = new HashSet<>(); removedWatchers.put(EventType.DataWatchRemoved, dataWatchersToRem); + HashSet persistentWatchersToRem = new HashSet<>(); + removedWatchers.put(EventType.PersistentWatchRemoved, persistentWatchersToRem); boolean removedWatcher = false; switch (watcherType) { case Children: { @@ -324,10 +343,23 @@ public Map> removeWatcher( boolean removedDataWatcher = removeWatches(dataWatches, watcher, clientPath, local, rc, dataWatchersToRem); removedWatcher |= removedDataWatcher; } + synchronized (existWatches) { boolean removedDataWatcher = removeWatches(existWatches, watcher, clientPath, local, rc, dataWatchersToRem); removedWatcher |= removedDataWatcher; } + + synchronized (persistentWatches) { + boolean removedPersistentWatcher = removeWatches(persistentWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistentWatcher; + } + + synchronized (persistentRecursiveWatches) { + boolean removedPersistentRecursiveWatcher = removeWatches(persistentRecursiveWatches, + watcher, clientPath, local, rc, persistentWatchersToRem); + removedWatcher |= removedPersistentRecursiveWatcher; + } } } // Watcher function doesn't exists for the specified params @@ -373,6 +405,18 @@ void containsWatcher(String path, Watcher watcher, WatcherType watcherType) thro synchronized (childWatches) { containsWatcher = contains(path, watcher, childWatches); } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } break; } case Data: { @@ -384,6 +428,18 @@ void containsWatcher(String path, Watcher watcher, WatcherType watcherType) thro boolean contains_temp = contains(path, watcher, existWatches); containsWatcher |= contains_temp; } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } break; } case Any: { @@ -395,10 +451,23 @@ void containsWatcher(String path, Watcher watcher, WatcherType watcherType) thro boolean contains_temp = contains(path, watcher, dataWatches); containsWatcher |= contains_temp; } + synchronized (existWatches) { boolean contains_temp = contains(path, watcher, existWatches); containsWatcher |= contains_temp; } + + synchronized (persistentWatches) { + boolean contains_temp = contains(path, watcher, + persistentWatches); + containsWatcher |= contains_temp; + } + + synchronized (persistentRecursiveWatches) { + boolean contains_temp = contains(path, watcher, + persistentRecursiveWatches); + containsWatcher |= contains_temp; + } } } // Watcher function doesn't exists for the specified params @@ -490,6 +559,18 @@ public Set materialize( } } + synchronized (persistentWatches) { + for (Set ws: persistentWatches.values()) { + result.addAll(ws); + } + } + + synchronized (persistentRecursiveWatches) { + for (Set ws: persistentRecursiveWatches.values()) { + result.addAll(ws); + } + } + return result; case NodeDataChanged: case NodeCreated: @@ -499,11 +580,13 @@ public Set materialize( synchronized (existWatches) { addTo(existWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; case NodeChildrenChanged: synchronized (childWatches) { addTo(childWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; case NodeDeleted: synchronized (dataWatches) { @@ -520,6 +603,7 @@ public Set materialize( synchronized (childWatches) { addTo(childWatches.remove(clientPath), result); } + addPersistentWatches(clientPath, result); break; default: String errorMsg = String.format( @@ -534,6 +618,16 @@ public Set materialize( return result; } + private void addPersistentWatches(String clientPath, Set result) { + synchronized (persistentWatches) { + addTo(persistentWatches.get(clientPath), result); + } + synchronized (persistentRecursiveWatches) { + for (String path : PathParentIterator.forAll(clientPath).asIterable()) { + addTo(persistentRecursiveWatches.get(path), result); + } + } + } } /** @@ -627,6 +721,31 @@ protected Map> getWatches(int rc) { } + class AddWatchRegistration extends WatchRegistration { + private final AddWatchMode mode; + + public AddWatchRegistration(Watcher watcher, String clientPath, AddWatchMode mode) { + super(watcher, clientPath); + this.mode = mode; + } + + @Override + protected Map> getWatches(int rc) { + switch (mode) { + case PERSISTENT: + return watchManager.persistentWatches; + case PERSISTENT_RECURSIVE: + return watchManager.persistentRecursiveWatches; + } + throw new IllegalArgumentException("Mode not supported: " + mode); + } + + @Override + protected boolean shouldAddWatch(int rc) { + return rc == 0 || rc == KeeperException.Code.NONODE.intValue(); + } + } + @InterfaceAudience.Public public enum States { CONNECTING, @@ -3035,6 +3154,91 @@ public void removeAllWatches(String path, WatcherType watcherType, boolean local removeWatches(ZooDefs.OpCode.removeWatches, path, null, watcherType, local, cb, ctx); } + /** + * Add a watch to the given znode using the given mode. Note: not all + * watch types can be set with this method. Only the modes available + * in {@link AddWatchMode} can be set with this method. + * + * @param basePath the path that the watcher applies to + * @param watcher the watcher + * @param mode type of watcher to add + * @throws InterruptedException If the server transaction is interrupted. + * @throws KeeperException If the server signals an error with a non-zero + * error code. + * @since 3.6.0 + */ + public void addWatch(String basePath, Watcher watcher, AddWatchMode mode) + throws KeeperException, InterruptedException { + PathUtils.validatePath(basePath); + String serverPath = prependChroot(basePath); + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.addWatch); + AddWatchRequest request = new AddWatchRequest(serverPath, mode.getMode()); + ReplyHeader r = cnxn.submitRequest(h, request, new ErrorResponse(), + new AddWatchRegistration(watcher, basePath, mode)); + if (r.getErr() != 0) { + throw KeeperException.create(KeeperException.Code.get(r.getErr()), + basePath); + } + } + + /** + * Add a watch to the given znode using the given mode. Note: not all + * watch types can be set with this method. Only the modes available + * in {@link AddWatchMode} can be set with this method. In this version of the method, + * the default watcher is used + * + * @param basePath the path that the watcher applies to + * @param mode type of watcher to add + * @throws InterruptedException If the server transaction is interrupted. + * @throws KeeperException If the server signals an error with a non-zero + * error code. + * @since 3.6.0 + */ + public void addWatch(String basePath, AddWatchMode mode) + throws KeeperException, InterruptedException { + addWatch(basePath, watchManager.defaultWatcher, mode); + } + + /** + * Async version of {@link #addWatch(String, Watcher, AddWatchMode)} (see it for details) + * + * @param basePath the path that the watcher applies to + * @param watcher the watcher + * @param mode type of watcher to add + * @param cb a handler for the callback + * @param ctx context to be provided to the callback + * @throws IllegalArgumentException if an invalid path is specified + * @since 3.6.0 + */ + public void addWatch(String basePath, Watcher watcher, AddWatchMode mode, + VoidCallback cb, Object ctx) { + PathUtils.validatePath(basePath); + String serverPath = prependChroot(basePath); + + RequestHeader h = new RequestHeader(); + h.setType(ZooDefs.OpCode.addWatch); + AddWatchRequest request = new AddWatchRequest(serverPath, mode.getMode()); + cnxn.queuePacket(h, new ReplyHeader(), request, new ErrorResponse(), cb, + basePath, serverPath, ctx, new AddWatchRegistration(watcher, basePath, mode)); + } + + /** + * Async version of {@link #addWatch(String, AddWatchMode)} (see it for details) + * + * @param basePath the path that the watcher applies to + * @param mode type of watcher to add + * @param cb a handler for the callback + * @param ctx context to be provided to the callback + * @throws IllegalArgumentException if an invalid path is specified + * @since 3.6.0 + */ + public void addWatch(String basePath, AddWatchMode mode, + VoidCallback cb, Object ctx) { + addWatch(basePath, watchManager.defaultWatcher, mode, cb, ctx); + } + private void validateWatcher(Watcher watcher) { if (watcher == null) { throw new IllegalArgumentException("Invalid Watcher, shouldn't be null!"); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java index c9f49a16ec2..857b16fcc70 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZooKeeperMain.java @@ -38,6 +38,7 @@ import org.apache.yetus.audience.InterfaceAudience; import org.apache.zookeeper.admin.ZooKeeperAdmin; import org.apache.zookeeper.cli.AddAuthCommand; +import org.apache.zookeeper.cli.AddWatchCommand; import org.apache.zookeeper.cli.CliCommand; import org.apache.zookeeper.cli.CliException; import org.apache.zookeeper.cli.CloseCommand; @@ -123,6 +124,7 @@ public boolean getPrintWatches() { new GetEphemeralsCommand().addToMap(commandMapCli); new GetAllChildrenNumberCommand().addToMap(commandMapCli); new VersionCommand().addToMap(commandMapCli); + new AddWatchCommand().addToMap(commandMapCli); // add all to commandMap for (Entry entry : commandMapCli.entrySet()) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/AddWatchCommand.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/AddWatchCommand.java new file mode 100644 index 00000000000..1e34b10159c --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/AddWatchCommand.java @@ -0,0 +1,87 @@ +/* + * 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.cli; + +import java.util.Arrays; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.commons.cli.Parser; +import org.apache.commons.cli.PosixParser; +import org.apache.zookeeper.AddWatchMode; +import org.apache.zookeeper.KeeperException; + +/** + * addWatch command for cli. + * Matches the ZooKeeper API addWatch() + */ +public class AddWatchCommand extends CliCommand { + + private static final Options options = new Options(); + private static final AddWatchMode defaultMode = AddWatchMode.PERSISTENT_RECURSIVE; + + private CommandLine cl; + private AddWatchMode mode = defaultMode; + + static { + options.addOption("m", true, ""); + } + + public AddWatchCommand() { + super("addWatch", "[-m mode] path # optional mode is one of " + + Arrays.toString(AddWatchMode.values()) + " - default is " + defaultMode.name()); + } + + @Override + public CliCommand parse(String[] cmdArgs) throws CliParseException { + Parser parser = new PosixParser(); + try { + cl = parser.parse(options, cmdArgs); + } catch (ParseException ex) { + throw new CliParseException(ex); + } + if (cl.getArgs().length != 2) { + throw new CliParseException(getUsageStr()); + } + + if (cl.hasOption("m")) { + try { + mode = AddWatchMode.valueOf(cl.getOptionValue("m").toUpperCase()); + } catch (IllegalArgumentException e) { + throw new CliParseException(getUsageStr()); + } + } + + return this; + } + + @Override + public boolean exec() throws CliException { + String path = cl.getArgs()[1]; + try { + zk.addWatch(path, mode); + } catch (KeeperException | InterruptedException ex) { + throw new CliWrapperException(ex); + } + + return false; + + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java index a9b08b0de56..766949ece5a 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java @@ -58,6 +58,7 @@ import org.apache.zookeeper.data.StatPersisted; import org.apache.zookeeper.server.watch.IWatchManager; import org.apache.zookeeper.server.watch.WatchManagerFactory; +import org.apache.zookeeper.server.watch.WatcherMode; import org.apache.zookeeper.server.watch.WatcherOrBitSet; import org.apache.zookeeper.server.watch.WatchesPathReport; import org.apache.zookeeper.server.watch.WatchesReport; @@ -701,6 +702,12 @@ public String getMaxPrefixWithQuota(String path) { } } + public void addWatch(String basePath, Watcher watcher, int mode) { + WatcherMode watcherMode = WatcherMode.fromZooDef(mode); + dataWatches.addWatch(basePath, watcher, watcherMode); + childWatches.addWatch(basePath, watcher, watcherMode); + } + public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperException.NoNodeException { DataNode n = nodes.get(path); byte[] data = null; @@ -1499,7 +1506,8 @@ public void removeCnxn(Watcher watcher) { childWatches.removeWatcher(watcher); } - public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, Watcher watcher) { + public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, + List persistentWatches, List persistentRecursiveWatches, Watcher watcher) { for (String path : dataWatches) { DataNode node = getNode(path); WatchedEvent e = null; @@ -1529,6 +1537,14 @@ public void setWatches(long relativeZxid, List dataWatches, List this.childWatches.addWatch(path, watcher); } } + for (String path : persistentWatches) { + this.childWatches.addWatch(path, watcher, WatcherMode.PERSISTENT); + this.dataWatches.addWatch(path, watcher, WatcherMode.PERSISTENT); + } + for (String path : persistentRecursiveWatches) { + this.childWatches.addWatch(path, watcher, WatcherMode.PERSISTENT_RECURSIVE); + this.dataWatches.addWatch(path, watcher, WatcherMode.PERSISTENT_RECURSIVE); + } } /** diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java index 93d70d86943..79a69771805 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Set; @@ -47,9 +48,11 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.proto.AddWatchRequest; import org.apache.zookeeper.proto.CheckWatchesRequest; import org.apache.zookeeper.proto.Create2Response; import org.apache.zookeeper.proto.CreateResponse; +import org.apache.zookeeper.proto.ErrorResponse; import org.apache.zookeeper.proto.ExistsRequest; import org.apache.zookeeper.proto.ExistsResponse; import org.apache.zookeeper.proto.GetACLRequest; @@ -69,6 +72,7 @@ import org.apache.zookeeper.proto.SetACLResponse; import org.apache.zookeeper.proto.SetDataResponse; import org.apache.zookeeper.proto.SetWatches; +import org.apache.zookeeper.proto.SetWatches2; import org.apache.zookeeper.proto.SyncRequest; import org.apache.zookeeper.proto.SyncResponse; import org.apache.zookeeper.server.DataTree.ProcessTxnResult; @@ -365,7 +369,7 @@ public void processRequest(Request request) { case OpCode.setWatches: { lastOp = "SETW"; SetWatches setWatches = new SetWatches(); - // TODO We really should NOT need this!!!! + // TODO we really should not need this request.request.rewind(); ByteBufferInputStream.byteBuffer2Record(request.request, setWatches); long relativeZxid = setWatches.getRelativeZxid(); @@ -375,9 +379,36 @@ public void processRequest(Request request) { setWatches.getDataWatches(), setWatches.getExistWatches(), setWatches.getChildWatches(), + Collections.emptyList(), + Collections.emptyList(), cnxn); break; } + case OpCode.setWatches2: { + lastOp = "STW2"; + SetWatches2 setWatches = new SetWatches2(); + // TODO we really should not need this + request.request.rewind(); + ByteBufferInputStream.byteBuffer2Record(request.request, setWatches); + long relativeZxid = setWatches.getRelativeZxid(); + zks.getZKDatabase().setWatches(relativeZxid, + setWatches.getDataWatches(), + setWatches.getExistWatches(), + setWatches.getChildWatches(), + setWatches.getPersistentWatches(), + setWatches.getPersistentRecursiveWatches(), + cnxn); + break; + } + case OpCode.addWatch: { + lastOp = "ADDW"; + AddWatchRequest addWatcherRequest = new AddWatchRequest(); + ByteBufferInputStream.byteBuffer2Record(request.request, + addWatcherRequest); + zks.getZKDatabase().addWatch(addWatcherRequest.getPath(), cnxn, addWatcherRequest.getMode()); + rsp = new ErrorResponse(0); + break; + } case OpCode.getACL: { lastOp = "GETA"; GetACLRequest getACLRequest = new GetACLRequest(); diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java index c393684c4d8..70d989a34ac 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/PrepRequestProcessor.java @@ -797,10 +797,12 @@ protected void pRequest(Request request) throws RequestProcessorException { case OpCode.getChildren2: case OpCode.ping: case OpCode.setWatches: + case OpCode.setWatches2: case OpCode.checkWatches: case OpCode.removeWatches: case OpCode.getEphemerals: case OpCode.multiRead: + case OpCode.addWatch: zks.sessionTracker.checkSession(request.sessionId, request.getOwner()); break; default: diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java index bab21944628..122f0cab097 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java @@ -243,9 +243,11 @@ static boolean isValid(int type) { case OpCode.setACL: case OpCode.setData: case OpCode.setWatches: + case OpCode.setWatches2: case OpCode.sync: case OpCode.checkWatches: case OpCode.removeWatches: + case OpCode.addWatch: return true; default: return false; @@ -334,6 +336,8 @@ public static String op2String(int op) { return "auth"; case OpCode.setWatches: return "setWatches"; + case OpCode.setWatches2: + return "setWatches2"; case OpCode.sasl: return "sasl"; case OpCode.getEphemerals: @@ -364,6 +368,7 @@ public String toString() { String path = "n/a"; if (type != OpCode.createSession && type != OpCode.setWatches + && type != OpCode.setWatches2 && type != OpCode.closeSession && request != null && request.remaining() >= 4) { diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java index a753b8a8da4..c9c6d54cdb8 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZKDatabase.java @@ -516,10 +516,27 @@ public byte[] getData(String path, Stat stat, Watcher watcher) throws KeeperExce * @param dataWatches the data watches the client wants to reset * @param existWatches the exists watches the client wants to reset * @param childWatches the child watches the client wants to reset + * @param persistentWatches the persistent watches the client wants to reset + * @param persistentRecursiveWatches the persistent recursive watches the client wants to reset * @param watcher the watcher function */ - public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, Watcher watcher) { - dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, watcher); + public void setWatches(long relativeZxid, List dataWatches, List existWatches, List childWatches, + List persistentWatches, List persistentRecursiveWatches, Watcher watcher) { + dataTree.setWatches(relativeZxid, dataWatches, existWatches, childWatches, persistentWatches, persistentRecursiveWatches, watcher); + } + + /** + * Add a watch + * + * @param basePath + * watch base + * @param watcher + * the watcher + * @param mode + * a mode from ZooDefs.AddWatchModes + */ + public void addWatch(String basePath, Watcher watcher, int mode) { + dataTree.addWatch(basePath, watcher, mode); } /** diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java index 9ef430cef9f..f3ec1fcea92 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/RequestPathMetricsCollector.java @@ -32,6 +32,7 @@ import static org.apache.zookeeper.ZooDefs.OpCode.removeWatches; import static org.apache.zookeeper.ZooDefs.OpCode.setACL; import static org.apache.zookeeper.ZooDefs.OpCode.setData; +import static org.apache.zookeeper.ZooDefs.OpCode.setWatches2; import static org.apache.zookeeper.ZooDefs.OpCode.sync; import java.io.PrintWriter; import java.util.Arrays; @@ -131,6 +132,7 @@ public RequestPathMetricsCollector(boolean accurateMode) { requestsMap.put(Request.op2String(getChildren2), new PathStatsQueue(getChildren2)); requestsMap.put(Request.op2String(checkWatches), new PathStatsQueue(checkWatches)); requestsMap.put(Request.op2String(removeWatches), new PathStatsQueue(removeWatches)); + requestsMap.put(Request.op2String(setWatches2), new PathStatsQueue(setWatches2)); requestsMap.put(Request.op2String(sync), new PathStatsQueue(sync)); this.immutableRequestsMap = java.util.Collections.unmodifiableMap(requestsMap); } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java index 286c7db1f0b..1bc44c805a0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/IWatchManager.java @@ -34,6 +34,22 @@ public interface IWatchManager { */ boolean addWatch(String path, Watcher watcher); + /** + * Add watch to specific path. + * + * @param path znode path + * @param watcher watcher object reference + * @param watcherMode the watcher mode to use + * + * @return true if the watcher added is not already present + */ + default boolean addWatch(String path, Watcher watcher, WatcherMode watcherMode) { + if (watcherMode == WatcherMode.DEFAULT_WATCHER_MODE) { + return addWatch(path, watcher); + } + throw new UnsupportedOperationException(); // custom implementations must defeat this + } + /** * Checks the specified watcher exists for the given path. * @@ -129,4 +145,12 @@ public interface IWatchManager { */ void dumpWatches(PrintWriter pwriter, boolean byPath); + /** + * Return the current number of recursive watchers + * + * @return qty + */ + default int getRecursiveWatchQty() { + return 0; + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/PathParentIterator.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/PathParentIterator.java new file mode 100644 index 00000000000..a6aa8cd2542 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/PathParentIterator.java @@ -0,0 +1,106 @@ +/** + * 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.watch; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * Iterates over a ZooKeeper path. Each iteration goes up one parent path. Thus, the + * effect of the iterator is to iterate over the initial path and then all of its parents. + */ +public class PathParentIterator implements Iterator { + private String path; + private final int maxLevel; + private int level = -1; + + /** + * Return a new PathParentIterator that iterates from the + * given path to all parents. + * + * @param path initial path + */ + public static PathParentIterator forAll(String path) { + return new PathParentIterator(path, Integer.MAX_VALUE); + } + + /** + * Return a new PathParentIterator that only returns the given path - i.e. + * does not iterate to parent paths. + * + * @param path initial path + */ + public static PathParentIterator forPathOnly(String path) { + return new PathParentIterator(path, 0); + } + + private PathParentIterator(String path, int maxLevel) { + // NOTE: asserts that the path has already been validated + this.path = path; + this.maxLevel = maxLevel; + } + + /** + * Return an Iterable view so that this Iterator can be used in for each + * statements. IMPORTANT: the returned Iterable is single use only + * @return Iterable + */ + public Iterable asIterable() { + return () -> PathParentIterator.this; + } + + @Override + public boolean hasNext() { + return !path.isEmpty() && (level < maxLevel); + } + + /** + * Returns true if this iterator is currently at a parent path as opposed + * to the initial path given to the constructor + * + * @return true/false + */ + public boolean atParentPath() { + return level > 0; + } + + @Override + public String next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + String localPath = path; + ++level; + if (path.equals("/")) { + path = ""; + } else { + path = path.substring(0, path.lastIndexOf('/')); + if (path.length() == 0) { + path = "/"; + } + } + return localPath; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } +} \ No newline at end of file diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java index 39327fcaa7a..c5b133059b2 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatchManager.java @@ -21,6 +21,7 @@ import java.io.PrintWriter; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -42,9 +43,11 @@ public class WatchManager implements IWatchManager { private static final Logger LOG = LoggerFactory.getLogger(WatchManager.class); - private final Map> watchTable = new HashMap>(); + private final Map> watchTable = new HashMap<>(); - private final Map> watch2Paths = new HashMap>(); + private final Map> watch2Paths = new HashMap<>(); + + private final WatcherModeManager watcherModeManager = new WatcherModeManager(); @Override public synchronized int size() { @@ -55,12 +58,17 @@ public synchronized int size() { return result; } - boolean isDeadWatcher(Watcher watcher) { + private boolean isDeadWatcher(Watcher watcher) { return watcher instanceof ServerCnxn && ((ServerCnxn) watcher).isStale(); } @Override - public synchronized boolean addWatch(String path, Watcher watcher) { + public boolean addWatch(String path, Watcher watcher) { + return addWatch(path, watcher, WatcherMode.DEFAULT_WATCHER_MODE); + } + + @Override + public synchronized boolean addWatch(String path, Watcher watcher, WatcherMode watcherMode) { if (isDeadWatcher(watcher)) { LOG.debug("Ignoring addWatch with closed cnxn"); return false; @@ -71,7 +79,7 @@ public synchronized boolean addWatch(String path, Watcher watcher) { // don't waste memory if there are few watches on a node // rehash when the 4th entry is added, doubling size thereafter // seems like a good compromise - list = new HashSet(4); + list = new HashSet<>(4); watchTable.put(path, list); } list.add(watcher); @@ -79,9 +87,12 @@ public synchronized boolean addWatch(String path, Watcher watcher) { Set paths = watch2Paths.get(watcher); if (paths == null) { // cnxns typically have many watches, so use default cap here - paths = new HashSet(); + paths = new HashSet<>(); watch2Paths.put(watcher, paths); } + + watcherModeManager.setWatcherMode(watcher, path, watcherMode); + return paths.add(path); } @@ -99,6 +110,7 @@ public synchronized void removeWatcher(Watcher watcher) { watchTable.remove(p); } } + watcherModeManager.removeWatcher(watcher, p); } } @@ -110,22 +122,45 @@ public WatcherOrBitSet triggerWatch(String path, EventType type) { @Override public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet supress) { WatchedEvent e = new WatchedEvent(type, KeeperState.SyncConnected, path); - Set watchers; + Set watchers = new HashSet<>(); + PathParentIterator pathParentIterator = getPathParentIterator(path); synchronized (this) { - watchers = watchTable.remove(path); - if (watchers == null || watchers.isEmpty()) { - if (LOG.isTraceEnabled()) { - ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK, "No watchers for " + path); + for (String localPath : pathParentIterator.asIterable()) { + Set thisWatchers = watchTable.get(localPath); + if (thisWatchers == null || thisWatchers.isEmpty()) { + continue; } - return null; - } - for (Watcher w : watchers) { - Set paths = watch2Paths.get(w); - if (paths != null) { - paths.remove(path); + Iterator iterator = thisWatchers.iterator(); + while (iterator.hasNext()) { + Watcher watcher = iterator.next(); + WatcherMode watcherMode = watcherModeManager.getWatcherMode(watcher, localPath); + if (watcherMode.isRecursive()) { + if (type != EventType.NodeChildrenChanged) { + watchers.add(watcher); + } + } else if (!pathParentIterator.atParentPath()) { + watchers.add(watcher); + if (!watcherMode.isPersistent()) { + iterator.remove(); + Set paths = watch2Paths.get(watcher); + if (paths != null) { + paths.remove(localPath); + } + } + } + } + if (thisWatchers.isEmpty()) { + watchTable.remove(localPath); } } } + if (watchers.isEmpty()) { + if (LOG.isTraceEnabled()) { + ZooTrace.logTraceMessage(LOG, ZooTrace.EVENT_DELIVERY_TRACE_MASK, "No watchers for " + path); + } + return null; + } + for (Watcher w : watchers) { if (supress != null && supress.contains(w)) { continue; @@ -134,24 +169,24 @@ public WatcherOrBitSet triggerWatch(String path, EventType type, WatcherOrBitSet } switch (type) { - case NodeCreated: - ServerMetrics.getMetrics().NODE_CREATED_WATCHER.add(watchers.size()); - break; - - case NodeDeleted: - ServerMetrics.getMetrics().NODE_DELETED_WATCHER.add(watchers.size()); - break; - - case NodeDataChanged: - ServerMetrics.getMetrics().NODE_CHANGED_WATCHER.add(watchers.size()); - break; - - case NodeChildrenChanged: - ServerMetrics.getMetrics().NODE_CHILDREN_WATCHER.add(watchers.size()); - break; - default: - // Other types not logged. - break; + case NodeCreated: + ServerMetrics.getMetrics().NODE_CREATED_WATCHER.add(watchers.size()); + break; + + case NodeDeleted: + ServerMetrics.getMetrics().NODE_DELETED_WATCHER.add(watchers.size()); + break; + + case NodeDataChanged: + ServerMetrics.getMetrics().NODE_CHANGED_WATCHER.add(watchers.size()); + break; + + case NodeChildrenChanged: + ServerMetrics.getMetrics().NODE_CHILDREN_WATCHER.add(watchers.size()); + break; + default: + // Other types not logged. + break; } return new WatcherOrBitSet(watchers); @@ -197,8 +232,20 @@ public synchronized void dumpWatches(PrintWriter pwriter, boolean byPath) { @Override public synchronized boolean containsWatcher(String path, Watcher watcher) { - Set paths = watch2Paths.get(watcher); - return paths != null && paths.contains(path); + WatcherMode watcherMode = watcherModeManager.getWatcherMode(watcher, path); + PathParentIterator pathParentIterator = getPathParentIterator(path); + for (String localPath : pathParentIterator.asIterable()) { + Set watchers = watchTable.get(localPath); + if (!pathParentIterator.atParentPath()) { + if (watchers != null) { + return true; // at the leaf node, all watcher types match + } + } + if (watcherMode.isRecursive()) { + return true; + } + } + return false; } @Override @@ -217,15 +264,17 @@ public synchronized boolean removeWatcher(String path, Watcher watcher) { watchTable.remove(path); } + watcherModeManager.removeWatcher(watcher, path); + return true; } @Override public synchronized WatchesReport getWatches() { - Map> id2paths = new HashMap>(); + Map> id2paths = new HashMap<>(); for (Entry> e : watch2Paths.entrySet()) { Long id = ((ServerCnxn) e.getKey()).getSessionId(); - Set paths = new HashSet(e.getValue()); + Set paths = new HashSet<>(e.getValue()); id2paths.put(id, paths); } return new WatchesReport(id2paths); @@ -233,9 +282,9 @@ public synchronized WatchesReport getWatches() { @Override public synchronized WatchesPathReport getWatchesByPath() { - Map> path2ids = new HashMap>(); + Map> path2ids = new HashMap<>(); for (Entry> e : watchTable.entrySet()) { - Set ids = new HashSet(e.getValue().size()); + Set ids = new HashSet<>(e.getValue().size()); path2ids.put(e.getKey(), ids); for (Watcher watcher : e.getValue()) { ids.add(((ServerCnxn) watcher).getSessionId()); @@ -256,4 +305,15 @@ public synchronized WatchesSummary getWatchesSummary() { @Override public void shutdown() { /* do nothing */ } + @Override + public int getRecursiveWatchQty() { + return watcherModeManager.getRecursiveQty(); + } + + private PathParentIterator getPathParentIterator(String path) { + if (watcherModeManager.getRecursiveQty() == 0) { + return PathParentIterator.forPathOnly(path); + } + return PathParentIterator.forAll(path); + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherMode.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherMode.java new file mode 100644 index 00000000000..b8a1dda7408 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherMode.java @@ -0,0 +1,56 @@ +/** + * 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.watch; + +import org.apache.zookeeper.ZooDefs; + +public enum WatcherMode { + STANDARD(false, false), + PERSISTENT(true, false), + PERSISTENT_RECURSIVE(true, true) + ; + + public static final WatcherMode DEFAULT_WATCHER_MODE = WatcherMode.STANDARD; + + public static WatcherMode fromZooDef(int mode) { + switch (mode) { + case ZooDefs.AddWatchModes.persistent: + return PERSISTENT; + case ZooDefs.AddWatchModes.persistentRecursive: + return PERSISTENT_RECURSIVE; + } + throw new IllegalArgumentException("Unsupported mode: " + mode); + } + + private final boolean isPersistent; + private final boolean isRecursive; + + WatcherMode(boolean isPersistent, boolean isRecursive) { + this.isPersistent = isPersistent; + this.isRecursive = isRecursive; + } + + public boolean isPersistent() { + return isPersistent; + } + + public boolean isRecursive() { + return isRecursive; + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherModeManager.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherModeManager.java new file mode 100644 index 00000000000..c1a8225f8ae --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/watch/WatcherModeManager.java @@ -0,0 +1,96 @@ +/** + * 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.watch; + +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.zookeeper.Watcher; + +class WatcherModeManager { + private final Map watcherModes = new ConcurrentHashMap<>(); + private final AtomicInteger recursiveQty = new AtomicInteger(0); + + private static class Key { + private final Watcher watcher; + private final String path; + + Key(Watcher watcher, String path) { + this.watcher = watcher; + this.path = path; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Key key = (Key) o; + return watcher.equals(key.watcher) && path.equals(key.path); + } + + @Override + public int hashCode() { + return Objects.hash(watcher, path); + } + } + + // VisibleForTesting + Map getWatcherModes() { + return watcherModes; + } + + void setWatcherMode(Watcher watcher, String path, WatcherMode mode) { + if (mode == WatcherMode.DEFAULT_WATCHER_MODE) { + removeWatcher(watcher, path); + } else { + adjustRecursiveQty(watcherModes.put(new Key(watcher, path), mode), mode); + } + } + + WatcherMode getWatcherMode(Watcher watcher, String path) { + return watcherModes.getOrDefault(new Key(watcher, path), WatcherMode.DEFAULT_WATCHER_MODE); + } + + void removeWatcher(Watcher watcher, String path) { + adjustRecursiveQty(watcherModes.remove(new Key(watcher, path)), WatcherMode.DEFAULT_WATCHER_MODE); + } + + int getRecursiveQty() { + return recursiveQty.get(); + } + + // recursiveQty is an optimization to avoid having to walk the map every time this value is needed + private void adjustRecursiveQty(WatcherMode oldMode, WatcherMode newMode) { + if (oldMode == null) { + oldMode = WatcherMode.DEFAULT_WATCHER_MODE; + } + if (oldMode.isRecursive() != newMode.isRecursive()) { + if (newMode.isRecursive()) { + recursiveQty.incrementAndGet(); + } else { + recursiveQty.decrementAndGet(); + } + } + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/PathParentIteratorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/PathParentIteratorTest.java new file mode 100644 index 00000000000..59bb17adaa3 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/PathParentIteratorTest.java @@ -0,0 +1,84 @@ +/** + * 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.watch; + +import org.junit.Assert; +import org.junit.Test; + +public class PathParentIteratorTest { + @Test + public void testRoot() { + PathParentIterator pathParentIterator = PathParentIterator.forAll("/"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertFalse(pathParentIterator.atParentPath()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertFalse(pathParentIterator.hasNext()); + } + + @Test + public void test1Level() { + PathParentIterator pathParentIterator = PathParentIterator.forAll("/a"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertFalse(pathParentIterator.atParentPath()); + Assert.assertEquals(pathParentIterator.next(), "/a"); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } + + @Test + public void testLong() { + PathParentIterator pathParentIterator = PathParentIterator.forAll("/a/b/c/d"); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c/d"); + Assert.assertFalse(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/"); + Assert.assertTrue(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } + + @Test + public void testForPathOnly() { + PathParentIterator pathParentIterator = PathParentIterator.forPathOnly("/a/b/c/d"); + Assert.assertTrue(pathParentIterator.hasNext()); + Assert.assertEquals(pathParentIterator.next(), "/a/b/c/d"); + Assert.assertFalse(pathParentIterator.atParentPath()); + + Assert.assertFalse(pathParentIterator.hasNext()); + } +} \ No newline at end of file diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/RecursiveWatchQtyTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/RecursiveWatchQtyTest.java new file mode 100644 index 00000000000..067cb2af94a --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/RecursiveWatchQtyTest.java @@ -0,0 +1,197 @@ +/** + * 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.watch; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.junit.Before; +import org.junit.Test; + +public class RecursiveWatchQtyTest { + private WatchManager watchManager; + + private static final int clientQty = 25; + private static final int iterations = 1000; + + private static class DummyWatcher implements Watcher { + @Override + public void process(WatchedEvent event) { + // NOP + } + } + + @Before + public void setup() { + watchManager = new WatchManager(); + } + + @Test + public void testRecursiveQty() { + WatcherModeManager manager = new WatcherModeManager(); + DummyWatcher watcher = new DummyWatcher(); + manager.setWatcherMode(watcher, "/a", WatcherMode.DEFAULT_WATCHER_MODE); + assertEquals(0, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a", WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(1, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a/b", WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(2, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a", WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(2, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a/b", WatcherMode.PERSISTENT); + assertEquals(1, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a/b", WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(2, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a/b", WatcherMode.DEFAULT_WATCHER_MODE); + assertEquals(1, manager.getRecursiveQty()); + manager.setWatcherMode(watcher, "/a", WatcherMode.PERSISTENT); + assertEquals(0, manager.getRecursiveQty()); + } + + @Test + public void testAddRemove() { + Watcher watcher1 = new DummyWatcher(); + Watcher watcher2 = new DummyWatcher(); + + watchManager.addWatch("/a", watcher1, WatcherMode.PERSISTENT_RECURSIVE); + watchManager.addWatch("/b", watcher2, WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(2, watchManager.getRecursiveWatchQty()); + assertTrue(watchManager.removeWatcher("/a", watcher1)); + assertTrue(watchManager.removeWatcher("/b", watcher2)); + assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testAddRemoveAlt() { + Watcher watcher1 = new DummyWatcher(); + Watcher watcher2 = new DummyWatcher(); + + watchManager.addWatch("/a", watcher1, WatcherMode.PERSISTENT_RECURSIVE); + watchManager.addWatch("/b", watcher2, WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(2, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher1); + watchManager.removeWatcher(watcher2); + assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testDoubleAdd() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatcherMode.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a", watcher, WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(1, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher); + assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testSameWatcherMultiPath() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatcherMode.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a/b", watcher, WatcherMode.PERSISTENT_RECURSIVE); + watchManager.addWatch("/a/b/c", watcher, WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(3, watchManager.getRecursiveWatchQty()); + assertTrue(watchManager.removeWatcher("/a/b", watcher)); + assertEquals(2, watchManager.getRecursiveWatchQty()); + watchManager.removeWatcher(watcher); + assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testChangeType() { + Watcher watcher = new DummyWatcher(); + + watchManager.addWatch("/a", watcher, WatcherMode.PERSISTENT); + assertEquals(0, watchManager.getRecursiveWatchQty()); + watchManager.addWatch("/a", watcher, WatcherMode.PERSISTENT_RECURSIVE); + assertEquals(1, watchManager.getRecursiveWatchQty()); + watchManager.addWatch("/a", watcher, WatcherMode.STANDARD); + assertEquals(0, watchManager.getRecursiveWatchQty()); + assertTrue(watchManager.removeWatcher("/a", watcher)); + assertEquals(0, watchManager.getRecursiveWatchQty()); + } + + @Test + public void testRecursiveQtyConcurrency() { + ThreadLocalRandom random = ThreadLocalRandom.current(); + WatcherModeManager manager = new WatcherModeManager(); + ExecutorService threadPool = Executors.newFixedThreadPool(clientQty); + List> tasks = null; + CountDownLatch completedLatch = new CountDownLatch(clientQty); + try { + tasks = IntStream.range(0, clientQty) + .mapToObj(__ -> threadPool.submit(() -> iterate(manager, completedLatch))) + .collect(Collectors.toList()); + try { + completedLatch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } finally { + if (tasks != null) { + tasks.forEach(t -> t.cancel(true)); + } + threadPool.shutdownNow(); + } + + int expectedRecursiveQty = (int) manager.getWatcherModes().values() + .stream() + .filter(mode -> mode == WatcherMode.PERSISTENT_RECURSIVE) + .count(); + assertEquals(expectedRecursiveQty, manager.getRecursiveQty()); + } + + private void iterate(WatcherModeManager manager, CountDownLatch completedLatch) { + ThreadLocalRandom random = ThreadLocalRandom.current(); + try { + for (int i = 0; i < iterations; ++i) { + String path = "/" + random.nextInt(clientQty); + boolean doSet = random.nextInt(100) > 33; // 2/3 will be sets + if (doSet) { + WatcherMode mode = WatcherMode.values()[random.nextInt(WatcherMode.values().length)]; + manager.setWatcherMode(new DummyWatcher(), path, mode); + } else { + manager.removeWatcher(new DummyWatcher(), path); + } + + int sleepMillis = random.nextInt(2); + if (sleepMillis > 0) { + try { + Thread.sleep(sleepMillis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } finally { + completedLatch.countDown(); + } + } +} \ No newline at end of file diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java index 0ce0a59a0da..e29dab90649 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/watch/WatchManagerTest.java @@ -49,7 +49,7 @@ public class WatchManagerTest extends ZKTestCase { protected static final Logger LOG = LoggerFactory.getLogger(WatchManagerTest.class); - private static final String PATH_PREFIX = "path"; + private static final String PATH_PREFIX = "/path"; private ConcurrentHashMap watchers; private Random r; diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java new file mode 100644 index 00000000000..67f19dc0549 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentRecursiveWatcherTest.java @@ -0,0 +1,174 @@ +/** + * 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.test; + +import static org.apache.zookeeper.AddWatchMode.PERSISTENT_RECURSIVE; +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import org.apache.zookeeper.AsyncCallback; +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.ZooKeeper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PersistentRecursiveWatcherTest extends ClientBase { + private static final Logger LOG = LoggerFactory.getLogger(PersistentRecursiveWatcherTest.class); + private BlockingQueue events; + private Watcher persistentWatcher; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + + events = new LinkedBlockingQueue<>(); + persistentWatcher = event -> events.add(event); + } + + @Test + public void testBasic() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT_RECURSIVE); + internalTestBasic(zk); + } + } + + @Test + public void testBasicAsync() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.VoidCallback cb = (rc, path, ctx) -> { + if (rc == 0) { + latch.countDown(); + } + }; + zk.addWatch("/a/b", persistentWatcher, PERSISTENT_RECURSIVE, cb, null); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + internalTestBasic(zk); + } + } + + private void internalTestBasic(ZooKeeper zk) throws KeeperException, InterruptedException { + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b/c/d/e", new byte[0], -1); + zk.delete("/a/b/c/d/e", -1); + zk.create("/a/b/c/d/e", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b/c/d/e"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c/d/e"); + } + + @Test + public void testRemoval() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT_RECURSIVE); + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b/c"); + + zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); + zk.create("/a/b/c/d", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); + } + } + + @Test + public void testDisconnect() throws Exception { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT_RECURSIVE); + stopServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + startServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + internalTestBasic(zk); + } + } + + @Test + public void testMultiClient() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk1 = createClient(new CountdownWatcher(), hostPort); ZooKeeper zk2 = createClient(new CountdownWatcher(), hostPort)) { + + zk1.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + zk1.addWatch("/a/b", persistentWatcher, PERSISTENT_RECURSIVE); + zk1.setData("/a/b/c", "one".getBytes(), -1); + Thread.sleep(1000); // give some time for the event to arrive + + zk2.setData("/a/b/c", "two".getBytes(), -1); + zk2.setData("/a/b/c", "three".getBytes(), -1); + zk2.setData("/a/b/c", "four".getBytes(), -1); + + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b/c"); + } + } + + @Test + public void testRootWatcher() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/", persistentWatcher, PERSISTENT_RECURSIVE); + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/b/c"); + } + } + + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) + throws InterruptedException { + WatchedEvent event = events.poll(5, TimeUnit.SECONDS); + Assert.assertNotNull(event); + Assert.assertEquals(eventType, event.getType()); + Assert.assertEquals(path, event.getPath()); + } +} \ No newline at end of file diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherTest.java new file mode 100644 index 00000000000..bffa8e0a2db --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/PersistentWatcherTest.java @@ -0,0 +1,211 @@ +/** + * 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.test; + +import static org.apache.zookeeper.AddWatchMode.PERSISTENT; +import java.io.IOException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import org.apache.zookeeper.AsyncCallback; +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.ZooKeeper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PersistentWatcherTest extends ClientBase { + private static final Logger LOG = LoggerFactory.getLogger(PersistentWatcherTest.class); + private BlockingQueue events; + private Watcher persistentWatcher; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + + events = new LinkedBlockingQueue<>(); + persistentWatcher = event -> events.add(event); + } + + @Test + public void testBasic() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT); + internalTestBasic(zk); + } + } + + @Test + public void testDefaultWatcher() + throws IOException, InterruptedException, KeeperException { + CountdownWatcher watcher = new CountdownWatcher() { + @Override + public synchronized void process(WatchedEvent event) { + super.process(event); + events.add(event); + } + }; + try (ZooKeeper zk = createClient(watcher, hostPort)) { + zk.addWatch("/a/b", PERSISTENT); + events.clear(); // clear any events added during client connection + internalTestBasic(zk); + } + } + + @Test + public void testBasicAsync() + throws IOException, InterruptedException, KeeperException { + CountdownWatcher watcher = new CountdownWatcher() { + @Override + public synchronized void process(WatchedEvent event) { + super.process(event); + events.add(event); + } + }; + try (ZooKeeper zk = createClient(watcher, hostPort)) { + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.VoidCallback cb = (rc, path, ctx) -> { + if (rc == 0) { + latch.countDown(); + } + }; + zk.addWatch("/a/b", persistentWatcher, PERSISTENT, cb, null); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + events.clear(); // clear any events added during client connection + internalTestBasic(zk); + } + } + + @Test + public void testAsyncDefaultWatcher() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + final CountDownLatch latch = new CountDownLatch(1); + AsyncCallback.VoidCallback cb = (rc, path, ctx) -> { + if (rc == 0) { + latch.countDown(); + } + }; + zk.addWatch("/a/b", persistentWatcher, PERSISTENT, cb, null); + Assert.assertTrue(latch.await(5, TimeUnit.SECONDS)); + internalTestBasic(zk); + } + } + + private void internalTestBasic(ZooKeeper zk) throws KeeperException, InterruptedException { + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a/b", new byte[0], -1); + zk.delete("/a/b/c", -1); + zk.delete("/a/b", -1); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDeleted, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + } + + @Test + public void testRemoval() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT); + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.create("/a/b/c", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeCreated, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/a/b"); + + zk.removeWatches("/a/b", persistentWatcher, Watcher.WatcherType.Any, false); + zk.delete("/a/b/c", -1); + zk.delete("/a/b", -1); + assertEvent(events, Watcher.Event.EventType.PersistentWatchRemoved, "/a/b"); + } + } + + @Test + public void testDisconnect() throws Exception { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/a/b", persistentWatcher, PERSISTENT); + stopServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + startServer(); + assertEvent(events, Watcher.Event.EventType.None, null); + internalTestBasic(zk); + } + } + + @Test + public void testMultiClient() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk1 = createClient(new CountdownWatcher(), hostPort); + ZooKeeper zk2 = createClient(new CountdownWatcher(), hostPort)) { + + zk1.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk1.create("/a/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + + zk1.addWatch("/a/b", persistentWatcher, PERSISTENT); + zk1.setData("/a/b", "one".getBytes(), -1); + Thread.sleep(1000); // give some time for the event to arrive + + zk2.setData("/a/b", "two".getBytes(), -1); + zk2.setData("/a/b", "three".getBytes(), -1); + zk2.setData("/a/b", "four".getBytes(), -1); + + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + assertEvent(events, Watcher.Event.EventType.NodeDataChanged, "/a/b"); + } + } + + @Test + public void testRootWatcher() + throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(new CountdownWatcher(), hostPort)) { + zk.addWatch("/", persistentWatcher, PERSISTENT); + zk.create("/a", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + zk.setData("/a", new byte[0], -1); + zk.create("/b", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/"); + assertEvent(events, Watcher.Event.EventType.NodeChildrenChanged, "/"); + } + } + + private void assertEvent(BlockingQueue events, Watcher.Event.EventType eventType, String path) + throws InterruptedException { + WatchedEvent event = events.poll(5, TimeUnit.SECONDS); + Assert.assertNotNull(event); + Assert.assertEquals(eventType, event.getType()); + Assert.assertEquals(path, event.getPath()); + } +} \ No newline at end of file diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java new file mode 100644 index 00000000000..95b5569bb08 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/test/UnsupportedAddWatcherTest.java @@ -0,0 +1,124 @@ +/* + * 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.test; + +import java.io.IOException; +import java.io.PrintWriter; +import java.util.Collections; +import org.apache.zookeeper.AddWatchMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.server.watch.IWatchManager; +import org.apache.zookeeper.server.watch.WatchManagerFactory; +import org.apache.zookeeper.server.watch.WatcherOrBitSet; +import org.apache.zookeeper.server.watch.WatchesPathReport; +import org.apache.zookeeper.server.watch.WatchesReport; +import org.apache.zookeeper.server.watch.WatchesSummary; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class UnsupportedAddWatcherTest extends ClientBase { + + public static class StubbedWatchManager implements IWatchManager { + @Override + public boolean addWatch(String path, Watcher watcher) { + return false; + } + + @Override + public boolean containsWatcher(String path, Watcher watcher) { + return false; + } + + @Override + public boolean removeWatcher(String path, Watcher watcher) { + return false; + } + + @Override + public void removeWatcher(Watcher watcher) { + // NOP + } + + @Override + public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type) { + return new WatcherOrBitSet(Collections.emptySet()); + } + + @Override + public WatcherOrBitSet triggerWatch(String path, Watcher.Event.EventType type, WatcherOrBitSet suppress) { + return new WatcherOrBitSet(Collections.emptySet()); + } + + @Override + public int size() { + return 0; + } + + @Override + public void shutdown() { + // NOP + } + + @Override + public WatchesSummary getWatchesSummary() { + return null; + } + + @Override + public WatchesReport getWatches() { + return null; + } + + @Override + public WatchesPathReport getWatchesByPath() { + return null; + } + + @Override + public void dumpWatches(PrintWriter pwriter, boolean byPath) { + // NOP + } + } + + @Before + public void setUp() throws Exception { + System.setProperty(WatchManagerFactory.ZOOKEEPER_WATCH_MANAGER_NAME, StubbedWatchManager.class.getName()); + super.setUp(); + } + + @After + public void tearDown() throws Exception { + try { + super.tearDown(); + } finally { + System.clearProperty(WatchManagerFactory.ZOOKEEPER_WATCH_MANAGER_NAME); + } + } + + @Test(expected = KeeperException.MarshallingErrorException.class) + public void testBehavior() throws IOException, InterruptedException, KeeperException { + try (ZooKeeper zk = createClient(hostPort)) { + // the server will generate an exception as our custom watch manager doesn't implement + // the new version of addWatch() + zk.addWatch("/foo", event -> {}, AddWatchMode.PERSISTENT_RECURSIVE); + } + } +} From 79f99af81842f415b97e1c3c18c953df5bd129b2 Mon Sep 17 00:00:00 2001 From: Fangmin Lyu Date: Mon, 11 Nov 2019 13:55:55 +0100 Subject: [PATCH 3/4] ZOOKEEPER-3598: Fix potential data inconsistency issue due to CommitProcessor not gracefully shutdown Note: use exit code 16 for SHUTDOWN_UNGRACEFULLY, since internally we've already using 15 for other exit code, which will be upstreamed later. Author: Fangmin Lyu Reviewers: Enrico Olivelli , Michael Han Closes #1130 from lvfangmin/ZOOKEEPER-3598 --- .../org/apache/zookeeper/server/ExitCode.java | 5 +- .../server/quorum/CommitProcessor.java | 15 ++++ .../server/quorum/CommitProcessorTest.java | 80 ++++++++++++++++++- 3 files changed, 97 insertions(+), 3 deletions(-) diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ExitCode.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ExitCode.java index 67af2c8df8f..810be278b83 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ExitCode.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ExitCode.java @@ -48,7 +48,10 @@ public enum ExitCode { QUORUM_PACKET_ERROR(13), /** Unable to bind to the quorum (election) port after multiple retry */ - UNABLE_TO_BIND_QUORUM_PORT(14); + UNABLE_TO_BIND_QUORUM_PORT(14), + + /** Failed to shutdown the request processor pipeline gracefully **/ + SHUTDOWN_UNGRACEFULLY(16); private final int value; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java index 4e521874b6e..8044e657ea7 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/CommitProcessor.java @@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.zookeeper.ZooDefs.OpCode; import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.server.ExitCode; import org.apache.zookeeper.server.Request; import org.apache.zookeeper.server.RequestProcessor; import org.apache.zookeeper.server.ServerMetrics; @@ -621,6 +622,20 @@ public void shutdown() { workerPool.join(workerShutdownTimeoutMS); } + try { + this.join(workerShutdownTimeoutMS); + } catch (InterruptedException e) { + LOG.warn("Interrupted while waiting for CommitProcessor to finish"); + Thread.currentThread().interrupt(); + } + + if (this.isAlive()) { + LOG.warn("CommitProcessor does not shutdown gracefully after " + + "waiting for {} ms, exit to avoid potential " + + "inconsistency issue", workerShutdownTimeoutMS); + System.exit(ExitCode.SHUTDOWN_UNGRACEFULLY.getValue()); + } + if (nextProcessor != null) { nextProcessor.shutdown(); } diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java index d939dc0eb86..e83ed73bad4 100644 --- a/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/quorum/CommitProcessorTest.java @@ -28,7 +28,9 @@ import java.util.ArrayList; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.jute.BinaryOutputArchive; @@ -82,13 +84,18 @@ public class CommitProcessorTest extends ZKTestCase { File tmpDir; ArrayList testClients = new ArrayList(); CommitProcessor commitProcessor; + DelayRequestProcessor delayProcessor; public void setUp(int numCommitThreads, int numClientThreads, int writePercent) throws Exception { + setUp(numCommitThreads, numClientThreads, writePercent, false); + } + + public void setUp(int numCommitThreads, int numClientThreads, int writePercent, boolean withDelayProcessor) throws Exception { stopped = false; System.setProperty(CommitProcessor.ZOOKEEPER_COMMIT_PROC_NUM_WORKER_THREADS, Integer.toString(numCommitThreads)); tmpDir = ClientBase.createTmpDir(); ClientBase.setupTestEnv(); - zks = new TestZooKeeperServer(tmpDir, tmpDir, 4000); + zks = new TestZooKeeperServer(tmpDir, tmpDir, 4000, withDelayProcessor); zks.startup(); for (int i = 0; i < numClientThreads; ++i) { TestClientThread client = new TestClientThread(writePercent); @@ -211,6 +218,23 @@ public void testNoCommitWorkersReadOnlyWorkload() throws Exception { assertTrue("Write requests processed", processedWriteRequests.get() == numClients); } + @Test + public void testWaitingForWriteToFinishBeforeShutdown() throws Exception { + setUp(1, 0, 0, true); + + // send a single write request + TestClientThread client = new TestClientThread(0); + client.sendWriteRequest(); + + // wait for request being committed + delayProcessor.waitRequestProcessing(); + + zks.shutdown(); + + // Make sure we've finished the in-flight request before shutdown returns + assertFalse(commitProcessor.isAlive()); + } + @Test public void testNoCommitWorkersMixedWorkload() throws Exception { int numClients = 10; @@ -287,8 +311,15 @@ private synchronized void failTest(String reason) { private class TestZooKeeperServer extends ZooKeeperServer { + final boolean withDelayProcessor; + public TestZooKeeperServer(File snapDir, File logDir, int tickTime) throws IOException { + this(snapDir, logDir, tickTime, false); + } + + public TestZooKeeperServer(File snapDir, File logDir, int tickTime, boolean withDelayProcessor) throws IOException { super(snapDir, logDir, tickTime); + this.withDelayProcessor = withDelayProcessor; } public PrepRequestProcessor getFirstProcessor() { @@ -303,7 +334,12 @@ protected void setupRequestProcessors() { // ValidateProcessor is set up in a similar fashion to ToBeApplied // processor, so it can do pre/post validating of requests ValidateProcessor validateProcessor = new ValidateProcessor(finalProcessor); - commitProcessor = new CommitProcessor(validateProcessor, "1", true, null); + if (withDelayProcessor) { + delayProcessor = new DelayRequestProcessor(validateProcessor); + commitProcessor = new CommitProcessor(delayProcessor, "1", true, null); + } else { + commitProcessor = new CommitProcessor(validateProcessor, "1", true, null); + } validateProcessor.setCommitProcessor(commitProcessor); commitProcessor.start(); MockProposalRequestProcessor proposalProcessor = new MockProposalRequestProcessor(commitProcessor); @@ -314,6 +350,46 @@ protected void setupRequestProcessors() { } + private class DelayRequestProcessor implements RequestProcessor { + // delay 1s for each request + static final int DEFAULT_DELAY = 1000; + RequestProcessor nextProcessor; + CountDownLatch waitingProcessRequestBeingCalled; + + public DelayRequestProcessor(RequestProcessor nextProcessor) { + this.nextProcessor = nextProcessor; + this.waitingProcessRequestBeingCalled = new CountDownLatch(1); + } + + @Override + public void processRequest(Request request) throws RequestProcessorException { + try { + this.waitingProcessRequestBeingCalled.countDown(); + LOG.info("Sleeping {} ms for request {}", DEFAULT_DELAY, request); + Thread.sleep(DEFAULT_DELAY); + } catch (InterruptedException e) { /* ignore */ } + nextProcessor.processRequest(request); + } + + public void waitRequestProcessing() { + try { + if (!waitingProcessRequestBeingCalled.await(3000, TimeUnit.MILLISECONDS)) { + LOG.info("Did not see request processing in 3s"); + } + } catch (InterruptedException e) { + LOG.info("Interrupted when waiting for processRequest being called"); + } + } + + @Override + public void shutdown() { + LOG.info("shutdown DelayRequestProcessor"); + if (nextProcessor != null) { + nextProcessor.shutdown(); + } + } + } + private class MockProposalRequestProcessor extends Thread implements RequestProcessor { private final CommitProcessor commitProcessor; From 794adf11220151496a1ff877df3d2627719c9505 Mon Sep 17 00:00:00 2001 From: Mohammad Arshad Date: Mon, 11 Nov 2019 13:59:08 +0100 Subject: [PATCH 4/4] ZOOKEEPER-1260: Audit logging in ZooKeeper servers. Author: Mohammad Arshad Reviewers: Enrico Olivelli , Andor Molnar Closes #1133 from arshadmohammad/ZOOKEEPER-1260-AuditLog-master --- conf/log4j.properties | 17 + .../main/resources/markdown/html/header.html | 3 + .../resources/markdown/images/zkAuditLogs.jpg | Bin 0 -> 28450 bytes .../src/main/resources/markdown/index.md | 1 + .../main/resources/markdown/zookeeperAdmin.md | 12 + .../resources/markdown/zookeeperAuditLogs.md | 129 +++++ .../main/java/org/apache/zookeeper/Login.java | 8 + .../java/org/apache/zookeeper/ZKUtil.java | 46 +- .../zookeeper/audit/AuditConstants.java | 34 ++ .../apache/zookeeper/audit/AuditEvent.java | 98 ++++ .../apache/zookeeper/audit/AuditHelper.java | 232 +++++++++ .../apache/zookeeper/audit/AuditLogger.java | 34 ++ .../zookeeper/audit/Log4jAuditLogger.java | 38 ++ .../zookeeper/audit/ZKAuditProvider.java | 133 ++++++ .../apache/zookeeper/cli/GetAclCommand.java | 25 +- .../org/apache/zookeeper/server/DataTree.java | 24 +- .../server/FinalRequestProcessor.java | 4 +- .../org/apache/zookeeper/server/Request.java | 28 ++ .../apache/zookeeper/server/ServerCnxn.java | 22 + .../zookeeper/server/ServerCnxnFactory.java | 15 + .../zookeeper/server/ZooKeeperServerMain.java | 7 + .../server/auth/AuthenticationProvider.java | 13 + .../auth/DigestAuthenticationProvider.java | 9 + .../server/quorum/QuorumPeerMain.java | 7 + .../zookeeper/server/util/AuthUtil.java | 39 ++ .../zookeeper/audit/AuditEventTest.java | 45 ++ .../zookeeper/audit/AuditLogPerfReading.java | 74 +++ .../zookeeper/audit/Log4jAuditLoggerTest.java | 442 ++++++++++++++++++ .../audit/ZKAuditLoggerPerformance.java | 150 ++++++ .../zookeeper/server/util/AuthUtilTest.java | 71 +++ 30 files changed, 1731 insertions(+), 29 deletions(-) create mode 100644 zookeeper-docs/src/main/resources/markdown/images/zkAuditLogs.jpg create mode 100644 zookeeper-docs/src/main/resources/markdown/zookeeperAuditLogs.md create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditConstants.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditEvent.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditLogger.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/Log4jAuditLogger.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/audit/ZKAuditProvider.java create mode 100644 zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AuthUtil.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditEventTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditLogPerfReading.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/audit/Log4jAuditLoggerTest.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/audit/ZKAuditLoggerPerformance.java create mode 100644 zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AuthUtilTest.java diff --git a/conf/log4j.properties b/conf/log4j.properties index 4a2ede95503..9e12a38d333 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -63,3 +63,20 @@ log4j.appender.TRACEFILE.File=${zookeeper.tracelog.dir}/${zookeeper.tracelog.fil log4j.appender.TRACEFILE.layout=org.apache.log4j.PatternLayout ### Notice we are including log4j's NDC here (%x) log4j.appender.TRACEFILE.layout.ConversionPattern=%d{ISO8601} [myid:%X{myid}] - %-5p [%t:%C{1}@%L][%x] - %m%n +# +# zk audit logging +# +zookeeper.auditlog.file=zookeeper_audit.log +zookeeper.auditlog.threshold=INFO +audit.logger=INFO, RFAAUDIT +log4j.logger.org.apache.zookeeper.audit.Log4jAuditLogger=${audit.logger} +log4j.additivity.org.apache.zookeeper.audit.Log4jAuditLogger=false +log4j.appender.RFAAUDIT=org.apache.log4j.RollingFileAppender +log4j.appender.RFAAUDIT.File=${zookeeper.log.dir}/${zookeeper.auditlog.file} +log4j.appender.RFAAUDIT.layout=org.apache.log4j.PatternLayout +log4j.appender.RFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n +log4j.appender.RFAAUDIT.Threshold=${zookeeper.auditlog.threshold} + +# Max log file size of 10MB +log4j.appender.RFAAUDIT.MaxFileSize=10MB +log4j.appender.RFAAUDIT.MaxBackupIndex=10 diff --git a/zookeeper-docs/src/main/resources/markdown/html/header.html b/zookeeper-docs/src/main/resources/markdown/html/header.html index 306de2dadcb..ce82ecea565 100644 --- a/zookeeper-docs/src/main/resources/markdown/html/header.html +++ b/zookeeper-docs/src/main/resources/markdown/html/header.html @@ -104,6 +104,9 @@

+ diff --git a/zookeeper-docs/src/main/resources/markdown/images/zkAuditLogs.jpg b/zookeeper-docs/src/main/resources/markdown/images/zkAuditLogs.jpg new file mode 100644 index 0000000000000000000000000000000000000000..fd6c376561017daf00514803e6dfb82fdf315553 GIT binary patch literal 28450 zcmeFZ1yoht*EhN;0Vx4VDG8D8jsq$sO7|fIq`M^#p`fIIfS?l6E!`l}-QC^YbcM$Yhn@gWm4FCV{vppfw6 zCsNWfvT{#V)zmdKwX}7NUz(VjnOj&oIlH*Jy>j;mc>N|Y=c6TYUV zrDtSjW#<%^l$MoOR9030Xlib0ZENr792^=R866v+m|R#~T3%UQTi@6`I6OK&IXyeS zxctc%5`gj-TmRPe4g8OSkBWAm>lT5eGPf$;E9G??Dy&vI#_U^vAW-iw}((ZGMjcWUH1Rz^Ec@|!2O7WU{#=y?! zyki?ZRU^m$=swE_qal}j$>eebojlhfPoj*c*DRB+HuE`x@*S}&NQ#p@19cI=oM680 zK*BL&!C zm~>7FK>#E&*K`SXH{!5)X9RGqXK5Nb(0%3i|E)pV9>+dZ`?HB0WlXk<72FtSl#i5< zXd*`?H}s^f-koPYh$9IIt!}npNOQ`)YwN~8`W>YzrK2D@GRbOj4n=>OtuB9MdWhdO z%_F3Q!l_?ghuUs`?|OfvVV~OoE}~DFvazI4+;9**`54vzfhP;SpqlOru3bk+_AZwz zrKjFt#y~+~8AchFw=&i}=Qm0%-{~wU9U?zpQs;}CP2;`1f(60W0`n1oUlam3m@G&@ z0I&6S@eP#NY~Zz67Z5l2+1{&lkFjvZccPNGF(!*bE*K1D)m4Y#iQoLn9A$8Z6CpP) zVTJNOd<}PK>6_#P9PVA*Z{7dm+qX1Tz$3$3g8+13AE_?P3eM$~f_h7Aq!(Ft=7^3% zc{U>1wEpq>LmUHWY)7d#UDGqe(n{aX66n;sPv*VIxpy$<*F=xhULLr;)gMzVdU$$P z(nBkHFqId@ouzWSRe`4u@>D2Enr_NxZei|XAs06PvfpOhoQwkjP!^Ji zGaitP=e=+X#;W4OD~`{23v5a>5pBCK-?mu`aj7GwG-`EUsH-WBq7PnvLbA7fm>LdA z%4av5yM%x2v-}6G;=jMC778a+8u%yk@W@1M&1Hr;1g#A(xs@eq@AEnr74-0voB zQY%TVI@lt!IfpjFvNc_%Yi}-kONM6`{VAMlpWdy(*t4sp5~sA)1(W;U(2bpe<U2I@HWtj6`)OlgVEGGM-p-;k@`MTZ zu0&I}k&i)EYW$SFXX4Z;0>DK8eK=51Zy6B4QmJ=CQ^Qe{2B_5?u=5>vqWY3WO5^4` z<5Sa*jRJc`;$yzzsx8~1h&Obm6dy6*5{?(;wZOz-0 zE*rZaaantoE=(~Lsib>pG@P4wdmh6G;16+Sv??@!THL}yva)h;&lCIhsSC-N!fSk8 zaWro;k!GyaiphFTskpAjst`55I{&F&$(2N<(WRs@D86O-CLSlOv|qyZDX-N zrlG%miaoUI-E4vYeh6hCfT1-wx*`H-8Tg0k(uq&ytx9r;hwX=3G#p=m3Ni#Lh%@Yu zNuwF&JrsSFWdHx5|IJGv5R`ZHakVyL!*K@zq!9kxyo>Z{Y$oSVSN1xyE4KZxE};)} zNvXJygMkwwR8>r}$jHSQ@oN=qZUk^rDH@LB9G~NMia$sD`NF9!iTMS$=F~LRQo}GZ zU#7mY&uP|kGG{gW_MUnnCHjLe#WX^Vr(%tpcqu!BX$8wenor(kwz8K6xsrwVNY5#9 z`K@h|Ab`jD+oRI{kO6jjx?A$`cnXywkW-!E9``ba9{$e9N75GYseCBZ2|UVjtPboO zl*-;(^Xjl25y0NgreSXr0fartdBi>#WxxkFR*s$^agw%-JN*)LjR2sqIZ)ooA*I`p zwaY0uyzqSZ%EO?${seBk9A|dMy=&g10w4N@0Ft)st}YvvWd4{YR>!ew;DG=*|6e=y zDW3>|J%`OZ1W=F*=k@%HHNy_}zFl_H&|%BXO)^sM#Mcx7@(E*(GOJa4WUut+S|)it zT)LGz$a{--++LC)1*xyq*_np+qsLBM-y6zf7rpWnk&3qB=n*Zzcp-_(FWLSeDbwqI z$zY+>N*Zylus>0P#XVDxdHfa+AM+l--5G+;%639jA3oY+s(EzRz1Ujg1S`}hCWX2Q z#thvcViPccwe@aLzW$8gPKV;}p_#mz5_>p_^3Ikv^isr6t~55qCo{gGu?D>;ynAZO zolEEy0vKfIBpgG}LZ=i~i}Io^j(Z;1m~Z|1PWk$2`$9~FY9;payxTU*z^{!x{k5?J zH<^YA08&18b^>dSY>YQ6A7afTQ3!6G-Ri9jTX|1*ci;WCoZty(6X!fgY(h$7+bS{d%vVO@zi0S{He6o)iBgV01>n4+XZBO9;8VW|g#FnmO?i)jk{JM zbm8M|hLbgaFbK04hxP4L+}s!(Wd0)WxB+t~V<`RGD>j_k60!-e37%D|JX~(bZ6hz* zlIQHtMYcyxXJuv=(i~gr+>C{`=IT5c{{wypS0n6K^?J=3eb6fQy@wB>Qpl|BkBe(W zirVQIYFw_jN_y18FGn@7D7)DeC5lm0S98wLYy*B70A6fR+IV*h{+@?PLD`$%O78u~w37in`8 zI;~JicN>pX?QcQ-Lvy1Kjlec-K$+ zZoH;p>udSXMiu;Fi(0U))g=TVBWAH$xl;4c$0F)!(UU`g87WpD|IiVeCkUXqcn@CL z0Qa{Nj&gXNR?T?9)uJy)y1nBFcV2Qzln{MQ6>xndpAg}4FZW0|hCxNa$<6wb4~^)h z9B9gSQ(Yvs|NaHZ=-I%}VOdxNpxN@wnvu|o<~2>7k={gZ22DTJrhpPLgTIr>!x+KP z$km7IGKIsMQVGku&>LcF>Up~rJq4?+O2^U_1?Q;&ojH+24MWtlwCac+vJBJEu|cX= z;4Ybhrb@FeKCUZ}pLsAkmhw?}<=HxfJXmn)tr4&K35}yqed$zd%T2?LWAsJ(gUz+i zb*qK~dxW;BE|06jMns#ZKe^n?p&YhN{mA|tyn(wZpxjOJ03yt-K>)fhLGg+D zY_doD^@NY-m$MtRh47ljC-6w%{o;q3b=}4Rt5QaL zs_fW+8j`w+Bj3B$S68A2mv>_rG%Iz>D4S3L(-Jx8=Y{dP6*qROm0 zwL53ltd7d8ju`8|{1xj!lYM0@AMB9~-MXjBc<)MjMPvvsy3hhs&QuxR(sT;;`l6cQPww1tztW(K_u zwq1*u8NweDy9M6Tf2xfXpoVJqF*AOT?~J!unJs{05_&h{Ge>%rJS1AKRHDQERcV?mY*M!!WePMo zmW=$NzRR&)lj;G59-d(yX)7LS662?Sg$3{t2ihPDR6}&pd8BdYliaJ^aQWEhcE(y) z9o&CuTaL`X)eh4hHUNhC_6UG^(@FPBBLa|)yJ@=4&FS&h#DR6q#cjH4Csc5ByPEe1 z@S4`A9k!W<4mnAiBZnQdoHX1xfhxn}LhgqEo>hRMh$Dx!Q$P)x zHKW`u)gkZ8tpFqo($OrFFN@mUo^8y~zBtWgjq`1@^9Z%seJd>vjoH}bNmR49`WT>D zJNTvaIeSYUJHzlZr#53&)ihYu+*{F}th6NtXr>@KZI1BQ{FKe9tQmc$%+1+{W2nUY zn_Swx<#D*(w84segR10&{+$gwfFg~bdd5L{5zLHo#P`)=g1i+00WSw-6O+5c>YXZT zW}uCXvGA#@VP~HrsJBO99J&X%={XyhR&ya<&z(zTjoz$8(sJF&yHm_^y`li2afo;{ z^HlSZhq*i`M#La{N4J-2Kn_6@K+))^H(b8+MFDcThyX4L)I_pJ^S{VJwn7j9Z%#mM zq>|4eUx~CuRVv@K3ofYKtD*fUzpiuy_3%h1o&c0t$Y)PL&q(1(!NXDQNJzGBbsZ3P~l|-&YYBjkJz|e2_2nN zaa~TM6$ACPBYbm&2HJX-N;54a7m#cg@vQ_v3w2J=(Gx4F%KG4kv_*2O z;iLoCr`%8Dr!jA1NXVid^`G?Ft*W!BcX)^pgYVC|_~0I+`?bo-``%_@yiB~zL*26a zFPY^oEQOa#Bu7!%7ct_o6T|EgH{2l?btc-Lp@W~!B8w7up7)Rypk)0er@oZGu9I); zY#@MOPgQVjRkKsOJJ&@1zUz)`tD`bp-H0O`Kt~fd}s|*L7Wc z+4XOCmTC9Zf|`GUGrVy$1Ih2ennCnrdBhIZR&#L3hPnxx?$UF)u@T0O2&SVH&VBgnf-i$@5pgKU8&l%SK7av>M#4JTEO7M}&Bs-V zP1%jcoQ82>fm6Fvw^Q}F@e123$om=rkDXSYJxu}S>Ld|r&T-ZqwkJljqZnS5*J+1K zSY^UHr0ZhYQ5Mx+<&F~_#@3@lT<-U|*xQU8S)Sh2DBi>^Oh2ArN za=YtJID-KzR~Op%C3@`3k-A^ryT`tYA6yNQtQi?j-QS&iT8YqbH0U;tbaJaRimW$T zer%$BXtF7e@rD2^Cum0_`XRu1o~Cvq+*X7xP{_9l_7Dh(X; zX77|Yhm1VhXL@pp{X!j_1AgLYw<)?KY5nR#oIwn1#t~?u~ zQ93gI?|GS(xrVSpZ^w0?rGlSG&Q(gBviy=;{*$C1rHhaw-`pqpdNBTS-gEeBvm$;^ zCVo%x1H1N|^yOdC!ayep>t3q2#*x@M_*ubIKwwN^Bur+FM0}SPeS%a=QKpE6WZP+oP zil#nEHaE+9+YJ%|_&eq_H!X1vfcyVj2MbrLOu!h_lH~*UWDn`)x|$E|mjoWwhuKms zC*-bfLcY^Zjk{t@PEZmZYfapj(p*)W677&T8t_)w#iha32?ChV{4v_j_5PERZdcvd zrZ z&^g3PzD5Ky7Ar0WYHyVC4iI1#c*rP9c+#XzEL7tNU44B-!M%0PlLwkI^zS|#)nm(5 zpY(>UtzZ1eV_At&X->>jKz{~683*gQoC963E6h8kCGnR*x${WYly^cOG zSf&1Zti^}7A51>)plsjrInH%2%4AeH4_O1(d+KT|~y>yBs_IbZ?;oY`P zmJs})59vLGiTO+d&T61jDfcPQw#I)p zO8LaHE?~T`2nvSseWz*-M~6vgpxr)DUyVZm1UhV%I8<#?^_~~$TCptS^}UX5z!Iy& z#lL2)L$hYoccNhBpRhKJ{g;=2Jb9V6DU#i%>!R9zM#?C2#cXhKp?Gx+JI@i%*Zk&5 zcFF=q1lsV4ZTU&wb4SY-pJ)+7|GIMlKzMoEl&wb zh~up$S50SNDYpKV^SFJX`zw0}{?366w}LyoDJn1Y#j4IwKl_w@s;~a^mbUtJ3-hk( zrDt#l4DWLi`@Y|lmpdedXznpSMpLXF?$gqjE3e`|tX!WO9G;4oGAbrQHa=Q1nc%7Zw31LMz?-{P$aD#{-31uY`DU$1=(E^ZXB z4#CZZ{5~mfaF?nz(^$mY+MIcAgU5KqC=0KECuvQ&EgAUUb_tg+UT zQdTN}JJR>I`H}LddWiQw;ft0a2i`=+?tXe^0Do@Vra^3(a4(f zlF3Qp9+)Y7GR(J_j4pMpkUe7Lz%pS_Jd6C+bspT10Q@O5zaX){cX!~gF9q{dUBs(h zwYvK)8roh3;RznUJvF$r*f{eD1PG@$d1j{T0%J-$gRIY<@lZBYTq(la=Ut2haim1a z(lg)Y@#Tr=P15_;eCgq-zjbE2jO{o663I7NW2LN?-*v;zj-cf>HuJ;pMC|6sEK?%h zkH#DAKSkg7!&Z+8l2#E?5y5ZIAwC-;_ilF8NlSd;I%VhS`&D&97nfKu1I4w~HSF22 z+nvWQb_{jRa;)6!V=`peVi?Jo;6icFPN@y<(v`(xb=5G67$$X@MR7<)*Wk%)-R$jA z+7(ZOR;RY%tTFi(7+}D=oDhIGXo%)+eHGH6@yD1CnEV<0{~IR!b4~t%A^!X_i+zu6 z+a_^w(kvhn%{H*yun{aNe>*c*ybFCbj+1)TJA?0W=q@OT{VGFe zm1>M_q2|Z$x>9zD1BC^`MO_q$0lr97T{}+kuSdx(5J0TTVE(o8%|ZfiEaN>ridz~~uc^u+ zZ<4&zt|y-*dS`7JI^F*facOqH!{AIoGx;8xHHh7#iQpKGDY?CoZ*n8;E-%3;ZWWW= zg-nv67xBUP9wC6|rNfbE-DX~9&MK)yQFNPmLg*V~b&9DQ9!+*~yZO8Maj(A|Mw<=9 zXo4iavOxfDb%0h6S#Fz5C5p)6b>F!<3sH#5B%AG5GtM+P6kP^m#Xl-9IW)$wecK#~ zN7~XVSBUl0Jv7tRq4ZMSx*z3Fr}Hl5e5zNJE|1W$o;fw zg}>rJp5qeuJH{KYJ`n2BFIdg82)ZHumr2w63+Wk1ur=?Y`7^>qn#kpx+#r$e^_FJv z47nE|Rzf*fV^rhRhVK6gqW>euG`Y4WxV3Zf&~*wMzqDB3YgWe%8elNUcFkZO3AKpK zvq*o=ES*E&tVlDxlvMW~K5s^W?}xwN7ZCQX;)(@BDq+H!peKR}>Vezf4t+oO%9Ss2 z;mX2>SKh&~#x`Td!Cs8veVpMrF^wmZPJA-@gHLH2_7@&nkqO%5+A9g#3{leGfs!iL zX+U@%?T@tolhfBnXL#<$EaUoBU0t{MJ7F7Nsbg-qrRRn5l@NgSv_%7Zg>8{EWaSdj zLD_A}gN3WXJj!`M^d)^ZDP!wS>llB(Ziax=eITaqQJxUB5?KAxNHyt=&y`hju4uUB zd`(R&dBQsPl=?AoG4CZW0(cSg&h$0=mOGx~z=6MK46~x6`E$4Jf^S=#-u{ms<+W5m z+dYjGI2BXT7|YE8_;hv2lKNog8yW2#9w!pGus}o$rW>=+bk#yl9~R#%rY04}LndBXJMRRk2lDMCDnz=_9{&R&&xs z7%@KsK@(-&na38kXCwr9O37U!6i7-2mYf#*x!rPRHjz&PI9OmVGER-+-wEuI_*7`yUP14(XhLY-pfzFyK%+gF44bEjqZIYQ_+swnaigvSj zO-~Q^^s7hN-pu9_{;1y4f^v*6&0s<;@DfNP>OLO~QB@U;WDZctxDL^>AF(xY`C(e9 z0R`S#0H;UjZHJnHt{`DYV(qZ+;!j^bmBvBRV(H7%m-41L8V(zmaTZ7 zw&cfIb&XKtRi54N@fj}7RwWipv1{~egkczCtEio?Pc5TZg=Bd%U%#f;|2anmXkPhzl)yEYA+?PE(V{v z4L{tx=vcJNXAXJ~EN2SR`94bvj&ft!UVxY-`Ey;1@x@X*+)bS$^Rrp{ zA=^Q#+^Y0yzY0InV5GZ-2jN%`_VG|FJemyfHouwM(Ca9q5NC@Ay(oFcX2RYX{jKtm zG^QPgsS>9t3UZzCmFwEod$`QCIeexHz6^;3U*K&^7rq}8I?6xR%o;f^=k!UdzLDXx z{A;CF!8BJF6>o=LS&Fg82|>_?k%HQHHpC;%(2A*g&2Dr`YDNpvT@0a}VYP1~JA2Eo zfGDiW)|X|UZ6eVNxFxA~nCNCdy$GZDpIJoYjBl_-IoRLEwUk9b3vxi;0Os5lkYhp} z^|3db&qC`!V~`5wcm`n7yFQj1pCjx5R(#^%^=1fQpiryf4xiNwVIz5XYUcChnWmNP z=oi+eQmReJ!h}B|F&8j-jz9pnw^|#HRUn5!c)v0NmO^rHvMd7VqyXc78h$YEqlOcW zg~8U-U@N#kv!HWLeLe*66RG8U5VH2g6uSJ*CKmbP+qVbt_sQ#=J* zU(JdC9sa{AM!yEK<)hFe2}+!><}rSyJ_qa3!RlRe?SQ+d^bQF{32g}Ajmwh3B@^|< z04|{UTFPGZ*1ys#8S)Z2 zxXq@#{K6#R>}hc4Ebm^dac$CzhR1pi2d?=k9M|f?EDYbavNys`KdJ6I9MN2YB!Yc2 zW*Z5uU-G7~!uU_IvT8~daK+wk;uYmqfi=*C&0;#btd}nyvtav+;XD9~TwpQjrDCSn zB*Tv;QUnmkRNPN(vPXT5zCu02$5viNsLPOkt!Vga4fAfZ4VYyP2~e}#p!b9)YlK^} ze0s&fHaojYy85P-T<9+Mx3gQmtfrx()Z;EVJYTq%c8HhYy1Ub#@lNTNBT{WC0|zv$ z%W|!%;GK(LFwAXZOJepS^0&q8P0P+6>%#+?Hp@|m}mEJ1wQ^pz{V&u5D1r~2=2c!YgW8uG`N6P zB;$VKHLXf2_j&^A0th6>)Od(b91z0ajlnf`)=@G*^v7%`2_Lfl1-4>48}a)%V53Ak z&Xwn}T~53YqxwNi3n4 zT*c7NeC0lVWrE-cm2n<4JJPq&+9~uT8Mgss>Rz2)>nUda(f4oA%(INKo%cRv-IfrN zg5&Zjr-dtG@z~34LeSuR8%Ej$)0;JNyOtXH5cAAoUaKjsG%^ASsUHvf4vbNKXsf?) zw2D)Wx`yEBk80XRmTZH8K9V){pszrQv}Ik-FtO{Ng9EeFZS0b&??iF%Px;<^!c7Wd zRr2rJ8eV~z4mARBqVQF8$m!y1ny^sLJA3Hb#k1DoivUVxX6;VV#kke(SX$w5iwIFI z%2BPJO1@(^S$PZJ3=eMD4#iz#r0=ss<&XWD=z#BFrkzLHX4d@=g4?@=Wjl1wzPuRb zgWi+Vc6+N!D*EEUX^b}Q)mPTjD^>e@x@(-?Z(&07pH&1z?U)*F( zLwCca>cjLOoM_<0h|iCK8~0B_{>LY}^Vcm^9*g1bZC9Uu$*u~`FYQ+(RJ@l|Q-{<9 zMg(_IW;_zc_RpHd&^7ya3fWKOu~_1CpWBK3kZxZpK%rQw6c4P!ILB-JiG^Zh1f5$P zzz+B*4Df32VcOIa4TLX6*LU(?7YioiYQhP?a1Yks0%juf*07)W)IWA8z3$w5A%0=P zYT@QKh{ZJA#9$@07qGk?h2Nso^24n$w4n0Y)~7C41!30*Ll@B!_sWGwauM9_ylA6jo{4Kx4Av0>n?ersKRMh1 zQ$o|IC#v$FRO`2`3gVG~8*+ZL^GH%C29is#h^&h z_T%Hx@VHZV;t9itxR}!~k80Z5QA&SJoa`9_Q&w${Hrh$sD#|3~`T?M4_PY=VbX5&_+dHdi6|G z)2Za1=e_i(L#&r31k-7DWkzgoJBwNoz(89G=i0|k4{~ZEvzswhdq-?0 zz6;K~d==m4Sts{j?HoLCozZC$K5i}wq|SGs#^+cYkO^`@H7M;f6KnpCAHfyX>(nbI zyQ6hh&vbAr+-3)|aKEg<=4tP&gBwGsp|y_oQiwgKZ~K$iS=OI3S*5V_e76h4sb2}N zAj>UyQv)xoOP)YJ0;8e`;8ra<&&9j1-iZo*c22A>eUOSSH5l67iYmp$Rf`pr+AWt4 zziIelMN~W^YBVj|NbpR$m-Kq}`#kQ<&sa!c)Qcn2lHl2cD04q|(P-jTbo6EV9U=74 z1mc|eV^>kwBFT?B!bWlYVK3$6+0D^}u#~qc(83S9F?ZI&+5*M4>U$m^RKVqMYY>1f zXw;M`A-w-Wq_m*cW!dMXBUitsb>$ln!I1v1`0S0Hc!3AEF>Ri4N-?RLvuK zBl0`^L&_Mov^UFMZOmM9bmBQpkrE^&(pH4sBc%-|!7lQW*XGF6G-vum@KlRq4L+YP z<rShJ&~=}~l+oIAZrhp0L5 zzS$qyapH09Jat@Ut}DFkTea>b*%!4eQ=sUFzekd#oN4%79M*T_vA40lq0rCcWz*k2 zU@Ck$B>LRYW|1XdXLd?qOet+}PV}Iye)vnE@)5sos4dEhD$ZN z>-~|u*922DJWt$orSWc0-3wo)aBX48_uFu)`&z~EHX+)gp5k>#yv^&g$VWpA<7Mh9 zaw4mZT-mKyNQu(IV>k;*-$YNjeR`9uP-N|oX?!;qo&=inWHMjB9vmqxKB6bwW9_X7kM?K7;o?9Oiap(`r11j z^F4yydZn{*-4%gjept^{wQXb^k*%{J3iW1)izo~*kT=ZiO|TX?ixdh=d0jRp(l(9e z&t%J0leML0G=?(Wb}MD^%hRbJ8)9qYE?oBN1y3<*%XnPuv7T2%H<*bO$%8eh!mT6p zs_DM{u0|5FxrH!ZGr0Q?LeN{anD*^M7p^ZDn>K?+MdIU*@q!nU52Tan`U`E#uHL2d zYbdbA-sG>Dy4{aAGGTJQmqOOL6iR;P;X1f5+7@ulF|<^h|NL5VBPqS3<&>AoH>!v54lSEWB=4KVmm~u>sl6LXbbIce^?DOqy<9!| z(F?7S;t8kGGX`Dst!|Z7?rL2kou9})#ya?W%R1{3jJQ>HeTw2sG$IaJ;Cq}gz*-v3w>GiE z`e3MkPp?%-UHzSi&*|M4`(j~QqqU5Pj0OF}!L+Yr6lqPjD*K zPvSDhi!y(1Q=GD@#`wj4E8F90NTu29T>UE4feDg>Eg|}w&);Ug zeKwd9&D~p4LRf(oklaSv>WG6sWY2hD_l|M@XkAlP_C;Bgh0Z0P(sFC;V7nTH>q8;Z zUgnVNN43S0ROt&HFKVj%g-XxwS8MwW)ISMj;RmE~Xw9;KeFc`}Q7Al>!t9$PjZO$o2#t*__%xiocLA$*1vn*FPP?hk9h9#a|AKoR)E^FUvmEEUi#$DCu zP)83P*hF=kBPehlPPn&2a8G{RPNN4tl!aQZ!R4NkU=ws(5-@RR{H$q_W=TC+EtP)p z`ny@EP-u1i^yze3zzKT7qB8`=GJ@!4H8sytL-S0tb%l&E*lc2j)zjn_gsH^eo&@V~ z{y;A=D0S);OsZ@!uj?aQv{5|9w0j35fD3|tRQ_+uQ`TR>U&98xKqSg=N9N-8vG7qH zHV8YNgTZanWW*uSddx2@N}ld2yJ2e;c0)eW?WF=au;sJ6G6u&*RYO(%gfDe_l3Kt> zR%ccGVbXbpaN!M@@C#=)CG^j=~1^%8oXsFs;M4o`7o+dmWRVLRU<51C50CJC5-^7jVzmVoezfElL}4K zIL$X9O(5NoPHb*gcVhL?=;?G#RP$BvT&-yM#_N;|qQ_sD+c&k^Ca>QdIfQy|n`Kff zu=LcbRM$6TD8ft@wrV74>aN7QT(wuJ&2C4@CYxw~G?ZLJvP>Ryi95LJ6V;?shEK3Z zw`FvhzdJWtOG3!*d5UC;T(L6cFN&$U>FkAS?J{L1%oUruTV7DJ%1yktONL!KL z3sDMY>Y86Z9}TazX-zU?PsmEKNhIQ8E_}vB3b-B@(okW>a-fZ0cJ8LlRC|irSs~+d*A-S!Mxu76zuVruP z!F9sVi;|-E%7=RU;STQmg;BnWP}XMxcg9Iux$#C*1Ab7Emj%k^UqjMcC?ZQ0vdw+D z`30Dk-rEbF67{`e9?}`~A)nEiwTNtSvmNuFoSc}!3fon~&SHr^bGQ~!^uiM>8Y&Zc zWhdT}KAT>nqdhGfJE`7M=}ucE6D@9N8mkoFWpD5qB{=gy3raVE{<@Z*cWfxoF9VCw zfesq-1f%yB3Q;R&8`+k2w$prS=0|uJc+O6H-sTi~A9zOagg#3--l6RPKk~C_Wk{Mg zv{@h_Ha*qy@Z88ojrW_J38w{6CKc8#I6JV1UZIy3+*Y~Yvkk`&oVvnN;g4+e*s@_I zAW^ei%(GT`xBZER>ITW%#xGvrg|CQWtx-G zu`EZMakoP0mM?-MK2f}935!!uwuC+!I0<3n*pBVUYaWodJ>&NN%DtJgKPb2I?lJdr z`8>DWUE6z90cl>M{dv4KpYBcFW0PdJ^%@?dOm+(>f+pd20k}w6=*&WAr}9oq+IXK2EWMD-TrIIJ zePvQ=>*p9){RSQ2CD5(+{=YuPz$}tG;Vr z7B2{m;ng$`=bne094jx@iIZrZgm{KgsYv4-qP6#N8SjNEMNSlxCaQ@R*p!DOLbSl|LwsQS<&~|qsG|*O)HByf zHRUJGCl46zW1i9RfsMAB=PAU=HqH&C4>G|PoWjH7t(RO>f+B%4<8aD}jrT}BfaeC1 zA1X4Qk`<%&{z-YsGS7^Gj@<2PmI>WbMk|0!L6FXGimsfcJ^!STTv2$c&CZH6_%6k^5+}fz=%23{ z;cft;G3#JidWCp4;@>OYk4F6_6VBEOf^9Yp$4+C03C;WU;1e;%g4lQ?N^df^uMmJ| ziRn&n&k!TEC#99~pDFIY4|Dt-#Ugde=gTeF1*hu7JsIW065>@XHd83$j=Y})6ZISL zc4{w(gOd;?-C)*pfr#4+5H;Kcd%A$2f7}H7*WR?4A%I7Ap|E`|$ayu0D*N95=S|3h zgEojGeOjWvCdPxagWz&92JFv!5$C!Zu;T~kZ3qAjL<~c+OTY><^*?T={^RZcy(Hms zG}(wwa)j)e71b0&cT`Sy9~LZfb*xnTbK|YFN=p{X<>5SAldm?|d(w3gs&V78;1NxS zKirr-^TcQSV>}GsAHvtSq?_4-f&hrO)Db{AYr4-@wnM)?a-faET?A-Q#s-TEakUL92X;Paf(YD??t`Ek!ldi` zCJ>QRMgUkmt<=W~u)_*alJK=cPncP{U<;d9%Zk~h2NCA z@fZ72F_AFaJ3?rYu335z!2g0yslnqgh*T+mwN0Mu0Z%+BqlWRw$3!{tjDoTM)j~*+ z8{C5}zu#&&-2-hO{;+m|eCzCkgP>5a``*d0A6Tc5GIv-5d@S&J%}dRzO0wQ*mpz?A zck!&9Tc1}&Oc+e+NSDvFN`@~hB>P{knWLV0yw0~D6?ROEGo~`Gd>~@)jMdwzIn)D)@KKy`PJTa{ehM})uu&)QKsfT`H@qA^&Hi-2q_ThU&WoaPu=K&OU zw)q;rl&V0XuXeRjG$s%wD}gKQ$)ohSX>H@QCl}vt`1xHUUb5(b1$U&F(;J(?O$jh5&HvZzlS|hn^Momht;gc)mCS>NicO!kW#tqMdD-v=&Gqo)r=DJ(mZB$UJ zUd}#2&k|qs1y3iB3Y?mpBni7_KWH4p>NittET-^b_4xp^*3)sw-iUsZns-`N8F7g6 z>RJVcQaPEeQ(TpI7*#`X38Uq zqOojj7U3gBQzr+@N1Z;`Ha|Mx=yONg(qTktU(@owui&us9sx%juin?qP1z*DDdr<4)teQ-3e#NVQP z6J)#r+8aL=uu`Pvtz+nzXx^g>Zz?*j0?Rc2%$eENR?TJPvYB#PPGV z|NFyEEZ%gwXUOx~KRN88Z6jwgbE82+$!7ePU1-hwMSQx?evG;5cCv1F_qGRfmUWSs z?`rncxhcy9T5smvgH{Z4>*qt?LId9SKE7kH?pkOaSsjrp%dXsfIX6cD-N-!0U}5MR ztIn*@*zViccwFa=r zqw!JqfB&kb@;UuM_U~IDp3hkYwn-KILG|x@Fp@#S^bg6%sTkFf{sJVd?5Yt@23PR# zh)@OR+++2AL#+{KUxmHjA5D(d*q#Bg15OSkR%S%|0)S>+My_xlO^v0;j7 z?2dAmyORCRv=vMH?IT(p*)pQU&oH2PWg;Z#aF6muysO+c-Pk_zt>lgM6m?rIzhm*tnQlSH#!VctxXP&XNrn<$Bmwx<(4@-7SBw5Qa}o3q-f z?JtUTWhLlRs0C0xMx7HWtA^4n*z}i_%NZX`T6GYsQE!WH3k+N!0L!4j1gtn7X^U8V z^XiaVF1}$m<l)(3>NP ztrF_X$xdrbkqW<63F>sigi8p=`AL2FDHyM@7HV4g5YWIa+h(gF|Czw93EebMR*(aRx@RQ;4Bfk@D<%oIzB%;WFubzJiM|G

VUL?r*a1Df8b|mU1SztLE54fmtp6oe@BaYWoH_Dt^lds&n{w z{AtWUbtQAX7jHwAsa2$CJRvv){s3}}Rtr+_9(K?PSp;JN325Kd1v&zt$Ug1gC4O0| zW^e$COfuWr&2<0*2nS0)Tzgn+ED|_u;wv=K2|I9m^d7z-Yj~Ui87!+zpFchneFX%(wfAkKk}gun7|kRNqD1AlX6omZ`5D zBW!RNOy$+eaGr_~QalY4o1QqdN&P?Cx$y`kz6Sp4E9%<==>dn%WjAK(|V0_Vs{hgJTA9i!?D?B&~>(4~`S# zI=od5TSQb6db?OKHFc{Lwo2%ig22t5J)faga*-`4wW;*f-Y<8viwzJkx{7*M9 zUvD_ImWc8bSkl$of{wV$o}7P$$G!ih9>1`hy6Yc*a!}<6Mm{xC%jBd|#^_GHvI`n= zbx~c?x%TqmnVyfPYCz@qbF}>hK*avdR<=sXVD;rja2!Vn)*-X5)#;*VsOw{8qcEBr zuO!-*JATb83OPk*e3^;}NiS>iTl$`#F^V@Fb zp$NQ733QQavj+nxk(d(QuMOYoCZJJKv!=wY7O`c;;!b)PHp%E@VmeuG)wXOYk2pd4 z!Vcn2GCLFhfQKnud#MJYzJ2F>s8HFnyawG$cG|L_Jvbkr7zR#BIL8coJsdZ7v!ch} z9G%Q##vdb{$uK6T7SyGUuyG%uTCxpsS*|--3EY_=H*!=3LELkVJMod(*$HeSSlba* zlN9oi6+1~gy~1ti66}Czv-bbO@BQ1v^AGm^(7nrUC7{+~!b_LNEcD9> zcZO;harD#6}XPIVWaBWDv$O)^jwO9tTpsyqWU17QjC! zbq9beG(F!TYjltua4EGPXbD$banEVR1A49%Puv8T{^)8@XHfovc0q11z*iJg_jozY z=V;;2Bw09#uJ{F(=|44hhV0s{{7&;hy#pVMEL?%_16LL;%s)A`{cYeY*Mch)zV|GG zD>wV^!Koilb=^rQX!LMrzy03Mg;zgARrOUcoEHMQ3c-+$PpI{?ok2)v4dTNqHCNAinhDhS3gaI|tOM3pvsA8RTMSpu14_hRD9w1J{FiDB`nn z5p)^}BpW)&#R{d)3;RHw{n4Bm?GhtKT4fcjx`yWdUE1v56h{SA@L(FB~F6;l~D1e?GP`>9^qBb zJzje5X{~-9e5)l>=i3W>5&Q!0+%!}qnS7vRtB(u86Y^#hQS)iOSw!KiCz-CMyOW$~ za)jM8UFL*ro&Ub+JshW41n*uDXzc%`W6G+(D;q=)T;|+GlWE)W_Xn|eG9TMYe1yIl zoe%6~`$p?V&IX{@sm~Yy1cp!*tvr2y*uW`8qilNkXh^o8*OPplyh}oQH6CTgap^%q zw)J);Ic5V^5S03Y8V@4|sp~vC)*R0X0i{{Ns6!DHfj}Hmzg)xMeVY7+#RMbB`j=ij z!EAYz1IWMvSnD^~(wXV#%6sY`4qqF*6xH9%+g2|F?v>bCP46D=N}xPVCOJ+7RG&DY zTY0CPUe0z|go+YNfo7a0&l=_u@{wbwNC>g2nLvj5EDr6OhivtSmE8V0EN7PCj2F)& z*%(P$gDq1I|Fjrg=$04##?T2mbe*~T%w2kTGcYgM-?2!nsJb5^-Me$nK=;GF9jq=F z-^sWk-+hO?NZwyGi2$l&s1>7Xq|TlrxqD8epVE1ihDqG2z3MawI>=#KPC71IQgJrX z^Amb{94cD)9a1sMPJF;6-&BZ^FM#W#^sG#pv+K2%avxaKj*;|+netq(N`@RAA{mya z$z9J}NQKP|y68<@>EZZrnQ;?;Vfo^UiP0XVTtdV*ExV#o(~i8H`xJ%CvwM+bM;YNY zZ&RPu>Q*|f_LmJF(Ae_I9R2&Uk8T>2y4QCVcZpr#u`4gC0Bs;PirydhhCLq1anF-p z`qZ)XHLos#4?pT%fva$|gUw533@9e>lb-1i_x25(uCB#X({JBvyL*MO(XLY3d5hLo zoXEM%;G{nSBISz#b_(6{%UkE6vD(875=&-=GF5w!$PXL#FFY*iS{hRb%~3qe4B zoR;QBjw}bt9gE0(@@8KHG-hIs-p2uSd5}7&QZ2VI@n$Z}fkm05`8xk(+H0cz69mCp zV^IVB3hHXRc?q;M@>~8gsK=-l1ipwv%_4@Y^1R)Ncz&C!R-gf}43s8I(Fn42oPBl} zy0Yv6j=Ym`GM@!X?S47$@Rj@Mpz97`7Y|Er7lQ34e=(TPx(dAXxhO{Ysb%L+#kece zfr3DtvVcXJ)5m3g^li8k&&c<)T|Fe{dK*ur*9i{=u6qOk0i+#?miN8sJdW!#PXbIN$f9-(uU5ND-|z}4r`lZ#(brlt+&jifS59=cQ} z_)2gPU!`Q;GCR&YvE5^Pey+B{9ow~VcfOF+w|Zh(x?>M`qtv-yb;XP&VVSwtc7>2Q zu2ur9liTrqRChKr=TW_A`*HlzWw}( zQrA9yAr{^POUuuN5!vgk6Uu}*P&qi&89}oTrMfR<_;e!3o6<*3JwVmTzAhp>>jS(3vNCffyEo*;Ny=o7h>ro1rla_ zqkVkivx7);lf|;*vdhbEYLF`BP=$2Vs^gqc@SI>-xeDc=v&uqBc$1EQ*sUIW%n0I+ z8h*RM^9BK^Bg`@U2uf9R?aXh3kuAyw0k{1!+j6fi&h2n)@!6Xah1-vW$X*D$RrOg3 zp#>WADvH7vmLh1bZ&$!|JYJ-4fuQ;w-%sznVTG^UnyisU3GWWisgjplWm^IqxIwv? zz3$?%Zd}~Fm{;?q(%!L(lI7Ih#aT9Y_jp3;8~CyO=h5P~jpTd#PKMir!}nc+!?_`Q z?z8$Nn15Z(1pAPknJ(6NuAny$B zUPz>5bliM(f=}=GGQ2T2IOweRF~lVSpnkZE^E=|`M0OH?WE@AGy-KSgi69Jngv*`g@}c|QVz}#>1yCNd&8I@L;tdRm)Nxi1 zpTcU;;UVJ7CY+kPtd4FJ_S|$BZQF)?1l_m0 znVOJJ5RKhj1OB4fjBJS+Mh-fGQ>A+vtws#dY7-@zOc&T{ZmR#e^{QnOY1;!TIO)EI z_onfgO0Ef;K(X3t9p>`GMr0V>&;>@$swCB4uj^;0o9?)L=Z+~kha`8_<~yQ={coT5 z->9EC=%#qTMX#euSgl1{Vlb&$Ag?*8%PmiPS$|O|+`(IMxN~kqQ{h8AH|ZS8ZM)40 zPvgB>JN9)Z zBRGFPi)k4n(IJ^mwT9;x#qA%Mas_0J>=Ia3c_}b)Q9ksN=Dd*?AAX$tI%q~?r% zNvJFhNYXG|&Kq)kvpFDtQZ?eDs_uud7y*=4?!07#BpP?OV7;#v9lAH4d5gR_2FbIQZEJ0Vmbc)v8 zdy*s}*y?P=>%OUu0kL-_VX8%<0g{;D(Nl>w3hk*=dmkGa0Y&@d>?srH|B$~Tn zsuCJ(yezydA{RbjUYoQMGtU5RweIMRcUv}L%}Qyi*v()D!t`47G{3eUS^YU^g%q^n zp07y}1>ubmKnqexD9%S8?wMf3HPAb9=5uD%7Vo$rdp&5mh)~8!sa(*@T{=QKV|{4F zZ-_dzl&Da9(@dwVvP;ZcOpb~W&uvc~6tJ-MRPPlu@#KW>*sErD(^tieBC?-iC7P0!t0$MMrvkW1Jy4ua`DL>X z@e4_;WIeeseYql@kPTVeMB%U6?mI*-)(^H>=TGf5?==>~H?4NeT#T6M8B#fU%SvU= zeF97b{J|S?8^$|o=cQwYu1te);PYXI(+!1r|M-SAXV##T?&EkajI(QnJ6Rp1CO`2L zp8Feq;Z(~Yp2=->zurq>pCv00sqLVr9KTM*Yq8KMMuiOHH^)KCSJu#ZENGrA9duK8 zBXY$OB{XR4LsOgT2Avo1yJyb|{&rbTmpLA_Qy}xL-HDLVmVlL9M3-Aeu!-^@*J)?% zrU}>=&IX)AS6YmwwgO5!&dL00@qsP~i@8mKQnv~;wBT)&O6(yg&PW9bg}bamn?ot;S8i`c%?d_*I!DjcBT^cX^Ra%VY^(m>#5~A37!yTe(p`@ zxlFr^fo2yE?6i)H=!xhK&8A3gZqBOLp~u4>CCgH?`!fbjf~@^5v8ZXI!U&c0(Dbv< z-dwg8lzb`LEYwyR_o{d=voh+oyy5d7tva)1n_I3L48BZB%?a04T(l?7!x+Cg_t07k zT}{O|z!CZDqlE;b)>{xWcY@L+dPHnK-Ne7>cQo@bC;z~8*EDB>G^nrv{^N!Z{MmRj z23bXo^I{&`p|R!GtMdb@IaB8B*avf>(Lo@+TPPW zz<)hDQl_OXkLkFrYm&G!#M^a2FbS_~Wfz%pPPM>n-6KWdkagM-hsm1&k#NL$vkX{)8pn{Fr2DKH)f$SFf!`>ySXr7`sqj$vYCd@ zZ{-1Zp%vS2J<<2Le@S*}gDGq!yTtV3!x~22G`zf|GTupghp>vajwFgxp>WLRX&dOG z)M6ZuGIXb@otAiBU)P~<#wwK}BNv>YoodVs_G_()ahqh;`x13Sa`EKBYjYTH zHSOYz_4K)qNA;iL;7@!47m=S;!7dc+I-j$*4+W!50)TQ6RBF`)`qAXT3&KF9TKz8wb?b1)fXQQZA`1 zFLmR5)q$-jY(Yw@=@81(bA_J%VFoWm-80hk&V^hL<-Id-U8bFou5Vyy?R3Fj%9arc zXebA|a35?9Q1OJ;Z`VQMUqDL}pNb3E&Hm?8{UYStM{u zvHcDij9_fQz5q8yQ_9q#S%(&EurcCucsPxeXH0?yr1IF@0`>H&V1~3T%d3pYiMVn=B9;a2YLjU#) zjvIXBO^F}4fmqrIAQzF_&3F-Hu@hQ&6NJ>zjTsQ8nBz(h4HrvK@jTSHq_|Tatn$?3 z%nSCP!i8%G4*_9hgTbvG|JWRvY`_NyfHv3}4pJCE9bDpw-_~Lw$)u>9L1=|-N`YUC z_;v%lVF7oIWj1wtlRCE{&-ZeXI#sKtcsP=|53oS#$Ti^NSe3SX-uOEt``fYBdAsir zCu{lqvPUz^V2-ejNiavK1DXM}iL60C4R#>RrsdmJWB})^sD}62-k@Y84PX`oUBLY9 z8z87%LH-=iYBTNe9ASjB7=Uo)J7gZiIGBwPfk~Z=xTeL3mIjVj2T%JXZ)*1fb%IQWB!NCE7ul|4pa%IC-CA^-0S0)~pLeB|P)Z(Yz z=m#mCaTSsCS!s!Beq=STuC~lrz>EKJhSc|r{NVstbrO_h{G2j(d_w_POe=#1`=aqT zm;h@4FfdwBL=U~+(AhT_^x-c!g#Y@=^&d(w&0sFx=tcft0OSbSmUm2rg%6T#_Cv6r zu^}}bHSha63^l||JG0L^RTe42moYu*LQ0Ykb<{>DnCdp;FjJ-D&6 zckAewjnkmu9XU%};39tedQg8xWR z3iLG!u)SyBZyEc)Eg&_*O0VE%=+@!?aRq9}{sYl}Ao@!~|D#1)T*d$2&~`-y>NO+x r!1_v#Gx8jN* + +# ZooKeeper Audit Logging + +* [ZooKeeper Audit Logs](#ch_auditLogs) +* [ZooKeeper Audit Log Configuration](#ch_reconfig_format) +* [Who is taken as user in audit logs?](#ch_zkAuditUser) + + +## ZooKeeper Audit Logs + +Apache ZooKeeper supports audit logs from version 3.6.0. By default audit logs are disabled. To enable audit logs + configure audit.enable=true in conf/zoo.cfg. Audit logs are not logged on all the ZooKeeper servers, but logged only on the servers where client is connected as depicted in below figure. + +![Audit Logs](images/zkAuditLogs.jpg) + + +The audit log captures detailed information for the operations that are selected to be audited. The audit information is written as a set of key=value pairs for the following keys + +| Key | Value | +| ----- | ----- | +|session | client session id | +|user | comma separated list of users who are associate with a client session. For more on this, see [Who is taken as user in audit logs](#ch_zkAuditUser). +|ip | client IP address +|operation | any one of the selected operations for audit. Possible values are(serverStart, serverStop, create, delete, setData, setAcl, multiOperation, reconfig, ephemeralZNodeDeleteOnSessionClose) +|znode | path of the znode +|znode type | type of znode in case of creation operation +|acl | String representation of znode ACL like cdrwa(create, delete,read, write, admin). This is logged only for setAcl operation +|result | result of the operation. Possible values are (success/failure/invoked). Result "invoked" is used for serverStop operation because stop is logged before ensuring that server actually stopped. + +Below are sample audit logs for all operations, where client is connected from 192.168.1.2, client principal is zkcli@HADOOP.COM, server principal is zookeeper/192.168.1.3@HADOOP.COM + + user=zookeeper/192.168.1.3 operation=serverStart result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/a znode_type=persistent result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/a znode_type=persistent result=failure + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/a result=failure + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/a result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setAcl znode=/a acl=world:anyone:cdrwa result=failure + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setAcl znode=/a acl=world:anyone:cdrwa result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/b znode_type=persistent result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=setData znode=/b result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/b result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=multiOperation result=failure + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/a result=failure + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=delete znode=/a result=success + session=0x19344730001 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=create znode=/ephemral znode_type=ephemral result=success + session=0x19344730001 user=zookeeper/192.168.1.3 operation=ephemeralZNodeDeletionOnSessionCloseOrExpire znode=/ephemral result=success + session=0x19344730000 user=192.168.1.2,zkcli@HADOOP.COM ip=192.168.1.2 operation=reconfig znode=/zookeeper/config result=success + user=zookeeper/192.168.1.3 operation=serverStop result=invoked + + + +## ZooKeeper Audit Log Configuration + +By default audit logs are disabled. To enable audit logs configure audit.enable=true in conf/zoo.cfg. Audit logging is done using log4j. Following is the default log4j configuration for audit logs in conf/log4j.properties + + # + # zk audit logging + # + zookeeper.auditlog.file=zookeeper_audit.log + zookeeper.auditlog.threshold=INFO + audit.logger=INFO, RFAAUDIT + log4j.logger.org.apache.zookeeper.audit.Log4jAuditLogger=${audit.logger} + log4j.additivity.org.apache.zookeeper.audit.Log4jAuditLogger=false + log4j.appender.RFAAUDIT=org.apache.log4j.RollingFileAppender + log4j.appender.RFAAUDIT.File=${zookeeper.log.dir}/${zookeeper.auditlog.file} + log4j.appender.RFAAUDIT.layout=org.apache.log4j.PatternLayout + log4j.appender.RFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n + log4j.appender.RFAAUDIT.Threshold=${zookeeper.auditlog.threshold} + + # Max log file size of 10MB + log4j.appender.RFAAUDIT.MaxFileSize=10MB + log4j.appender.RFAAUDIT.MaxBackupIndex=10 + +Change above configuration to customize the auditlog file, number of backups, max file size, custom audit logger etc. + + + +## Who is taken as user in audit logs? + +By default there are only four authentication provider: + +* IPAuthenticationProvider +* SASLAuthenticationProvider +* X509AuthenticationProvider +* DigestAuthenticationProvider + +User is decided based on the configured authentication provider: + +* When IPAuthenticationProvider is configured then authenticated IP is taken as user +* When SASLAuthenticationProvider is configured then client principal is taken as user +* When X509AuthenticationProvider is configured then client certificate is taken as user +* When DigestAuthenticationProvider is configured then authenticated user is user + +Custom authentication provider can override org.apache.zookeeper.server.auth.AuthenticationProvider.getUserName(String id) + to provide user name. If authentication provider is not overriding this method then whatever is stored in + org.apache.zookeeper.data.Id.id is taken as user. + Generally only user name is stored in this field but it is up to the custom authentication provider what they store in it. + For audit logging value of org.apache.zookeeper.data.Id.id would be taken as user. + +In ZooKeeper Server not all the operations are done by clients but some operations are done by the server itself. For example when client closes the session, ephemeral znodes are deleted by the Server. These deletion are not done by clients directly but it is done the server itself these are called system operations. For these system operations the user associated with the ZooKeeper server are taken as user while audit logging these operations. For example if in ZooKeeper server principal is zookeeper/hadoop.hadoop.com@HADOOP.COM then this becomes the system user and all the system operations will be logged with this user name. + + user=zookeeper/hadoop.hadoop.com@HADOOP.COM operation=serverStart result=success + + +If there is no user associate with ZooKeeper server then the user who started the ZooKeeper server is taken as the user. For example if server started by root then root is taken as the system user + + user=root operation=serverStart result=success + + +Single client can attach multiple authentication schemes to a session, in this case all authenticated schemes will taken taken as user and will be presented as comma separated list. For example if a client is authenticate with principal zkcli@HADOOP.COM and ip 127.0.0.1 then create znode audit log will be as: + + session=0x10c0bcb0000 user=zkcli@HADOOP.COM,127.0.0.1 ip=127.0.0.1 operation=create znode=/a result=success + + diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java b/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java index 275fdaae16a..83af16256a3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/Login.java @@ -47,6 +47,7 @@ public class Login { private static final String KINIT_COMMAND_DEFAULT = "/usr/bin/kinit"; private static final Logger LOG = LoggerFactory.getLogger(Login.class); + public static final String SYSTEM_USER = System.getProperty("user.name", ""); public CallbackHandler callbackHandler; // LoginThread will sleep until 80% of time from last refresh to @@ -295,6 +296,13 @@ public Subject getSubject() { return subject; } + public String getUserName() { + if (principal == null || principal.isEmpty()) { + return SYSTEM_USER; + } + return principal; + } + public String getLoginContextName() { return loginContextName; } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java index c6b1ff58170..481f8e9a081 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/ZKUtil.java @@ -23,7 +23,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.zookeeper.AsyncCallback.MultiCallback; @@ -31,12 +33,14 @@ import org.apache.zookeeper.AsyncCallback.VoidCallback; import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.common.PathUtils; +import org.apache.zookeeper.data.ACL; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class ZKUtil { private static final Logger LOG = LoggerFactory.getLogger(ZKUtil.class); + private static final Map permCache = new ConcurrentHashMap(); /** * Recursively delete the node with the given path. *

@@ -234,4 +238,44 @@ private static void visitSubTreeDFSHelper( } } -} + /** + * @param perms + * ACL permissions + * @return string representation of permissions + */ + public static String getPermString(int perms) { + return permCache.computeIfAbsent(perms, k -> constructPermString(k)); + } + + private static String constructPermString(int perms) { + StringBuilder p = new StringBuilder(); + if ((perms & ZooDefs.Perms.CREATE) != 0) { + p.append('c'); + } + if ((perms & ZooDefs.Perms.DELETE) != 0) { + p.append('d'); + } + if ((perms & ZooDefs.Perms.READ) != 0) { + p.append('r'); + } + if ((perms & ZooDefs.Perms.WRITE) != 0) { + p.append('w'); + } + if ((perms & ZooDefs.Perms.ADMIN) != 0) { + p.append('a'); + } + return p.toString(); + } + + public static String aclToString(List acls) { + StringBuilder sb = new StringBuilder(); + for (ACL acl : acls) { + sb.append(acl.getId().getScheme()); + sb.append(":"); + sb.append(acl.getId().getId()); + sb.append(":"); + sb.append(getPermString(acl.getPerms())); + } + return sb.toString(); + } +} \ No newline at end of file diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditConstants.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditConstants.java new file mode 100644 index 00000000000..22fd8567abe --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditConstants.java @@ -0,0 +1,34 @@ +/* + * 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.audit; + +public final class AuditConstants { + private AuditConstants() { + //Utility classes should not have public constructors + } + + static final String OP_START = "serverStart"; + static final String OP_STOP = "serverStop"; + public static final String OP_CREATE = "create"; + public static final String OP_DELETE = "delete"; + public static final String OP_SETDATA = "setData"; + public static final String OP_SETACL = "setAcl"; + public static final String OP_MULTI_OP = "multiOperation"; + public static final String OP_RECONFIG = "reconfig"; + public static final String OP_DEL_EZNODE_EXP = "ephemeralZNodeDeletionOnSessionCloseOrExpire"; +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditEvent.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditEvent.java new file mode 100644 index 00000000000..e499552a948 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditEvent.java @@ -0,0 +1,98 @@ +/* + * 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.audit; + +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +public final class AuditEvent { + private static final char PAIR_SEPARATOR = '\t'; + private static final String KEY_VAL_SEPARATOR = "="; + // Holds the entries which to be logged. + private Map logEntries = new LinkedHashMap<>(); + private Result result; + + AuditEvent(Result result) { + this.result = result; + } + + /** + * Gives all entries to be logged. + * + * @return log entries + */ + public Set> getLogEntries() { + return logEntries.entrySet(); + } + + void addEntry(FieldName fieldName, String value) { + if (value != null) { + logEntries.put(fieldName.name().toLowerCase(), value); + } + } + + public String getValue(FieldName fieldName) { + return logEntries.get(fieldName.name().toLowerCase()); + } + + public Result getResult() { + return result; + } + + /** + * Gives the string to be logged, ignores fields with null values + * + * @return String + */ + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + boolean first = true; + for (Map.Entry entry : logEntries.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (null != value) { + // if first field then no need to add the tabs + if (first) { + first = false; + } else { + buffer.append(PAIR_SEPARATOR); + } + buffer.append(key).append(KEY_VAL_SEPARATOR) + .append(value); + } + } + //add result field + if (buffer.length() > 0) { + buffer.append(PAIR_SEPARATOR); + } + buffer.append("result").append(KEY_VAL_SEPARATOR) + .append(result.name().toLowerCase()); + return buffer.toString(); + } + + public enum FieldName { + USER, OPERATION, IP, ACL, ZNODE, SESSION, ZNODE_TYPE + } + + public enum Result { + SUCCESS, FAILURE, INVOKED + } +} + diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java new file mode 100644 index 00000000000..c39980133a2 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditHelper.java @@ -0,0 +1,232 @@ +/* + * 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.audit; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import org.apache.jute.Record; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.MultiOperationRecord; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.ZKUtil; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.apache.zookeeper.proto.CreateRequest; +import org.apache.zookeeper.proto.DeleteRequest; +import org.apache.zookeeper.proto.SetACLRequest; +import org.apache.zookeeper.proto.SetDataRequest; +import org.apache.zookeeper.server.ByteBufferInputStream; +import org.apache.zookeeper.server.DataTree.ProcessTxnResult; +import org.apache.zookeeper.server.Request; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper class to decouple audit log code. + */ +public final class AuditHelper { + private static final Logger LOG = LoggerFactory.getLogger(AuditHelper.class); + + public static void addAuditLog(Request request, ProcessTxnResult rc) { + addAuditLog(request, rc, false); + } + + /** + * Add audit log if audit log is enabled and operation is of type which to be audit logged. + * + * @param request user request + * @param txnResult ProcessTxnResult + * @param failedTxn whether audit is being done failed transaction for normal transaction + */ + public static void addAuditLog(Request request, ProcessTxnResult txnResult, boolean failedTxn) { + if (!ZKAuditProvider.isAuditEnabled()) { + return; + } + String op = null; + //For failed transaction rc.path is null + String path = txnResult.path; + String acls = null; + String createMode = null; + try { + switch (request.type) { + case ZooDefs.OpCode.create: + case ZooDefs.OpCode.create2: + case ZooDefs.OpCode.createContainer: + op = AuditConstants.OP_CREATE; + if (failedTxn) { + CreateRequest createRequest = new CreateRequest(); + deserialize(request, createRequest); + path = createRequest.getPath(); + createMode = + getCreateMode(createRequest); + } else { + createMode = getCreateMode(request); + } + break; + case ZooDefs.OpCode.delete: + case ZooDefs.OpCode.deleteContainer: + op = AuditConstants.OP_DELETE; + if (failedTxn) { + DeleteRequest deleteRequest = new DeleteRequest(); + deserialize(request, deleteRequest); + path = deleteRequest.getPath(); + } + break; + case ZooDefs.OpCode.setData: + op = AuditConstants.OP_SETDATA; + if (failedTxn) { + SetDataRequest setDataRequest = new SetDataRequest(); + deserialize(request, setDataRequest); + path = setDataRequest.getPath(); + } + break; + case ZooDefs.OpCode.setACL: + op = AuditConstants.OP_SETACL; + if (failedTxn) { + SetACLRequest setACLRequest = new SetACLRequest(); + deserialize(request, setACLRequest); + path = setACLRequest.getPath(); + acls = ZKUtil.aclToString(setACLRequest.getAcl()); + } else { + acls = getACLs(request); + } + break; + case ZooDefs.OpCode.multi: + if (failedTxn) { + op = AuditConstants.OP_MULTI_OP; + } else { + logMultiOperation(request, txnResult); + //operation si already logged + return; + } + break; + case ZooDefs.OpCode.reconfig: + op = AuditConstants.OP_RECONFIG; + break; + default: + //Not an audit log operation + return; + } + Result result = getResult(txnResult, failedTxn); + log(request, path, op, acls, createMode, result); + } catch (Throwable e) { + LOG.error("Failed to audit log request {}", request.type, e); + } + } + + private static void deserialize(Request request, Record record) throws IOException { + ByteBufferInputStream.byteBuffer2Record(getRequestData(request), record); + } + + private static ByteBuffer getRequestData(Request request) { + ByteBuffer reqData = request.request.slice(); + reqData.rewind(); + return reqData; + } + + private static Result getResult(ProcessTxnResult rc, boolean failedTxn) { + if (failedTxn) { + return Result.FAILURE; + } else { + return rc.err == KeeperException.Code.OK.intValue() ? Result.SUCCESS : Result.FAILURE; + } + } + + private static void logMultiOperation(Request request, ProcessTxnResult rc) throws IOException, KeeperException { + Map createModes = AuditHelper.getCreateModes(request); + boolean multiFailed = false; + for (ProcessTxnResult subTxnResult : rc.multiResult) { + switch (subTxnResult.type) { + case ZooDefs.OpCode.create: + case ZooDefs.OpCode.create2: + case ZooDefs.OpCode.createTTL: + case ZooDefs.OpCode.createContainer: + log(request, subTxnResult.path, AuditConstants.OP_CREATE, null, + createModes.get(subTxnResult.path), Result.SUCCESS); + break; + case ZooDefs.OpCode.delete: + case ZooDefs.OpCode.deleteContainer: + log(request, subTxnResult.path, AuditConstants.OP_DELETE, null, + null, Result.SUCCESS); + break; + case ZooDefs.OpCode.setData: + log(request, subTxnResult.path, AuditConstants.OP_SETDATA, null, + null, Result.SUCCESS); + break; + case ZooDefs.OpCode.error: + multiFailed = true; + break; + default: + // Do nothing, it ok, we do not log all multi operations + } + } + if (multiFailed) { + log(request, rc.path, AuditConstants.OP_MULTI_OP, null, + null, Result.FAILURE); + } + } + + private static void log(Request request, String path, String op, String acls, String createMode, Result result) { + log(request.getUsers(), op, path, acls, createMode, + request.cnxn.getSessionIdHex(), request.cnxn.getHostAddress(), result); + } + + private static void log(String user, String operation, String znode, String acl, + String createMode, String session, String ip, Result result) { + ZKAuditProvider.log(user, operation, znode, acl, createMode, session, ip, result); + } + + private static String getACLs(Request request) throws IOException { + SetACLRequest setACLRequest = new SetACLRequest(); + deserialize(request, setACLRequest); + return ZKUtil.aclToString(setACLRequest.getAcl()); + } + + private static String getCreateMode(Request request) throws IOException, KeeperException { + CreateRequest createRequest = new CreateRequest(); + deserialize(request, createRequest); + return getCreateMode(createRequest); + } + + private static String getCreateMode(CreateRequest createRequest) throws KeeperException { + return CreateMode.fromFlag(createRequest.getFlags()).toString().toLowerCase(); + } + + private static Map getCreateModes(Request request) + throws IOException, KeeperException { + Map createModes = new HashMap<>(); + if (!ZKAuditProvider.isAuditEnabled()) { + return createModes; + } + MultiOperationRecord multiRequest = new MultiOperationRecord(); + deserialize(request, multiRequest); + for (Op op : multiRequest) { + if (op.getType() == ZooDefs.OpCode.create || op.getType() == ZooDefs.OpCode.create2 + || op.getType() == ZooDefs.OpCode.createContainer) { + CreateRequest requestRecord = (CreateRequest) op.toRequestRecord(); + createModes.put(requestRecord.getPath(), + getCreateMode(requestRecord)); + } + } + return createModes; + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditLogger.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditLogger.java new file mode 100644 index 00000000000..0f84488f339 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/AuditLogger.java @@ -0,0 +1,34 @@ +/* + * 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.audit; + +public interface AuditLogger { + + /** + * Called during initialization of the logger. + */ + default void initialize() { + } + + /** + * Called to log an audit event. + * + * @param auditEvent contains all the fields to be logged + */ + void logAuditEvent(AuditEvent auditEvent); +} \ No newline at end of file diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/Log4jAuditLogger.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/Log4jAuditLogger.java new file mode 100644 index 00000000000..c4efc1c4e08 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/Log4jAuditLogger.java @@ -0,0 +1,38 @@ +/* + * 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.audit; + +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Log4j based audit logger + */ +public class Log4jAuditLogger implements AuditLogger { + private static final Logger LOG = LoggerFactory.getLogger(Log4jAuditLogger.class); + + @Override + public void logAuditEvent(AuditEvent auditEvent) { + if (auditEvent.getResult() == Result.FAILURE) { + LOG.error(auditEvent.toString()); + } else { + LOG.info(auditEvent.toString()); + } + } +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/audit/ZKAuditProvider.java b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/ZKAuditProvider.java new file mode 100644 index 00000000000..00646baf871 --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/audit/ZKAuditProvider.java @@ -0,0 +1,133 @@ +/* + * 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.audit; + +import static org.apache.zookeeper.audit.AuditEvent.FieldName; +import java.lang.reflect.Constructor; +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZKAuditProvider { + static final String AUDIT_ENABLE = "zookeeper.audit.enable"; + static final String AUDIT_IMPL_CLASS = "zookeeper.audit.impl.class"; + private static final Logger LOG = LoggerFactory.getLogger(ZKAuditProvider.class); + // By default audit logging is disabled + private static boolean auditEnabled; + private static AuditLogger auditLogger; + + static { + auditEnabled = Boolean.getBoolean(AUDIT_ENABLE); + if (auditEnabled) { + //initialise only when audit logging is enabled + auditLogger = getAuditLogger(); + LOG.info("ZooKeeper audit is enabled."); + } else { + LOG.info("ZooKeeper audit is disabled."); + } + } + + private static AuditLogger getAuditLogger() { + String auditLoggerClass = System.getProperty(AUDIT_IMPL_CLASS); + if (auditLoggerClass == null) { + auditLoggerClass = Log4jAuditLogger.class.getName(); + } + try { + Constructor clientCxnConstructor = Class.forName(auditLoggerClass) + .getDeclaredConstructor(); + AuditLogger auditLogger = (AuditLogger) clientCxnConstructor.newInstance(); + auditLogger.initialize(); + return auditLogger; + } catch (Exception e) { + throw new RuntimeException("Couldn't instantiate " + auditLoggerClass, e); + } + } + + /** + * @return true if audit log is enabled + */ + public static boolean isAuditEnabled() { + return auditEnabled; + } + + public static void log(String user, String operation, String znode, String acl, + String createMode, String session, String ip, Result result) { + auditLogger.logAuditEvent(createLogEvent(user, operation, znode, acl, createMode, session, ip, result)); + } + + /** + * A helper api for creating an AuditEvent object. + */ + static AuditEvent createLogEvent(String user, String operation, Result result) { + AuditEvent event = new AuditEvent(result); + event.addEntry(FieldName.USER, user); + event.addEntry(FieldName.OPERATION, operation); + return event; + } + + /** + * A helper api for creating an AuditEvent object. + */ + static AuditEvent createLogEvent(String user, String operation, String znode, String acl, + String createMode, String session, String ip, Result result) { + AuditEvent event = new AuditEvent(result); + event.addEntry(FieldName.SESSION, session); + event.addEntry(FieldName.USER, user); + event.addEntry(FieldName.IP, ip); + event.addEntry(FieldName.OPERATION, operation); + event.addEntry(FieldName.ZNODE, znode); + event.addEntry(FieldName.ZNODE_TYPE, createMode); + event.addEntry(FieldName.ACL, acl); + return event; + } + + /** + * Add audit log for server start and register server stop log. + */ + public static void addZKStartStopAuditLog() { + if (isAuditEnabled()) { + log(getZKUser(), AuditConstants.OP_START, Result.SUCCESS); + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + log(getZKUser(), AuditConstants.OP_STOP, Result.INVOKED); + })); + } + } + + /** + * Add audit log for server start fail. + */ + public static void addServerStartFailureAuditLog() { + if (isAuditEnabled()) { + log(ZKAuditProvider.getZKUser(), AuditConstants.OP_START, Result.FAILURE); + } + } + + private static void log(String user, String operation, Result result) { + auditLogger.logAuditEvent(createLogEvent(user, operation, result)); + } + + /** + * User who has started the ZooKeeper server user, it will be the logged-in + * user. If no user logged-in then system user. + */ + public static String getZKUser() { + return ServerCnxnFactory.getUserName(); + } + +} diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/GetAclCommand.java b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/GetAclCommand.java index f397f0e1c22..553395093f0 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/cli/GetAclCommand.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/cli/GetAclCommand.java @@ -24,7 +24,7 @@ import org.apache.commons.cli.Parser; import org.apache.commons.cli.PosixParser; import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZKUtil; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; @@ -75,7 +75,7 @@ public boolean exec() throws CliException { } for (ACL a : acl) { - out.println(a.getId() + ": " + getPermString(a.getPerms())); + out.println(a.getId() + ": " + ZKUtil.getPermString(a.getPerms())); } if (cl.hasOption("s")) { @@ -83,25 +83,4 @@ public boolean exec() throws CliException { } return false; } - - private static String getPermString(int perms) { - StringBuilder p = new StringBuilder(); - if ((perms & ZooDefs.Perms.CREATE) != 0) { - p.append('c'); - } - if ((perms & ZooDefs.Perms.DELETE) != 0) { - p.append('d'); - } - if ((perms & ZooDefs.Perms.READ) != 0) { - p.append('r'); - } - if ((perms & ZooDefs.Perms.WRITE) != 0) { - p.append('w'); - } - if ((perms & ZooDefs.Perms.ADMIN) != 0) { - p.append('a'); - } - return p.toString(); - } - } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java index 766949ece5a..f370d88f400 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/DataTree.java @@ -52,6 +52,9 @@ import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.audit.AuditConstants; +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.apache.zookeeper.audit.ZKAuditProvider; import org.apache.zookeeper.common.PathTrie; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; @@ -1171,14 +1174,27 @@ void killSession(long session, long zxid, Set paths2DeleteLocal, void deleteNodes(long session, long zxid, Iterable paths2Delete) { for (String path : paths2Delete) { + boolean deleted = false; + String sessionHex = "0x" + Long.toHexString(session); try { deleteNode(path, zxid); - LOG.debug("Deleting ephemeral node {} for session 0x{}", path, Long.toHexString(session)); + deleted = true; + LOG.debug("Deleting ephemeral node {} for session {}", path, sessionHex); } catch (NoNodeException e) { LOG.warn( - "Ignoring NoNodeException for path {} while removing ephemeral for dead session 0x{}", - path, - Long.toHexString(session)); + "Ignoring NoNodeException for path {} while removing ephemeral for dead session {}", + path, sessionHex); + } + if (ZKAuditProvider.isAuditEnabled()) { + if (deleted) { + ZKAuditProvider.log(ZKAuditProvider.getZKUser(), + AuditConstants.OP_DEL_EZNODE_EXP, path, null, null, + sessionHex, null, Result.SUCCESS); + } else { + ZKAuditProvider.log(ZKAuditProvider.getZKUser(), + AuditConstants.OP_DEL_EZNODE_EXP, path, null, null, + sessionHex, null, Result.FAILURE); + } } } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java index 79a69771805..0ffa5af25ea 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/FinalRequestProcessor.java @@ -44,6 +44,7 @@ import org.apache.zookeeper.Watcher.WatcherType; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooDefs.OpCode; +import org.apache.zookeeper.audit.AuditHelper; import org.apache.zookeeper.common.Time; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; @@ -165,6 +166,7 @@ public void processRequest(Request request) { String path = null; try { if (request.getHdr() != null && request.getHdr().getType() == OpCode.error) { + AuditHelper.addAuditLog(request, rc, true); /* * When local session upgrading is disabled, leader will * reject the ephemeral node creation due to session expire. @@ -192,7 +194,7 @@ public void processRequest(Request request) { if (request.isStale()) { ServerMetrics.getMetrics().STALE_REPLIES.add(1); } - + AuditHelper.addAuditLog(request, rc); switch (request.type) { case OpCode.ping: { lastOp = "PING"; diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java index 122f0cab097..63cc30b7eb6 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/Request.java @@ -28,6 +28,7 @@ import org.apache.zookeeper.metrics.Summary; import org.apache.zookeeper.metrics.SummarySet; import org.apache.zookeeper.server.quorum.flexible.QuorumVerifier; +import org.apache.zookeeper.server.util.AuthUtil; import org.apache.zookeeper.txn.TxnHeader; /** @@ -434,4 +435,31 @@ public void logLatency(SummarySet metric, String key) { logLatency(metric, key, Time.currentWallTime()); } + + /** + * Returns comma separated list of users authenticated in the current + * session + */ + public String getUsers() { + if (authInfo == null) { + return (String) null; + } + if (authInfo.size() == 1) { + return AuthUtil.getUser(authInfo.get(0)); + } + StringBuilder users = new StringBuilder(); + boolean first = true; + for (Id id : authInfo) { + String user = AuthUtil.getUser(id); + if (user != null) { + if (first) { + first = false; + } else { + users.append(","); + } + users.append(user); + } + } + return users.toString(); + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java index 4d95c8156c8..bf5a6c7c90a 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxn.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; +import java.net.InetAddress; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.security.cert.Certificate; @@ -555,4 +556,25 @@ public void cleanupWriterSocket(PrintWriter pwriter) { } } + /** + * Returns the IP address or empty string. + */ + public String getHostAddress() { + InetSocketAddress remoteSocketAddress = getRemoteSocketAddress(); + if (remoteSocketAddress == null) { + return ""; + } + InetAddress address = remoteSocketAddress.getAddress(); + if (address == null) { + return ""; + } + return address.getHostAddress(); + } + + /** + * Get session id in hexadecimal notation. + */ + public String getSessionIdHex() { + return "0x" + Long.toHexString(getSessionId()); + } } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java index 85c7ba92376..996fb32573d 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ServerCnxnFactory.java @@ -54,6 +54,8 @@ public abstract class ServerCnxnFactory { // sessionMap is used by closeSession() final ConcurrentHashMap sessionMap = new ConcurrentHashMap(); + private static String loginUser = Login.SYSTEM_USER; + public void addSession(long sessionId, ServerCnxn cnxn) { sessionMap.put(sessionId, cnxn); } @@ -264,6 +266,7 @@ protected void configureSaslLogin() throws IOException { try { saslServerCallbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration()); login = new Login(serverSection, saslServerCallbackHandler, new ZKConfig()); + setLoginUser(login.getUserName()); login.startThreadIfNeeded(); } catch (LoginException e) { throw new IOException("Could not configure server because SASL configuration did not allow the " @@ -272,4 +275,16 @@ protected void configureSaslLogin() throws IOException { } } + private static void setLoginUser(String name) { + //Created this method to avoid ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD find bug issue + loginUser = name; + } + /** + * User who has started the ZooKeeper server user, it will be the logged-in + * user. If no user logged-in then system user + */ + public static String getUserName() { + return loginUser; + } + } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java index 3c67d370618..4a1b6009596 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/ZooKeeperServerMain.java @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit; import javax.management.JMException; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.audit.ZKAuditProvider; import org.apache.zookeeper.jmx.ManagedUtil; import org.apache.zookeeper.metrics.MetricsProvider; import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException; @@ -67,21 +68,26 @@ public static void main(String[] args) { LOG.error("Invalid arguments, exiting abnormally", e); LOG.info(USAGE); System.err.println(USAGE); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.INVALID_INVOCATION.getValue()); } catch (ConfigException e) { LOG.error("Invalid config, exiting abnormally", e); System.err.println("Invalid config, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.INVALID_INVOCATION.getValue()); } catch (DatadirException e) { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); } catch (AdminServerException e) { LOG.error("Unable to start AdminServer, exiting abnormally", e); System.err.println("Unable to start AdminServer, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); } LOG.info("Exiting normally"); @@ -165,6 +171,7 @@ public void runFromConfig(ServerConfig config) throws IOException, AdminServerEx Integer.getInteger("znode.container.checkIntervalMs", (int) TimeUnit.MINUTES.toMillis(1)), Integer.getInteger("znode.container.maxPerMinute", 10000)); containerManager.start(); + ZKAuditProvider.addZKStartStopAuditLog(); // Watch status of ZooKeeper server. It will do a graceful shutdown // if the server is not running or hits an internal error. diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/AuthenticationProvider.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/AuthenticationProvider.java index 706b3b869cb..179eac8dfbf 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/AuthenticationProvider.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/AuthenticationProvider.java @@ -81,4 +81,17 @@ public interface AuthenticationProvider { */ boolean isValid(String id); + /** + * id represents the authentication info which is set in server connection. + * id may contain both user name as well as password. + * This method should be implemented to extract the user name. + * + * @param id authentication info set by client. + * @return String user name + */ + default String getUserName(String id) { + // Most of the authentication providers id contains only user name. + return id; + } + } diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java index 3a1c5e85f17..2330000a6f3 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/auth/DigestAuthenticationProvider.java @@ -120,6 +120,15 @@ public boolean matches(String id, String aclExpr) { return id.equals(aclExpr); } + @Override + public String getUserName(String id) { + /** + * format is already enforced in server code. so no need to check it + * again, just assume it is in correct format + */ + return id.split(":")[0]; + } + /** Call with a single argument of user:pass to generate authdata. * Authdata output can be used when setting superDigest for example. * @param args single argument of user:pass diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java index 943affd6f2a..bf97488e97b 100644 --- a/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/quorum/QuorumPeerMain.java @@ -22,6 +22,7 @@ import javax.management.JMException; import javax.security.sasl.SaslException; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.zookeeper.audit.ZKAuditProvider; import org.apache.zookeeper.jmx.ManagedUtil; import org.apache.zookeeper.metrics.MetricsProvider; import org.apache.zookeeper.metrics.MetricsProviderLifeCycleException; @@ -90,21 +91,26 @@ public static void main(String[] args) { LOG.error("Invalid arguments, exiting abnormally", e); LOG.info(USAGE); System.err.println(USAGE); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.INVALID_INVOCATION.getValue()); } catch (ConfigException e) { LOG.error("Invalid config, exiting abnormally", e); System.err.println("Invalid config, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.INVALID_INVOCATION.getValue()); } catch (DatadirException e) { LOG.error("Unable to access datadir, exiting abnormally", e); System.err.println("Unable to access datadir, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.UNABLE_TO_ACCESS_DATADIR.getValue()); } catch (AdminServerException e) { LOG.error("Unable to start AdminServer, exiting abnormally", e); System.err.println("Unable to start AdminServer, exiting abnormally"); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.ERROR_STARTING_ADMIN_SERVER.getValue()); } catch (Exception e) { LOG.error("Unexpected exception, exiting abnormally", e); + ZKAuditProvider.addServerStartFailureAuditLog(); System.exit(ExitCode.UNEXPECTED_ERROR.getValue()); } LOG.info("Exiting normally"); @@ -215,6 +221,7 @@ public void runFromConfig(QuorumPeerConfig config) throws IOException, AdminServ } quorumPeer.start(); + ZKAuditProvider.addZKStartStopAuditLog(); quorumPeer.join(); } catch (InterruptedException e) { // warn, but generally this is ok diff --git a/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AuthUtil.java b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AuthUtil.java new file mode 100644 index 00000000000..3b0b7e43d7e --- /dev/null +++ b/zookeeper-server/src/main/java/org/apache/zookeeper/server/util/AuthUtil.java @@ -0,0 +1,39 @@ +/* + * 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.util; + +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.auth.AuthenticationProvider; +import org.apache.zookeeper.server.auth.ProviderRegistry; + +public final class AuthUtil { + private AuthUtil() { + //Utility classes should not have public constructors + } + /** + * Gives user name + * + * @param id contains scheme and authentication info + * @return returns null if authentication scheme does not exist or + * authentication provider returns null as user + */ + public static String getUser(Id id) { + AuthenticationProvider provider = ProviderRegistry.getProvider(id.getScheme()); + return provider == null ? null : provider.getUserName(id.getId()); + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditEventTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditEventTest.java new file mode 100644 index 00000000000..02d9ac0bb85 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditEventTest.java @@ -0,0 +1,45 @@ +/* + * 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.audit; + +import static org.junit.Assert.assertEquals; +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.junit.Test; + +public class AuditEventTest { + + @Test + public void testFormat() { + AuditEvent auditEvent = new AuditEvent(Result.SUCCESS); + auditEvent.addEntry(AuditEvent.FieldName.USER, "Value1"); + auditEvent.addEntry(AuditEvent.FieldName.OPERATION, "Value2"); + String actual = auditEvent.toString(); + String expected = "user=Value1\toperation=Value2\tresult=success"; + assertEquals(expected, actual); + } + + @Test + public void testFormatShouldIgnoreKeyIfValueIsNull() { + AuditEvent auditEvent = new AuditEvent(Result.SUCCESS); + auditEvent.addEntry(AuditEvent.FieldName.USER, null); + auditEvent.addEntry(AuditEvent.FieldName.OPERATION, "Value2"); + String actual = auditEvent.toString(); + String expected = "operation=Value2\tresult=success"; + assertEquals(expected, actual); + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditLogPerfReading.java b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditLogPerfReading.java new file mode 100644 index 00000000000..38ef96c3cd3 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/AuditLogPerfReading.java @@ -0,0 +1,74 @@ +/* + * 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.audit; + +/** + * Audit log performance reading + */ +public final class AuditLogPerfReading { + // time taken by create operations + private long create; + // time taken by setData operations + private long setData; + // time taken by delete operations + private long delete; + + public long getCreate() { + return create; + } + + public void setCreate(long create) { + this.create = create; + } + + public long getSetData() { + return setData; + } + + public void setSetData(long setData) { + this.setData = setData; + } + + public long getDelete() { + return delete; + } + + public void setDelete(long delete) { + this.delete = delete; + } + + public String report() { + StringBuilder builder = new StringBuilder(); + builder.append("create="); + builder.append(create); + builder.append(" ms\n"); + builder.append("setData="); + builder.append(setData); + builder.append(" ms\n"); + builder.append("delete="); + builder.append(delete); + builder.append(" ms\n"); + return builder.toString(); + } + + @Override + public String toString() { + return "create=" + create + ", setData=" + setData + ", delete=" + + delete; + } +} \ No newline at end of file diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/audit/Log4jAuditLoggerTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/Log4jAuditLoggerTest.java new file mode 100644 index 00000000000..0c408110a17 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/Log4jAuditLoggerTest.java @@ -0,0 +1,442 @@ +/* + * 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.audit; + +import static org.apache.zookeeper.test.ClientBase.CONNECTION_TIMEOUT; +import static org.junit.Assert.assertEquals; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.LineNumberReader; +import java.io.StringReader; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.List; +import org.apache.log4j.Layout; +import org.apache.log4j.Level; +import org.apache.log4j.Logger; +import org.apache.log4j.SimpleLayout; +import org.apache.log4j.WriterAppender; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.Code; +import org.apache.zookeeper.Op; +import org.apache.zookeeper.PortAssignment; +import org.apache.zookeeper.ZKUtil; +import org.apache.zookeeper.ZooDefs; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.audit.AuditEvent.Result; +import org.apache.zookeeper.data.ACL; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.server.Request; +import org.apache.zookeeper.server.ServerCnxn; +import org.apache.zookeeper.server.quorum.QuorumPeerTestBase; +import org.apache.zookeeper.test.ClientBase; +import org.apache.zookeeper.test.ClientBase.CountdownWatcher; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + + +public class Log4jAuditLoggerTest extends QuorumPeerTestBase { + private static final Logger LOG = Logger.getLogger(Log4jAuditLoggerTest.class); + private static int SERVER_COUNT = 3; + private static MainThread[] mt; + private static ZooKeeper zk; + private static Logger zlogger; + private static WriterAppender appender; + private static ByteArrayOutputStream os; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + System.setProperty(ZKAuditProvider.AUDIT_ENABLE, "true"); + // setup the logger to capture all logs + Layout layout = new SimpleLayout(); + os = new ByteArrayOutputStream(); + appender = new WriterAppender(layout, os); + appender.setImmediateFlush(true); + appender.setThreshold(Level.INFO); + zlogger = Logger.getLogger(Log4jAuditLogger.class); + zlogger.addAppender(appender); + mt = startQuorum(); + zk = ClientBase.createZKClient("127.0.0.1:" + mt[0].getQuorumPeer().getClientPort()); + //Verify start audit log here itself + String expectedAuditLog = getStartLog(); + List logs = readAuditLog(os, SERVER_COUNT); + verifyLogs(expectedAuditLog, logs); + } + + @Before + public void setUp() { + os.reset(); + } + + @Test + public void testCreateAuditLogs() + throws KeeperException, InterruptedException, IOException { + String path = "/createPath"; + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + // success log + String createMode = CreateMode.PERSISTENT.toString().toLowerCase(); + verifyLog( + getAuditLog(AuditConstants.OP_CREATE, path, Result.SUCCESS, + null, createMode), readAuditLog(os)); + try { + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + } catch (KeeperException exception) { + Code code = exception.code(); + assertEquals(Code.NODEEXISTS, code); + } + // Verify create operation log + verifyLog( + getAuditLog(AuditConstants.OP_CREATE, path, Result.FAILURE, + null, createMode), readAuditLog(os)); + } + + @Test + public void testDeleteAuditLogs() + throws InterruptedException, IOException, KeeperException { + String path = "/deletePath"; + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + os.reset(); + try { + zk.delete(path, -100); + } catch (KeeperException exception) { + Code code = exception.code(); + assertEquals(Code.BADVERSION, code); + } + verifyLog(getAuditLog(AuditConstants.OP_DELETE, path, + Result.FAILURE), + readAuditLog(os)); + zk.delete(path, -1); + verifyLog(getAuditLog(AuditConstants.OP_DELETE, path), + readAuditLog(os)); + } + + @Test + public void testSetDataAuditLogs() + throws InterruptedException, IOException, KeeperException { + String path = "/setDataPath"; + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + os.reset(); + try { + zk.setData(path, "newData".getBytes(), -100); + } catch (KeeperException exception) { + Code code = exception.code(); + assertEquals(Code.BADVERSION, code); + } + verifyLog(getAuditLog(AuditConstants.OP_SETDATA, path, + Result.FAILURE), + readAuditLog(os)); + zk.setData(path, "newdata".getBytes(), -1); + verifyLog(getAuditLog(AuditConstants.OP_SETDATA, path), + readAuditLog(os)); + } + + @Test + public void testSetACLAuditLogs() + throws InterruptedException, IOException, KeeperException { + ArrayList openAclUnsafe = ZooDefs.Ids.OPEN_ACL_UNSAFE; + String path = "/aclPath"; + zk.create(path, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + os.reset(); + try { + zk.setACL(path, openAclUnsafe, -100); + } catch (KeeperException exception) { + Code code = exception.code(); + assertEquals(Code.BADVERSION, code); + } + verifyLog( + getAuditLog(AuditConstants.OP_SETACL, path, Result.FAILURE, + ZKUtil.aclToString(openAclUnsafe), null), readAuditLog(os)); + zk.setACL(path, openAclUnsafe, -1); + verifyLog( + getAuditLog(AuditConstants.OP_SETACL, path, Result.SUCCESS, + ZKUtil.aclToString(openAclUnsafe), null), readAuditLog(os)); + } + + @Test + public void testMultiOperationAuditLogs() + throws InterruptedException, KeeperException, IOException { + List ops = new ArrayList<>(); + + String multiop = "/b"; + Op create = Op.create(multiop, "".getBytes(), + ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + Op setData = Op.setData(multiop, "newData".getBytes(), -1); + // check does nothing so it is audit logged + Op check = Op.check(multiop, -1); + Op delete = Op.delete(multiop, -1); + + String createMode = CreateMode.PERSISTENT.toString().toLowerCase(); + + ops.add(create); + ops.add(setData); + ops.add(check); + ops.add(delete); + + zk.multi(ops); + List multiOpLogs = readAuditLog(os, 3); + // verify that each multi operation success is logged + verifyLog(getAuditLog(AuditConstants.OP_CREATE, multiop, + Result.SUCCESS, null, createMode), + multiOpLogs.get(0)); + verifyLog(getAuditLog(AuditConstants.OP_SETDATA, multiop), + multiOpLogs.get(1)); + verifyLog(getAuditLog(AuditConstants.OP_DELETE, multiop), + multiOpLogs.get(2)); + + ops = new ArrayList<>(); + ops.add(create); + ops.add(create); + try { + zk.multi(ops); + } catch (KeeperException exception) { + Code code = exception.code(); + assertEquals(Code.NODEEXISTS, code); + } + + // Verify that multi operation failure is logged, and there is no path + // mentioned in the audit log + verifyLog(getAuditLog(AuditConstants.OP_MULTI_OP, null, + Result.FAILURE), + readAuditLog(os)); + } + + @Test + public void testEphemralZNodeAuditLogs() + throws Exception { + String ephemralPath = "/ephemral"; + CountdownWatcher watcher2 = new CountdownWatcher(); + ZooKeeper zk2 = new ZooKeeper( + "127.0.0.1:" + mt[0].getQuorumPeer().getClientPort(), + ClientBase.CONNECTION_TIMEOUT, watcher2); + watcher2.waitForConnected(ClientBase.CONNECTION_TIMEOUT); + zk2.create(ephemralPath, "".getBytes(), ZooDefs.Ids.OPEN_ACL_UNSAFE, + CreateMode.EPHEMERAL); + String session2 = "0x" + Long.toHexString(zk2.getSessionId()); + verifyLog(getAuditLog(AuditConstants.OP_CREATE, ephemralPath, + Result.SUCCESS, null, + CreateMode.EPHEMERAL.toString().toLowerCase(), + session2), readAuditLog(os)); + zk2.close(); + waitForDeletion(zk, ephemralPath); + // verify that ephemeral node deletion on session close are captured + // in audit log + // Because these operations are done by ZooKeeper server itself, + // there are no IP user is zkServer user, not any client user + verifyLogs(getAuditLog(AuditConstants.OP_DEL_EZNODE_EXP, ephemralPath, + Result.SUCCESS, null, null, session2, + ZKAuditProvider.getZKUser(), null), readAuditLog(os, SERVER_COUNT)); + } + + + private static String getStartLog() { + // user=userName operation=ZooKeeperServer start result=success + AuditEvent logEvent = ZKAuditProvider.createLogEvent(ZKAuditProvider.getZKUser(), + AuditConstants.OP_START, Result.SUCCESS); + return logEvent.toString(); + } + + private String getAuditLog(String operation, String znode) { + return getAuditLog(operation, znode, Result.SUCCESS); + } + + private String getAuditLog(String operation, String znode, Result result) { + return getAuditLog(operation, znode, result, null, null); + } + + private String getAuditLog(String operation, String znode, Result result, + String acl, String createMode) { + String session = getSession(); + return getAuditLog(operation, znode, result, acl, createMode, session); + } + + private String getAuditLog(String operation, String znode, Result result, + String acl, String createMode, String session) { + String user = getUser(); + String ip = getIp(); + return getAuditLog(operation, znode, result, acl, createMode, session, + user, ip); + } + + private String getAuditLog(String operation, String znode, Result result, + String acl, String createMode, String session, String user, String ip) { + AuditEvent logEvent = ZKAuditProvider.createLogEvent(user, operation, znode, acl, createMode, session, ip, + result); + String auditLog = logEvent.toString(); + LOG.info("expected audit log for operation '" + operation + "' is '" + + auditLog + "'"); + return auditLog; + } + + private String getSession() { + return "0x" + Long.toHexString(zk.getSessionId()); + } + + private String getUser() { + ServerCnxn next = getServerCnxn(); + Request request = new Request(next, -1, -1, -1, null, + next.getAuthInfo()); + return request.getUsers(); + } + + private String getIp() { + ServerCnxn next = getServerCnxn(); + InetSocketAddress remoteSocketAddress = next.getRemoteSocketAddress(); + InetAddress address = remoteSocketAddress.getAddress(); + return address.getHostAddress(); + } + + private ServerCnxn getServerCnxn() { + Iterable connections = mt[0].getQuorumPeer() + .getActiveServer() + .getServerCnxnFactory().getConnections(); + return connections.iterator().next(); + } + + private static void verifyLog(String expectedLog, String log) { + String searchString = " - "; + int logStartIndex = log.indexOf(searchString); + String auditLog = log.substring(logStartIndex + searchString.length()); + assertEquals(expectedLog, auditLog); + + } + + private static void verifyLogs(String expectedLog, List logs) { + for (String log : logs) { + verifyLog(expectedLog, log); + } + } + + private String readAuditLog(ByteArrayOutputStream os) throws IOException { + return readAuditLog(os, 1).get(0); + } + + private static List readAuditLog(ByteArrayOutputStream os, + int numberOfLogEntry) + throws IOException { + return readAuditLog(os, numberOfLogEntry, false); + } + + private static List readAuditLog(ByteArrayOutputStream os, + int numberOfLogEntry, + boolean skipEphemralDeletion) throws IOException { + List logs = new ArrayList<>(); + LineNumberReader r = new LineNumberReader( + new StringReader(os.toString())); + String line; + while ((line = r.readLine()) != null) { + if (skipEphemralDeletion + && line.contains(AuditConstants.OP_DEL_EZNODE_EXP)) { + continue; + } + logs.add(line); + } + os.reset(); + assertEquals( + "Expected number of log entries are not generated. Logs are " + + logs, + numberOfLogEntry, logs.size()); + return logs; + + } + + private static MainThread[] startQuorum() throws IOException { + final int[] clientPorts = new int[SERVER_COUNT]; + StringBuilder sb = new StringBuilder(); + sb.append("4lw.commands.whitelist=*"); + sb.append("\n"); + String server; + + for (int i = 0; i < SERVER_COUNT; i++) { + clientPorts[i] = PortAssignment.unique(); + server = "server." + i + "=127.0.0.1:" + PortAssignment.unique() + + ":" + + PortAssignment.unique() + ":participant;127.0.0.1:" + + clientPorts[i]; + sb.append(server); + sb.append("\n"); + } + String currentQuorumCfgSection = sb.toString(); + MainThread[] mt = new MainThread[SERVER_COUNT]; + + // start all the servers + for (int i = 0; i < SERVER_COUNT; i++) { + mt[i] = new MainThread(i, clientPorts[i], currentQuorumCfgSection, + false); + mt[i].start(); + } + + // ensure all servers started + for (int i = 0; i < SERVER_COUNT; i++) { + Assert.assertTrue("waiting for server " + i + " being up", + ClientBase.waitForServerUp("127.0.0.1:" + clientPorts[i], + CONNECTION_TIMEOUT)); + } + return mt; + } + + private void waitForDeletion(ZooKeeper zooKeeper, String path) + throws Exception { + long elapsedTime = 0; + long waitInterval = 10; + int timeout = 100; + Stat exists = zooKeeper.exists(path, false); + while (exists != null && elapsedTime < timeout) { + try { + Thread.sleep(waitInterval); + } catch (InterruptedException e) { + Assert.fail("CurrentEpoch update failed"); + } + elapsedTime = elapsedTime + waitInterval; + exists = zooKeeper.exists(path, false); + } + Assert.assertNull("Node " + path + " not deleted in " + timeout + " ms", + exists); + } + + @AfterClass + public static void tearDownAfterClass() { + System.clearProperty(ZKAuditProvider.AUDIT_ENABLE); + for (int i = 0; i < SERVER_COUNT; i++) { + try { + if (mt[i] != null) { + mt[i].shutdown(); + } + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + try { + zlogger.removeAppender(appender); + os.close(); + } catch (IOException e) { + e.printStackTrace(); + } + } + +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/audit/ZKAuditLoggerPerformance.java b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/ZKAuditLoggerPerformance.java new file mode 100644 index 00000000000..da64197ed55 --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/audit/ZKAuditLoggerPerformance.java @@ -0,0 +1,150 @@ +/* + * 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.audit; + +import java.io.IOException; +import java.util.concurrent.TimeoutException; +import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.ZooDefs.Ids; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.common.Time; +import org.apache.zookeeper.data.Stat; +import org.apache.zookeeper.test.ClientBase.CountdownWatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ZKAuditLoggerPerformance { + private static final Logger LOG = LoggerFactory + .getLogger(ZKAuditLoggerPerformance.class); + private ZooKeeper zkClient; + private String parentPath; + private int numberOfRecords; + + public ZKAuditLoggerPerformance(ZooKeeper zkClient, String parentPath, + int numberOfRecords) { + this.zkClient = zkClient; + this.parentPath = parentPath; + this.numberOfRecords = numberOfRecords; + } + + public void create() throws Exception { + for (int i = 0; i < numberOfRecords; i++) { + zkClient.create(getPath(i), "0123456789".getBytes(), + Ids.OPEN_ACL_UNSAFE, + CreateMode.PERSISTENT); + + } + } + + public void setData() throws Exception { + for (int i = 0; i < numberOfRecords; i++) { + zkClient.setData(getPath(i), "9876543210".getBytes(), -1); + } + } + + public void delete() throws Exception { + for (int i = 0; i < numberOfRecords; i++) { + zkClient.delete(getPath(i), -1); + } + } + + public AuditLogPerfReading doOperations() throws Exception { + AuditLogPerfReading perfReading = new AuditLogPerfReading(); + // create + long startTime = Time.currentElapsedTime(); + create(); + perfReading.setCreate(Time.currentElapsedTime() - startTime); + + // setData + startTime = Time.currentElapsedTime(); + setData(); + perfReading.setSetData(Time.currentElapsedTime() - startTime); + + // delete + startTime = Time.currentElapsedTime(); + delete(); + perfReading.setDelete(Time.currentElapsedTime() - startTime); + return perfReading; + } + + private String getPath(int i) { + return parentPath + "zNode" + i; + } + + public static void main(String[] args) { + if (args.length != 3) { + System.err.println( + "USAGE: ZKAuditLoggerPerformance connectionString parentPath numberOfRecords"); + System.exit(1); + } + String cxnString = args[0]; + CountdownWatcher watcher = new CountdownWatcher(); + ZooKeeper zkClient = null; + try { + zkClient = new ZooKeeper(cxnString, 60000, watcher); + watcher.waitForConnected(30000); + } catch (InterruptedException | TimeoutException | IOException e) { + String msg = "ZooKeeper client can not connect to " + cxnString; + logErrorAndExit(e, msg); + } + String parentPath = args[1]; + try { + Stat exists = zkClient.exists(parentPath, false); + if (exists == null) { + System.err.println( + "Parent path '" + parentPath + "' must exist."); + System.exit(1); + } + } catch (KeeperException | InterruptedException e1) { + String msg = "Error while checking the existence of parent path"; + logErrorAndExit(e1, msg); + } + int recordCount = 0; + try { + recordCount = Integer.parseInt(args[2]); + } catch (NumberFormatException e) { + String msg = "Failed to parse '" + args[2] + "' to integer"; + LOG.error(msg, e); + System.err.println(msg); + System.exit(1); + } + ZKAuditLoggerPerformance auditLoggingPerf = new ZKAuditLoggerPerformance( + zkClient, + parentPath, recordCount); + AuditLogPerfReading doOperations = null; + try { + doOperations = auditLoggingPerf.doOperations(); + } catch (Exception e) { + String msg = "Error while doing operations."; + LOG.error(msg, e); + System.err.println(msg); + System.exit(1); + } + System.out + .println("Time taken for " + recordCount + " operations are:"); + System.out.println(doOperations.report()); + System.exit(0); + } + + private static void logErrorAndExit(Exception e, String msg) { + LOG.error(msg, e); + System.err.println(msg + ", error=" + e.getMessage()); + System.exit(1); + } +} diff --git a/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AuthUtilTest.java b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AuthUtilTest.java new file mode 100644 index 00000000000..801a76944ec --- /dev/null +++ b/zookeeper-server/src/test/java/org/apache/zookeeper/server/util/AuthUtilTest.java @@ -0,0 +1,71 @@ +/* + * 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.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import org.apache.zookeeper.data.Id; +import org.apache.zookeeper.server.auth.ProviderRegistry; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class AuthUtilTest { + + @BeforeClass + public static void beforeClassSetUp() { + ProviderRegistry.reset(); + System.setProperty("zookeeper.authProvider.sasl", + "org.apache.zookeeper.server.auth.SASLAuthenticationProvider"); + System.setProperty("zookeeper.authProvider.x509", + "org.apache.zookeeper.server.auth.X509AuthenticationProvider"); + } + + @AfterClass + public static void afterClassTearDown() { + System.clearProperty("zookeeper.authProvider.sasl"); + System.clearProperty("zookeeper.authProvider.x509"); + } + + @Test + public void testGetUserFromAllAuthenticationScheme() { + String user = "zkUser"; + Id id = new Id("digest", user + ":password"); + String result = AuthUtil.getUser(id); + assertEquals(user, result); + + String principal = "zkCli/hadoop.hadoop.com"; + id = new Id("sasl", principal); + assertEquals(principal, AuthUtil.getUser(id)); + + String ip = "192.168.1.2"; + id = new Id("ip", ip); + assertEquals(ip, AuthUtil.getUser(id)); + + String certificate = "CN=host-192.168.1.2,OU=OrganizationUnit,O=Organization,L=Location,ST=State,C=IN"; + id = new Id("x509", certificate); + assertEquals(certificate, AuthUtil.getUser(id)); + } + + @Test + public void testGetUserShouldReturnNullIfAuthenticationNotConfigured() { + Id id = new Id("invalid Authentication Scheme", "user"); + String result = AuthUtil.getUser(id); + assertNull(result); + } +}