You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@mesos.apache.org by ti...@apache.org on 2015/02/23 06:55:29 UTC

[1/2] mesos git commit: Introduced anonymous modules.

Repository: mesos
Updated Branches:
  refs/heads/master 336997936 -> 7ee3b7b67


Introduced anonymous modules.

New kind of module that does __not__ receive any callbacks but simply
coexists with its parent (OS-)process.

Unlike other named modules, an anonymous module does not directly
replace or provide essential Mesos functionality (such as an Isolator
module does). Unlike a decorator module it does not directly add or
inject data into Mesos core either.

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


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/384587ee
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/384587ee
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/384587ee

Branch: refs/heads/master
Commit: 384587ee8695e45f9a3c5a854c3b9981093045c9
Parents: 3369979
Author: Till Toenshoff <to...@me.com>
Authored: Mon Feb 23 05:26:54 2015 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Mon Feb 23 05:26:55 2015 +0100

----------------------------------------------------------------------
 include/mesos/module/anonymous.hpp | 81 +++++++++++++++++++++++++++++++++
 src/Makefile.am                    |  1 +
 src/local/local.cpp                | 23 +++++++++-
 src/master/main.cpp                | 22 ++++++++-
 src/module/manager.cpp             |  4 ++
 src/module/manager.hpp             | 23 ++++++++++
 src/slave/main.cpp                 | 22 ++++++++-
 7 files changed, 173 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/include/mesos/module/anonymous.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module/anonymous.hpp b/include/mesos/module/anonymous.hpp
