You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by be...@apache.org on 2012/10/27 00:07:10 UTC

svn commit: r1402690 - in /incubator/mesos/trunk/src: ./ jvm/ sched/ tests/

Author: benh
Date: Fri Oct 26 22:07:09 2012
New Revision: 1402690

URL: http://svn.apache.org/viewvc?rev=1402690&view=rev
Log:
Refactored and simplified the ZooKeeper test fixture(s) and tests. In
particular:

(a) Eliminated the verbose logging coming from the
    AllocatorZooKeeperTest tests that was causing a lot of output when
    the tests ran.

(b) Made sure all the ZooKeeper clients are properly shutdown in the
    AllocatorZooKeeperTest tests (this was mainly just invoking
    MasterDetector::destroy on all created detectors).

(c) Renamed ZooKeeperServer to ZooKeeperServerTest so as not to
    conflate it with the Java class
    org.apache.zookeeper.server.ZooKeeperServer.

(d) Added an AssertZKGet helper to get better error messages.

Updated BaseZooKeeperTest fixture to shutdown embedeed JVM after the
tests have completed.

Review: https://reviews.apache.org/r/7713

Added:
    incubator/mesos/trunk/src/tests/zookeeper_test.cpp
      - copied, changed from r1402689, incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp
    incubator/mesos/trunk/src/tests/zookeeper_test.hpp
      - copied, changed from r1402689, incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp
    incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp
      - copied, changed from r1402689, incubator/mesos/trunk/src/tests/zookeeper_server.cpp
    incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp
      - copied, changed from r1402689, incubator/mesos/trunk/src/tests/zookeeper_server.hpp
Removed:
    incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp
    incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp
    incubator/mesos/trunk/src/tests/zookeeper_server.cpp
    incubator/mesos/trunk/src/tests/zookeeper_server.hpp
    incubator/mesos/trunk/src/tests/zookeeper_server_tests.cpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/jvm/jvm.hpp
    incubator/mesos/trunk/src/sched/sched.cpp
    incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
    incubator/mesos/trunk/src/tests/state_tests.cpp
    incubator/mesos/trunk/src/tests/zookeeper_tests.cpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Fri Oct 26 22:07:09 2012
@@ -212,8 +212,8 @@ libmesos_no_third_party_la_SOURCES += co
 	slave/process_based_isolation_module.hpp slave/reaper.hpp	\
 	slave/slave.hpp slave/solaris_project_isolation_module.hpp	\
 	slave/webui.hpp tests/external_test.hpp				\
-	tests/base_zookeeper_test.hpp tests/utils.hpp			\
-	tests/zookeeper_server.hpp zookeeper/authentication.hpp		\
+	tests/zookeeper_test.hpp tests/utils.hpp			\
+	tests/zookeeper_test_server.hpp zookeeper/authentication.hpp	\
 	zookeeper/group.hpp zookeeper/watcher.hpp			\
 	zookeeper/zookeeper.hpp zookeeper/url.hpp
 
@@ -806,9 +806,8 @@ if OS_LINUX
 endif
 
 if HAS_JAVA
-  mesos_tests_SOURCES += tests/zookeeper_server.cpp		\
-                         tests/base_zookeeper_test.cpp		\
-                         tests/zookeeper_server_tests.cpp	\
+  mesos_tests_SOURCES += tests/zookeeper_test.cpp		\
+                         tests/zookeeper_test_server.cpp	\
                          tests/zookeeper_tests.cpp		\
                          tests/allocator_zookeeper_tests.cpp
   mesos_tests_CPPFLAGS += $(JAVA_CPPFLAGS)

Modified: incubator/mesos/trunk/src/jvm/jvm.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/jvm/jvm.hpp?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/jvm/jvm.hpp (original)
+++ incubator/mesos/trunk/src/jvm/jvm.hpp Fri Oct 26 22:07:09 2012
@@ -22,7 +22,7 @@ namespace internal {
 // Facilitates embedding a jvm and calling into it.
 //
 // TODO(John Sirois): Fix variadic methods.  Possibly a way to do this with
-// typelists, type concatenation and unwinding builder inheritance
+// typelists, type concatenation and unwinding builder inheritance.
 //
 // TODO(John Sirois): Support finding static methods.
 class Jvm
@@ -196,7 +196,7 @@ public:
     ~Attach();
 
   private:
-      Jvm* _jvm;
+    Jvm* _jvm;
   };
 
   friend class Attach;

