You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by ma...@apache.org on 2008/11/19 00:43:34 UTC

svn commit: r718801 - in /hadoop/zookeeper/branches/branch-3.0: CHANGES.txt src/java/main/org/apache/zookeeper/server/DataNode.java src/java/test/org/apache/zookeeper/test/NullDataTest.java

Author: mahadev
Date: Tue Nov 18 15:43:34 2008
New Revision: 718801

URL: http://svn.apache.org/viewvc?rev=718801&view=rev
Log:
ZOOKEEPER-226. fix exists calls that fail on server if node has null data. (mahadev)

Added:
    hadoop/zookeeper/branches/branch-3.0/src/java/test/org/apache/zookeeper/test/NullDataTest.java
Modified:
    hadoop/zookeeper/branches/branch-3.0/CHANGES.txt
    hadoop/zookeeper/branches/branch-3.0/src/java/main/org/apache/zookeeper/server/DataNode.java

Modified: hadoop/zookeeper/branches/branch-3.0/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.0/CHANGES.txt?rev=718801&r1=718800&r2=718801&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.0/CHANGES.txt (original)
+++ hadoop/zookeeper/branches/branch-3.0/CHANGES.txt Tue Nov 18 15:43:34 2008
@@ -25,6 +25,9 @@
    ZOOKEEPER-206. documentation tab should contain the version number and 
    other small site changes. (pat via mahadev) 
 
+   ZOOKEEPER-226. fix exists calls that fail on server if node has null data.
+   (mahadev) 
+
 Release 3.0.0 - 2008-10-21
 
 Non-backward compatible changes:

Modified: hadoop/zookeeper/branches/branch-3.0/src/java/main/org/apache/zookeeper/server/DataNode.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.0/src/java/main/org/apache/zookeeper/server/DataNode.java?rev=718801&r1=718800&r2=718801&view=diff
==============================================================================
--- hadoop/zookeeper/branches/branch-3.0/src/java/main/org/apache/zookeeper/server/DataNode.java (original)
+++ hadoop/zookeeper/branches/branch-3.0/src/java/main/org/apache/zookeeper/server/DataNode.java Tue Nov 18 15:43:34 2008
@@ -96,7 +96,7 @@
         to.setPzxid(stat.getPzxid());
         to.setVersion(stat.getVersion());
         to.setEphemeralOwner(stat.getEphemeralOwner());
-        to.setDataLength(data.length);
+        to.setDataLength(data == null ? 0 : data.length);
         to.setNumChildren(children.size());
     }
 

Added: hadoop/zookeeper/branches/branch-3.0/src/java/test/org/apache/zookeeper/test/NullDataTest.java
URL: http://svn.apache.org/viewvc/hadoop/zookeeper/branches/branch-3.0/src/java/test/org/apache/zookeeper/test/NullDataTest.java?rev=718801&view=auto
==============================================================================
--- hadoop/zookeeper/branches/branch-3.0/src/java/test/org/apache/zookeeper/test/NullDataTest.java (added)
+++ hadoop/zookeeper/branches/branch-3.0/src/java/test/org/apache/zookeeper/test/NullDataTest.java Tue Nov 18 15:43:34 2008
@@ -0,0 +1,72 @@
+/**
+ * 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.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.Stat;
+import org.apache.zookeeper.AsyncCallback.StatCallback;
+
+public class NullDataTest extends ClientBase implements StatCallback {
+    String snapCount;
+    CountDownLatch cn = new CountDownLatch(1);
+    
+    @Override
+    protected void setUp() throws Exception {
+        // Change the snapcount to happen more often
+        snapCount = System.getProperty("zookeeper.snapCount", "1024");
+        System.setProperty("zookeeper.snapCount", "10");
+        super.setUp();
+    }
+    
+    @Override
+    protected void tearDown() throws Exception {
+        System.setProperty("zookeeper.snapCount", snapCount);
+        super.tearDown();
+    }
+    
+    public void testNullData() throws IOException, 
+        InterruptedException, KeeperException {
+        String path = "/SIZE";
+        ZooKeeper zk = null;
+        zk = createClient();
+        try {
+            zk.create(path, null, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            // try sync zk exists 
+            zk.exists(path, false);
+            zk.exists(path, false, this , null);
+            cn.await(10, TimeUnit.SECONDS);
+            assertSame(0L, cn.getCount());
+        } finally {
+            if(zk != null)
+                zk.close();
+        }
+        
+    }
+
+    public void processResult(int rc, String path, Object ctx, Stat stat) {
+        cn.countDown();
+    }
+}
\ No newline at end of file