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:26 UTC

svn commit: r1402696 - in /incubator/mesos/trunk/src: ./ java/jni/ log/ tests/

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

URL: http://svn.apache.org/viewvc?rev=1402696&view=rev
Log:
Refactored the libprocess test "filter" so that tests only need to use
EXPECT_MESSAGE(from, to, body) and no longer need to explicitly
construct, set, unset, or destruct an actual filter instance. This was
done in order to guarantee a filter is properly removed after a test
finishes (abnormally or otherwise).

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

Added:
    incubator/mesos/trunk/src/tests/filter.cpp
    incubator/mesos/trunk/src/tests/filter.hpp
Modified:
    incubator/mesos/trunk/src/Makefile.am
    incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
    incubator/mesos/trunk/src/log/coordinator.cpp
    incubator/mesos/trunk/src/log/coordinator.hpp
    incubator/mesos/trunk/src/log/log.hpp
    incubator/mesos/trunk/src/tests/allocator_tests.cpp
    incubator/mesos/trunk/src/tests/allocator_zookeeper_tests.cpp
    incubator/mesos/trunk/src/tests/environment.cpp
    incubator/mesos/trunk/src/tests/exception_tests.cpp
    incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
    incubator/mesos/trunk/src/tests/gc_tests.cpp
    incubator/mesos/trunk/src/tests/log_tests.cpp
    incubator/mesos/trunk/src/tests/master_tests.cpp
    incubator/mesos/trunk/src/tests/utils.hpp

Modified: incubator/mesos/trunk/src/Makefile.am
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/Makefile.am?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/Makefile.am (original)
+++ incubator/mesos/trunk/src/Makefile.am Fri Oct 26 22:07:26 2012
@@ -213,6 +213,7 @@ libmesos_no_third_party_la_SOURCES += co
 	slave/slave.hpp slave/solaris_project_isolation_module.hpp	\
 	slave/webui.hpp tests/environment.hpp tests/external.hpp	\
 	tests/zookeeper_test.hpp tests/flags.hpp tests/utils.hpp	\
+	tests/filter.hpp                                                \
 	tests/zookeeper_test_server.hpp zookeeper/authentication.hpp	\
 	zookeeper/group.hpp zookeeper/watcher.hpp			\
 	zookeeper/zookeeper.hpp zookeeper/url.hpp
@@ -765,7 +766,7 @@ balloon_executor_LDADD = libmesos.la
 
 check_PROGRAMS += mesos-tests
 
-mesos_tests_SOURCES = tests/main.cpp tests/utils.cpp			\
+mesos_tests_SOURCES = tests/main.cpp tests/utils.cpp tests/filter.cpp  	\
 	              tests/environment.cpp				\
 	              tests/master_tests.cpp tests/state_tests.cpp	\
 	              tests/slave_state_tests.cpp			\

Modified: incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp (original)
+++ incubator/mesos/trunk/src/java/jni/org_apache_mesos_Log.cpp Fri Oct 26 22:07:26 2012
@@ -34,6 +34,8 @@
 using namespace mesos;
 using namespace mesos::internal::log;
 