Modified: incubator/mesos/trunk/src/sched/sched.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/sched/sched.cpp?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/sched/sched.cpp (original)
+++ incubator/mesos/trunk/src/sched/sched.cpp Fri Oct 26 22:07:09 2012
@@ -113,7 +113,9 @@ protected:
   {
     // The master detector needs to be created after this process is
     // running so that the "master detected" message is not dropped.
-    detector = MasterDetector::create(url, self(), false, false);
+    // TODO(benh): Get access to flags so that we can decide whether
+    // or not to make ZooKeeper verbose.
+    detector = MasterDetector::create(url, self(), false);
     if (detector.isError()) {
       driver->abort();
       scheduler->error(driver, detector.error());

Modified: incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp Fri Oct 26 22:07:09 2012
@@ -23,7 +23,7 @@
 #include "master/allocator.hpp"
 #include "master/master.hpp"
 
-#include "tests/base_zookeeper_test.hpp"
+#include "tests/zookeeper_test.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -50,13 +50,8 @@ using testing::SaveArg;
 
 
 template <typename T = AllocatorProcess>
-class AllocatorZooKeeperTest : public BaseZooKeeperTest
+class AllocatorZooKeeperTest : public ZooKeeperTest
 {
-public:
-  static void SetUpTestCase() {
-    BaseZooKeeperTest::SetUpTestCase();
-  }
-
 protected:
   T allocator1;
   MockAllocator<T> allocator2;
@@ -104,11 +99,10 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
   Master m(&this->allocator1, &files);
   PID<Master> master1 = process::spawn(&m);
 
-  string zk = "zk://" + this->zks->connectString() + "/znode";
+  string zk = "zk://" + this->server->connectString() + "/znode";
   Try<MasterDetector*> detector =
     MasterDetector::create(zk, master1, true, true);
-  CHECK(!detector.isError())
-    << "Failed to create a master detector: " << detector.error();
+  ASSERT_SOME(detector);
 
   MockExecutor exec;
 
@@ -129,10 +123,9 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
   Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
-  Try<MasterDetector*> slave_detector =
+  Try<MasterDetector*> slaveDetector =
     MasterDetector::create(zk, slave, false, true);
-  CHECK(!slave_detector.isError())
-    << "Failed to create a master detector: " << slave_detector.error();
+  ASSERT_SOME(slaveDetector);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO,zk);
@@ -188,8 +181,7 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
 
   Try<MasterDetector*> detector2 =
     MasterDetector::create(zk, master2, true, true);
-  CHECK(!detector2.isError())
-    << "Failed to create a master detector: " << detector2.error();
+  ASSERT_SOME(detector2);
 
   WAIT_UNTIL(frameworkAddedTrigger);
 
@@ -211,11 +203,13 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
 
   process::terminate(slave);
   process::wait(slave);
+  MasterDetector::destroy(slaveDetector.get());
 
   WAIT_UNTIL(slaveRemovedTrigger);
 
   process::terminate(master2);
   process::wait(master2);
+  MasterDetector::destroy(detector2.get());
 }
 
 
@@ -256,11 +250,10 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
   Master m(&this->allocator1, &files);
   PID<Master> master1 = process::spawn(&m);
 
-  string zk = "zk://" + this->zks->connectString() + "/znode";
+  string zk = "zk://" + this->server->connectString() + "/znode";
   Try<MasterDetector*> detector =
     MasterDetector::create(zk, master1, true, true);
-  CHECK(!detector.isError())
-    << "Failed to create a master detector: " << detector.error();
+  ASSERT_SOME(detector);
 
   MockExecutor exec;
 
@@ -281,10 +274,9 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
   Slave s(resources, true, &isolationModule, &files);
   PID<Slave> slave = process::spawn(&s);
 
-  Try<MasterDetector*> slave_detector =
+  Try<MasterDetector*> slaveDetector =
     MasterDetector::create(zk, slave, false, true);
-  CHECK(!slave_detector.isError())
-    << "Failed to create a master detector: " << slave_detector.error();
+  ASSERT_SOME(slaveDetector);
 
   MockScheduler sched;
   MesosSchedulerDriver driver(&sched, DEFAULT_FRAMEWORK_INFO,zk);
@@ -340,8 +332,7 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
 
   Try<MasterDetector*> detector2 =
     MasterDetector::create(zk, master2, true, true);
-  CHECK(!detector2.isError())
-    << "Failed to create a master detector: " << detector2.error();
+  ASSERT_SOME(detector2);
 
   WAIT_UNTIL(slaveAddedTrigger);
 
@@ -363,9 +354,11 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
 
   process::terminate(slave);
   process::wait(slave);
+  MasterDetector::destroy(slaveDetector.get());
 
   WAIT_UNTIL(slaveRemovedTrigger);
 
   process::terminate(master2);
   process::wait(master2);
+  MasterDetector::destroy(detector2.get());
 }

Modified: incubator/mesos/trunk/src/tests/state_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/state_tests.cpp?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/state_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/state_tests.cpp Fri Oct 26 22:07:09 2012
@@ -40,7 +40,7 @@
 #include "state/zookeeper.hpp"
 
 #ifdef MESOS_HAS_JAVA
-#include "tests/base_zookeeper_test.hpp"
+#include "tests/zookeeper_test.hpp"
 #endif
 #include "tests/utils.hpp"
 
@@ -287,7 +287,7 @@ TEST_F(LevelDBStateTest, Names)
 
 
 #ifdef MESOS_HAS_JAVA
