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 2014/10/27 21:15:48 UTC

git commit: Added support for module-specific command-line parameters.

Repository: mesos
Updated Branches:
  refs/heads/master 1ff259e96 -> 17ecc0c0c


Added support for module-specific command-line parameters.

The Modules protobuf is enhanced to support key-value parameters based
on mesos::Parameter and mesos::Parameters.

Bumped Module API version since we modified the signature of the
create() method.

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


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

Branch: refs/heads/master
Commit: 17ecc0c0c49908e5a0b31fdea642438301feb2b3
Parents: 1ff259e
Author: Kapil Arya <ka...@mesosphere.io>
Authored: Mon Oct 27 21:13:53 2014 +0100
Committer: Till Toenshoff <to...@me.com>
Committed: Mon Oct 27 21:13:53 2014 +0100

----------------------------------------------------------------------
 include/mesos/module.hpp              |   2 +-
 src/examples/example_module_impl.cpp  |  51 +++++++++++++-
 src/examples/test_isolator_module.cpp |   7 +-
 src/examples/test_module.hpp          |  12 +++-
 src/master/flags.hpp                  |  23 ++++++-
 src/messages/messages.proto           |  10 ++-
 src/module/isolator.hpp               |   6 +-
 src/module/manager.cpp                |  17 +++--
 src/module/manager.hpp                |   5 +-
 src/slave/flags.hpp                   |  23 ++++++-
 src/tests/flags.hpp                   |  23 ++++++-
 src/tests/module.cpp                  |   3 +-
 src/tests/module_tests.cpp            | 105 +++++++++++++++++++++++++++--
 13 files changed, 255 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/include/mesos/module.hpp
----------------------------------------------------------------------
diff --git a/include/mesos/module.hpp b/include/mesos/module.hpp
index 5bafb40..5942126 100644
--- a/include/mesos/module.hpp
+++ b/include/mesos/module.hpp
@@ -46,7 +46,7 @@
 
 #include <mesos/version.hpp>
 
