You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by he...@apache.org on 2011/06/25 08:13:03 UTC

svn commit: r1139494 - in /zookeeper/trunk: CHANGES.txt src/java/main/org/apache/zookeeper/server/DataTree.java src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java

Author: henry
Date: Sat Jun 25 06:13:03 2011
New Revision: 1139494

URL: http://svn.apache.org/viewvc?rev=1139494&view=rev
Log:
ZOOKEEPER-1097. Quota is not correctly rehydrated on snapshot reload (camille via henryr)

Modified:
    zookeeper/trunk/CHANGES.txt
    zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
    zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java

Modified: zookeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/trunk/CHANGES.txt?rev=1139494&r1=1139493&r2=1139494&view=diff
==============================================================================
--- zookeeper/trunk/CHANGES.txt (original)
+++ zookeeper/trunk/CHANGES.txt Sat Jun 25 06:13:03 2011
@@ -239,7 +239,9 @@ BUGFIXES: 
   ZOOKEEPER-1103. In QuorumTest, use the same "for ( .. try { break }
   catch { } )" pattern in testFollowersStartAfterLeaders as in
   testSessionMove. (Eugene Koontz via phunt)
-  ZOOKEEPER-1046. Creating a new sequential node results in a ZNODEEXISTS error. (breed via camille) 
+  ZOOKEEPER-1046. Creating a new sequential node results in a ZNODEEXISTS error. (breed via camille)
+
+  ZOOKEEPER-1097. Quota is not correctly rehydrated on snapshot reload (camille via henryr)
 
 IMPROVEMENTS:
   ZOOKEEPER-724. Improve junit test integration - log harness information 