-class ZooKeeperStateTest : public mesos::internal::test::BaseZooKeeperTest
+class ZooKeeperStateTest : public mesos::internal::test::ZooKeeperTest
 {
 public:
   ZooKeeperStateTest()
@@ -296,9 +296,9 @@ public:
 protected:
   virtual void SetUp()
   {
-    BaseZooKeeperTest::SetUp();
+    ZooKeeperTest::SetUp();
     state = new ZooKeeperState<ProtobufSerializer>(
-        zks->connectString(),
+        server->connectString(),
         NO_TIMEOUT,
         "/state/");
   }
@@ -306,7 +306,7 @@ protected:
   virtual void TearDown()
   {
     delete state;
-    BaseZooKeeperTest::TearDown();
+    ZooKeeperTest::TearDown();
   }
 
   State<ProtobufSerializer>* state;

Copied: incubator/mesos/trunk/src/tests/zookeeper_test.cpp (from r1402689, incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test.cpp?p2=incubator/mesos/trunk/src/tests/zookeeper_test.cpp&p1=incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp&r1=1402689&r2=1402690&rev=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/base_zookeeper_test.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test.cpp Fri Oct 26 22:07:09 2012
@@ -18,33 +18,34 @@
 
 #include <signal.h>
 
-#include <queue>
-
-#include <glog/logging.h>
-
 #include <gtest/gtest.h>
 
+#include <queue>
+
 #include <tr1/functional>
 
+#include <stout/lambda.hpp>
+
 #include "common/lock.hpp"
 
+#include "logging/logging.hpp"
+
 #include "jvm/jvm.hpp"
 
-#include "tests/base_zookeeper_test.hpp"
 #include "tests/utils.hpp"
-#include "tests/zookeeper_server.hpp"
-
-using mesos::internal::test::mesosSourceDirectory;
-using std::tr1::bind;
-using std::tr1::function;
-
-namespace params = std::tr1::placeholders;
+#include "tests/zookeeper_test.hpp"
+#include "tests/zookeeper_test_server.hpp"
 
 namespace mesos {
 namespace internal {
 namespace test {
 
-const Milliseconds BaseZooKeeperTest::NO_TIMEOUT(5000);
+const Milliseconds ZooKeeperTest::NO_TIMEOUT(5000);
+
+// Note that we NEVER delete the Jvm instance because you can only
+// create one JVM since destructing a JVM has issues (see:
+// http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4712793).
+Jvm* ZooKeeperTest::jvm = NULL;
 
 
 static void silenceServerLogs(Jvm* jvm)
@@ -52,14 +53,18 @@ static void silenceServerLogs(Jvm* jvm)
   Jvm::Attach attach(jvm);
 
   Jvm::JClass loggerClass = Jvm::JClass::forName("org/apache/log4j/Logger");
-  jobject rootLogger = jvm->invokeStatic<jobject>(
-      jvm->findStaticMethod(loggerClass.method("getRootLogger")
-          .returns(loggerClass)));
+  jobject rootLogger =jvm->invokeStatic<jobject>(
+      jvm->findStaticMethod(loggerClass
+                            .method("getRootLogger")
+                            .returns(loggerClass)));
 
   Jvm::JClass levelClass = Jvm::JClass::forName("org/apache/log4j/Level");
-  jvm->invoke<void>(rootLogger,
-      jvm->findMethod(loggerClass.method("setLevel").parameter(levelClass)
-          .returns(jvm->voidClass)),
+  jvm->invoke<void>(
+      rootLogger,
+      jvm->findMethod(loggerClass
+                      .method("setLevel")
+                      .parameter(levelClass)
+                      .returns(jvm->voidClass)),
       jvm->getStaticField<jobject>(jvm->findStaticField(levelClass, "OFF")));
 }
 
@@ -71,49 +76,43 @@ static void silenceClientLogs()
 }
 
 
-void BaseZooKeeperTest::SetUpTestCase()
+void ZooKeeperTest::SetUpTestCase()
 {
-  static Jvm* singleton;
-  if (singleton == NULL) {
-    std::vector<std::string> opts;
-
+  if (jvm == NULL) {
     std::string zkHome = mesosBuildDirectory +
-        "/third_party/zookeeper-" ZOOKEEPER_VERSION;
+      "/third_party/zookeeper-" ZOOKEEPER_VERSION;
+
     std::string classpath = "-Djava.class.path=" +
-        zkHome + "/zookeeper-" ZOOKEEPER_VERSION ".jar:" +
-        zkHome + "/lib/log4j-1.2.15.jar";
+      zkHome + "/zookeeper-" ZOOKEEPER_VERSION ".jar:" +
+      zkHome + "/lib/log4j-1.2.15.jar";
+
     LOG(INFO) << "Using classpath setup: " << classpath << std::endl;
+
+    std::vector<std::string> opts;
     opts.push_back(classpath);
-    singleton = new Jvm(opts);
+    jvm = new Jvm(opts);
 
-    silenceServerLogs(singleton);
+    silenceServerLogs(jvm);
     silenceClientLogs();
   }
-
-  // TODO(John Sirois): Introduce a mechanism to contribute classpath
-  // requirements to a singleton Jvm, then access the singleton here.
-  jvm = singleton;
 }
 
 
-void BaseZooKeeperTest::SetUp()
+void ZooKeeperTest::SetUp()
 {
-  zks = new ZooKeeperServer(jvm);
-  zks->startNetwork();
+  server = new ZooKeeperTestServer(jvm);
+  server->startNetwork();
 };
 
 
-void BaseZooKeeperTest::TearDown()
+void ZooKeeperTest::TearDown()
 {
-  zks->shutdownNetwork();
-  delete zks;
+  delete server;
+  server = NULL;
 };
 
 
-Jvm* BaseZooKeeperTest::jvm = NULL;
-
-
-BaseZooKeeperTest::TestWatcher::TestWatcher()
+ZooKeeperTest::TestWatcher::TestWatcher()
 {
   pthread_mutexattr_t attr;
   pthread_mutexattr_init(&attr);
@@ -124,17 +123,18 @@ BaseZooKeeperTest::TestWatcher::TestWatc
 }
 
 
-BaseZooKeeperTest::TestWatcher::~TestWatcher()
+ZooKeeperTest::TestWatcher::~TestWatcher()
 {
   pthread_mutex_destroy(&mutex);
   pthread_cond_destroy(&cond);
 }
 
 
-void BaseZooKeeperTest::TestWatcher::process(ZooKeeper* zk,
-                                             int type,
-                                             int state,
-                                             const std::string& path)
+void ZooKeeperTest::TestWatcher::process(
+    ZooKeeper* zk,
+    int type,
+    int state,
+    const std::string& path)
 {
   Lock lock(&mutex);
   events.push(Event(type, state, path));
@@ -142,34 +142,36 @@ void BaseZooKeeperTest::TestWatcher::pro
 }
 
 
-static bool isSessionState(const BaseZooKeeperTest::TestWatcher::Event& event,
-                           int state)
+static bool isSessionState(
+    const ZooKeeperTest::TestWatcher::Event& event,
+    int state)
 {
   return event.type == ZOO_SESSION_EVENT && event.state == state;
 }
 
 
-void BaseZooKeeperTest::TestWatcher::awaitSessionEvent(int state)
+void ZooKeeperTest::TestWatcher::awaitSessionEvent(int state)
 {
-  awaitEvent(bind(&isSessionState, params::_1, state));
+  awaitEvent(lambda::bind(&isSessionState, lambda::_1, state));
 }
 
 
-static bool isCreated(const BaseZooKeeperTest::TestWatcher::Event& event,
-                      const std::string& path)
+static bool isCreated(
+    const ZooKeeperTest::TestWatcher::Event& event,
+    const std::string& path)
 {
   return event.type == ZOO_CHILD_EVENT && event.path == path;
 }
 
 
-void BaseZooKeeperTest::TestWatcher::awaitCreated(const std::string& path)
+void ZooKeeperTest::TestWatcher::awaitCreated(const std::string& path)
 {
-  awaitEvent(bind(&isCreated, params::_1, path));
+  awaitEvent(lambda::bind(&isCreated, lambda::_1, path));
 }
 
 
-BaseZooKeeperTest::TestWatcher::Event
-BaseZooKeeperTest::TestWatcher::awaitEvent()
+ZooKeeperTest::TestWatcher::Event
+ZooKeeperTest::TestWatcher::awaitEvent()
 {
   Lock lock(&mutex);
   while (true) {
@@ -183,8 +185,9 @@ BaseZooKeeperTest::TestWatcher::awaitEve
 }
 
 
-BaseZooKeeperTest::TestWatcher::Event
-BaseZooKeeperTest::TestWatcher::awaitEvent(function<bool(Event)> matches)
+ZooKeeperTest::TestWatcher::Event
+ZooKeeperTest::TestWatcher::awaitEvent(
+    const std::tr1::function<bool(Event)>& matches)
 {
   while (true) {
     Event event = awaitEvent();

Copied: incubator/mesos/trunk/src/tests/zookeeper_test.hpp (from r1402689, incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test.hpp?p2=incubator/mesos/trunk/src/tests/zookeeper_test.hpp&p1=incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp&r1=1402689&r2=1402690&rev=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/base_zookeeper_test.hpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test.hpp Fri Oct 26 22:07:09 2012
@@ -16,38 +16,37 @@
  * limitations under the License.
  */
 
-#ifndef __BASE_ZOOKEEPER_TEST_HPP__
-#define __BASE_ZOOKEEPER_TEST_HPP__
+#ifndef __ZOOKEEPER_TEST_HPP__
+#define __ZOOKEEPER_TEST_HPP__
 
-#include <queue>
+#include <pthread.h>
 
 #include <gtest/gtest.h>
 
+#include <queue>
+
 #include <tr1/functional>
 
 #include <stout/duration.hpp>
 
 #include "jvm/jvm.hpp"
 
-#include "tests/zookeeper_server.hpp"
-
-using std::tr1::function;
+#include "tests/zookeeper_test_server.hpp"
 
 namespace mesos {
 namespace internal {
 namespace test {
 
-// A baseclass for tests that need to interact with a ZooKeeper server ensemble.
-// Tests classes need only extend from this base and implement test methods
-// using TEST_F to access the in process ZooKeeperServer, zks.  This test base
-// ensures the server is started before each test and shutdown after it so that
-// each test is presented with a ZooKeeper ensemble with no data or watches.
-class BaseZooKeeperTest : public ::testing::Test
+// A fixture for tests that need to interact with a ZooKeeper server
+// ensemble. Tests can access the in process ZooKeeperTestServer via
+// the variable 'server'. This test fixture ensures the server is
+// started before each test and shutdown after it so that each test is
+// presented with a ZooKeeper ensemble with no data or watches.
+class ZooKeeperTest : public ::testing::Test
 {
 public:
-
-  // A watcher that is useful to install in a ZooKeeper client for tests.
-  // Allows easy blocking on expected events.
+  // A watcher that is useful to install in a ZooKeeper client for
+  // tests. Allows easy blocking on expected events.
   class TestWatcher : public Watcher
   {
   public:
@@ -64,7 +63,10 @@ public:
     TestWatcher();
     virtual ~TestWatcher();
 
-    virtual void process(ZooKeeper* zk, int type, int state,
+    virtual void process(
+        ZooKeeper* zk,
+        int type,
+        int state,
         const std::string& path);
 
     // Blocks until the session event of the given state fires.
@@ -74,7 +76,7 @@ public:
     void awaitCreated(const std::string& path);
 
     // Blocks until an event is fired matching the given predicate.
-    Event awaitEvent(function<bool(Event)> matches);
+    Event awaitEvent(const std::tr1::function<bool(Event)>& matches);
 
     // Blocks until an event is fired.
     Event awaitEvent();
@@ -85,16 +87,18 @@ public:
     pthread_cond_t cond;
   };
 
-protected:
+  ZooKeeperTest() : server(NULL) {}
+
   static void SetUpTestCase();
 
+protected:
   virtual void SetUp();
   virtual void TearDown();
 
   // A very long session timeout that simulates no timeout for test cases.
   static const Milliseconds NO_TIMEOUT;
 
-  ZooKeeperServer* zks;
+  ZooKeeperTestServer* server;
 
 private:
   static Jvm* jvm;
@@ -104,4 +108,4 @@ private:
 } // namespace internal
 } // namespace mesos
 
-#endif /* __BASE_ZOOKEEPER_TEST_HPP__ */
+#endif // __ZOOKEEPER_TEST_HPP__

Copied: incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp (from r1402689, incubator/mesos/trunk/src/tests/zookeeper_server.cpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp?p2=incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp&p1=incubator/mesos/trunk/src/tests/zookeeper_server.cpp&r1=1402689&r2=1402690&rev=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_server.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test_server.cpp Fri Oct 26 22:07:09 2012
@@ -28,108 +28,122 @@
 
 #include "jvm/jvm.hpp"
 
-#include "tests/zookeeper_server.hpp"
+#include "tests/zookeeper_test_server.hpp"
 
 namespace mesos {
 namespace internal {
 namespace test {
 
-ZooKeeperServer::ZooKeeperServer(Jvm* _jvm) : jvm(_jvm), port(0), started(false)
+ZooKeeperTestServer::ZooKeeperTestServer(Jvm* _jvm)
+  : jvm(_jvm),
+    port(0),
+    started(false)
 {
   Jvm::Attach attach(jvm);
 
   Jvm::JClass fileClass = Jvm::JClass::forName("java/io/File");
-  fileConstructor =
-      new Jvm::JConstructor(
-          jvm->findConstructor(
-              fileClass.constructor().parameter(jvm->stringClass)));
+  fileConstructor = new Jvm::JConstructor(
+      jvm->findConstructor(
+          fileClass
+          .constructor()
+          .parameter(jvm->stringClass)));
 
   Jvm::JClass inetSocketAddressClass =
       Jvm::JClass::forName("java/net/InetSocketAddress");
-  inetSocketAddressConstructor =
-      new Jvm::JConstructor(
-          jvm->findConstructor(
-              inetSocketAddressClass.constructor().parameter(jvm->intClass)));
+  inetSocketAddressConstructor = new Jvm::JConstructor(
+      jvm->findConstructor(
+          inetSocketAddressClass
+          .constructor()
+          .parameter(jvm->intClass)));
 
   Jvm::JClass cnxnFactoryClass =
       Jvm::JClass::forName("org/apache/zookeeper/server/NIOServerCnxn$Factory");
-  cnxnFactoryConstructor =
-      new Jvm::JConstructor(
-          jvm->findConstructor(cnxnFactoryClass.constructor()
-              .parameter(inetSocketAddressClass)));
+  cnxnFactoryConstructor = new Jvm::JConstructor(
+      jvm->findConstructor(
+          cnxnFactoryClass
+          .constructor()
+          .parameter(inetSocketAddressClass)));
 
   Jvm::JClass zkServerClass =
       Jvm::JClass::forName("org/apache/zookeeper/server/ZooKeeperServer");
-  startup =
-      new Jvm::JMethod(
-          jvm->findMethod(cnxnFactoryClass.method("startup")
-              .parameter(zkServerClass)
-              .returns(jvm->voidClass)));
-
-  isAlive =
-      new Jvm::JMethod(
-          jvm->findMethod(cnxnFactoryClass.method("isAlive")
-              .returns(jvm->booleanClass)));
-  shutdown =
-      new Jvm::JMethod(
-          jvm->findMethod(cnxnFactoryClass.method("shutdown")
-              .returns(jvm->voidClass)));
+  startup = new Jvm::JMethod(
+      jvm->findMethod(
+          cnxnFactoryClass
+          .method("startup")
+          .parameter(zkServerClass)
+          .returns(jvm->voidClass)));
+
+  isAlive = new Jvm::JMethod(
+      jvm->findMethod(
+          cnxnFactoryClass
+          .method("isAlive")
+          .returns(jvm->booleanClass)));
+
+  shutdown = new Jvm::JMethod(
+      jvm->findMethod(
+          cnxnFactoryClass
+          .method("shutdown")
+          .returns(jvm->voidClass)));
 
   dataDir = createTempDir();
   snapDir = createTempDir();
   Jvm::JClass snapLogClass =
-      Jvm::JClass::forName("org/apache/zookeeper/server/"
-                           "persistence/FileTxnSnapLog");
-
-  snapLog =
-      jvm->newGlobalRef(
-          jvm->invoke(
-              jvm->findConstructor(snapLogClass.constructor()
-                  .parameter(fileClass).parameter(fileClass)),
-              dataDir->file,
-              snapDir->file));
+    Jvm::JClass::forName(
+        "org/apache/zookeeper/server/"
+        "persistence/FileTxnSnapLog");
+
+  snapLog = jvm->newGlobalRef(
+      jvm->invoke(
+          jvm->findConstructor(snapLogClass
+                               .constructor()
+                               .parameter(fileClass)
+                               .parameter(fileClass)),
+          dataDir->file,
+          snapDir->file));
 
-  dataTreeBuilder =
-      jvm->newGlobalRef(
-          jvm->invoke(
-              jvm->findConstructor(
-                  Jvm::JClass::forName("org/apache/zookeeper/server/"
-                                       "ZooKeeperServer$BasicDataTreeBuilder")
-                                       .constructor())));
+  dataTreeBuilder = jvm->newGlobalRef(
+      jvm->invoke(
+          jvm->findConstructor(
+              Jvm::JClass::forName(
+                  "org/apache/zookeeper/server/"
+                  "ZooKeeperServer$BasicDataTreeBuilder").constructor())));
 
   Jvm::JClass dataTreeBuilderClass(
       Jvm::JClass::forName("org/apache/zookeeper/server/"
                            "ZooKeeperServer$DataTreeBuilder"));
 
-  zooKeeperServer =
-      jvm->newGlobalRef(
-          jvm->invoke(
-              jvm->findConstructor(zkServerClass.constructor()
-                  .parameter(snapLogClass).parameter(dataTreeBuilderClass)),
-              snapLog,
-              dataTreeBuilder));
-
-  getClientPort =
-      new Jvm::JMethod(
-          jvm->findMethod(zkServerClass.method("getClientPort")
-              .returns(jvm->intClass)));
-  closeSession =
-      new Jvm::JMethod(
-          jvm->findMethod(zkServerClass.method("closeSession")
-              .parameter(jvm->longClass).returns(jvm->voidClass)));
+  zooKeeperServer = jvm->newGlobalRef(
+      jvm->invoke(
+          jvm->findConstructor(zkServerClass
+                               .constructor()
+                               .parameter(snapLogClass)
+                               .parameter(dataTreeBuilderClass)),
+          snapLog,
+          dataTreeBuilder));
+
+  getClientPort = new Jvm::JMethod(
+      jvm->findMethod(zkServerClass
+                      .method("getClientPort")
+                      .returns(jvm->intClass)));
+
+  closeSession = new Jvm::JMethod(
+      jvm->findMethod(zkServerClass
+                      .method("closeSession")
+                      .parameter(jvm->longClass)
+                      .returns(jvm->voidClass)));
 }
 
 
-const ZooKeeperServer::TemporaryDirectory* ZooKeeperServer::createTempDir()
+const ZooKeeperTestServer::TemporaryDirectory* ZooKeeperTestServer::createTempDir()
 {
   std::string tmpdir = "/tmp/zks-" + UUID::random().toString();
   jobject file =
-      jvm->newGlobalRef(jvm->invoke(*fileConstructor, jvm->string(tmpdir)));
+    jvm->newGlobalRef(jvm->invoke(*fileConstructor, jvm->string(tmpdir)));
   return new TemporaryDirectory(jvm, tmpdir, file);
 }
 
 
-ZooKeeperServer::~ZooKeeperServer()
+ZooKeeperTestServer::~ZooKeeperTestServer()
 {
   shutdownNetwork();
 
@@ -157,7 +171,7 @@ ZooKeeperServer::~ZooKeeperServer()
 }
 
 
-void ZooKeeperServer::expireSession(int64_t sessionId)
+void ZooKeeperTestServer::expireSession(int64_t sessionId)
 {
   Jvm::Attach attach(jvm);
 
@@ -165,25 +179,25 @@ void ZooKeeperServer::expireSession(int6
 }
 
 
-std::string ZooKeeperServer::connectString() const
+std::string ZooKeeperTestServer::connectString() const
 {
   checkStarted();
   return "127.0.0.1:" + stringify(port);
 }
 
 
-void ZooKeeperServer::shutdownNetwork()
+void ZooKeeperTestServer::shutdownNetwork()
 {
   Jvm::Attach attach(jvm);
 
   if (started && jvm->invoke<bool>(connectionFactory, *isAlive)) {
     jvm->invoke<void>(connectionFactory, *shutdown);
-    LOG(INFO) << "Shutdown ZooKeeperServer on port " << port << std::endl;
+    LOG(INFO) << "Shutdown ZooKeeperTestServer on port " << port << std::endl;
   }
 }
 
 
-int ZooKeeperServer::startNetwork()
+int ZooKeeperTestServer::startNetwork()
 {
   Jvm::Attach attach(jvm);
 
@@ -195,13 +209,13 @@ int ZooKeeperServer::startNetwork()
 
   jvm->invoke<void>(connectionFactory, *startup, zooKeeperServer);
   port = jvm->invoke<int>(zooKeeperServer, *getClientPort);
-  LOG(INFO) << "Started ZooKeeperServer on port " << port;
+  LOG(INFO) << "Started ZooKeeperTestServer on port " << port;
   started = true;
   return port;
 }
 
 
-void ZooKeeperServer::checkStarted() const
+void ZooKeeperTestServer::checkStarted() const
 {
   CHECK(port > 0) << "Illegal state, must call startNetwork first!";
 }

Copied: incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp (from r1402689, incubator/mesos/trunk/src/tests/zookeeper_server.hpp)
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp?p2=incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp&p1=incubator/mesos/trunk/src/tests/zookeeper_server.hpp&r1=1402689&r2=1402690&rev=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_server.hpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_test_server.hpp Fri Oct 26 22:07:09 2012
@@ -16,8 +16,8 @@
  * limitations under the License.
  */
 
-#ifndef __TESTING_ZOO_KEEPER_SERVER_HPP__
-#define __TESTING_ZOO_KEEPER_SERVER_HPP__
+#ifndef __ZOOKEEPER_TEST_SERVER_HPP__
+#define __ZOOKEEPER_TEST_SERVER_HPP__
 
 #include <jni.h>
 
@@ -33,14 +33,15 @@ namespace mesos {
 namespace internal {
 namespace test {
 
-// An in-process ZooKeeper server that can be manipulated to control repeatable
-// client tests.  Sessions can be programmatically expired and network
-// partitions can be forced to simulate common failure modes.
-class ZooKeeperServer
+// An in-process ZooKeeper server that can be manipulated to control
+// repeatable client tests. Sessions can be programmatically expired
+// and network partitions can be forced to simulate common failure
+// modes.
+class ZooKeeperTestServer
 {
 public:
-  ZooKeeperServer(Jvm* jvm);
-  ~ZooKeeperServer();
+  ZooKeeperTestServer(Jvm* jvm);
+  ~ZooKeeperTestServer();
 
   // Gets a connection string that can be used to attach a ZooKeeper client to
   // this server.
@@ -111,4 +112,4 @@ private:
 } // namespace internal
 } // namespace mesos
 
-#endif // __TESTING_ZOO_KEEPER_SERVER_HPP__
+#endif // __ZOOKEEPER_TEST_SERVER_HPP__

Modified: incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_tests.cpp?rev=1402690&r1=1402689&r2=1402690&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_tests.cpp Fri Oct 26 22:07:09 2012
@@ -34,8 +34,8 @@
 
 #include "messages/messages.hpp"
 
-#include "tests/base_zookeeper_test.hpp"
 #include "tests/utils.hpp"
+#include "tests/zookeeper_test.hpp"
 
 #include "zookeeper/authentication.hpp"
 #include "zookeeper/group.hpp"
@@ -49,26 +49,42 @@ using testing::_;
 using testing::Return;
 
 
-#define assertGet(zk, path, expected)				\
-  do {								\
-    std::string result;						\
-    ASSERT_EQ(ZOK, zk.get(path, false, &result, NULL));		\
-    ASSERT_EQ(expected, result);				\
-  } while (false)
-
-
-#define assertNotSet(zk, path, value)		\
-  ASSERT_EQ(ZNOAUTH, zk.set(path, value, -1))
-
+// Helper for invoking ZooKeeper::get(path, ...) in order to check the
+// data stored at a specified znode path.
+::testing::AssertionResult AssertZKGet(
+    const char* expectedExpr,
+    const char* zkExpr,
+    const char* pathExpr,
+    const std::string& expected,
+    ZooKeeper* zk,
+    const std::string& path)
+{
+  std::string result;
+  int code = zk->get(path, false, &result, NULL);
+  if (code == ZOK) {
+    if (expected == result) {
+      return ::testing::AssertionSuccess();
+    } else {
+      return ::testing::AssertionFailure()
+        << "Expected data at znode '" << pathExpr << "' "
+        << "to be '" << expected << "', but actually '" << result << "'";
+    }
+  } else {
+    return ::testing::AssertionFailure()
+      << "(" << zkExpr << ").get(" << pathExpr << ", ...): "
+      << zk->message(code);
+  }
+}
 
-class ZooKeeperTest : public BaseZooKeeperTest {};
+#define ASSERT_ZK_GET(expected, zk, path)               \
+  ASSERT_PRED_FORMAT3(AssertZKGet, expected, zk, path)
 
 
 TEST_F(ZooKeeperTest, Auth)
 {
-  BaseZooKeeperTest::TestWatcher watcher;
+  ZooKeeperTest::TestWatcher watcher;
 
-  ZooKeeper authenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper authenticatedZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
   authenticatedZk.authenticate("digest", "creator:creator");
   authenticatedZk.create("/test",
@@ -76,26 +92,26 @@ TEST_F(ZooKeeperTest, Auth)
                          zookeeper::EVERYONE_READ_CREATOR_ALL,
                          0,
                          NULL);
-  assertGet(authenticatedZk, "/test", "42");
+  ASSERT_ZK_GET("42", &authenticatedZk, "/test");
 
-  ZooKeeper unauthenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper unauthenticatedZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
-  assertGet(unauthenticatedZk, "/test", "42");
-  assertNotSet(unauthenticatedZk, "/test", "37");
+  ASSERT_ZK_GET("42", &unauthenticatedZk, "/test");
+  ASSERT_EQ(ZNOAUTH, unauthenticatedZk.set("/test", "", -1));
 
-  ZooKeeper nonOwnerZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper nonOwnerZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
   nonOwnerZk.authenticate("digest", "non-owner:non-owner");
-  assertGet(nonOwnerZk, "/test", "42");
-  assertNotSet(nonOwnerZk, "/test", "37");
+  ASSERT_ZK_GET("42", &nonOwnerZk, "/test");
+  ASSERT_EQ(ZNOAUTH, nonOwnerZk.set("/test", "", -1));
 }
 
 
 TEST_F(ZooKeeperTest, Create)
 {
-  BaseZooKeeperTest::TestWatcher watcher;
+  ZooKeeperTest::TestWatcher watcher;
 
-  ZooKeeper authenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper authenticatedZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
   authenticatedZk.authenticate("digest", "creator:creator");
   EXPECT_EQ(ZOK, authenticatedZk.create("/foo/bar",
@@ -109,9 +125,9 @@ TEST_F(ZooKeeperTest, Create)
                          zookeeper::EVERYONE_CREATE_AND_READ_CREATOR_ALL,
                          0,
                          NULL);
-  assertGet(authenticatedZk, "/foo/bar/baz", "43");
+  ASSERT_ZK_GET("43", &authenticatedZk, "/foo/bar/baz");
 
-  ZooKeeper nonOwnerZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper nonOwnerZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
   nonOwnerZk.authenticate("digest", "non-owner:non-owner");
   EXPECT_EQ(ZNOAUTH, nonOwnerZk.create("/foo/bar/baz",
@@ -126,7 +142,7 @@ TEST_F(ZooKeeperTest, Create)
                                    0,
                                    NULL,
                                    true));
-  assertGet(nonOwnerZk, "/foo/bar/baz/bam", "44");
+  ASSERT_ZK_GET("44", &nonOwnerZk, "/foo/bar/baz/bam");
 
   std::string result;
   EXPECT_EQ(ZOK, nonOwnerZk.create("/foo/bar/baz/",
@@ -171,7 +187,7 @@ TEST_F(ZooKeeperTest, MasterDetector)
   EXPECT_CALL(mock, newMasterDetected(mock.self()))
     .WillOnce(Trigger(&newMasterDetectedCall));
 
-  std::string master = "zk://" + zks->connectString() + "/mesos";
+  std::string master = "zk://" + server->connectString() + "/mesos";
 
   Try<MasterDetector*> detector =
     MasterDetector::create(master, mock.self(), true, true);
@@ -196,7 +212,7 @@ TEST_F(ZooKeeperTest, MasterDetectors)
   EXPECT_CALL(mock1, newMasterDetected(mock1.self()))
     .WillOnce(Trigger(&newMasterDetectedCall1));
 
-  std::string master = "zk://" + zks->connectString() + "/mesos";
+  std::string master = "zk://" + server->connectString() + "/mesos";
 
   Try<MasterDetector*> detector1 =
     MasterDetector::create(master, mock1.self(), true, true);
@@ -247,7 +263,7 @@ TEST_F(ZooKeeperTest, MasterDetectorShut
   EXPECT_CALL(mock, newMasterDetected(mock.self()))
     .WillOnce(Trigger(&newMasterDetectedCall1));
 
-  std::string master = "zk://" + zks->connectString() + "/mesos";
+  std::string master = "zk://" + server->connectString() + "/mesos";
 
   Try<MasterDetector*> detector =
     MasterDetector::create(master, mock.self(), true, true);
@@ -260,7 +276,7 @@ TEST_F(ZooKeeperTest, MasterDetectorShut
   EXPECT_CALL(mock, noMasterDetected())
     .WillOnce(Trigger(&noMasterDetectedCall));
 
-  zks->shutdownNetwork();
+  server->shutdownNetwork();
 
   Clock::advance(10.0); // TODO(benh): Get session timeout from detector.
 
@@ -270,7 +286,7 @@ TEST_F(ZooKeeperTest, MasterDetectorShut
   EXPECT_CALL(mock, newMasterDetected(mock.self()))
     .WillOnce(Trigger(&newMasterDetectedCall2));
 
-  zks->startNetwork();
+  server->startNetwork();
 
   WAIT_FOR(newMasterDetectedCall2, Seconds(5.0));
 
@@ -285,7 +301,7 @@ TEST_F(ZooKeeperTest, MasterDetectorShut
 
 TEST_F(ZooKeeperTest, Group)
 {
-  zookeeper::Group group(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group(server->connectString(), NO_TIMEOUT, "/test/");
 
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
@@ -319,16 +335,16 @@ TEST_F(ZooKeeperTest, Group)
 
 TEST_F(ZooKeeperTest, GroupJoinWithDisconnect)
 {
-  zookeeper::Group group(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group(server->connectString(), NO_TIMEOUT, "/test/");
 
-  zks->shutdownNetwork();
+  server->shutdownNetwork();
 
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
 
   EXPECT_TRUE(membership.isPending());
 
-  zks->startNetwork();
+  server->startNetwork();
 
   ASSERT_FUTURE_WILL_SUCCEED(membership);
 
@@ -343,7 +359,7 @@ TEST_F(ZooKeeperTest, GroupJoinWithDisco
 
 TEST_F(ZooKeeperTest, GroupDataWithDisconnect)
 {
-  zookeeper::Group group(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group(server->connectString(), NO_TIMEOUT, "/test/");
 
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
@@ -357,13 +373,13 @@ TEST_F(ZooKeeperTest, GroupDataWithDisco
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 
-  zks->shutdownNetwork();
+  server->shutdownNetwork();
 
   process::Future<std::string> data = group.data(membership.get());
 
   EXPECT_TRUE(data.isPending());
 
-  zks->startNetwork();
+  server->startNetwork();
 
   EXPECT_FUTURE_WILL_EQ("hello world", data);
 }
@@ -371,7 +387,7 @@ TEST_F(ZooKeeperTest, GroupDataWithDisco
 
 TEST_F(ZooKeeperTest, GroupCancelWithDisconnect)
 {
-  zookeeper::Group group(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group(server->connectString(), NO_TIMEOUT, "/test/");
 
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
@@ -389,13 +405,13 @@ TEST_F(ZooKeeperTest, GroupCancelWithDis
 
   EXPECT_FUTURE_WILL_EQ("hello world", data);
 
-  zks->shutdownNetwork();
+  server->shutdownNetwork();
 
   process::Future<bool> cancellation = group.cancel(membership.get());
 
   EXPECT_TRUE(cancellation.isPending());
 
-  zks->startNetwork();
+  server->startNetwork();
 
   EXPECT_FUTURE_WILL_EQ(true, cancellation);
 
@@ -411,7 +427,7 @@ TEST_F(ZooKeeperTest, GroupCancelWithDis
 
 TEST_F(ZooKeeperTest, GroupWatchWithSessionExpiration)
 {
-  zookeeper::Group group(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group(server->connectString(), NO_TIMEOUT, "/test/");
 
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
@@ -432,7 +448,7 @@ TEST_F(ZooKeeperTest, GroupWatchWithSess
 
   memberships = group.watch(memberships.get());
 
-  zks->expireSession(session.get().get());
+  server->expireSession(session.get().get());
 
   ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(0u, memberships.get().size());
@@ -444,8 +460,8 @@ TEST_F(ZooKeeperTest, GroupWatchWithSess
 
 TEST_F(ZooKeeperTest, MultipleGroups)
 {
-  zookeeper::Group group1(zks->connectString(), NO_TIMEOUT, "/test/");
-  zookeeper::Group group2(zks->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group1(server->connectString(), NO_TIMEOUT, "/test/");
+  zookeeper::Group group2(server->connectString(), NO_TIMEOUT, "/test/");
 
   process::Future<zookeeper::Group::Membership> membership1 =
     group1.join("group 1");
@@ -488,7 +504,7 @@ TEST_F(ZooKeeperTest, MultipleGroups)
   ASSERT_FUTURE_WILL_SUCCEED(session1);
   ASSERT_SOME(session1.get());
 
-  zks->expireSession(session1.get().get());
+  server->expireSession(session1.get().get());
 
   ASSERT_FUTURE_WILL_EQ(false, cancelled);
 }
@@ -496,9 +512,9 @@ TEST_F(ZooKeeperTest, MultipleGroups)
 
 TEST_F(ZooKeeperTest, GroupPathWithRestrictivePerms)
 {
-  BaseZooKeeperTest::TestWatcher watcher;
+  ZooKeeperTest::TestWatcher watcher;
 
-  ZooKeeper authenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
+  ZooKeeper authenticatedZk(server->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
   authenticatedZk.authenticate("digest", "creator:creator");
   authenticatedZk.create("/read-only",
@@ -506,31 +522,31 @@ TEST_F(ZooKeeperTest, GroupPathWithRestr
                          zookeeper::EVERYONE_READ_CREATOR_ALL,
                          0,
                          NULL);
-  assertGet(authenticatedZk, "/read-only", "42");
+  ASSERT_ZK_GET("42", &authenticatedZk, "/read-only");
   authenticatedZk.create("/read-only/writable",
                          "37",
                          ZOO_OPEN_ACL_UNSAFE,
                          0,
                          NULL);
-  assertGet(authenticatedZk, "/read-only/writable", "37");
+  ASSERT_ZK_GET("37", &authenticatedZk, "/read-only/writable");
 
   zookeeper::Authentication auth("digest", "non-creator:non-creator");
 
-  zookeeper::Group failedGroup1(zks->connectString(), NO_TIMEOUT,
+  zookeeper::Group failedGroup1(server->connectString(), NO_TIMEOUT,
                                 "/read-only/", auth);
   process::Future<zookeeper::Group::Membership> failedMembership1 =
     failedGroup1.join("fail");
 
   ASSERT_FUTURE_WILL_FAIL(failedMembership1);
 
-  zookeeper::Group failedGroup2(zks->connectString(), NO_TIMEOUT,
+  zookeeper::Group failedGroup2(server->connectString(), NO_TIMEOUT,
                                 "/read-only/new", auth);
   process::Future<zookeeper::Group::Membership> failedMembership2 =
     failedGroup2.join("fail");
 
   ASSERT_FUTURE_WILL_FAIL(failedMembership2);
 
-  zookeeper::Group successGroup(zks->connectString(), NO_TIMEOUT,
+  zookeeper::Group successGroup(server->connectString(), NO_TIMEOUT,
                                 "/read-only/writable/", auth);
   process::Future<zookeeper::Group::Membership> successMembership =
     successGroup.join("succeed");