-#define MESOS_MODULE_API_VERSION "1"
+#define MESOS_MODULE_API_VERSION "2"
 
 namespace mesos {
 namespace modules {

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/examples/example_module_impl.cpp
----------------------------------------------------------------------
diff --git a/src/examples/example_module_impl.cpp b/src/examples/example_module_impl.cpp
index 6a48d67..1e28221 100644
--- a/src/examples/example_module_impl.cpp
+++ b/src/examples/example_module_impl.cpp
@@ -16,8 +16,14 @@
  * limitations under the License.
  */
 
+#include <string>
+
 #include <mesos/module.hpp>
 
+#include <stout/hashmap.hpp>
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
 #include "test_module.hpp"
 
 // Mesos core receives an object of type TestModuleImpl when
@@ -27,6 +33,28 @@
 class TestModuleImpl : public TestModule
 {
 public:
+  Try<Nothing> initialize(const mesos::Parameters& parameters)
+  {
+    foreach (const mesos::Parameter& parameter, parameters.parameter()) {
+      if (parameter.has_key() && parameter.has_value()) {
+        flags[parameter.key()] = parameter.value();
+      } else {
+        return Error("Invalid key-value parameters");
+      }
+    }
+
+    // We expect that when specifying the module, a module parameter
+    // was also specified, i.e.:
+    // "modules": [{"name": "org_apache_mesos_TestModule",
+    //              "flags": [{"key": "operation", "value": "sum"}]}]
+    // The expected value for the key "operation" is "sum".
+    if (flags.contains("operation") && flags["operation"] != "sum") {
+      return Error("Invalid 'operation'");
+    }
+
+    return Nothing();
+  }
+
   virtual int foo(char a, long b)
   {
     return a + b;
@@ -36,6 +64,17 @@ public:
   {
     return a * b;
   }
+
+  virtual int baz(int a, int b)
+  {
+    if (flags["operation"] == "sum") {
+      return a + b;
+    }
+    return -1;
+  }
+
+private:
+  hashmap<std::string, std::string> flags;
 };
 
 
@@ -45,9 +84,17 @@ static bool compatible()
 }
 
 
-static TestModule* create()
+static TestModule* create(const mesos::Parameters& parameters)
 {
-  return new TestModuleImpl();
+  TestModule *testModule = new TestModuleImpl();
+  Try<Nothing> result = testModule->initialize(parameters);
+  if (result.isError()) {
+    delete testModule;
+    // TODO(karya): make the return type Try<TestModule*> to pass the
+    // error message as well.
+    return NULL;
+  }
+  return testModule;
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/examples/test_isolator_module.cpp
----------------------------------------------------------------------
diff --git a/src/examples/test_isolator_module.cpp b/src/examples/test_isolator_module.cpp
index 2d6b427..dc107a1 100644
--- a/src/examples/test_isolator_module.cpp
+++ b/src/examples/test_isolator_module.cpp
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 
+#include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
 #include <stout/try.hpp>
@@ -26,6 +27,8 @@
 #include "slave/containerizer/isolators/posix.hpp"
 #include "slave/flags.hpp"
 
+using namespace mesos;
+
 using mesos::internal::slave::Flags;
 using mesos::internal::slave::Isolator;
 using mesos::internal::slave::PosixCpuIsolatorProcess;
@@ -40,7 +43,7 @@ static bool compatible()
 }
 
 
-static Isolator* createCpuIsolator()
+static Isolator* createCpuIsolator(const Parameters& parameters)
 {
   Flags flags;
   Try<Isolator*> result = PosixCpuIsolatorProcess::create(flags);
@@ -51,7 +54,7 @@ static Isolator* createCpuIsolator()
 }
 
 
-static Isolator* createMemIsolator()
+static Isolator* createMemIsolator(const Parameters& parameters)
 {
   Flags flags;
   Try<Isolator*> result = PosixMemIsolatorProcess::create(flags);

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/examples/test_module.hpp
----------------------------------------------------------------------
diff --git a/src/examples/test_module.hpp b/src/examples/test_module.hpp
index 820df23..041570e 100644
--- a/src/examples/test_module.hpp
+++ b/src/examples/test_module.hpp
@@ -19,8 +19,12 @@
 #ifndef __TEST_MODULE_HPP__
 #define __TEST_MODULE_HPP__
 
+#include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
+#include <stout/nothing.hpp>
+#include <stout/try.hpp>
+
 // Each module "kind" has a base class associated with it that is
 // inherited by the module instances.  Mesos core uses the base
 // class interface to bind with the module instances.
@@ -36,9 +40,13 @@ public:
   // cleanup that may be required for the derived object.
   virtual ~TestModule() {}
 
+  virtual Try<Nothing> initialize(const mesos::Parameters& parameters) = 0;
+
   virtual int foo(char a, long b) = 0;
 
   virtual int bar(float a, double b) = 0;
+
+  virtual int baz(int a, int b) = 0;
 };
 
 
@@ -62,7 +70,7 @@ struct Module<TestModule> : ModuleBase
       const char* _authorEmail,
       const char* _description,
       bool (*_compatible)(),
-      TestModule* (*_create)())
+      TestModule* (*_create)(const Parameters& parameters))
     : ModuleBase(
         _moduleApiVersion,
         _mesosVersion,
@@ -74,7 +82,7 @@ struct Module<TestModule> : ModuleBase
       create(_create)
   { }
 
-  TestModule* (*create)();
+  TestModule* (*create)(const Parameters& parameters);
 };
 
 } // namespace modules {

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/master/flags.hpp
----------------------------------------------------------------------
diff --git a/src/master/flags.hpp b/src/master/flags.hpp
index 9d06856..c931fd9 100644
--- a/src/master/flags.hpp
+++ b/src/master/flags.hpp
@@ -301,6 +301,11 @@ public:
         "This helps fairness when running frameworks that hold on to offers,\n"
         "or frameworks that accidentally drop offers.\n");
 
