You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2021/03/04 21:37:47 UTC

[accumulo] branch main updated: Convert tests from MAC to ZooKeeperTestingServer (#1952)

This is an automated email from the ASF dual-hosted git repository.

ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/main by this push:
     new 58aa28e  Convert tests from MAC to ZooKeeperTestingServer (#1952)
58aa28e is described below

commit 58aa28ee2a7f4f35e4419a5bbae8ad01acf04a43
Author: Dom G <47...@users.noreply.github.com>
AuthorDate: Thu Mar 4 16:37:35 2021 -0500

    Convert tests from MAC to ZooKeeperTestingServer (#1952)
    
    * Convert ZooLockITs use of MAC to ZooKeeperTestingServer
    * Make ZKTestingServer Auto-closeable
    * Move ZooKeeperTestingServer into try-with-resources block
    * Minor, unrelated improvements in FateMetricsIT
---
 .../categories/ZooKeeperTestingServerTests.java    | 25 ++++++++++++
 .../accumulo/test/fate/zookeeper/ZooLockIT.java    | 44 ++++++++++++----------
 .../zookeeper}/ZooMutatorIT.java                   | 38 +++++++++----------
 .../{FateMetricsTest.java => FateMetricsIT.java}   | 12 +++---
 .../test/zookeeper/ZooKeeperTestingServer.java     |  3 +-
 5 files changed, 75 insertions(+), 47 deletions(-)

diff --git a/start/src/main/java/org/apache/accumulo/test/categories/ZooKeeperTestingServerTests.java b/start/src/main/java/org/apache/accumulo/test/categories/ZooKeeperTestingServerTests.java
new file mode 100644
index 0000000..00c758c
--- /dev/null
+++ b/start/src/main/java/org/apache/accumulo/test/categories/ZooKeeperTestingServerTests.java
@@ -0,0 +1,25 @@
+/*
+ * 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.test.categories;
+
+/**
+ * Interface to be used with JUnit Category annotation to denote that the IntegrationTest requires
+ * the use of a ZooKeeperTestingServer.
+ */
+public interface ZooKeeperTestingServerTests {}
diff --git a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooLockIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooLockIT.java
index f65bff3..06dd0b0 100644
--- a/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooLockIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooLockIT.java
@@ -42,7 +42,8 @@ import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooLock.AccumuloLockWatcher;
 import org.apache.accumulo.fate.zookeeper.ZooLock.LockLossReason;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.harness.SharedMiniClusterBase;
+import org.apache.accumulo.test.categories.ZooKeeperTestingServerTests;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
@@ -54,19 +55,24 @@ import org.apache.zookeeper.data.ACL;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ZooLockIT extends SharedMiniClusterBase {
+@Category({ZooKeeperTestingServerTests.class})
+public class ZooLockIT {
+
+  private static ZooKeeperTestingServer szk = null;
 
   @BeforeClass
   public static void setup() throws Exception {
-    SharedMiniClusterBase.startMiniCluster();
+    szk = new ZooKeeperTestingServer();
+    szk.initPaths("/accumulo/" + UUID.randomUUID().toString());
   }
 
   @AfterClass
-  public static void teardown() {
-    SharedMiniClusterBase.stopMiniCluster();
+  public static void teardown() throws Exception {
+    szk.close();
   }
 
   static class ZooKeeperWrapper extends ZooKeeper {
@@ -76,9 +82,9 @@ public class ZooLockIT extends SharedMiniClusterBase {
       super(connectString, sessionTimeout, watcher);
     }
 
-    public String createOnce(String path, byte[] data, List<ACL> acl, CreateMode createMode)
+    public void createOnce(String path, byte[] data, List<ACL> acl, CreateMode createMode)
         throws KeeperException, InterruptedException {
-      return super.create(path, data, acl, createMode);
+      super.create(path, data, acl, createMode);
     }
 
     @Override
@@ -185,7 +191,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
 
   private static ZooLock getZooLock(String parent, UUID uuid) {
     Map<String,String> props = new HashMap<>();
-    props.put(Property.INSTANCE_ZK_HOST.toString(), getCluster().getZooKeepers());
+    props.put(Property.INSTANCE_ZK_HOST.toString(), szk.getConn());
     props.put(Property.INSTANCE_ZK_TIMEOUT.toString(), "30000");
     props.put(Property.INSTANCE_SECRET.toString(), "secret");
     return new ZooLock(new ConfigurationCopy(props), parent, uuid);
@@ -203,7 +209,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
 
     assertFalse(zl.isLocked());
 
-    ZooReaderWriter zk = new ZooReaderWriter(getCluster().getZooKeepers(), 30000, "secret");
+    ZooReaderWriter zk = new ZooReaderWriter(szk.getConn(), 30000, "secret");
 
     // intentionally created parent after lock
     zk.mkdirs(parent);
@@ -250,7 +256,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
   public void testDeleteLock() throws Exception {
     String parent = "/zltestDeleteLock-" + this.hashCode() + "-l" + pdCount.incrementAndGet();
 
-    ZooReaderWriter zk = new ZooReaderWriter(getCluster().getZooKeepers(), 30000, "secret");
+    ZooReaderWriter zk = new ZooReaderWriter(szk.getConn(), 30000, "secret");
     zk.mkdirs(parent);
 
     ZooLock zl = getZooLock(parent, UUID.randomUUID());
@@ -281,7 +287,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
   public void testDeleteWaiting() throws Exception {
     String parent = "/zltestDeleteWaiting-" + this.hashCode() + "-l" + pdCount.incrementAndGet();
 
-    ZooReaderWriter zk = new ZooReaderWriter(getCluster().getZooKeepers(), 30000, "secret");
+    ZooReaderWriter zk = new ZooReaderWriter(szk.getConn(), 30000, "secret");
     zk.mkdirs(parent);
 
     ZooLock zl = getZooLock(parent, UUID.randomUUID());
@@ -348,7 +354,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
     String parent = "/zltestUnexpectedEvent-" + this.hashCode() + "-l" + pdCount.incrementAndGet();
 
     ConnectedWatcher watcher = new ConnectedWatcher();
-    try (ZooKeeper zk = new ZooKeeper(getCluster().getZooKeepers(), 30000, watcher)) {
+    try (ZooKeeper zk = new ZooKeeper(szk.getConn(), 30000, watcher)) {
       zk.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
 
       while (!watcher.isConnected()) {
@@ -394,9 +400,8 @@ public class ZooLockIT extends SharedMiniClusterBase {
 
     ConnectedWatcher watcher1 = new ConnectedWatcher();
     ConnectedWatcher watcher2 = new ConnectedWatcher();
-    try (ZooKeeperWrapper zk1 = new ZooKeeperWrapper(getCluster().getZooKeepers(), 30000, watcher1);
-        ZooKeeperWrapper zk2 =
-            new ZooKeeperWrapper(getCluster().getZooKeepers(), 30000, watcher2)) {
+    try (ZooKeeperWrapper zk1 = new ZooKeeperWrapper(szk.getConn(), 30000, watcher1);
+        ZooKeeperWrapper zk2 = new ZooKeeperWrapper(szk.getConn(), 30000, watcher2)) {
 
       zk1.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
       zk2.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
@@ -508,8 +513,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
     public void run() {
       try {
         ConnectedWatcher watcher = new ConnectedWatcher();
-        try (ZooKeeperWrapper zk =
-            new ZooKeeperWrapper(getCluster().getZooKeepers(), 30000, watcher)) {
+        try (ZooKeeperWrapper zk = new ZooKeeperWrapper(szk.getConn(), 30000, watcher)) {
           zk.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
           while (!watcher.isConnected()) {
             Thread.sleep(50);
@@ -558,7 +562,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
     String parent = "/zlParallel";
 
     ConnectedWatcher watcher = new ConnectedWatcher();
-    try (ZooKeeperWrapper zk = new ZooKeeperWrapper(getCluster().getZooKeepers(), 30000, watcher)) {
+    try (ZooKeeperWrapper zk = new ZooKeeperWrapper(szk.getConn(), 30000, watcher)) {
       zk.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
 
       while (!watcher.isConnected()) {
@@ -631,7 +635,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
     ZooLock zl = getZooLock(parent, UUID.randomUUID());
 
     ConnectedWatcher watcher = new ConnectedWatcher();
-    try (ZooKeeper zk = new ZooKeeper(getCluster().getZooKeepers(), 30000, watcher)) {
+    try (ZooKeeper zk = new ZooKeeper(szk.getConn(), 30000, watcher)) {
       zk.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
 
       while (!watcher.isConnected()) {
@@ -666,7 +670,7 @@ public class ZooLockIT extends SharedMiniClusterBase {
   public void testChangeData() throws Exception {
     String parent = "/zltestChangeData-" + this.hashCode() + "-l" + pdCount.incrementAndGet();
     ConnectedWatcher watcher = new ConnectedWatcher();
-    try (ZooKeeper zk = new ZooKeeper(getCluster().getZooKeepers(), 30000, watcher)) {
+    try (ZooKeeper zk = new ZooKeeper(szk.getConn(), 30000, watcher)) {
       zk.addAuthInfo("digest", "accumulo:secret".getBytes(UTF_8));
 
       while (!watcher.isConnected()) {
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ZooMutatorIT.java b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java
similarity index 76%
rename from test/src/main/java/org/apache/accumulo/test/functional/ZooMutatorIT.java
rename to test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java
index 114dc27..27e3b7f 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ZooMutatorIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/fate/zookeeper/ZooMutatorIT.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.test.functional;
+package org.apache.accumulo.test.fate.zookeeper;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.junit.Assert.assertEquals;
@@ -24,33 +24,31 @@ import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import org.apache.accumulo.core.client.Accumulo;
-import org.apache.accumulo.core.clientImpl.ClientContext;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.harness.AccumuloClusterHarness;
-import org.apache.accumulo.test.categories.MiniClusterOnlyTests;
+import org.apache.accumulo.test.categories.ZooKeeperTestingServerTests;
+import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.google.common.hash.Hashing;
 
-@Category(MiniClusterOnlyTests.class)
-public class ZooMutatorIT extends AccumuloClusterHarness {
+@Category({ZooKeeperTestingServerTests.class})
+public class ZooMutatorIT {
   /**
    * This test uses multiple threads to update the data in a single zookeeper node using
-   * {@link ZooReaderWriter#mutateOrCreate(String, byte[], org.apache.accumulo.fate.zookeeper.ZooReaderWriter.Mutator)}
-   * and tries to detect errors and race conditions in that code. Each thread uses
-   * {@link #nextValue(String)} to compute a new value for the ZK node based on the current value,
-   * producing a new unique value. Its expected that multiple threads calling
-   * {@link #nextValue(String)} as previously described should yield the same final value as a
-   * single thread repeatedly calling {@link #nextValue(String)} the same number of times. There are
-   * many things that can go wrong in the multithreaded case. This test tries to ensure the
-   * following are true for the multithreaded case.
+   * {@link ZooReaderWriter#mutateOrCreate(String, byte[], ZooReaderWriter.Mutator)} and tries to
+   * detect errors and race conditions in that code. Each thread uses {@link #nextValue(String)} to
+   * compute a new value for the ZK node based on the current value, producing a new unique value.
+   * Its expected that multiple threads calling {@link #nextValue(String)} as previously described
+   * should yield the same final value as a single thread repeatedly calling
+   * {@link #nextValue(String)} the same number of times. There are many things that can go wrong in
+   * the multithreaded case. This test tries to ensure the following are true for the multithreaded
+   * case.
    *
    * <ul>
    * <li>All expected updates are made, none were skipped.
@@ -75,12 +73,10 @@ public class ZooMutatorIT extends AccumuloClusterHarness {
    */
   @Test
   public void concurrentMutatorTest() throws Exception {
-    try (var client = Accumulo.newClient().from(getClientProps()).build();
-        var context = (ClientContext) client) {
-      String secret = cluster.getSiteConfiguration().get(Property.INSTANCE_SECRET);
 
-      ZooReaderWriter zk = new ZooReaderWriter(context.getZooKeepers(),
-          context.getZooKeepersSessionTimeOut(), secret);
+    try (ZooKeeperTestingServer szk = new ZooKeeperTestingServer()) {
+      szk.initPaths("/accumulo/" + UUID.randomUUID().toString());
+      ZooReaderWriter zk = new ZooReaderWriter(szk.getConn(), 10_0000, "aPasswd");
 
       var executor = Executors.newFixedThreadPool(16);
 
diff --git a/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java b/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsIT.java
similarity index 96%
rename from test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java
rename to test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsIT.java
index f6695c8..74587e7 100644
--- a/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/metrics/fate/FateMetricsIT.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.manager.Manager;
 import org.apache.accumulo.manager.metrics.fate.FateMetrics;
 import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.categories.ZooKeeperTestingServerTests;
 import org.apache.accumulo.test.zookeeper.ZooKeeperTestingServer;
 import org.apache.zookeeper.ZooKeeper;
 import org.apache.zookeeper.data.Stat;
@@ -43,6 +44,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,14 +53,14 @@ import org.slf4j.LoggerFactory;
  * zookeeper server is used an the FATE repos are stubs, but this should represent the metrics
  * collection execution without needed to stand up a mini cluster to exercise these execution paths.
  */
-public class FateMetricsTest {
+@Category({ZooKeeperTestingServerTests.class})
+public class FateMetricsIT {
 
   public static final String INSTANCE_ID = "1234";
   public static final String MOCK_ZK_ROOT = "/accumulo/" + INSTANCE_ID;
   public static final String A_FAKE_SECRET = "aPasswd";
-  private static final Logger log = LoggerFactory.getLogger(FateMetricsTest.class);
+  private static final Logger log = LoggerFactory.getLogger(FateMetricsIT.class);
   private static ZooKeeperTestingServer szk = null;
-  private static ZooReaderWriter zooReaderWriter;
   private ZooStore<Manager> zooStore = null;
   private ZooKeeper zookeeper = null;
   private ServerContext context = null;
@@ -87,7 +89,7 @@ public class FateMetricsTest {
   @Before
   public void init() throws Exception {
 
-    zooReaderWriter = new ZooReaderWriter(szk.getConn(), 10_0000, "aPasswd");
+    ZooReaderWriter zooReaderWriter = new ZooReaderWriter(szk.getConn(), 10_0000, A_FAKE_SECRET);
     zookeeper = zooReaderWriter.getZooKeeper();
 
     clear(MOCK_ZK_ROOT);
@@ -179,7 +181,7 @@ public class FateMetricsTest {
   /**
    * Seeds the zoo store with a "fake" repo operation with a step, and sets the prop_debug field.
    * This emulates the actions performed with {@link org.apache.accumulo.fate.Fate} for what is
-   * expected in zookeeeper / the zoo store for an IN_PROGRESS transaction.
+   * expected in zookeeper / the zoo store for an IN_PROGRESS transaction.
    *
    * @throws Exception
    *           any exception is a test failure.
diff --git a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java
index c112aef..2eafa73 100644
--- a/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java
+++ b/test/src/main/java/org/apache/accumulo/test/zookeeper/ZooKeeperTestingServer.java
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
  * Uses Apache Curator to create a running zookeeper server for internal tests. The zookeeper port
  * is randomly assigned in case multiple instances are created by concurrent tests.
  */
-public class ZooKeeperTestingServer {
+public class ZooKeeperTestingServer implements AutoCloseable {
 
   private static final Logger log = LoggerFactory.getLogger(ZooKeeperTestingServer.class);
 
@@ -137,6 +137,7 @@ public class ZooKeeperTestingServer {
     }
   }
 
+  @Override
   public void close() throws IOException {
     if (zkServer != null) {
       zkServer.stop();