You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/10/20 03:05:47 UTC

[06/16] git commit: ACCUMULO-3242 Add some tests for ZooReaderWriter

ACCUMULO-3242 Add some tests for ZooReaderWriter


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/6e716364
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/6e716364
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/6e716364

Branch: refs/heads/master
Commit: 6e71636462b3a5886502afcaed008e78ced0b4ef
Parents: c023f74
Author: Josh Elser <el...@apache.org>
Authored: Sat Oct 18 14:17:21 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun Oct 19 20:16:04 2014 -0400

----------------------------------------------------------------------
 .../accumulo/fate/zookeeper/ZooReader.java      |  20 +-
 .../fate/zookeeper/ZooReaderWriter.java         |   6 +-
 .../fate/zookeeper/ZooReaderWriterTest.java     | 193 +++++++++++++++++++
 3 files changed, 208 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e716364/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
index 994f395..fd7938a 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReader.java
@@ -33,7 +33,7 @@ public class ZooReader implements IZooReader {
 
   protected String keepers;
   protected int timeout;
-  protected final RetryFactory retryFactory;
+  private final RetryFactory retryFactory;
 
   protected ZooKeeper getSession(String keepers, int timeout, String scheme, byte[] auth) {
     return ZooSession.getSession(keepers, timeout, scheme, auth);
@@ -43,6 +43,10 @@ public class ZooReader implements IZooReader {
     return getSession(keepers, timeout, null, null);
   }
 
+  protected RetryFactory getRetryFactory() {
+    return retryFactory;
+  }
+
   protected void retryOrThrow(Retry retry, KeeperException e) throws KeeperException {
     log.warn("Saw (possibly) transient exception communicating with ZooKeeper", e);
     if (retry.canRetry()) {
@@ -61,7 +65,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public byte[] getData(String zPath, boolean watch, Stat stat) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().getData(zPath, watch, stat);
@@ -80,7 +84,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public Stat getStatus(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, false);
@@ -99,7 +103,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public Stat getStatus(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, watcher);
@@ -118,7 +122,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public List<String> getChildren(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().getChildren(zPath, false);
@@ -137,7 +141,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public List<String> getChildren(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().getChildren(zPath, watcher);
@@ -156,7 +160,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public boolean exists(String zPath) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, false) != null;
@@ -175,7 +179,7 @@ public class ZooReader implements IZooReader {
 
   @Override
   public boolean exists(String zPath, Watcher watcher) throws KeeperException, InterruptedException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         return getZooKeeper().exists(zPath, watcher) != null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e716364/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
index b29b88a..79c2219 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriter.java
@@ -106,7 +106,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
 
   @Override
   public void delete(String path, int version) throws InterruptedException, KeeperException {
-    final Retry retry = retryFactory.create();
+    final Retry retry = getRetryFactory().create();
     while (true) {
       try {
         getZooKeeper().delete(path, version);
@@ -137,7 +137,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
   public byte[] mutate(String zPath, byte[] createValue, List<ACL> acl, Mutator mutator) throws Exception {
     if (createValue != null) {
       while (true) {
-        final Retry retry = retryFactory.create();
+        final Retry retry = getRetryFactory().create();
         try {
           getZooKeeper().create(zPath, createValue, acl, CreateMode.PERSISTENT);
           return createValue;
@@ -157,7 +157,7 @@ public class ZooReaderWriter extends ZooReader implements IZooReaderWriter {
       }
     }
     do {
-      final Retry retry = retryFactory.create();
+      final Retry retry = getRetryFactory().create();
       Stat stat = new Stat();
       byte[] data = getData(zPath, false, stat);
       data = mutator.mutate(data);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6e716364/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
----------------------------------------------------------------------
diff --git a/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
new file mode 100644
index 0000000..59fb498
--- /dev/null
+++ b/fate/src/test/java/org/apache/accumulo/fate/zookeeper/ZooReaderWriterTest.java
@@ -0,0 +1,193 @@
+/*
+ * 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.accumulo.fate.zookeeper;
+
+import java.lang.reflect.Method;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.BadVersionException;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.KeeperException.ConnectionLossException;
+import org.apache.zookeeper.KeeperException.NoNodeException;
+import org.apache.zookeeper.KeeperException.NodeExistsException;
+import org.apache.zookeeper.KeeperException.SessionExpiredException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Stat;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ZooReaderWriterTest {
+
+  private ZooReaderWriter zrw;
+  private ZooKeeper zk;
+  private RetryFactory retryFactory;
+  private Retry retry;
+
+  @Before
+  public void setup() {
+    zk = EasyMock.createMock(ZooKeeper.class);
+    zrw = EasyMock.createMockBuilder(ZooReaderWriter.class).addMockedMethods("getRetryFactory", "getZooKeeper").createMock();
+    retryFactory = EasyMock.createMock(RetryFactory.class);
+    retry = EasyMock.createMock(Retry.class);
+
+    EasyMock.expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
+    EasyMock.expect(zrw.getRetryFactory()).andReturn(retryFactory).anyTimes();
+    EasyMock.expect(retryFactory.create()).andReturn(retry).anyTimes();
+  }
+
+  @Test(expected = NoNodeException.class)
+  public void testDeleteFailOnInitialNoNode() throws Exception {
+    final String path = "/foo";
+    final int version = -1;
+
+    zk.delete(path, version);
+    EasyMock.expectLastCall().andThrow(KeeperException.create(Code.NONODE));
+    EasyMock.expect(retry.hasRetried()).andReturn(false);
+
+    EasyMock.replay(zk, zrw, retryFactory, retry);
+
+    zrw.delete(path, version);
+  }
+
+  @Test
+  public void testDeleteFailOnRetry() throws Exception {
+    final String path = "/foo";
+    final int version = -1;
+
+    zk.delete(path, version);
+    EasyMock.expectLastCall().andThrow(KeeperException.create(Code.CONNECTIONLOSS));
+    EasyMock.expect(retry.canRetry()).andReturn(true);
+    retry.useRetry();
+    EasyMock.expectLastCall().once();
+    retry.waitForNextAttempt();
+    EasyMock.expectLastCall().once();
+    zk.delete(path, version);
+    EasyMock.expectLastCall().andThrow(KeeperException.create(Code.NONODE));
+    EasyMock.expect(retry.hasRetried()).andReturn(true);
+
+    EasyMock.replay(zk, zrw, retryFactory, retry);
+
+    zrw.delete(path, version);
+
+    EasyMock.verify(zk, zrw, retryFactory, retry);
+  }
+
+  @Test(expected = SessionExpiredException.class)
+  public void testMutateNodeCreationFails() throws Exception {
+    final String path = "/foo";
+    final byte[] value = new byte[]{0};
+    final List<ACL> acls = Collections.<ACL> emptyList();
+    Mutator mutator = new Mutator() {
+      @Override
+      public byte[] mutate(byte[] currentValue) throws Exception {
+        return new byte[] {1};
+      }
+    };
+
+    zk.create(path, value, acls, CreateMode.PERSISTENT);
+    EasyMock.expectLastCall().andThrow(new SessionExpiredException()).once();
+    EasyMock.expect(retry.canRetry()).andReturn(false);
+    EasyMock.expect(retry.retriesCompleted()).andReturn(1l).once();
+
+    EasyMock.replay(zk, zrw, retryFactory, retry);
+
+    zrw.mutate(path, value, acls, mutator);
+  }
+
+  @Test
+  public void testMutateWithBadVersion() throws Exception {
+    final String path = "/foo";
+    final byte[] value = new byte[] {0};
+    final List<ACL> acls = Collections.<ACL> emptyList();
+    final byte[] mutatedBytes = new byte[] {1};
+    Mutator mutator = new Mutator() {
+      @Override
+      public byte[] mutate(byte[] currentValue) throws Exception {
+        return mutatedBytes;
+      }
+    };
+
+    Method getDataMethod = ZooReaderWriter.class.getMethod("getData", String.class, boolean.class, Stat.class);
+    zrw = EasyMock.createMockBuilder(ZooReaderWriter.class).addMockedMethods("getRetryFactory", "getZooKeeper").addMockedMethod(getDataMethod).createMock();
+    EasyMock.expect(zrw.getRetryFactory()).andReturn(retryFactory).anyTimes();
+    EasyMock.expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
+
+    Stat stat = new Stat();
+
+    zk.create(path, value, acls, CreateMode.PERSISTENT);
+    EasyMock.expectLastCall().andThrow(new NodeExistsException()).once();
+    EasyMock.expect(zrw.getData(path, false, stat)).andReturn(new byte[] {3}).times(2);
+    // BadVersionException should retry
+    EasyMock.expect(zk.setData(path, mutatedBytes, 0)).andThrow(new BadVersionException());
+    // Let 2nd setData succeed
+    EasyMock.expect(zk.setData(path, mutatedBytes, 0)).andReturn(null);
+
+    EasyMock.replay(zk, zrw, retryFactory, retry);
+
+    Assert.assertArrayEquals(new byte[] {1}, zrw.mutate(path, value, acls, mutator));
+
+    EasyMock.verify(zk, zrw, retryFactory, retry);
+  }
+
+  @Test
+  public void testMutateWithRetryOnSetData() throws Exception {
+    final String path = "/foo";
+    final byte[] value = new byte[] {0};
+    final List<ACL> acls = Collections.<ACL> emptyList();
+    final byte[] mutatedBytes = new byte[] {1};
+    Mutator mutator = new Mutator() {
+      @Override
+      public byte[] mutate(byte[] currentValue) throws Exception {
+        return mutatedBytes;
+      }
+    };
+
+    Method getDataMethod = ZooReaderWriter.class.getMethod("getData", String.class, boolean.class, Stat.class);
+    zrw = EasyMock.createMockBuilder(ZooReaderWriter.class).addMockedMethods("getRetryFactory", "getZooKeeper").addMockedMethod(getDataMethod).createMock();
+    EasyMock.expect(zrw.getRetryFactory()).andReturn(retryFactory).anyTimes();
+    EasyMock.expect(zrw.getZooKeeper()).andReturn(zk).anyTimes();
+
+    Stat stat = new Stat();
+
+    zk.create(path, value, acls, CreateMode.PERSISTENT);
+    EasyMock.expectLastCall().andThrow(new NodeExistsException()).once();
+    EasyMock.expect(zrw.getData(path, false, stat)).andReturn(new byte[] {3}).times(2);
+    // BadVersionException should retry
+    EasyMock.expect(zk.setData(path, mutatedBytes, 0)).andThrow(new ConnectionLossException());
+
+    EasyMock.expect(retry.canRetry()).andReturn(true);
+    retry.useRetry();
+    EasyMock.expectLastCall();
+    retry.waitForNextAttempt();
+    EasyMock.expectLastCall();
+    // Let 2nd setData succeed
+    EasyMock.expect(zk.setData(path, mutatedBytes, 0)).andReturn(null);
+
+    EasyMock.replay(zk, zrw, retryFactory, retry);
+
+    Assert.assertArrayEquals(new byte[] {1}, zrw.mutate(path, value, acls, mutator));
+
+    EasyMock.verify(zk, zrw, retryFactory, retry);
+  }
+}