+    // This help message for --modules flag is the same for
+    // {master,slave,tests}/flags.hpp and should always be kept in
+    // sync.
+    // TODO(karya): Remove the JSON example and add reference to the
+    // doc file explaining the --modules flag.
     add(&Flags::modules,
         "modules",
         "List of modules to be loaded and be available to the internal\n"
@@ -318,14 +323,26 @@ public:
         "    {\n"
         "      \"file\": \"/path/to/libfoo.so\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_bar\",\n"
-        "        \"org_apache_mesos_baz\"\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_bar\",\n"
+        "          \"flags\": [\n"
+        "            {\n"
+        "              \"key\": \"X\",\n"
+        "              \"value\": \"Y\",\n"
+        "            }\n"
+        "          ]\n"
+        "        },\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_baz\"\n"
+        "        }\n"
         "      ]\n"
         "    },\n"
         "    {\n"
         "      \"name\": \"qux\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_norf\",\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_norf\"\n"
+        "        }\n"
         "      ]\n"
         "    }\n"
         "  ]\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/messages/messages.proto
----------------------------------------------------------------------
diff --git a/src/messages/messages.proto b/src/messages/messages.proto
index 6e49fe7..76e3980 100644
--- a/src/messages/messages.proto
+++ b/src/messages/messages.proto
@@ -437,7 +437,15 @@ message Modules {
     // Linux and ".dylib" for OS X) to the "name".
     optional string name = 2;
 
-    repeated string modules = 3;
+    message Module {
+      // Module name.
+      optional string name = 1;
+
+      // Module specific flags.
+      optional Parameters parameters = 2;
+    }
+
+    repeated Module modules = 3;
   }
 
   repeated Library libraries = 1;

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/module/isolator.hpp
----------------------------------------------------------------------
diff --git a/src/module/isolator.hpp b/src/module/isolator.hpp
index fc78b07..0f71e70 100644
--- a/src/module/isolator.hpp
+++ b/src/module/isolator.hpp
@@ -19,6 +19,7 @@
 #ifndef __MODULE_ISOLATOR_HPP__
 #define __MODULE_ISOLATOR_HPP__
 
+#include <mesos/mesos.hpp>
 #include <mesos/module.hpp>
 
 #include "slave/containerizer/isolator.hpp"
@@ -43,7 +44,8 @@ struct Module<mesos::internal::slave::Isolator> : ModuleBase
       const char* _authorEmail,
       const char* _description,
       bool (*_compatible)(),
-      mesos::internal::slave::Isolator* (*_create)())
+      mesos::internal::slave::Isolator*
+        (*_create)(const Parameters& parameters))
     : ModuleBase(
         _moduleApiVersion,
         _mesosVersion,
@@ -55,7 +57,7 @@ struct Module<mesos::internal::slave::Isolator> : ModuleBase
       create(_create)
   { }
 
-  mesos::internal::slave::Isolator* (*create)();
+  mesos::internal::slave::Isolator* (*create)(const Parameters& parameters);
 };
 
 } // namespace modules {

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/module/manager.cpp
----------------------------------------------------------------------
diff --git a/src/module/manager.cpp b/src/module/manager.cpp
index 1613e39..7a6c884 100644
--- a/src/module/manager.cpp
+++ b/src/module/manager.cpp
@@ -45,6 +45,7 @@ pthread_mutex_t ModuleManager::mutex = PTHREAD_MUTEX_INITIALIZER;
 hashmap<const string, string> ModuleManager::kindToVersion;
 hashmap<const string, ModuleBase*> ModuleManager::moduleBases;
 hashmap<const string, Owned<DynamicLibrary>> ModuleManager::dynamicLibraries;
+hashmap<const std::string, Parameters> ModuleManager::moduleParameters;
 
 
 void ModuleManager::initialize()
@@ -200,28 +201,36 @@ Try<Nothing> ModuleManager::load(const Modules& modules)
     }
 
     // Load module manifests.
