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/09/17 23:15:49 UTC

svn commit: r1386847 - in /incubator/mesos/trunk: src/ src/tests/ third_party/libprocess/include/process/ third_party/libprocess/src/

Author: benh
Date: Mon Sep 17 21:15:49 2012
New Revision: 1386847

URL: http://svn.apache.org/viewvc?rev=1386847&view=rev
Log:
Used new ASSERT/EXPECT_FUTURE_* within some existing tests as well as
added ASSERT/EXPECT_FUTURE_WILL_EQ and
EXPECT_RESPONSE_BODY_WILL_EQ. Also added new tests for logging HTTP
interface thanks to process::http::get
(https://reviews.apache.org/r/7008).

Added:
    incubator/mesos/trunk/src/tests/logging_tests.cpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/tests/files_tests.cpp
    incubator/mesos/trunk/src/tests/utils.hpp
    incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
    incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
    incubator/mesos/trunk/third_party/libprocess/src/process.cpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Mon Sep 17 21:15:49 2012
@@ -768,9 +768,8 @@ mesos_tests_SOURCES = tests/main.cpp tes
 	              tests/gc_tests.cpp				\
 	              tests/resource_offers_tests.cpp			\
 	              tests/fault_tolerance_tests.cpp			\
-	              tests/files_tests.cpp				\
-	              tests/flags_tests.cpp tests/log_tests.cpp		\
-	              tests/resources_tests.cpp				\
+	              tests/files_tests.cpp tests/flags_tests.cpp	\
+	              tests/log_tests.cpp tests/resources_tests.cpp	\
 	              tests/external_tests.cpp				\
 	              tests/sample_frameworks_tests.cpp			\
 	              tests/configurator_tests.cpp			\
@@ -783,8 +782,8 @@ mesos_tests_SOURCES = tests/main.cpp tes
 	              tests/exception_tests.cpp				\
 	              tests/attributes_tests.cpp			\
 	              tests/master_detector_tests.cpp			\
-	              tests/sorter_tests.cpp				\
-	              tests/allocator_tests.cpp
+	              tests/sorter_tests.cpp tests/allocator_tests.cpp	\
+	              tests/logging_tests.cpp
 
 mesos_tests_CPPFLAGS = $(MESOS_CPPFLAGS)
 mesos_tests_CPPFLAGS += -DSOURCE_DIR=\"$(abs_top_srcdir)\"

Modified: incubator/mesos/trunk/src/tests/files_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/files_tests.cpp?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/files_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/files_tests.cpp Mon Sep 17 21:15:49 2012
@@ -18,8 +18,6 @@
 
 #include <gmock/gmock.h>
 
-#include <string>
-
 #include <process/future.hpp>
 #include <process/http.hpp>
 #include <process/pid.hpp>
@@ -36,29 +34,17 @@ using namespace mesos::internal;
 using namespace mesos::internal::test;
 
 using process::Future;
+
 using process::http::BadRequest;
-using process::http::InternalServerError;
 using process::http::NotFound;
 using process::http::OK;
 using process::http::Response;
-using process::http::Request;
-using process::http::ServiceUnavailable;
-
-using std::string;
-
-
-void checkResponse(const Future<Response>& response,
-                   const string& status,
-                   const JSON::Value& expected)
-{
-  EXPECT_RESPONSE_STATUS_WILL_EQ(status, response);
-  EXPECT_EQ(stringify(expected), response.get().body);
-}
 
 
 TEST_WITH_WORKDIR(FilesTest, AttachTest)
 {
   Files files;
+
   ASSERT_TRUE(os::write("file", "body").get());
   ASSERT_TRUE(os::mkdir("dir"));
 
@@ -91,6 +77,27 @@ TEST_WITH_WORKDIR(FilesTest, ReadTest)
   Files files;
   const process::PID<>& pid = files.pid();
 
+  Future<Response> response =
+    process::http::get(pid, "read.json");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ("Expecting 'path=value' in query.\n", response);
+
+  response = process::http::get(pid, "read.json?path=none&offset=hello");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Failed to parse offset: Failed to convert 'hello' to number",
+      response);
+
+  response = process::http::get(pid, "read.json?path=none&length=hello");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Failed to parse length: Failed to convert 'hello' to number",
+      response);
+
+  // Now write a file.
   ASSERT_TRUE(os::write("file", "body").get());
   EXPECT_FUTURE_WILL_SUCCEED(files.attach("file", "/myname"));
   EXPECT_FUTURE_WILL_SUCCEED(files.attach("file", "myname"));