Modified: zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java?rev=1139494&r1=1139493&r2=1139494&view=diff
==============================================================================
--- zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java (original)
+++ zookeeper/trunk/src/java/main/org/apache/zookeeper/server/DataTree.java Sat Jun 25 06:13:03 2011
@@ -379,11 +379,11 @@ public class DataTree {
         synchronized (node) {
             thisStats = new StatsTrack(new String(node.data));
         }
-        if (thisStats.getCount() < updatedStat.getCount()) {
+        if (thisStats.getCount() > -1 && (thisStats.getCount() < updatedStat.getCount())) {
             LOG
-                    .warn("Quota exceeded: " + lastPrefix + " count="
-                            + updatedStat.getCount() + " limit="
-                            + thisStats.getCount());
+            .warn("Quota exceeded: " + lastPrefix + " count="
+                    + updatedStat.getCount() + " limit="
+                    + thisStats.getCount());
         }
     }
 
@@ -425,11 +425,11 @@ public class DataTree {
         synchronized (node) {
             thisStats = new StatsTrack(new String(node.data));
         }
-        if (thisStats.getBytes() < updatedStat.getBytes()) {
+        if (thisStats.getBytes() > -1 && (thisStats.getBytes() < updatedStat.getBytes())) {
             LOG
-                    .warn("Quota exceeded: " + lastPrefix + " bytes="
-                            + updatedStat.getBytes() + " limit="
-                            + thisStats.getBytes());
+            .warn("Quota exceeded: " + lastPrefix + " bytes="
+                    + updatedStat.getBytes() + " limit="
+                    + thisStats.getBytes());
         }
     }
 
@@ -508,8 +508,8 @@ public class DataTree {
             }
         }
         // also check to update the quotas for this node
-        String lastPrefix = pTrie.findMaxPrefix(path);
-        if (!rootZookeeper.equals(lastPrefix) && !("".equals(lastPrefix))) {
+        String lastPrefix;
+        if((lastPrefix = getMaxPrefixWithQuota(path)) != null) {
             // ok we have some match and need to update
             updateCount(lastPrefix, 1);
             updateBytes(lastPrefix, data == null ? 0 : data.length);
@@ -567,8 +567,8 @@ public class DataTree {
         }
 
         // also check to update the quotas for this node
-        String lastPrefix = pTrie.findMaxPrefix(path);
-        if (!rootZookeeper.equals(lastPrefix) && !("".equals(lastPrefix))) {
+        String lastPrefix;
+        if((lastPrefix = getMaxPrefixWithQuota(path)) != null) {
             // ok we have some match and need to update
             updateCount(lastPrefix, -1);
             int bytes = 0;
@@ -607,16 +607,33 @@ public class DataTree {
             n.copyStat(s);
         }
         // now update if the path is in a quota subtree.
-        String lastPrefix = pTrie.findMaxPrefix(path);
+        String lastPrefix;
+        if((lastPrefix = getMaxPrefixWithQuota(path)) != null) {
+          this.updateBytes(lastPrefix, (data == null ? 0 : data.length)
+              - (lastdata == null ? 0 : lastdata.length));
+        }
+        dataWatches.triggerWatch(path, EventType.NodeDataChanged);
+        return s;
+    }
+
+    /**
+     * If there is a quota set, return the appropriate prefix for that quota
+     * Else return null
+     * @param The ZK path to check for quota
+     * @return Max quota prefix, or null if none
+     */
+    public String getMaxPrefixWithQuota(String path) {
         // do nothing for the root.
         // we are not keeping a quota on the zookeeper
         // root node for now.
+        String lastPrefix = pTrie.findMaxPrefix(path);
+
         if (!rootZookeeper.equals(lastPrefix) && !("".equals(lastPrefix))) {
-            this.updateBytes(lastPrefix, (data == null ? 0 : data.length)
-                    - (lastdata == null ? 0 : lastdata.length));
+            return lastPrefix;
+        }
+        else {
+            return null;
         }
-        dataWatches.triggerWatch(path, EventType.NodeDataChanged);
-        return s;
     }
 
     public byte[] getData(String path, Stat stat, Watcher watcher)
@@ -919,26 +936,24 @@ public class DataTree {
                 children = childs.toArray(new String[childs.size()]);
             }
         }
-        if (children != null) {
-            if (children.length == 0) {
-                // this node does not have a child
-                // is the leaf node
-                // check if its the leaf node
-                String endString = "/" + Quotas.limitNode;
-                if (path.endsWith(endString)) {
-                    // ok this is the limit node
-                    // get the real node and update
-                    // the count and the bytes
-                    String realPath = path.substring(Quotas.quotaZookeeper
-                            .length(), path.indexOf(endString));
-                    updateQuotaForPath(realPath);
-                    this.pTrie.addPath(realPath);
-                }
-                return;
-            }
-            for (String child : children) {
-                traverseNode(path + "/" + child);
+        if (children == null || children.length == 0) {
+            // this node does not have a child
+            // is the leaf node
+            // check if its the leaf node
+            String endString = "/" + Quotas.limitNode;
+            if (path.endsWith(endString)) {
+                // ok this is the limit node
+                // get the real node and update
+                // the count and the bytes
+                String realPath = path.substring(Quotas.quotaZookeeper
+                        .length(), path.indexOf(endString));
+                updateQuotaForPath(realPath);
+                this.pTrie.addPath(realPath);
             }
+            return;
+        }
+        for (String child : children) {
+            traverseNode(path + "/" + child);
         }
     }
 

Modified: zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java
URL: http://svn.apache.org/viewvc/zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java?rev=1139494&r1=1139493&r2=1139494&view=diff
==============================================================================
--- zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java (original)
+++ zookeeper/trunk/src/java/test/org/apache/zookeeper/test/ZooKeeperQuotaTest.java Sat Jun 25 06:13:03 2011
@@ -20,8 +20,6 @@ package org.apache.zookeeper.test;
 
 import java.io.IOException;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Quotas;
@@ -30,16 +28,21 @@ import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.ZooKeeperMain;
 import org.apache.zookeeper.ZooDefs.Ids;
 import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.server.ZooKeeperServer;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class ZooKeeperQuotaTest extends ClientBase {
+
+
     private static final Logger LOG = LoggerFactory.getLogger(
             ZooKeeperQuotaTest.class);
 
     @Test
     public void testQuota() throws IOException,
-        InterruptedException, KeeperException {
+        InterruptedException, KeeperException, Exception {
         final ZooKeeper zk = createClient();
         final String path = "/a/b/v";
         // making sure setdata works on /
@@ -55,18 +58,28 @@ public class ZooKeeperQuotaTest extends 
 
         zk.create("/a/b/v/d", "some".getBytes(), Ids.OPEN_ACL_UNSAFE,
                 CreateMode.PERSISTENT);
-        ZooKeeperMain.createQuota(zk, path, 1000L, 1000);
+        ZooKeeperMain.createQuota(zk, path, 5L, 10);
+
         // see if its set
         String absolutePath = Quotas.quotaZookeeper + path + "/" + Quotas.limitNode;
         byte[] data = zk.getData(absolutePath, false, new Stat());
         StatsTrack st = new StatsTrack(new String(data));
-        Assert.assertTrue("bytes are set", st.getBytes() == 1000L);
-        Assert.assertTrue("num count is set", st.getCount() == 1000);
+        Assert.assertTrue("bytes are set", st.getBytes() == 5L);
+        Assert.assertTrue("num count is set", st.getCount() == 10);
 
         String statPath = Quotas.quotaZookeeper + path + "/" + Quotas.statNode;
         byte[] qdata = zk.getData(statPath, false, new Stat());
         StatsTrack qst = new StatsTrack(new String(qdata));
         Assert.assertTrue("bytes are set", qst.getBytes() == 8L);
-        Assert.assertTrue("cound is set", qst.getCount() == 2);
+        Assert.assertTrue("count is set", qst.getCount() == 2);
+
+        //force server to restart and load from snapshot, not txn log
+        stopServer();
+        startServer();
+        stopServer();
+        startServer();
+        ZooKeeperServer server = getServer(serverFactory);
+        Assert.assertNotNull("Quota is still set",
+            server.getZKDatabase().getDataTree().getMaxPrefixWithQuota(path) != null);
     }
 }