-    foreach (const string& moduleName, library.modules()) {
-      if (moduleName.empty()) {
+    foreach (const Modules::Library::Module& module, library.modules()) {
+      if (!module.has_name()) {
         return Error(
-            "Error: module name not provided with library '" +
-            library.file() + "'");
+            "Error: module name not provided with library '" + libraryName +
+            "'");
       }
+
       // Check for possible duplicate module names.
+      const std::string moduleName = module.name();
       if (moduleBases.contains(moduleName)) {
         return Error("Error loading duplicate module '" + moduleName + "'");
       }
+
+      // Load ModuleBase.
       Try<void*> symbol = dynamicLibraries[libraryName]->loadSymbol(moduleName);
       if (symbol.isError()) {
         return Error(
             "Error loading module '" + moduleName + "': " + symbol.error());
       }
       ModuleBase* moduleBase = (ModuleBase*) symbol.get();
+
+      // Verify module compatibility including version, etc.
       Try<Nothing> result = verifyModule(moduleName, moduleBase);
       if (result.isError()) {
         return Error(
             "Error verifying module '" + moduleName + "': " + result.error());
       }
+
       moduleBases[moduleName] = (ModuleBase*) symbol.get();
+      moduleParameters[moduleName] = module.parameters();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/module/manager.hpp
----------------------------------------------------------------------
diff --git a/src/module/manager.hpp b/src/module/manager.hpp
index dc78921..04c0b18 100644
--- a/src/module/manager.hpp
+++ b/src/module/manager.hpp
@@ -90,7 +90,7 @@ public:
           "kind is '" + expectedKind + "'");
     }
 
-    T* instance = module->create();
+    T* instance = module->create(moduleParameters[moduleName]);
     if (instance == NULL) {
       return Error("Error creating Module instance for '" + moduleName + "'");
     }
@@ -127,6 +127,9 @@ private:
   // one specified in the protobuf Modules will be picked.
   static hashmap<const std::string, ModuleBase*> moduleBases;
 
+  // Module-specific command-line parameters.
+  static hashmap<const std::string, Parameters> moduleParameters;
+
   // A list of dynamic libraries to keep the object from getting
   // destructed. Destroying the DynamicLibrary object could result in
   // unloading the library from the process memory.

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/slave/flags.hpp
----------------------------------------------------------------------
diff --git a/src/slave/flags.hpp b/src/slave/flags.hpp
index 621d0de..f7a8cde 100644
--- a/src/slave/flags.hpp
+++ b/src/slave/flags.hpp
@@ -350,6 +350,11 @@ public:
         "This flag uses the Bytes type, defined in stout.");
 #endif // WITH_NETWORK_ISOLATOR
 
+    // This help message for --modules flag is the same for
+    // {master,slave,tests}/flags.hpp and should always be kept in
+    // sync.
+    // TODO(karya): Remove the JSON example and add reference to the
+    // doc file explaining the --modules flag.
     add(&Flags::modules,
         "modules",
         "List of modules to be loaded and be available to the internal\n"
@@ -367,14 +372,26 @@ public:
         "    {\n"
         "      \"file\": \"/path/to/libfoo.so\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_bar\",\n"
-        "        \"org_apache_mesos_baz\"\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_bar\",\n"
+        "          \"flags\": [\n"
+        "            {\n"
+        "              \"key\": \"X\",\n"
+        "              \"value\": \"Y\",\n"
+        "            }\n"
+        "          ]\n"
+        "        },\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_baz\"\n"
+        "        }\n"
         "      ]\n"
         "    },\n"
         "    {\n"
         "      \"name\": \"qux\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_norf\",\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_norf\"\n"
+        "        }\n"
         "      ]\n"
         "    }\n"
         "  ]\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/tests/flags.hpp
----------------------------------------------------------------------
diff --git a/src/tests/flags.hpp b/src/tests/flags.hpp
index bf3569f..2886e89 100644
--- a/src/tests/flags.hpp
+++ b/src/tests/flags.hpp
@@ -75,6 +75,11 @@ public:
         "Where to find docker executable",
         "docker");
 