new file mode 100644
index 0000000..22862bd
--- /dev/null
+++ b/include/mesos/module/anonymous.hpp
@@ -0,0 +1,81 @@
+/**
+ * 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 __MESOS_MODULE_ANONYMOUS_HPP__
+#define __MESOS_MODULE_ANONYMOUS_HPP__
+
+#include <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+namespace mesos {
+namespace modules {
+
+// The 'Anonymous' kind of module does __not__ receive any callbacks
+// but simply coexists with its parent (OS-)process.
+//
+// Unlike other named modules, an anonymous module does not directly
+// replace or provide essential Mesos functionality (such as an
+// Isolator module does). Unlike a decorator module it does not
+// directly add or inject data into Mesos core either.
+//
+// Anonymous modules do not require any specific selectors (flags) as
+// they are immediately instantiated when getting loaded via the
+// '--modules' flag by the Mesos master or slave.
+//
+class Anonymous
+{
+public:
+  Anonymous() {}
+
+  virtual ~Anonymous() {}
+};
+
+
+template <>
+inline const char* kind<Anonymous>()
+{
+  return "Anonymous";
+}
+
+
+template <>
+struct Module<Anonymous> : ModuleBase
+{
+  Module(const char* _moduleApiVersion,
+         const char* _mesosVersion,
+         const char* _authorName,
+         const char* _authorEmail,
+         const char* _description,
+         bool (*_compatible)(),
+         Anonymous* (*_create)(const Parameters& parameters))
+    : ModuleBase(_moduleApiVersion,
+                 _mesosVersion,
+                 mesos::modules::kind<Anonymous>(),
+                 _authorName,
+                 _authorEmail,
+                 _description,
+                 _compatible),
+      create(_create) {}
+
+  Anonymous* (*create)(const Parameters& parameters);
+};
+
+} // namespace modules {
+} // namespace mesos {
+
+#endif // __MESOS_MODULE_ANONYMOUS_HPP__

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index d372404..34dbd9c 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -396,6 +396,7 @@ nodist_fetcher_HEADERS = ../include/mesos/fetcher/fetcher.pb.h
 moduledir = $(pkgincludedir)/module
 
 module_HEADERS =							\
+  $(top_srcdir)/include/mesos/module/anonymous.hpp			\
   $(top_srcdir)/include/mesos/module/authenticatee.hpp			\
   $(top_srcdir)/include/mesos/module/authenticator.hpp			\
   $(top_srcdir)/include/mesos/module/hook.hpp				\

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/local/local.cpp
----------------------------------------------------------------------
diff --git a/src/local/local.cpp b/src/local/local.cpp
index 1ef04b8..5717bd6 100644
--- a/src/local/local.cpp
+++ b/src/local/local.cpp
@@ -21,6 +21,8 @@
 #include <sstream>
 #include <vector>
 
+#include <mesos/module/anonymous.hpp>
+
 #include <process/owned.hpp>
 #include <process/pid.hpp>
 
@@ -74,6 +76,9 @@ using mesos::master::Master;
 using mesos::master::Registrar;
 using mesos::master::Repairer;
 
+using mesos::modules::Anonymous;
+using mesos::modules::ModuleManager;
+
 using mesos::slave::Containerizer;
 using mesos::slave::Fetcher;
 using mesos::slave::GarbageCollector;
@@ -139,7 +144,7 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
     // Load modules. Note that this covers both, master and slave
     // specific modules as both use the same flag (--modules).
     if (flags.modules.isSome()) {
-      Try<Nothing> result = modules::ModuleManager::load(flags.modules.get());
+      Try<Nothing> result = ModuleManager::load(flags.modules.get());
       if (result.isError()) {
         EXIT(1) << "Error loading modules: " << result.error();
       }
@@ -194,6 +199,22 @@ PID<Master> launch(const Flags& flags, Allocator* _allocator)
       authorizer = authorizer__.release();
     }
 
+    // Create anonymous modules.
+    foreach (const string& name, ModuleManager::find<Anonymous>()) {
+      Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);
+      if (create.isError()) {
+        EXIT(1) << "Failed to create anonymous module named '" << name << "'";
+      }
+
+      // We don't bother keeping around the pointer to this anonymous
+      // module, when we exit that will effectively free it's memory.
+      //
+      // TODO(benh): We might want to add explicit finalization (and
+      // maybe explicit initialization too) in order to let the module
+      // do any housekeeping necessary when the master is cleanly
+      // terminating.
+    }
+
     master = new Master(
         _allocator,
         registrar,

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/master/main.cpp
----------------------------------------------------------------------
diff --git a/src/master/main.cpp b/src/master/main.cpp
index a1efd2b..4f5eb55 100644
--- a/src/master/main.cpp
+++ b/src/master/main.cpp
@@ -22,6 +22,8 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/module/anonymous.hpp>
+
 #include <process/owned.hpp>
 #include <process/pid.hpp>
 
@@ -72,6 +74,8 @@ using namespace mesos::state;
 using namespace zookeeper;
 
 using mesos::MasterInfo;
+
+using mesos::modules::Anonymous;
 using mesos::modules::ModuleManager;
 
 using process::Owned;
@@ -285,6 +289,22 @@ int main(int argc, char** argv)
     authorizer = authorizer__.release();
   }
 
+  // Create anonymous modules.
+  foreach (const string& name, ModuleManager::find<Anonymous>()) {
+    Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);
+    if (create.isError()) {
+      EXIT(1) << "Failed to create anonymous module named '" << name << "'";
+    }
+
+    // We don't bother keeping around the pointer to this anonymous
+    // module, when we exit that will effectively free it's memory.
+    //
+    // TODO(benh): We might want to add explicit finalization (and
+    // maybe explicit initialization too) in order to let the module
+    // do any housekeeping necessary when the master is cleanly
+    // terminating.
+  }
+
   LOG(INFO) << "Starting Mesos master";
 
   Master* master =
@@ -305,8 +325,8 @@ int main(int argc, char** argv)
   }
 
   process::spawn(master);
-
   process::wait(master->self());
+
   delete master;
   delete allocator;
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 44a3e65..7a5a81e 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -21,6 +21,7 @@
 
 #include <mesos/module.hpp>
 
+#include <mesos/module/anonymous.hpp>
 #include <mesos/module/module.hpp>
 
 #include <stout/json.hpp>
@@ -31,7 +32,9 @@
 #include <stout/version.hpp>
 
 #include "common/lock.hpp"
+
 #include "messages/messages.hpp"
+
 #include "module/manager.hpp"
 
 using std::list;
@@ -58,6 +61,7 @@ void ModuleManager::initialize()
   // current change.  Typically that should be the version currently
   // under development.
 
+  kindToVersion["Anonymous"] = MESOS_VERSION;
   kindToVersion["Authenticatee"] = MESOS_VERSION;
   kindToVersion["Authenticator"] = MESOS_VERSION;
   kindToVersion["Hook"] = MESOS_VERSION;

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/module/manager.hpp
----------------------------------------------------------------------
diff --git a/src/module/manager.hpp b/src/module/manager.hpp
index f071678..bebbb40 100644
--- a/src/module/manager.hpp
+++ b/src/module/manager.hpp
@@ -107,6 +107,29 @@ public:
             moduleBases[moduleName]->kind == stringify(kind<T>()));
   }
 
+  // Returns all module names that have been loaded that implement the
+  // specified interface 'T'. For example:
+  //
+  //   std::vector<std::string> modules = ModuleManager::find<Isolator>();
+  //
+  // Will return all of the module names for modules that implement
+  // the Isolator interface.
+  template <typename T>
+  static std::vector<std::string> find()
+  {
+    mesos::Lock lock(&mutex);
+
+    std::vector<std::string> names;
+
+    foreachpair (const std::string& name, ModuleBase* base, moduleBases) {
+      if (base->kind == stringify(kind<T>())) {
+        names.push_back(name);
+      }
+    }
+
+    return names;
+  }
+
   // Exposed just for testing so that we can unload a given
   // module  and remove it from the list of ModuleBases.
   static Try<Nothing> unload(const std::string& moduleName);

http://git-wip-us.apache.org/repos/asf/mesos/blob/384587ee/src/slave/main.cpp
----------------------------------------------------------------------
diff --git a/src/slave/main.cpp b/src/slave/main.cpp
index 173a1a3..d1126b7 100644
--- a/src/slave/main.cpp
+++ b/src/slave/main.cpp
@@ -20,6 +20,8 @@
 
 #include <mesos/mesos.hpp>
 
+#include <mesos/module/anonymous.hpp>
+
 #include <stout/check.hpp>
 #include <stout/flags.hpp>
 #include <stout/nothing.hpp>
@@ -46,7 +48,9 @@
 using namespace mesos;
 using namespace mesos::slave;
 
+using mesos::modules::Anonymous;
 using mesos::modules::ModuleManager;
+
 using mesos::SlaveInfo;
 
 using std::cerr;
@@ -177,6 +181,22 @@ int main(int argc, char** argv)
     EXIT(1) << "Failed to create a master detector: " << detector.error();
   }
 
+  // Create anonymous modules.
+  foreach (const string& name, ModuleManager::find<Anonymous>()) {
+    Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);
+    if (create.isError()) {
+      EXIT(1) << "Failed to create anonymous module named '" << name << "'";
+    }
+
+    // We don't bother keeping around the pointer to this anonymous
+    // module, when we exit that will effectively free it's memory.
+    //
+    // TODO(benh): We might want to add explicit finalization (and
+    // maybe explicit initialization too) in order to let the module
+    // do any housekeeping necessary when the slave is cleanly
+    // terminating.
+  }
+
   LOG(INFO) << "Starting Mesos slave";
 
   Files files;
@@ -192,8 +212,8 @@ int main(int argc, char** argv)
       &statusUpdateManager);
 
   process::spawn(slave);
-
   process::wait(slave->self());
+
   delete slave;
 
   delete detector.get();


[2/2] mesos git commit: Added anonymous module example and tests.

Posted by ti...@apache.org.
Added anonymous module example and tests.

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


Project: http://git-wip-us.apache.org/repos/asf/mesos/repo
Commit: http://git-wip-us.apache.org/repos/asf/mesos/commit/7ee3b7b6
Tree: http://git-wip-us.apache.org/repos/asf/mesos/tree/7ee3b7b6
Diff: http://git-wip-us.apache.org/repos/asf/mesos/diff/7ee3b7b6

Branch: refs/heads/master
Commit: 7ee3b7b672a4d8fee4fe4eb5f0aa2e7e3bf6b049
Parents: 384587e
Author: Till Toenshoff <to...@me.com>
Authored: Mon Feb 23 05:27:28 2015 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Mon Feb 23 05:27:29 2015 +0100

----------------------------------------------------------------------
 src/Makefile.am                        | 10 +++-
 src/examples/test_anonymous_module.cpp | 67 ++++++++++++++++++++++++
 src/examples/test_anonymous_module.hpp | 25 +++++++++
 src/module/manager.hpp                 |  2 +-
 src/tests/anonymous_tests.cpp          | 79 +++++++++++++++++++++++++++++
 src/tests/module.cpp                   | 40 +++++++++++----
 src/tests/module.hpp                   |  3 +-
 7 files changed, 214 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/Makefile.am
----------------------------------------------------------------------
diff --git a/src/Makefile.am b/src/Makefile.am
index 34dbd9c..65cba64 100644
--- a/src/Makefile.am
+++ b/src/Makefile.am
@@ -486,8 +486,9 @@ libmesos_no_3rdparty_la_SOURCES +=					\
 	common/status_utils.hpp						\
 	common/thread.hpp						\
 	credentials/credentials.hpp					\
-	examples/utils.hpp						\
+	examples/test_anonymous_module.hpp				\
 	examples/test_module.hpp					\
+	examples/utils.hpp						\
 	files/files.hpp							\
 	hdfs/hdfs.hpp							\
 	hook/manager.hpp						\
@@ -1278,6 +1279,12 @@ libexamplemodule_la_SOURCES = examples/example_module_impl.cpp
 libexamplemodule_la_CPPFLAGS = $(MESOS_CPPFLAGS)
 libexamplemodule_la_LDFLAGS = $(MESOS_MODULE_LDFLAGS)
 
+# Library containing the test anonymous modules.
+noinst_LTLIBRARIES += libtestanonymous.la
+libtestanonymous_la_SOURCES = examples/test_anonymous_module.cpp
+libtestanonymous_la_CPPFLAGS = $(MESOS_CPPFLAGS)
+libtestanonymous_la_LDFLAGS = $(MESOS_MODULE_LDFLAGS)
+
 # Library containing the test CRAM-MD5 authentication modules.
 # TODO(tillt): Add cyrus-sasl2 dependency while removing it from libmesos.
 # TODO(tillt): Enable optional building of this module library.
@@ -1300,6 +1307,7 @@ libtestisolator_la_CPPFLAGS = $(MESOS_CPPFLAGS)
 libtestisolator_la_LDFLAGS = $(MESOS_MODULE_LDFLAGS)
 
 mesos_tests_SOURCES =				\
+  tests/anonymous_tests.cpp			\
   tests/attributes_tests.cpp			\
   tests/authentication_tests.cpp		\
   tests/authorization_tests.cpp		        \

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/examples/test_anonymous_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_anonymous_module.cpp b/src/examples/test_anonymous_module.cpp
new file mode 100644
index 0000000..859f4e1
--- /dev/null
+++ b/src/examples/test_anonymous_module.cpp
@@ -0,0 +1,67 @@
+/**
+ * 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 <mesos/mesos.hpp>
+#include <mesos/module.hpp>
+
+#include <mesos/module/anonymous.hpp>
+
+#include <stout/foreach.hpp>
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+
+#include "test_anonymous_module.hpp"
+
+using namespace mesos;
+
+using mesos::modules::Anonymous;
+
+class TestAnonymous : public Anonymous
+{
+public:
+  TestAnonymous()
+  {
+    VLOG(1) << "Anonymous module constructor";
+    os::setenv(TEST_ANONYMOUS_ENVIRONMENT_VARIABLE, "42");
+  }
+
+  // TODO(tillt): Currently this destructor will only ever get called
+  // during the test runs. Fix this behavior by introducing anonymous
+  // module instance reference management.
+  virtual ~TestAnonymous()
+  {
+    VLOG(1) << "Anonymous module destructor";
+  }
+};
+
+
+static Anonymous* createAnonymous(const Parameters& parameters)
+{
+  return new TestAnonymous();
+}
+
+
+// Declares an anonymous module named 'TestAnonymous'.
+mesos::modules::Module<Anonymous> org_apache_mesos_TestAnonymous(
+  MESOS_MODULE_API_VERSION,
+  MESOS_VERSION,
+  "Apache Mesos",
+  "modules@mesos.apache.org",
+  "Test anonymous module.",
+  NULL,
+  createAnonymous);

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/examples/test_anonymous_module.hpp
----------------------------------------------------------------------
diff --git a/src/examples/test_anonymous_module.hpp b/src/examples/test_anonymous_module.hpp
new file mode 100644
index 0000000..04655ec
--- /dev/null
+++ b/src/examples/test_anonymous_module.hpp
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef __EXAMPLES_TEST_ANONYMOUS_MODULE_HPP__
+#define __EXAMPLES_TEST_ANONYMOUS_MODULE_HPP__
+
+// Test environment variable.
+static const char* TEST_ANONYMOUS_ENVIRONMENT_VARIABLE = "MESOS_TEST_ANONYMOUS";
+
+#endif

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/module/manager.hpp
----------------------------------------------------------------------
diff --git a/src/module/manager.hpp b/src/module/manager.hpp
index bebbb40..e8333d9 100644
--- a/src/module/manager.hpp
+++ b/src/module/manager.hpp
@@ -110,7 +110,7 @@ public:
   // Returns all module names that have been loaded that implement the
   // specified interface 'T'. For example:
   //
-  //   std::vector<std::string> modules = ModuleManager::find<Isolator>();
+  //   std::vector<std::string> modules = ModuleManager::find<Hook>();
   //
   // Will return all of the module names for modules that implement
   // the Isolator interface.

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/tests/anonymous_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/anonymous_tests.cpp b/src/tests/anonymous_tests.cpp
new file mode 100644
index 0000000..541de7a
--- /dev/null
+++ b/src/tests/anonymous_tests.cpp
@@ -0,0 +1,79 @@
+/**
+ * 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 <mesos/module.hpp>
+
+#include <mesos/module/anonymous.hpp>
+
+#include <process/owned.hpp>
+
+#include <stout/os.hpp>
+#include <stout/try.hpp>
+
+#include "examples/test_anonymous_module.hpp"
+
+#include "module/manager.hpp"
+
+#include "tests/mesos.hpp"
+
+using namespace mesos;
+
+using namespace mesos::modules;
+using namespace mesos::tests;
+
+using std::string;
+using std::vector;
+
+using testing::_;
+using testing::Return;
+
+namespace mesos {
+namespace internal {
+namespace tests {
+
+class AnonymousTest : public MesosTest {};
+
+
+// Test for the side effect of our test-module which mutates the
+// environment once it got loaded.
+TEST_F(AnonymousTest, Running)
+{
+  // Clear test relevant environment.
+  os::unsetenv(TEST_ANONYMOUS_ENVIRONMENT_VARIABLE);
+
+  // Creates an instance of all anonymous module implementations
+  // loaded.
+  vector<process::Owned<Anonymous>> modules;
+
+  foreach (const string& name, ModuleManager::find<Anonymous>()) {
+    Try<Anonymous*> create = ModuleManager::create<Anonymous>(name);
+    ASSERT_SOME(create);
+    modules.push_back(process::Owned<Anonymous>(create.get()));
+  }
+
+  // Test if the environment variables have been created by the
+  // anonymous module.
+  EXPECT_EQ("42", os::getenv(TEST_ANONYMOUS_ENVIRONMENT_VARIABLE));
+
+  // Clear test relevant environment.
+  os::unsetenv(TEST_ANONYMOUS_ENVIRONMENT_VARIABLE);
+}
+
+} // namespace tests {
+} // namespace internal {
+} // namespace mesos {

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/tests/module.cpp
----------------------------------------------------------------------
diff --git a/src/tests/module.cpp b/src/tests/module.cpp
index 3c9bab7..a8c0f9a 100644
--- a/src/tests/module.cpp
+++ b/src/tests/module.cpp
@@ -49,7 +49,7 @@ static void addModule(
 
 
 // Add available Isolator modules.
-static void addIsolatorModules(Modules& modules)
+static void addIsolatorModules(Modules* modules)
 {
   const string libraryPath = path::join(
       tests::flags.build_dir,
@@ -58,7 +58,7 @@ static void addIsolatorModules(Modules& modules)
       os::libraries::expandName("testisolator"));
 
   // Now add our test CPU and Memory isolator modules.
-  Modules::Library* library = modules.add_libraries();
+  Modules::Library* library = modules->add_libraries();
   library->set_file(libraryPath);
 
   // To add a new module from this library, create a new ModuleID enum
@@ -69,7 +69,7 @@ static void addIsolatorModules(Modules& modules)
 
 
 // Add available Authentication modules.
-static void addAuthenticationModules(Modules& modules)
+static void addAuthenticationModules(Modules* modules)
 {
   const string libraryPath = path::join(
       tests::flags.build_dir,
@@ -78,7 +78,7 @@ static void addAuthenticationModules(Modules& modules)
       os::libraries::expandName("testauthentication"));
 
   // Now add our test authentication modules.
-  Modules::Library* library = modules.add_libraries();
+  Modules::Library* library = modules->add_libraries();
   library->set_file(libraryPath);
 
   // To add a new module from this library, create a new ModuleID enum
@@ -92,7 +92,7 @@ static void addAuthenticationModules(Modules& modules)
 }
 
 
-static void addHookModules(Modules& modules)
+static void addHookModules(Modules* modules)
 {
   const string libraryPath = path::join(
       tests::flags.build_dir,
@@ -101,7 +101,7 @@ static void addHookModules(Modules& modules)
       os::libraries::expandName("testhook"));
 
   // Now add our test hook module.
-  Modules::Library* library = modules.add_libraries();
+  Modules::Library* library = modules->add_libraries();
   library->set_file(libraryPath);
 
   // To add a new module from this library, create a new ModuleID enum
@@ -110,6 +110,25 @@ static void addHookModules(Modules& modules)
 }
 
 
+static void addAnonymousModules(Modules* modules)
+{
+  const string libraryPath = path::join(
+      tests::flags.build_dir,
+      "src",
+      ".libs",
+      os::libraries::expandName("testanonymous"));
+
+  // Now add our test anonymous module.
+  Modules::Library* library = modules->add_libraries();
+  library->set_file(libraryPath);
+
+  // To add a new module from this library, create a new ModuleID enum
+  // and tie it with a module name.
+  addModule(
+      library, TestAnonymous, "org_apache_mesos_TestAnonymous");
+}
+
+
 Try<Nothing> tests::initModules(const Option<Modules>& modules)
 {
   // First get the user provided modules.
@@ -119,13 +138,16 @@ Try<Nothing> tests::initModules(const Option<Modules>& modules)
   }
 
   // Add isolator modules from testisolator library.
-  addIsolatorModules(mergedModules);
+  addIsolatorModules(&mergedModules);
 
   // Add authentication modules from testauthentication library.
-  addAuthenticationModules(mergedModules);
+  addAuthenticationModules(&mergedModules);
 
   // Add hook modules from testhook library.
-  addHookModules(mergedModules);
+  addHookModules(&mergedModules);
+
+  // Add anonymous modules from testanonymous library.
+  addAnonymousModules(&mergedModules);
 
   return ModuleManager::load(mergedModules);
 }

http://git-wip-us.apache.org/repos/asf/mesos/blob/7ee3b7b6/src/tests/module.hpp
----------------------------------------------------------------------
diff --git a/src/tests/module.hpp b/src/tests/module.hpp
index 5748add..af71b58 100644
--- a/src/tests/module.hpp
+++ b/src/tests/module.hpp
@@ -44,7 +44,8 @@ enum ModuleID
   TestCpuIsolator,
   TestCRAMMD5Authenticatee,
   TestCRAMMD5Authenticator,
-  TestHook
+  TestHook,
+  TestAnonymous
 };