+using namespace process;
+
 
 std::string identity(JNIEnv* env, jobject jposition)
 {

Modified: incubator/mesos/trunk/src/log/coordinator.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/coordinator.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/coordinator.cpp (original)
+++ incubator/mesos/trunk/src/log/coordinator.cpp Fri Oct 26 22:07:26 2012
@@ -27,6 +27,8 @@
 #include "log/coordinator.hpp"
 #include "log/replica.hpp"
 
+using namespace process;
+
 using std::list;
 using std::pair;
 using std::set;

Modified: incubator/mesos/trunk/src/log/coordinator.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/coordinator.hpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/coordinator.hpp (original)
+++ incubator/mesos/trunk/src/log/coordinator.hpp Fri Oct 26 22:07:26 2012
@@ -37,14 +37,10 @@ namespace mesos {
 namespace internal {
 namespace log {
 
-using namespace process;
-
 class Coordinator
 {
 public:
-  Coordinator(int quorum,
-              Replica* replica,
-              Network* group);
+  Coordinator(int quorum, Replica* replica, Network* group);
 
   ~Coordinator();
 
@@ -52,43 +48,45 @@ public:
   // coordinator failed to achieve a quorum (e.g., due to timeout) but
   // can be retried. A some result returns the last committed log
   // position.
-  Result<uint64_t> elect(const Timeout& timeout);
+  Result<uint64_t> elect(const process::Timeout& timeout);
   Result<uint64_t> demote();
 
   // Returns the result of trying to append the specified bytes. A
   // result of none means the append failed (e.g., due to timeout),
   // but can be retried.
-  Result<uint64_t> append(const std::string& bytes, const Timeout& timeout);
+  Result<uint64_t> append(
+      const std::string& bytes,
+      const process::Timeout& timeout);
 
   // Returns the result of trying to truncate the log (from the
   // beginning to the specified position exclusive). A result of
   // none means the truncate failed (e.g., due to timeout), but can be
   // retried.
-  Result<uint64_t> truncate(uint64_t to, const Timeout& timeout);
+  Result<uint64_t> truncate(uint64_t to, const process::Timeout& timeout);
 
 private:
   // Helper that tries to achieve consensus of the specified action. A
   // result of none means the write failed (e.g., due to timeout), but
   // can be retried.
-  Result<uint64_t> write(const Action& action, const Timeout& timeout);
+  Result<uint64_t> write(const Action& action, const process::Timeout& timeout);
 
   // Helper that handles commiting an action (i.e., writing to the
   // local replica and then sending out learned messages).
   Result<uint64_t> commit(const Action& action);
 
   // Helper that tries to fill a position in the log.
-  Result<Action> fill(uint64_t position, const Timeout& timeout);
+  Result<Action> fill(uint64_t position, const process::Timeout& timeout);
 
   // Helper that uses the specified protocol to broadcast a request to
   // our group and return a set of futures.
   template <typename Req, typename Res>
-  std::set<Future<Res> > broadcast(
+  std::set<process::Future<Res> > broadcast(
       const Protocol<Req, Res>& protocol,
       const Req& req);
 
   // Helper like broadcast, but excludes our local replica.
   template <typename Req, typename Res>
-  std::set<Future<Res> > remotecast(
+  std::set<process::Future<Res> > remotecast(
       const Protocol<Req, Res>& protocol,
       const Req& req);
 

Modified: incubator/mesos/trunk/src/log/log.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/log.hpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/log.hpp (original)
+++ incubator/mesos/trunk/src/log/log.hpp Fri Oct 26 22:07:26 2012
@@ -122,7 +122,7 @@ public:
     // those positions are invalid, in which case returns an error.
     Result<std::list<Entry> > read(const Position& from,
                                    const Position& to,
-                                   const Timeout& timeout);
+                                   const process::Timeout& timeout);
 
     // Returns the beginning position of the log from the perspective
     // of the local replica (which may be out of date if the log has
@@ -153,13 +153,17 @@ public:
     // means the operation timed out, otherwise the new ending
     // position of the log is returned or an error. Upon error a new
     // Writer must be created.
-    Result<Position> append(const std::string& data, const Timeout& timeout);
+    Result<Position> append(
+        const std::string& data,
+        const process::Timeout& timeout);
 
     // Attempts to truncate the log up to but not including the
     // specificed position. A none result means the operation timed
     // out, otherwise the new ending position of the log is returned
     // or an error. Upon error a new Writer must be created.
-    Result<Position> truncate(const Position& to, const Timeout& timeout);
+    Result<Position> truncate(
+        const Position& to,
+        const process::Timeout& timeout);
 
   private:
     Option<std::string> error;
@@ -276,7 +280,7 @@ Log::Reader::~Reader() {}
 Result<std::list<Log::Entry> > Log::Reader::read(
     const Log::Position& from,
     const Log::Position& to,
-    const Timeout& timeout)
+    const process::Timeout& timeout)
 {
   process::Future<std::list<Action> > actions =
     replica->read(from.value, to.value);
@@ -341,7 +345,7 @@ Log::Writer::Writer(Log* log, const Dura
     coordinator(log->quorum, log->replica, log->network)
 {
   do {
-    Result<uint64_t> result = coordinator.elect(Timeout(timeout));
+    Result<uint64_t> result = coordinator.elect(process::Timeout(timeout));
     if (result.isNone()) {
       retries--;
     } else if (result.isSome()) {
@@ -362,7 +366,7 @@ Log::Writer::~Writer()
 
 Result<Log::Position> Log::Writer::append(
     const std::string& data,
-    const Timeout& timeout)
+    const process::Timeout& timeout)
 {
   if (error.isSome()) {
     return Result<Log::Position>::error(error.get());
@@ -387,7 +391,7 @@ Result<Log::Position> Log::Writer::appen
 
 Result<Log::Position> Log::Writer::truncate(
     const Log::Position& to,
-    const Timeout& timeout)
+    const process::Timeout& timeout)
 {
   if (error.isSome()) {
     return Result<Log::Position>::error(error.get());

Modified: incubator/mesos/trunk/src/tests/allocator_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/allocator_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/allocator_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/allocator_tests.cpp Fri Oct 26 22:07:26 2012
@@ -29,6 +29,7 @@
 
 #include "slave/process_based_isolation_module.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -568,10 +569,7 @@ TYPED_TEST(AllocatorTest, OutOfOrderDisp
 
 TYPED_TEST(AllocatorTest, SchedulerFailover)
 {
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, Eq(UnregisterFrameworkMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(UnregisterFrameworkMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   EXPECT_CALL(this->allocator, initialize(_, _));
@@ -703,8 +701,6 @@ TYPED_TEST(AllocatorTest, SchedulerFailo
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 

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=1402696&r1=1402695&r2=1402696&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:26 2012
@@ -23,6 +23,7 @@
 #include "master/allocator.hpp"
 #include "master/master.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/zookeeper_test.hpp"
 #include "tests/utils.hpp"
 
@@ -84,15 +85,12 @@ TYPED_TEST(AllocatorZooKeeperTest, Frame
   EXPECT_CALL(this->allocator2, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  MockFilter filter;
-  process::filter(&filter);
-
   trigger shutdownMessageTrigger;
-  EXPECT_MESSAGE(filter, Eq(ShutdownMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ShutdownMessage().GetTypeName()), _, _)
     .WillRepeatedly(DoAll(Trigger(&shutdownMessageTrigger),
 			  Return(true)));
 
-  EXPECT_MESSAGE(filter, Eq(ReregisterSlaveMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ReregisterSlaveMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   Files files;
@@ -235,15 +233,12 @@ TYPED_TEST(AllocatorZooKeeperTest, Slave
   EXPECT_CALL(this->allocator2, resourcesRecovered(_, _, _))
     .WillRepeatedly(DoDefault());
 
-  MockFilter filter;
-  process::filter(&filter);
-
   trigger shutdownMessageTrigger;
-  EXPECT_MESSAGE(filter, Eq(ShutdownMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ShutdownMessage().GetTypeName()), _, _)
     .WillRepeatedly(DoAll(Trigger(&shutdownMessageTrigger),
 			  Return(true)));
 
-  EXPECT_MESSAGE(filter, Eq(ReregisterFrameworkMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ReregisterFrameworkMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   Files files;

Modified: incubator/mesos/trunk/src/tests/environment.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/environment.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/environment.cpp (original)
+++ incubator/mesos/trunk/src/tests/environment.cpp Fri Oct 26 22:07:26 2012
@@ -20,17 +20,34 @@
 
 #include <string>
 
+#include <process/clock.hpp>
+
 #include <stout/os.hpp>
 #include <stout/strings.hpp>
 
 #include "configurator/configurator.hpp"
 
 #include "tests/environment.hpp"
+#include "tests/filter.hpp"
 
 namespace mesos {
 namespace internal {
 namespace tests {
 
+// A simple test event listener that makes sure to resume the clock
+// after each test even if the previous test had a partial result
+// (i.e., an ASSERT_* failed).
+class ClockTestEventListener : public ::testing::EmptyTestEventListener
+{
+public:
+  virtual void OnTestEnd(const ::testing::TestInfo&)
+  {
+    if (process::Clock::paused()) {
+      process::Clock::resume();
+    }
+  }
+};
+
 // Returns true if we should enable a test case or test with the given
 // name. For now, this ONLY disables test cases and tests in two
 // circumstances:
@@ -59,7 +76,7 @@ static bool enable(const std::string& na
 // Setup special tests by updating the gtest filter (i.e., selectively
 // enable/disable tests based on certain "environmental" criteria,
 // such as whether or not the machine has 'cgroups' support).
-static void setupFilter()
+static void setupGtestFilter()
 {
   // First we split the current filter into positive and negative
   // components (which are separated by a '-').
@@ -106,12 +123,19 @@ void Environment::SetUp()
   // Clear any MESOS_ environment variables so they don't affect our tests.
   Configurator::clearMesosEnvironmentVars();
 
-  // Setup specific tests by updating the filter. We do this so that
-  // we can selectively run tests that require root or specific OS
-  // support (e.g., cgroups). Note that this should not effect any
+  // Setup specific tests by updating the gtest filter. We do this so
+  // that we can selectively run tests that require root or specific
+  // OS support (e.g., cgroups). Note that this should not effect any
   // other filters that have been put in place either on the command
   // line or via an environment variable.
-  setupFilter();
+  setupGtestFilter();
+
+  // Add our test event listeners.
+  ::testing::TestEventListeners& listeners =
+    ::testing::UnitTest::GetInstance()->listeners();
+
+  listeners.Append(new FilterTestEventListener());
+  listeners.Append(new ClockTestEventListener());
 }
 
 

Modified: incubator/mesos/trunk/src/tests/exception_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/exception_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/exception_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/exception_tests.cpp Fri Oct 26 22:07:26 2012
@@ -30,6 +30,7 @@
 #include "slave/process_based_isolation_module.hpp"
 #include "slave/slave.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -61,12 +62,6 @@ TEST(ExceptionTest, DeactiveFrameworkOnA
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   PID<Master> master = local::launch(1, 2, 1 * Gigabyte,1 * Gigabyte,  false);
 
   MockScheduler sched;
@@ -86,7 +81,7 @@ TEST(ExceptionTest, DeactiveFrameworkOnA
 
   trigger deactivateMsg;
 
-  EXPECT_MESSAGE(filter, Eq(DeactivateFrameworkMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(DeactivateFrameworkMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&deactivateMsg), Return(false)));
 
   driver.start();
@@ -99,8 +94,6 @@ TEST(ExceptionTest, DeactiveFrameworkOnA
 
   driver.stop();
   local::shutdown();
-
-  process::filter(NULL);
 }
 
 
@@ -142,12 +135,6 @@ TEST(ExceptionTest, DisallowSchedulerCal
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
@@ -184,14 +171,14 @@ TEST(ExceptionTest, DisallowSchedulerCal
   process::Message message;
   trigger rescindMsg, unregisterMsg;
 
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Return(false)));
 
-  EXPECT_MESSAGE(filter, Eq(RescindResourceOfferMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(RescindResourceOfferMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&rescindMsg), Return(false)));
 
-  EXPECT_MESSAGE(filter, Eq(UnregisterFrameworkMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(UnregisterFrameworkMessage().GetTypeName()), _, _)
       .WillOnce(DoAll(Trigger(&unregisterMsg), Return(false)));
 
   driver.start();
@@ -215,6 +202,4 @@ TEST(ExceptionTest, DisallowSchedulerCal
   WAIT_UNTIL(unregisterMsg); //Ensures reception of RescindResourceOfferMessage.
 
   local::shutdown();
-
-  process::filter(NULL);
 }

Modified: incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/fault_tolerance_tests.cpp Fri Oct 26 22:07:26 2012
@@ -34,6 +34,7 @@
 #include "slave/process_based_isolation_module.hpp"
 #include "slave/slave.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -132,21 +133,15 @@ TEST(FaultToleranceTest, SlavePartitione
 
   Clock::pause();
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   uint32_t pings = 0;
 
   // Set these expectations up before we spawn the slave (in
   // local::launch) so that we don't miss the first PING.
-  EXPECT_MESSAGE(filter, Eq("PING"), _, _)
+  EXPECT_MESSAGE(Eq("PING"), _, _)
     .WillRepeatedly(DoAll(Increment(&pings),
                           Return(false)));
 
-  EXPECT_MESSAGE(filter, Eq("PONG"), _, _)
+  EXPECT_MESSAGE(Eq("PONG"), _, _)
     .WillRepeatedly(Return(true));
 
   PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
@@ -191,8 +186,6 @@ TEST(FaultToleranceTest, SlavePartitione
 
   local::shutdown();
 
-  process::filter(NULL);
-
   Clock::resume();
 }
 
@@ -273,12 +266,6 @@ TEST(FaultToleranceTest, FrameworkReliab
 
   Clock::pause();
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
@@ -298,7 +285,7 @@ TEST(FaultToleranceTest, FrameworkReliab
   trigger frameworkRegisteredMsg;
 
   // Drop the first framework registered message, allow subsequent messages.
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&frameworkRegisteredMsg),
                     Return(true)))
     .WillRepeatedly(Return(false));
@@ -316,8 +303,6 @@ TEST(FaultToleranceTest, FrameworkReliab
 
   local::shutdown();
 
-  process::filter(NULL);
-
   Clock::resume();
 }
 
@@ -326,12 +311,6 @@ TEST(FaultToleranceTest, FrameworkReregi
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   PID<Master> master = local::launch(1, 2, 1 * Gigabyte, 1 * Gigabyte, false);
 
   MockScheduler sched;
@@ -353,7 +332,7 @@ TEST(FaultToleranceTest, FrameworkReregi
 
   process::Message message;
 
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Return(false)));
 
@@ -374,8 +353,6 @@ TEST(FaultToleranceTest, FrameworkReregi
   driver.join();
 
   local::shutdown();
-
-  process::filter(NULL);
 }
 
 
@@ -389,12 +366,6 @@ TEST(FaultToleranceTest, DISABLED_TaskLo
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -445,7 +416,7 @@ TEST(FaultToleranceTest, DISABLED_TaskLo
 
   process::Message message;
 
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Return(false)));
 
@@ -483,8 +454,6 @@ TEST(FaultToleranceTest, DISABLED_TaskLo
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 
@@ -494,12 +463,6 @@ TEST(FaultToleranceTest, SchedulerFailov
 
   Clock::pause();
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -555,7 +518,7 @@ TEST(FaultToleranceTest, SchedulerFailov
   EXPECT_CALL(sched1, error(&driver1, "Framework failed over"))
     .Times(1);
 
-  EXPECT_MESSAGE(filter, Eq(StatusUpdateMessage().GetTypeName()), _,
+  EXPECT_MESSAGE(Eq(StatusUpdateMessage().GetTypeName()), _,
              Not(AnyOf(Eq(master), Eq(slave))))
     .WillOnce(DoAll(Trigger(&statusUpdateMsg), Return(true)))
     .RetiresOnSaturation();
@@ -623,8 +586,6 @@ TEST(FaultToleranceTest, SchedulerFailov
   process::terminate(master);
   process::wait(master);
 
-  process::filter(NULL);
-
   Clock::resume();
 }
 
@@ -750,12 +711,6 @@ TEST(FaultToleranceTest, SchedulerExit)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -765,7 +720,7 @@ TEST(FaultToleranceTest, SchedulerExit)
   process::Message message;
 
   // Trigger on the second status update received.
-  EXPECT_MESSAGE(filter, Eq(StatusUpdateMessage().GetTypeName()), _, Eq(master))
+  EXPECT_MESSAGE(Eq(StatusUpdateMessage().GetTypeName()), _, Eq(master))
     .WillOnce(Return(false))
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Trigger(&statusUpdateMsg),Return(false)));
@@ -858,8 +813,6 @@ TEST(FaultToleranceTest, SchedulerExit)
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 
@@ -869,16 +822,10 @@ TEST(FaultToleranceTest, SlaveReliableRe
 
   Clock::pause();
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   trigger slaveRegisteredMsg;
 
   // Drop the first slave registered message, allow subsequent messages.
-  EXPECT_MESSAGE(filter, Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&slaveRegisteredMsg), Return(true)))
     .WillRepeatedly(Return(false));
 
@@ -925,8 +872,6 @@ TEST(FaultToleranceTest, SlaveReliableRe
   process::terminate(master);
   process::wait(master);
 
-  process::filter(NULL);
-
   Clock::resume();
 }
 
@@ -935,12 +880,6 @@ TEST(FaultToleranceTest, SlaveReregister
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -969,7 +908,7 @@ TEST(FaultToleranceTest, SlaveReregister
 
   trigger slaveReRegisterMsg;
 
-  EXPECT_MESSAGE(filter, Eq(SlaveReregisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(SlaveReregisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&slaveReRegisterMsg), Return(false)));
 
   driver.start();
@@ -994,6 +933,4 @@ TEST(FaultToleranceTest, SlaveReregister
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }

Added: incubator/mesos/trunk/src/tests/filter.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/filter.cpp?rev=1402696&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/filter.cpp (added)
+++ incubator/mesos/trunk/src/tests/filter.cpp Fri Oct 26 22:07:26 2012
@@ -0,0 +1,29 @@
+/**
+ * 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 "tests/filter.hpp"
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+TestsFilter* TestsFilter::instance = NULL;
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

Added: incubator/mesos/trunk/src/tests/filter.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/filter.hpp?rev=1402696&view=auto
==============================================================================
--- incubator/mesos/trunk/src/tests/filter.hpp (added)
+++ incubator/mesos/trunk/src/tests/filter.hpp Fri Oct 26 22:07:26 2012
@@ -0,0 +1,117 @@
+/**
+ * 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.
+ */
+
+#ifndef __TESTS_FILTER_HPP__
+#define __TESTS_FILTER_HPP__
+
+#include <gmock/gmock.h>
+
+#include <process/event.hpp>
+#include <process/filter.hpp>
+
+
+// This macro provides a mechanism for matching libprocess
+// messages. TODO(benh): Also add EXPECT_DISPATCH, EXPECT_HTTP, etc.
+#define EXPECT_MESSAGE(name, from, to)                                  \
+  EXPECT_CALL(*mesos::internal::tests::filter(),                        \
+              filter(testing::A<const process::MessageEvent&>()))       \
+    .With(mesos::internal::tests::MessageMatcher(name, from, to))
+
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+// A gtest matcher used by EXPECT_MESSAGE for matching a libprocess
+// MessageEvent.
+MATCHER_P3(MessageMatcher, name, from, to, "")
+{
+  const process::MessageEvent& event = ::std::tr1::get<0>(arg);
+  return (testing::Matcher<std::string>(name).Matches(event.message->name) &&
+          testing::Matcher<process::UPID>(from).Matches(event.message->from) &&
+          testing::Matcher<process::UPID>(to).Matches(event.message->to));
+}
+
+
+// A definition of a libprocess filter to enable waiting for events
+// (such as messages or dispatches) via WAIT_UNTIL in tests (i.e.,
+// using triggers). This is not meant to be used directly by tests;
+// tests should use macros like EXPECT_MESSAGE.
+class TestsFilter : public process::Filter
+{
+public:
+  // A "singleton" instance of Filter that gets used by a single
+  // test. The 'FilterTestEventListener' deletes this instance after
+  // each test to make sure no filter is set for subsequent tests. The
+  // 'filter' routine constructs a new "singleton" if no instance yet
+  // exists.
+  static TestsFilter* instance;
+
+  TestsFilter()
+  {
+    EXPECT_CALL(*this, filter(testing::A<const process::MessageEvent&>()))
+      .WillRepeatedly(testing::Return(false));
+    EXPECT_CALL(*this, filter(testing::A<const process::DispatchEvent&>()))
+      .WillRepeatedly(testing::Return(false));
+    EXPECT_CALL(*this, filter(testing::A<const process::HttpEvent&>()))
+      .WillRepeatedly(testing::Return(false));
+    EXPECT_CALL(*this, filter(testing::A<const process::ExitedEvent&>()))
+      .WillRepeatedly(testing::Return(false));
+
+    process::filter(this);
+  }
+
+  virtual ~TestsFilter()
+  {
+    process::filter(NULL);
+  }
+
+  MOCK_METHOD1(filter, bool(const process::MessageEvent&));
+  MOCK_METHOD1(filter, bool(const process::DispatchEvent&));
+  MOCK_METHOD1(filter, bool(const process::HttpEvent&));
+  MOCK_METHOD1(filter, bool(const process::ExitedEvent&));
+};
+
+
+inline TestsFilter* filter()
+{
+  if (TestsFilter::instance != NULL) {
+    return TestsFilter::instance;
+  }
+
+  return TestsFilter::instance = new TestsFilter();
+}
+
+
+class FilterTestEventListener : public ::testing::EmptyTestEventListener
+{
+public:
+  virtual void OnTestEnd(const ::testing::TestInfo&)
+  {
+    if (TestsFilter::instance != NULL) {
+      delete TestsFilter::instance;
+      TestsFilter::instance = NULL;
+    }
+  }
+};
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {
+
+#endif // __TESTS_FILTER_HPP__

Modified: incubator/mesos/trunk/src/tests/gc_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/gc_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/gc_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/gc_tests.cpp Fri Oct 26 22:07:26 2012
@@ -41,6 +41,7 @@
 #include "slave/flags.hpp"
 #include "slave/slave.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -84,11 +85,6 @@ protected:
   {
     ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-    process::filter(&filter);
-
-    EXPECT_MESSAGE(filter, _, _, _)
-      .WillRepeatedly(Return(false));
-
     a = new TestAllocatorProcess();
     files = new Files();
     m = new Master(a, files);
@@ -106,7 +102,6 @@ protected:
     delete m;
     delete a;
     delete files;
-    process::filter(NULL);
 
     os::rmdir(flags.work_dir);
   }
@@ -149,7 +144,6 @@ protected:
   MockScheduler sched;
   SlaveRegisteredMessage registeredMsg;
   TaskStatus status;
-  MockFilter filter;
   PID<Master> master;
   PID<Slave> slave;
   static flags::Flags<logging::Flags, slave::Flags> flags;
@@ -169,7 +163,7 @@ TEST_F(GarbageCollectorTest, Restart)
   // Messages expectations.
   process::Message message;
   trigger slaveRegisteredMsg1, slaveRegisteredMsg2;
-  EXPECT_MESSAGE(filter, Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(
         SaveArgField<0>(&process::MessageEvent::message, &message),
         Trigger(&slaveRegisteredMsg1),
@@ -177,7 +171,7 @@ TEST_F(GarbageCollectorTest, Restart)
     .WillOnce(DoAll(Trigger(&slaveRegisteredMsg2), Return(false)));
 
   trigger lostSlaveMsg;
-  EXPECT_MESSAGE(filter, Eq(LostSlaveMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(LostSlaveMessage().GetTypeName()), _, _)
     .WillRepeatedly(DoAll(Trigger(&lostSlaveMsg), Return(false)));
 
   // Executor expectations.
@@ -262,7 +256,7 @@ TEST_F(GarbageCollectorTest, ExitedExecu
 {
   // Messages expectations.
   trigger exitedExecutorMsg;
-  EXPECT_MESSAGE(filter, Eq(ExitedExecutorMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ExitedExecutorMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&exitedExecutorMsg), Return(false)));
 
   // Executor expectations.
@@ -340,7 +334,7 @@ TEST_F(GarbageCollectorTest, DiskUsage)
 {
   // Messages expectations.
   trigger exitedExecutorMsg;
-  EXPECT_MESSAGE(filter, Eq(ExitedExecutorMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(ExitedExecutorMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&exitedExecutorMsg), Return(false)));
 
   // Executor expectations.

Modified: incubator/mesos/trunk/src/tests/log_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/log_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/log_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/log_tests.cpp Fri Oct 26 22:07:26 2012
@@ -21,8 +21,11 @@
 #include <set>
 #include <string>
 
+#include <process/clock.hpp>
 #include <process/future.hpp>
+#include <process/pid.hpp>
 #include <process/protobuf.hpp>
+#include <process/timeout.hpp>
 
 #include <stout/option.hpp>
 #include <stout/os.hpp>
@@ -35,6 +38,7 @@
 
 #include "messages/messages.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -42,7 +46,10 @@ using namespace mesos::internal;
 using namespace mesos::internal::log;
 using namespace mesos::internal::tests;
 
-using namespace process;
+using process::Clock;
+using process::Future;
+using process::Timeout;
+using process::UPID;
 
 using testing::_;
 using testing::Eq;
@@ -686,13 +693,7 @@ TEST(CoordinatorTest, Fill)
 
 TEST(CoordinatorTest, NotLearnedFill)
 {
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
-  EXPECT_MESSAGE(filter, Eq(LearnedMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(LearnedMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   const std::string path1 = os::getcwd() + "/.log1";
@@ -760,8 +761,6 @@ TEST(CoordinatorTest, NotLearnedFill)
   os::rmdir(path1);
   os::rmdir(path2);
   os::rmdir(path3);
-
-  process::filter(NULL);
 }
 
 
@@ -815,13 +814,7 @@ TEST(CoordinatorTest, MultipleAppends)
 
 TEST(CoordinatorTest, MultipleAppendsNotLearnedFill)
 {
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
-  EXPECT_MESSAGE(filter, Eq(LearnedMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(LearnedMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   const std::string path1 = os::getcwd() + "/.log1";
@@ -887,8 +880,6 @@ TEST(CoordinatorTest, MultipleAppendsNot
   os::rmdir(path1);
   os::rmdir(path2);
   os::rmdir(path3);
-
-  process::filter(NULL);
 }
 
 
@@ -955,13 +946,7 @@ TEST(CoordinatorTest, Truncate)
 
 TEST(CoordinatorTest, TruncateNotLearnedFill)
 {
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
-  EXPECT_MESSAGE(filter, Eq(LearnedMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(LearnedMessage().GetTypeName()), _, _)
     .WillRepeatedly(Return(true));
 
   const std::string path1 = os::getcwd() + "/.log1";
@@ -1040,8 +1025,6 @@ TEST(CoordinatorTest, TruncateNotLearned
   os::rmdir(path1);
   os::rmdir(path2);
   os::rmdir(path3);
-
-  process::filter(NULL);
 }
 
 

Modified: incubator/mesos/trunk/src/tests/master_tests.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/master_tests.cpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/master_tests.cpp (original)
+++ incubator/mesos/trunk/src/tests/master_tests.cpp Fri Oct 26 22:07:26 2012
@@ -41,6 +41,7 @@
 
 #include "slave/slave.hpp"
 
+#include "tests/filter.hpp"
 #include "tests/utils.hpp"
 
 using namespace mesos;
@@ -664,12 +665,6 @@ TEST(MasterTest, MasterInfo)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -718,8 +713,6 @@ TEST(MasterTest, MasterInfo)
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 
@@ -727,12 +720,6 @@ TEST(MasterTest, MasterInfoOnReElection)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -770,7 +757,7 @@ TEST(MasterTest, MasterInfoOnReElection)
 
   process::Message message;
 
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Return(false)));
 
@@ -798,8 +785,6 @@ TEST(MasterTest, MasterInfoOnReElection)
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 
@@ -823,12 +808,6 @@ TEST_F(WhitelistFixture, WhitelistSlave)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   // Add some hosts to the white list.
   Try<string> hostname = os::hostname();
   ASSERT_SOME(hostname);
@@ -844,7 +823,7 @@ TEST_F(WhitelistFixture, WhitelistSlave)
 
   trigger slaveRegisteredMsg;
 
-  EXPECT_MESSAGE(filter, Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(SlaveRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(Trigger(&slaveRegisteredMsg), Return(false)));
 
   MockExecutor exec;
@@ -889,8 +868,6 @@ TEST_F(WhitelistFixture, WhitelistSlave)
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 
@@ -913,12 +890,6 @@ TEST(MasterTest, MasterLost)
 {
   ASSERT_TRUE(GTEST_IS_THREADSAFE);
 
-  MockFilter filter;
-  process::filter(&filter);
-
-  EXPECT_MESSAGE(filter, _, _, _)
-    .WillRepeatedly(Return(false));
-
   TestAllocatorProcess a;
   Files files;
   Master m(&a, &files);
@@ -956,7 +927,7 @@ TEST(MasterTest, MasterLost)
 
   process::Message message;
 
-  EXPECT_MESSAGE(filter, Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
+  EXPECT_MESSAGE(Eq(FrameworkRegisteredMessage().GetTypeName()), _, _)
     .WillOnce(DoAll(SaveArgField<0>(&process::MessageEvent::message, &message),
                     Return(false)));
 
@@ -978,8 +949,6 @@ TEST(MasterTest, MasterLost)
 
   process::terminate(master);
   process::wait(master);
-
-  process::filter(NULL);
 }
 
 // This fixture sets up expectations on the storage class

Modified: incubator/mesos/trunk/src/tests/utils.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/utils.hpp?rev=1402696&r1=1402695&r2=1402696&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/utils.hpp (original)
+++ incubator/mesos/trunk/src/tests/utils.hpp Fri Oct 26 22:07:26 2012
@@ -423,48 +423,6 @@ inline const ::testing::Matcher<const st
 }
 
 
-// Definition of a mock Filter so that messages can act as triggers.
-class MockFilter : public process::Filter
-{
-public:
-  MockFilter()
-  {
-    EXPECT_CALL(*this, filter(testing::A<const process::MessageEvent&>()))
-      .WillRepeatedly(testing::Return(false));
-    EXPECT_CALL(*this, filter(testing::A<const process::DispatchEvent&>()))
-      .WillRepeatedly(testing::Return(false));
-    EXPECT_CALL(*this, filter(testing::A<const process::HttpEvent&>()))
-      .WillRepeatedly(testing::Return(false));
-    EXPECT_CALL(*this, filter(testing::A<const process::ExitedEvent&>()))
-      .WillRepeatedly(testing::Return(false));
-  }
-
-  MOCK_METHOD1(filter, bool(const process::MessageEvent&));
-  MOCK_METHOD1(filter, bool(const process::DispatchEvent&));
-  MOCK_METHOD1(filter, bool(const process::HttpEvent&));
-  MOCK_METHOD1(filter, bool(const process::ExitedEvent&));
-};
-
-
-// A message can be matched against in conjunction with the MockFilter
-// (see above) to perform specific actions based for messages.
-MATCHER_P3(MsgMatcher, name, from, to, "")
-{
-  const process::MessageEvent& event = ::std::tr1::get<0>(arg);
-  return (testing::Matcher<std::string>(name).Matches(event.message->name) &&
-          testing::Matcher<process::UPID>(from).Matches(event.message->from) &&
-          testing::Matcher<process::UPID>(to).Matches(event.message->to));
-}
-
-
-// This macro provides some syntactic sugar for matching messages
-// using the message matcher (see above) as well as the MockFilter
-// (see above). We should also add EXPECT_DISPATCH, EXPECT_HTTP, etc.
-#define EXPECT_MESSAGE(mockFilter, name, from, to)              \
-  EXPECT_CALL(mockFilter, filter(testing::A<const process::MessageEvent&>())) \
-    .With(MsgMatcher(name, from, to))
-
-
 ACTION_TEMPLATE(SaveArgField,
                 HAS_1_TEMPLATE_PARAMS(int, k),
                 AND_2_VALUE_PARAMS(field, pointer))