+    // This help message for --modules flag is the same for
+    // {master,slave,tests}/flags.hpp and should always be kept in
+    // sync.
+    // TODO(karya): Remove the JSON example and add reference to the
+    // doc file explaining the --modules flag.
     add(&Flags::modules,
         "modules",
         "List of modules to be loaded and be available to the internal\n"
@@ -92,14 +97,26 @@ public:
         "    {\n"
         "      \"file\": \"/path/to/libfoo.so\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_bar\",\n"
-        "        \"org_apache_mesos_baz\"\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_bar\",\n"
+        "          \"flags\": [\n"
+        "            {\n"
+        "              \"key\": \"X\",\n"
+        "              \"value\": \"Y\",\n"
+        "            }\n"
+        "          ]\n"
+        "        },\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_baz\"\n"
+        "        }\n"
         "      ]\n"
         "    },\n"
         "    {\n"
         "      \"name\": \"qux\",\n"
         "      \"modules\": [\n"
-        "        \"org_apache_mesos_norf\",\n"
+        "        {\n"
+        "          \"name\": \"org_apache_mesos_norf\"\n"
+        "        }\n"
         "      ]\n"
         "    }\n"
         "  ]\n"

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/tests/module.cpp
----------------------------------------------------------------------
diff --git a/src/tests/module.cpp b/src/tests/module.cpp
index 45becad..482ed22 100644
--- a/src/tests/module.cpp
+++ b/src/tests/module.cpp
@@ -44,7 +44,8 @@ static void addModule(
     string moduleName)
 {
   moduleNames[moduleId] = moduleName;
-  library->add_modules(moduleNames[moduleId]);
+  Modules::Library::Module* module = library->add_modules();
+  module->set_name(moduleNames[moduleId]);
 }
 
 

http://git-wip-us.apache.org/repos/asf/mesos/blob/17ecc0c0/src/tests/module_tests.cpp
----------------------------------------------------------------------
diff --git a/src/tests/module_tests.cpp b/src/tests/module_tests.cpp
index 45125d8..e079dbe 100644
--- a/src/tests/module_tests.cpp
+++ b/src/tests/module_tests.cpp
@@ -89,7 +89,8 @@ protected:
     library->set_file(path::join(
         libraryDirectory,
         os::libraries::expandName(DEFAULT_MODULE_LIBRARY_NAME)));
-    library->add_modules(DEFAULT_MODULE_NAME);
+    Modules::Library::Module* module = library->add_modules();
+    module->set_name(DEFAULT_MODULE_NAME);
   }
 
   // During the per-test tear-down, we unload the module to allow
@@ -128,12 +129,26 @@ string ModuleTest::originalLdLibraryPath;
 string ModuleTest::libraryDirectory;
 
 
-static Modules getModules(const string& libraryName, const string& moduleName)
+static Modules getModules(
+    const string& libraryName,
+    const string& moduleName,
+    const string& parameterKey = "",
+    const string& parameterValue = "")
 {
   Modules modules;
   Modules::Library* library = modules.add_libraries();
   library->set_file(os::libraries::expandName(libraryName));
-  library->add_modules(moduleName);
+  Modules::Library::Module* module = library->add_modules();
+  module->set_name(moduleName);
+  if (!parameterKey.empty() || !parameterValue.empty()) {
+    Parameter* parameter = module->mutable_parameters()->add_parameter();
+    if (!parameterKey.empty()) {
+      parameter->set_key(parameterKey);
+    }
+    if (!parameterValue.empty()) {
+      parameter->set_value(parameterValue);
+    }
+  }
   return modules;
 }
 
@@ -151,9 +166,82 @@ TEST_F(ModuleTest, ExampleModuleLoadTest)
 
   // The TestModuleImpl module's implementation of foo() returns
   // the sum of the passed arguments, whereas bar() returns the
-  // product.
+  // product. baz() returns '-1' if "sum" is not specified as the
+  // "operation" in the command line parameters.
   EXPECT_EQ(module.get()->foo('A', 1024), 1089);
   EXPECT_EQ(module.get()->bar(0.5, 10.8), 5);