@@ -101,17 +108,27 @@ TEST_WITH_WORKDIR(FilesTest, ReadTest)
   expected.values["length"] = strlen("body");
   expected.values["data"] = "body";
 
-  checkResponse(process::http::get(pid, "read.json?path=/myname&offset=0"),
-                OK().status,
-                expected);
-  checkResponse(process::http::get(pid, "read.json?path=myname&offset=0"),
-                OK().status,
-                expected);
+  response = process::http::get(pid, "read.json?path=/myname&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
+
+  response = process::http::get(pid, "read.json?path=myname&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
 
   // Missing file.
   EXPECT_RESPONSE_STATUS_WILL_EQ(
       NotFound().status,
       process::http::get(pid, "read.json?path=missing"));
+}
+
+
+TEST_WITH_WORKDIR(FilesTest, ResolveTest)
+{
+  Files files;
+  const process::PID<>& pid = files.pid();
 
   // Test the directory / file resolution.
   ASSERT_TRUE(os::mkdir("1/2"));
@@ -125,25 +142,31 @@ TEST_WITH_WORKDIR(FilesTest, ReadTest)
   EXPECT_FUTURE_WILL_SUCCEED(files.attach("1/2", "one/two"));
 
   // Resolve 1/2/3 via each attached path.
-  JSON::Object expectedResponse;
-  expectedResponse.values["offset"] = 0;
-  expectedResponse.values["length"] = strlen("three");
-  expectedResponse.values["data"] = "three";
-
-  checkResponse(process::http::get(pid, "read.json?path=one/2/three&offset=0"),
-                OK().status,
-                expectedResponse);
-  checkResponse(process::http::get(pid, "read.json?path=/one/2/three&offset=0"),
-                OK().status,
-                expectedResponse);
-  checkResponse(process::http::get(pid, "read.json?path=two/three&offset=0"),
-                OK().status,
-                expectedResponse);
-  checkResponse(
-      process::http::get(pid, "read.json?path=one/two/three&offset=0"),
-      OK().status,
-      expectedResponse);
+  JSON::Object expected;
+  expected.values["offset"] = 0;
+  expected.values["length"] = strlen("three");
+  expected.values["data"] = "three";
+
+  Future<Response> response =
+    process::http::get(pid, "read.json?path=one/2/three&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
 
+  response = process::http::get(pid, "read.json?path=/one/2/three&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
+
+  response = process::http::get(pid, "read.json?path=two/three&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
+
+  response = process::http::get(pid, "read.json?path=one/two/three&offset=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
 
   // Reading dirs not allowed.
   EXPECT_RESPONSE_STATUS_WILL_EQ(
@@ -156,7 +179,7 @@ TEST_WITH_WORKDIR(FilesTest, ReadTest)
       BadRequest().status,
       process::http::get(pid, "read.json?path=one/"));
 
-  // Breaking out of sandbox
+  // Breaking out of sandbox.
   EXPECT_RESPONSE_STATUS_WILL_EQ(
       BadRequest().status,
       process::http::get(pid, "read.json?path=two/../two"));
@@ -182,17 +205,21 @@ TEST_WITH_WORKDIR(FilesTest, BrowseTest)
   expected.values.push_back(jsonFileInfo("one/three", false, 5u));
   expected.values.push_back(jsonFileInfo("one/two", false, 3u));
 
-  checkResponse(process::http::get(pid, "browse.json?path=one/"),
-                OK().status,
-                expected);
-  checkResponse(process::http::get(pid, "browse.json?path=one"),
-                OK().status,
-                expected);
+  Future<Response> response = process::http::get(pid, "browse.json?path=one/");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
+
+  response = process::http::get(pid, "browse.json?path=one");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(expected), response);
 
   // Empty listing.
-  checkResponse(process::http::get(pid, "browse.json?path=one/2"),
-                OK().status,
-                JSON::Array());
+  response = process::http::get(pid, "browse.json?path=one/2");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(stringify(JSON::Array()), response);
 
   // Missing dir.
   EXPECT_RESPONSE_STATUS_WILL_EQ(

Added: incubator/mesos/trunk/src/tests/logging_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/logging_tests.cpp?rev=1386847&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/logging_tests.cpp (added)
+++ incubator/mesos/trunk/src/tests/logging_tests.cpp Mon Sep 17 21:15:49 2012
@@ -0,0 +1,82 @@
+/**
+ * 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.
+ */
+
+#include <gmock/gmock.h>
+
+#include <process/future.hpp>
+#include <process/http.hpp>
+#include <process/pid.hpp>
+
+#include "logging/logging.hpp"
+
+#include "tests/utils.hpp"
+
+using namespace mesos::internal;
+using namespace mesos::internal::test;
+
+using process::http::BadRequest;
+using process::http::OK;
+using process::http::Response;
+
+
+TEST_WITH_WORKDIR(LoggingTest, Toggle)
+{
+  process::PID<> pid;
+  pid.id = "logging";
+  pid.ip = process::ip();
+  pid.port = process::port();
+
+  process::Future<Response> response = process::http::get(pid, "toggle");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(OK().status, response);
+
+  response = process::http::get(pid, "toggle?level=0");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Expecting 'duration=value' in query.\n",
+      response);
+
+  response = process::http::get(pid, "toggle?duration=10secs");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Expecting 'level=value' in query.\n",
+      response);
+
+  response = process::http::get(pid, "toggle?duration=10secs");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Expecting 'level=value' in query.\n",
+      response);
+
+  response = process::http::get(pid, "toggle?level=-1&duration=10secs");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Invalid level '-1'.\n",
+      response);
+
+  response = process::http::get(pid, "toggle?level=-1&duration=10secs");
+
+  EXPECT_RESPONSE_STATUS_WILL_EQ(BadRequest().status, response);
+  EXPECT_RESPONSE_BODY_WILL_EQ(
+      "Invalid level '-1'.\n",
+      response);
+}

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Mon Sep 17 21:15:49 2012
@@ -31,6 +31,7 @@
 #include <mesos/scheduler.hpp>
 
 #include <process/future.hpp>
+#include <process/http.hpp>
 #include <process/process.hpp>
 
 #include <stout/option.hpp>
@@ -120,6 +121,7 @@ template <typename T>
 void ASSERT_FUTURE_WILL_SUCCEED(const process::Future<T>& future)
 {
   ASSERT_TRUE(future.await());
+  ASSERT_FALSE(future.isDiscarded());
   ASSERT_FALSE(future.isFailed()) << future.failure();
 }
 
@@ -128,6 +130,7 @@ template <typename T>
 void EXPECT_FUTURE_WILL_SUCCEED(const process::Future<T>& future)
 {
   ASSERT_TRUE(future.await());
+  EXPECT_FALSE(future.isDiscarded());
   EXPECT_FALSE(future.isFailed()) << future.failure();
 }
 
@@ -148,15 +151,40 @@ void EXPECT_FUTURE_WILL_FAIL(const proce
 }
 
 
-template <typename T>
-void EXPECT_RESPONSE_STATUS_WILL_EQ(const std::string& expected,
-                                    const process::Future<T>& future)
+template <typename T1, typename T2>
+void ASSERT_FUTURE_WILL_EQ(const T1& t1, const process::Future<T2>& future)
+{
+  ASSERT_FUTURE_WILL_SUCCEED(future);
+  ASSERT_EQ(t1, future.get());
+}
+
+
+template <typename T1, typename T2>
+void EXPECT_FUTURE_WILL_EQ(const T1& t1, const process::Future<T2>& future)
+{
+  ASSERT_FUTURE_WILL_SUCCEED(future);
+  EXPECT_EQ(t1, future.get());
+}
+
+
+inline void EXPECT_RESPONSE_STATUS_WILL_EQ(
+    const std::string& expected,
+    const process::Future<process::http::Response>& future)
 {
   ASSERT_FUTURE_WILL_SUCCEED(future);
   EXPECT_EQ(expected, future.get().status);
 }
 
 
+inline void EXPECT_RESPONSE_BODY_WILL_EQ(
+    const std::string& expected,
+    const process::Future<process::http::Response>& future)
+{
+  ASSERT_FUTURE_WILL_SUCCEED(future);
+  EXPECT_EQ(expected, future.get().body);
+}
+
+
 /**
  * Definition of a mock Scheduler to be used in tests with gmock.
  */

Modified: incubator/mesos/trunk/src/tests/zookeeper_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/zookeeper_tests.cpp?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/zookeeper_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/zookeeper_tests.cpp Mon Sep 17 21:15:49 2012
@@ -23,12 +23,16 @@
 #include <gtest/gtest.h>
 
 #include "tests/base_zookeeper_test.hpp"
+#include "tests/utils.hpp"
 
 #include "zookeeper/authentication.hpp"
 #include "zookeeper/group.hpp"
 
+using namespace mesos::internal;
+using namespace mesos::internal::test;
 
-class ZooKeeperTest : public mesos::internal::test::BaseZooKeeperTest {
+
+class ZooKeeperTest : public BaseZooKeeperTest {
 protected:
   void assertGet(ZooKeeper* client,
                  const std::string& path,
@@ -48,7 +52,7 @@ protected:
 
 TEST_F(ZooKeeperTest, Auth)
 {
-  mesos::internal::test::BaseZooKeeperTest::TestWatcher watcher;
+  BaseZooKeeperTest::TestWatcher watcher;
 
   ZooKeeper authenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
@@ -80,44 +84,26 @@ TEST_F(ZooKeeperTest, Group)
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
 
-  membership.await();
-
-  ASSERT_FALSE(membership.isFailed()) << membership.failure();
-  ASSERT_FALSE(membership.isDiscarded());
-  ASSERT_TRUE(membership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships =
     group.watch();
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 
   process::Future<std::string> data = group.data(membership.get());
 
-  data.await();
-
-  ASSERT_FALSE(data.isFailed()) << data.failure();
-  ASSERT_FALSE(data.isDiscarded());
-  ASSERT_TRUE(data.isReady());
-  EXPECT_EQ("hello world", data.get());
+  EXPECT_FUTURE_WILL_EQ("hello world", data);
 
   process::Future<bool> cancellation = group.cancel(membership.get());
 
-  cancellation.await();
-
-  ASSERT_FALSE(cancellation.isFailed()) << cancellation.failure();
-  ASSERT_FALSE(cancellation.isDiscarded());
-  ASSERT_TRUE(cancellation.isReady());
-  EXPECT_TRUE(cancellation.get());
+  EXPECT_FUTURE_WILL_EQ(true, cancellation);
 
   memberships = group.watch(memberships.get());
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(0u, memberships.get().size());
 
   ASSERT_TRUE(membership.get().cancelled().isReady());
@@ -138,18 +124,12 @@ TEST_F(ZooKeeperTest, GroupJoinWithDisco
 
   zks->startNetwork();
 
-  membership.await();
-
-  ASSERT_FALSE(membership.isFailed()) << membership.failure();
-  ASSERT_FALSE(membership.isDiscarded());
-  ASSERT_TRUE(membership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships =
     group.watch();
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 }
@@ -162,18 +142,12 @@ TEST_F(ZooKeeperTest, GroupDataWithDisco
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
 
-  membership.await();
-
-  ASSERT_FALSE(membership.isFailed()) << membership.failure();
-  ASSERT_FALSE(membership.isDiscarded());
-  ASSERT_TRUE(membership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships =
     group.watch();
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 
@@ -185,12 +159,7 @@ TEST_F(ZooKeeperTest, GroupDataWithDisco
 
   zks->startNetwork();
 
-  data.await();
-
-  ASSERT_FALSE(data.isFailed()) << data.failure();
-  ASSERT_FALSE(data.isDiscarded());
-  ASSERT_TRUE(data.isReady());
-  EXPECT_EQ("hello world", data.get());
+  EXPECT_FUTURE_WILL_EQ("hello world", data);
 }
 
 
@@ -201,31 +170,18 @@ TEST_F(ZooKeeperTest, GroupCancelWithDis
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
 
-  membership.await();
-
-  ASSERT_FALSE(membership.isFailed()) << membership.failure();
-  ASSERT_FALSE(membership.isDiscarded());
-  ASSERT_TRUE(membership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships =
     group.watch();
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 
   process::Future<std::string> data = group.data(membership.get());
 
-  EXPECT_TRUE(data.isPending());
-
-  data.await();
-
-  ASSERT_FALSE(data.isFailed()) << data.failure();
-  ASSERT_FALSE(data.isDiscarded());
-  ASSERT_TRUE(data.isReady());
-  EXPECT_EQ("hello world", data.get());
+  EXPECT_FUTURE_WILL_EQ("hello world", data);
 
   zks->shutdownNetwork();
 
@@ -235,18 +191,11 @@ TEST_F(ZooKeeperTest, GroupCancelWithDis
 
   zks->startNetwork();
 
-  cancellation.await();
-
-  ASSERT_FALSE(cancellation.isFailed()) << cancellation.failure();
-  ASSERT_FALSE(cancellation.isDiscarded());
-  ASSERT_TRUE(cancellation.isReady());
-  EXPECT_TRUE(cancellation.get());
+  EXPECT_FUTURE_WILL_EQ(true, cancellation);
 
   memberships = group.watch(memberships.get());
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(0u, memberships.get().size());
 
   ASSERT_TRUE(membership.get().cancelled().isReady());
@@ -261,37 +210,25 @@ TEST_F(ZooKeeperTest, GroupWatchWithSess
   process::Future<zookeeper::Group::Membership> membership =
     group.join("hello world");
 
-  membership.await();
-
-  ASSERT_FALSE(membership.isFailed()) << membership.failure();
-  ASSERT_FALSE(membership.isDiscarded());
-  ASSERT_TRUE(membership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships =
     group.watch();
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(1u, memberships.get().size());
   EXPECT_EQ(1u, memberships.get().count(membership.get()));
 
   process::Future<Option<int64_t> > session = group.session();
 
-  session.await();
-
-  ASSERT_FALSE(session.isFailed()) << session.failure();
-  ASSERT_FALSE(session.isDiscarded());
-  ASSERT_TRUE(session.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(session);
   ASSERT_TRUE(session.get().isSome());
 
   memberships = group.watch(memberships.get());
 
   zks->expireSession(session.get().get());
 
-  memberships.await();
-
-  ASSERT_TRUE(memberships.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships);
   EXPECT_EQ(0u, memberships.get().size());
 
   ASSERT_TRUE(membership.get().cancelled().isReady());
@@ -307,27 +244,17 @@ TEST_F(ZooKeeperTest, MultipleGroups)
   process::Future<zookeeper::Group::Membership> membership1 =
     group1.join("group 1");
 
-  membership1.await();
-
-  ASSERT_FALSE(membership1.isFailed()) << membership1.failure();
-  ASSERT_FALSE(membership1.isDiscarded());
-  ASSERT_TRUE(membership1.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership1);
 
   process::Future<zookeeper::Group::Membership> membership2 =
     group2.join("group 2");
 
-  membership2.await();
-
-  ASSERT_FALSE(membership2.isFailed()) << membership2.failure();
-  ASSERT_FALSE(membership2.isDiscarded());
-  ASSERT_TRUE(membership2.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(membership2);
 
   process::Future<std::set<zookeeper::Group::Membership> > memberships1 =
     group1.watch();
 
-  memberships1.await();
-
-  ASSERT_TRUE(memberships1.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships1);
   EXPECT_EQ(2u, memberships1.get().size());
   EXPECT_EQ(1u, memberships1.get().count(membership1.get()));
   EXPECT_EQ(1u, memberships1.get().count(membership2.get()));
@@ -335,9 +262,7 @@ TEST_F(ZooKeeperTest, MultipleGroups)
   process::Future<std::set<zookeeper::Group::Membership> > memberships2 =
     group2.watch();
 
-  memberships2.await();
-
-  ASSERT_TRUE(memberships2.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(memberships2);
   EXPECT_EQ(2u, memberships2.get().size());
   EXPECT_EQ(1u, memberships2.get().count(membership1.get()));
   EXPECT_EQ(1u, memberships2.get().count(membership2.get()));
@@ -354,25 +279,18 @@ TEST_F(ZooKeeperTest, MultipleGroups)
 
   process::Future<Option<int64_t> > session1 = group1.session();
 
-  session1.await();
-
-  ASSERT_FALSE(session1.isFailed()) << session1.failure();
-  ASSERT_FALSE(session1.isDiscarded());
-  ASSERT_TRUE(session1.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(session1);
   ASSERT_TRUE(session1.get().isSome());
 
   zks->expireSession(session1.get().get());
 
-  cancelled.await();
-
-  ASSERT_TRUE(cancelled.isReady());
-  ASSERT_FALSE(cancelled.get());
+  ASSERT_FUTURE_WILL_EQ(false, cancelled);
 }
 
 
 TEST_F(ZooKeeperTest, GroupPathWithRestrictivePerms)
 {
-  mesos::internal::test::BaseZooKeeperTest::TestWatcher watcher;
+  BaseZooKeeperTest::TestWatcher watcher;
 
   ZooKeeper authenticatedZk(zks->connectString(), NO_TIMEOUT, &watcher);
   watcher.awaitSessionEvent(ZOO_CONNECTED_STATE);
@@ -396,23 +314,20 @@ TEST_F(ZooKeeperTest, GroupPathWithRestr
                                 "/read-only/", auth);
   process::Future<zookeeper::Group::Membership> failedMembership1 =
     failedGroup1.join("fail");
-  failedMembership1.await();
-  ASSERT_TRUE(failedMembership1.isFailed());
+
+  ASSERT_FUTURE_WILL_FAIL(failedMembership1);
 
   zookeeper::Group failedGroup2(zks->connectString(), NO_TIMEOUT,
                                 "/read-only/new", auth);
   process::Future<zookeeper::Group::Membership> failedMembership2 =
     failedGroup2.join("fail");
-  failedMembership2.await();
-  ASSERT_TRUE(failedMembership2.isFailed());
+
+  ASSERT_FUTURE_WILL_FAIL(failedMembership2);
 
   zookeeper::Group successGroup(zks->connectString(), NO_TIMEOUT,
                                 "/read-only/writable/", auth);
   process::Future<zookeeper::Group::Membership> successMembership =
     successGroup.join("succeed");
-  successMembership.await();
 
-  ASSERT_FALSE(successMembership.isFailed()) << successMembership.failure();
-  ASSERT_FALSE(successMembership.isDiscarded());
-  ASSERT_TRUE(successMembership.isReady());
+  ASSERT_FUTURE_WILL_SUCCEED(successMembership);
 }

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp Mon Sep 17 21:15:49 2012
@@ -249,6 +249,19 @@ void initialize(const std::string& deleg
 
 
 /**
+ * Returns the IP address associated with this instance of the
+ * library.
+ */
+uint32_t ip();
+
+
+/**
+ * Returns the port associated with this instance of the library.
+ */
+uint16_t port();
+
+
+/**
  * Spawn a new process.
  *
  * @param process process to be spawned

Modified: incubator/mesos/trunk/third_party/libprocess/src/process.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/process.cpp?rev=1386847&r1=1386846&r2=1386847&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/process.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/process.cpp Mon Sep 17 21:15:49 2012
@@ -1412,6 +1412,20 @@ void initialize(const string& delegate)
 }
 
 
+uint32_t ip()
+{
+  process::initialize();
+  return __ip__;
+}
+
+
+uint16_t port()
+{
+  process::initialize();
+  return __port__;
+}
+
+
 HttpProxy::HttpProxy(const Socket& _socket)
   : ProcessBase(ID::generate("__http__")),
     socket(_socket) {}