+  EXPECT_EQ(module.get()->baz(5, 10), -1);
+}
+
+
+// Test passing parameter without value.
+TEST_F(ModuleTest, ParameterWithoutValue)
+{
+  Modules modules =
+    getModules(DEFAULT_MODULE_LIBRARY_NAME, DEFAULT_MODULE_NAME, "operation");
+
+  EXPECT_SOME(ModuleManager::load(modules));
+  module = ModuleManager::create<TestModule>(DEFAULT_MODULE_NAME);
+  EXPECT_ERROR(module);
+}
+
+
+// Test passing parameter with invalid value.
+TEST_F(ModuleTest, ParameterWithInvalidValue)
+{
+  Modules modules = getModules(
+      DEFAULT_MODULE_LIBRARY_NAME,
+      DEFAULT_MODULE_NAME,
+      "operation",
+      "X");
+
+  EXPECT_SOME(ModuleManager::load(modules));
+  module = ModuleManager::create<TestModule>(DEFAULT_MODULE_NAME);
+  EXPECT_ERROR(module);
+}
+
+
+// Test passing parameter without key.
+TEST_F(ModuleTest, ParameterWithoutKey)
+{
+  Modules modules =
+    getModules(DEFAULT_MODULE_LIBRARY_NAME, DEFAULT_MODULE_NAME, "", "sum");
+
+  EXPECT_SOME(ModuleManager::load(modules));
+  module = ModuleManager::create<TestModule>(DEFAULT_MODULE_NAME);
+  EXPECT_ERROR(module);
+}
+
+
+// Test passing parameter with invalid key.
+TEST_F(ModuleTest, ParameterWithInvalidKey)
+{
+  Modules modules =
+    getModules(DEFAULT_MODULE_LIBRARY_NAME, DEFAULT_MODULE_NAME, "X", "sum");
+
+  EXPECT_SOME(ModuleManager::load(modules));
+  module = ModuleManager::create<TestModule>(DEFAULT_MODULE_NAME);
+  EXPECT_SOME(module);
+
+  // Since there was no valid key, baz() should return -1.
+  EXPECT_EQ(module.get()->baz(5, 10), -1);
+}
+
+
+// Test passing parameter with valid key and value.
+TEST_F(ModuleTest, ValidParameters)
+{
+  Modules modules = getModules(
+      DEFAULT_MODULE_LIBRARY_NAME,
+      DEFAULT_MODULE_NAME,
+      "operation",
+      "sum");
+
+  EXPECT_SOME(ModuleManager::load(modules));
+  module = ModuleManager::create<TestModule>(DEFAULT_MODULE_NAME);
+  EXPECT_SOME(module);
+
+  EXPECT_EQ(module.get()->baz(5, 10), 15);
 }
 
 
@@ -215,7 +303,8 @@ TEST_F(ModuleTest, LibraryNameWithoutExtension)
   Modules modules;
   Modules::Library* library = modules.add_libraries();
   library->set_name(DEFAULT_MODULE_LIBRARY_NAME);
-  library->add_modules(DEFAULT_MODULE_NAME);
+  Modules::Library::Module* module = library->add_modules();
+  module->set_name(DEFAULT_MODULE_NAME);
 
   EXPECT_SOME(ModuleManager::load(modules));
 }
@@ -228,7 +317,8 @@ TEST_F(ModuleTest, LibraryNameWithExtension)
   Modules modules;
   Modules::Library* library = modules.add_libraries();
   library->set_file(os::libraries::expandName(DEFAULT_MODULE_LIBRARY_NAME));
-  library->add_modules(DEFAULT_MODULE_NAME);
+  Modules::Library::Module* module = library->add_modules();
+  module->set_name(DEFAULT_MODULE_NAME);
 
   EXPECT_SOME(ModuleManager::load(modules));
 }
@@ -292,7 +382,8 @@ TEST_F(ModuleTest, DuplicateModule)
   // Add duplicate module.
   Modules::Library* library = defaultModules.add_libraries();
   library->set_name(DEFAULT_MODULE_LIBRARY_NAME);
-  library->add_modules(DEFAULT_MODULE_NAME);
+  Modules::Library::Module* module = library->add_modules();
+  module->set_name(DEFAULT_MODULE_NAME);
 
   EXPECT_ERROR(ModuleManager::load(defaultModules));
 }