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/05/01 00:23:43 UTC

svn commit: r1332449 - in /incubator/mesos/trunk: src/common/ src/configurator/ src/exec/ src/local/ src/log/ src/master/ src/mesos/ src/sched/ src/slave/ src/tests/ third_party/libprocess/include/process/ third_party/libprocess/src/

Author: benh
Date: Mon Apr 30 22:23:42 2012
New Revision: 1332449

URL: http://svn.apache.org/viewvc?rev=1332449&view=rev
Log:
Scratched some itches and cleaned up some aspects of logging and
configuration:

(1) Removed initializing glog from within libprocess (that should
    never have been done, if a user wants that they should just set
    GLOG_* environment variables before initializing libprocess).

(2) Make 'localquiet' actually be quiet again.

(3) Added a 'once' abstraction to libprocess (needed to only
    initialize logging once).

(4) Made mesos-tests actually use logging and configuration.

(5) Removed unused Configuration methods (and cleaned up existing APIs
    as well as how they were getting called/used).

Added:
    incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp
    incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp
Modified:
    incubator/mesos/trunk/src/common/logging.cpp
    incubator/mesos/trunk/src/common/logging.hpp
    incubator/mesos/trunk/src/common/webui_utils.cpp
    incubator/mesos/trunk/src/configurator/configuration.hpp
    incubator/mesos/trunk/src/exec/exec.cpp
    incubator/mesos/trunk/src/local/local.cpp
    incubator/mesos/trunk/src/local/main.cpp
    incubator/mesos/trunk/src/log/log.cpp
    incubator/mesos/trunk/src/log/main.cpp
    incubator/mesos/trunk/src/master/main.cpp
    incubator/mesos/trunk/src/master/webui.cpp
    incubator/mesos/trunk/src/mesos/main.cpp
    incubator/mesos/trunk/src/sched/sched.cpp
    incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
    incubator/mesos/trunk/src/slave/main.cpp
    incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
    incubator/mesos/trunk/src/slave/slave.cpp
    incubator/mesos/trunk/src/slave/slave.hpp
    incubator/mesos/trunk/src/slave/webui.cpp
    incubator/mesos/trunk/src/tests/main.cpp
    incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
    incubator/mesos/trunk/third_party/libprocess/src/process.cpp

Modified: incubator/mesos/trunk/src/common/logging.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/logging.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/logging.cpp (original)
+++ incubator/mesos/trunk/src/common/logging.cpp Mon Apr 30 22:23:42 2012
@@ -16,73 +16,79 @@
  * limitations under the License.
  */
 
-#include <sys/stat.h>
-
 #include <glog/logging.h>
 
-#include "fatal.hpp"
-#include "logging.hpp"
+#include <process/once.hpp>
 
-using std::string;
+#include "common/logging.hpp"
+#include "common/utils.hpp"
 
-// TODO(benh): Provide a mechanism to initialize the logging only
-// once, possibly using something like pthread_once. In particular, we
-// need to make sure we handle the case that another library is used
-// with Mesos that also uses glog.
-//
-//   static pthread_once_t glog_initialized = PTHREAD_ONCE_INIT;
-//
-//   pthread_once(&glog_initialized, initialize_glog);
+using std::string;
 
 namespace mesos {
 namespace internal {
+namespace logging {
 
-void Logging::registerOptions(Configurator* conf)
+void registerOptions(Configurator* configurator)
 {
-  conf->addOption<bool>("quiet", 'q', "Disable logging to stderr", false);
-  conf->addOption<string>("log_dir",
-                          "Where to put logs (default: glog default)");
-  conf->addOption<int>("log_buf_secs",
-                       "How many seconds to buffer log messages for\n",
-                       0);
+  configurator->addOption<bool>(
+      "quiet",
+      'q',
+      "Disable logging to stderr (default: false)",
+      false);
+
+  configurator->addOption<string>(
+      "log_dir",
+      "Location to put log files (no default, nothing"
+      " is written to disk unless specified; "
+      " does not affect logging to stderr)");
+
+  configurator->addOption<int>(
+      "logbufsecs",
+      "How many seconds to buffer log messages for (default: 0)",
+      0);
 }
 
 
-void Logging::init(const char* programName, const Configuration& conf)
+void initialize(const string& argv0, const Configuration& conf)
 {
-  // Set glog's parameters through Google Flags variables
-  string logDir = getLogDir(conf);
-  if (logDir == "") {
-    FLAGS_logtostderr = true;
-  } else {
-    if (mkdir(logDir.c_str(), 0755) < 0 && errno != EEXIST) {
-      fatalerror("Failed to create log directory %s", logDir.c_str());
+  static process::Once initialized;
+
+  if (initialized.once()) {
+    return;
+  }
+
+  Option<string> directory = conf.get<string>("log_dir");
+
+  // Set glog's parameters through Google Flags variables.
+  if (directory.isSome()) {
+    if (!utils::os::mkdir(directory.get())) {
+      std::cerr << "Could not initialize logging: Failed to create directory "
+                << directory.get() << std::endl;
+      exit(1);
     }
-    FLAGS_log_dir = logDir;
+    FLAGS_log_dir = directory.get();
   }
 
-  FLAGS_logbufsecs = conf.get<int>("log_buf_secs", 0);
 
-  google::InitGoogleLogging(programName);
+  // Log everything to stderr IN ADDITION to log files unless
+  // otherwise specified.
+  bool quiet = conf.get<bool>("quiet", false);
 
-  if (!isQuiet(conf)) {
-    google::SetStderrLogging(google::INFO);
+  if (!quiet) {
+    FLAGS_stderrthreshold = 0; // INFO.
   }
 
-  LOG(INFO) << "Logging to " << (FLAGS_logtostderr ? "<stderr>" : FLAGS_log_dir);
-}
-
+  FLAGS_logbufsecs = conf.get<int>("logbufsecs", 0);
 
-string Logging::getLogDir(const Configuration& conf)
-{
-  return conf.get("log_dir", "");
-}
+  google::InitGoogleLogging(argv0.c_str());
 
+  LOG(INFO) << "Logging to " <<
+    (directory.isSome() ? directory.get() : "STDERR");
 
-bool Logging::isQuiet(const Configuration& conf)
-{
-  return conf.get<bool>("quiet", false);
+  initialized.done();
 }
 
+} // namespace logging {
 } // namespace internal {
 } // namespace mesos {

Modified: incubator/mesos/trunk/src/common/logging.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/logging.hpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/logging.hpp (original)
+++ incubator/mesos/trunk/src/common/logging.hpp Mon Apr 30 22:23:42 2012
@@ -21,24 +21,20 @@
 
 #include <string>
 
-#include "configurator/configurator.hpp"
+#include <glog/logging.h> // Includes LOG(*), PLOG(*), CHECK, etc.
 
+#include "configurator/configuration.hpp"
+#include "configurator/configurator.hpp"
 
 namespace mesos {
 namespace internal {
+namespace logging {
 
-/**
- * Utility functions for configuring and initializing Mesos logging.
- */
-class Logging {
-public:
-  static void registerOptions(Configurator* conf);
-  static void init(const char* programName, const Configuration& conf);
-  static std::string getLogDir(const Configuration& conf);
-  static bool isQuiet(const Configuration& conf);
-};
+void registerOptions(Configurator* configurator);
+void initialize(const std::string& argv0, const Configuration& conf);
 
+} // namespace logging {
 } // namespace internal {
 } // namespace mesos {
 
-#endif
+#endif // __LOGGING_HPP__

Modified: incubator/mesos/trunk/src/common/webui_utils.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/common/webui_utils.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/common/webui_utils.cpp (original)
+++ incubator/mesos/trunk/src/common/webui_utils.cpp Mon Apr 30 22:23:42 2012
@@ -62,7 +62,7 @@ void start(const Configuration& conf,
   // is necessary for running out of the build directory before 'make
   // install') or the directory determined at build time via the
   // preprocessor macro '-DMESOS_WEBUI_DIR' set in the Makefile.
-  std::string directory = conf.get("webui_dir", MESOS_WEBUI_DIR);
+  std::string directory = conf.get<std::string>("webui_dir", MESOS_WEBUI_DIR);
 
   // Remove any trailing '/' in directory.
   directory = strings::remove(directory, "/", strings::SUFFIX);

Modified: incubator/mesos/trunk/src/configurator/configuration.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/configurator/configuration.hpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/configurator/configuration.hpp (original)
+++ incubator/mesos/trunk/src/configurator/configuration.hpp Mon Apr 30 22:23:42 2012
@@ -97,44 +97,26 @@ public:
     return params[key];
   }
 
-  Option<std::string> get(const std::string& key) const
-  {
-    if (!contains(key)) {
-      return Option<std::string>::none();
-    }
-    return get(key, "");
-  }
-
-  const std::string& get(const std::string& key,
-                         const std::string& defaultValue) const
+  template <typename T>
+  Option<T> get(const std::string& key) const
   {
     std::map<std::string, std::string>::const_iterator it = params.find(key);
-    return (it != params.end()) ? it->second : defaultValue;
-  }
-
-  int getInt(const std::string& key, int defaultValue) const
-  {
-    return get<int>(key, defaultValue);
-  }
-
-  int32_t getInt32(const std::string& key, int32_t defaultValue) const
-  {
-    return get<int32_t>(key, defaultValue);
-  }
+    if (it != params.end()) {
+      return boost::lexical_cast<T>(it->second);
+    }
 
-  int64_t getInt64(const std::string& key, int64_t defaultValue) const
-  {
-    return get<int64_t>(key, defaultValue);
+    return Option<T>::none();
   }
 
   template <typename T>
   T get(const std::string& key, const T& defaultValue) const
   {
     std::map<std::string, std::string>::const_iterator it = params.find(key);
-    if (it != params.end())
+    if (it != params.end()) {
       return boost::lexical_cast<T>(it->second);
-    else
-      return defaultValue;
+    }
+
+    return defaultValue;
   }
 
   template <typename T>

Modified: incubator/mesos/trunk/src/exec/exec.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/exec/exec.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/exec/exec.cpp (original)
+++ incubator/mesos/trunk/src/exec/exec.cpp Mon Apr 30 22:23:42 2012
@@ -293,10 +293,10 @@ MesosExecutorDriver::MesosExecutorDriver
   pthread_mutexattr_destroy(&attr);
   pthread_cond_init(&cond, 0);
 
-  // TODO(benh): Initialize glog.
+  // Initialize libprocess.
+  process::initialize();
 
-  // Initialize libprocess library (but not glog, done above).
-  process::initialize(false);
+  // TODO(benh): Initialize glog.
 }
 
 

Modified: incubator/mesos/trunk/src/local/local.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/local.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/local.cpp (original)
+++ incubator/mesos/trunk/src/local/local.cpp Mon Apr 30 22:23:42 2012
@@ -16,8 +16,6 @@
  * limitations under the License.
  */
 
-#include <pthread.h>
-
 #include <map>
 #include <sstream>
 #include <vector>
@@ -26,7 +24,6 @@
 
 #include "common/fatal.hpp"
 #include "common/foreach.hpp"
-#include "common/logging.hpp"
 
 #include "configurator/configurator.hpp"
 
@@ -67,12 +64,13 @@ static MasterDetector* detector = NULL;
 
 void registerOptions(Configurator* configurator)
 {
-  Logging::registerOptions(configurator);
   Master::registerOptions(configurator);
   Slave::registerOptions(configurator);
-  configurator->addOption<int>("num_slaves",
-                               "Number of slaves to create for local cluster",
-                               1);
+
+  configurator->addOption<int>(
+      "num_slaves",
+      "Number of slaves to create for local cluster",
+      1);
 }
 
 

Modified: incubator/mesos/trunk/src/local/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/local/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/local/main.cpp (original)
+++ incubator/mesos/trunk/src/local/main.cpp Mon Apr 30 22:23:42 2012
@@ -19,13 +19,14 @@
 #include <iostream>
 #include <string>
 
-#include "local.hpp"
+#include "common/logging.hpp"
+#include "common/utils.hpp"
 
 #include "configurator/configurator.hpp"
 
 #include "detector/detector.hpp"
 
-#include "common/logging.hpp"
+#include "local/local.hpp"
 
 #include "master/master.hpp"
 
@@ -41,14 +42,11 @@ using std::endl;
 using std::string;
 
 
-void usage(const char* programName, const Configurator& configurator)
+void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << programName
-       << " [--port=PORT] [--slaves=N] [--cpus=CPUS] [--mem=MEM] [...]" << endl
-       << endl
-       << "Launches a single-process cluster containing N slaves, each of "
-       << "which report" << endl << "CPUS cores and MEM bytes of memory."
+  cerr << "Usage: " << utils::os::basename(argv0) << " [...]" << endl
        << endl
+       << "Launches a cluster within a single OS process."
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();
@@ -60,7 +58,11 @@ int main(int argc, char **argv)
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
   Configurator configurator;
+
+  logging::registerOptions(&configurator);
+
   local::registerOptions(&configurator);
+
   configurator.addOption<int>("port", 'p', "Port to listen on", 5050);
   configurator.addOption<string>("ip", "IP address to listen on");
 
@@ -77,18 +79,18 @@ int main(int argc, char **argv)
     exit(1);
   }
 
-  Logging::init(argv[0], conf);
-
   if (conf.contains("port")) {
-    setenv("LIBPROCESS_PORT", conf["port"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_PORT", conf["port"]);
   }
 
   if (conf.contains("ip")) {
-    setenv("LIBPROCESS_IP", conf["ip"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_IP", conf["ip"]);
   }
 
-  // Initialize libprocess library (but not glog, done above).
-  process::initialize(false);
+  // Initialize libprocess.
+  process::initialize();
+
+  logging::initialize(argv[0], conf);
 
   process::wait(local::launch(conf, false));
 

Modified: incubator/mesos/trunk/src/log/log.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/log.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/log.cpp (original)
+++ incubator/mesos/trunk/src/log/log.cpp Mon Apr 30 22:23:42 2012
@@ -639,7 +639,7 @@ int main(int argc, char** argv)
     truncations[at] = to;
   }
 
-  process::initialize(true);
+  process::initialize();
 
   LogProcess log(quorum, file, servers, znode, end, truncations);
   spawn(log);

Modified: incubator/mesos/trunk/src/log/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/log/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/log/main.cpp (original)
+++ incubator/mesos/trunk/src/log/main.cpp Mon Apr 30 22:23:42 2012
@@ -45,7 +45,6 @@ void usage(const char* argv0, const Conf
 {
   cerr << "Usage: " << utils::os::basename(argv0) << " [...] path/to/log"
        << endl
-       << endl
        << "Supported options:" << endl
        << configurator.getUsage();
 }
@@ -54,7 +53,8 @@ void usage(const char* argv0, const Conf
 int main(int argc, char** argv)
 {
   Configurator configurator;
-  Logging::registerOptions(&configurator);
+
+  logging::registerOptions(&configurator);
 
   configurator.addOption<uint64_t>(
       "from",
@@ -83,11 +83,11 @@ int main(int argc, char** argv)
     exit(1);
   }
 
-  Logging::init(argv[0], conf);
+  process::initialize();
 
-  string path = argv[argc - 1];
+  logging::initialize(argv[0], conf);
 
-  process::initialize(false);
+  string path = argv[argc - 1];
 
   Replica replica(path);
 

Modified: incubator/mesos/trunk/src/master/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/main.cpp (original)
+++ incubator/mesos/trunk/src/master/main.cpp Mon Apr 30 22:23:42 2012
@@ -16,12 +16,10 @@
  * limitations under the License.
  */
 
-#include <libgen.h> // For chdir.
-
 #include "common/build.hpp"
-#include "common/fatal.hpp"
 #include "common/logging.hpp"
 #include "common/try.hpp"
+#include "common/utils.hpp"
 
 #include "configurator/configurator.hpp"
 
@@ -40,10 +38,9 @@ using std::endl;
 using std::string;
 
 
-void usage(const char* programName, const Configurator& configurator)
+void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << programName << " [...]" << endl
-       << endl
+  cerr << "Usage: " << utils::os::basename(argv0) << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();
@@ -55,7 +52,9 @@ int main(int argc, char **argv)
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
   Configurator configurator;
-  Logging::registerOptions(&configurator);
+
+  logging::registerOptions(&configurator);
+
   Master::registerOptions(&configurator);
 
   // The following options are executable specific (e.g., since we
@@ -87,35 +86,33 @@ int main(int argc, char **argv)
     exit(1);
   }
 
-  Logging::init(argv[0], conf);
-
   if (conf.contains("port")) {
-    setenv("LIBPROCESS_PORT", conf["port"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_PORT", conf["port"]);
   }
 
   if (conf.contains("ip")) {
-    setenv("LIBPROCESS_IP", conf["ip"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_IP", conf["ip"]);
   }
 
-  // Initialize libprocess library (but not glog, done above).
-  process::initialize(false);
+  // Initialize libprocess.
+  process::initialize();
 
-  string zk = conf.get("zk", "");
+  logging::initialize(argv[0], conf);
+
+  string zk = conf.get<std::string>("zk", "");
 
   LOG(INFO) << "Build: " << build::DATE << " by " << build::USER;
   LOG(INFO) << "Starting Mesos master";
 
-  if (chdir(dirname(argv[0])) != 0) {
-    fatalerror("Could not chdir into %s", dirname(argv[0]));
-  }
-
   Allocator* allocator = new SimpleAllocator();
 
   Master* master = new Master(allocator, conf);
   process::spawn(master);
 
+  bool quiet = conf.get<bool>("quiet", false);
+
   Try<MasterDetector*> detector =
-    MasterDetector::create(zk, master->self(), true, Logging::isQuiet(conf));
+    MasterDetector::create(zk, master->self(), true, quiet);
 
   CHECK(detector.isSome())
     << "Failed to create a master detector: " << detector.error();

Modified: incubator/mesos/trunk/src/master/webui.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/master/webui.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/master/webui.cpp (original)
+++ incubator/mesos/trunk/src/master/webui.cpp Mon Apr 30 22:23:42 2012
@@ -34,8 +34,8 @@ void start(const process::PID<Master>& m
 {
   std::vector<std::string> args(3);
   args[0] = "--master_port=" + utils::stringify(master.port);
-  args[1] = "--webui_port=" + conf.get("webui_port", "8080");
-  args[2] = "--log_dir=" + conf.get("log_dir", FLAGS_log_dir);
+  args[1] = "--webui_port=" + conf.get<std::string>("webui_port", "8080");
+  args[2] = "--log_dir=" + conf.get<std::string>("log_dir", FLAGS_log_dir);
 
   utils::webui::start(conf, "master/webui.py", args);
 }

Modified: incubator/mesos/trunk/src/mesos/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/mesos/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/mesos/main.cpp (original)
+++ incubator/mesos/trunk/src/mesos/main.cpp Mon Apr 30 22:23:42 2012
@@ -1,6 +1,8 @@
 #include <process/process.hpp>
 #include <process/protobuf.hpp>
 
+#include "common/utils.hpp"
+
 #include "configurator/configurator.hpp"
 
 #include "messages/messages.hpp"
@@ -15,10 +17,9 @@ using std::endl;
 using std::string;
 
 
-void usage(const char* programName, const Configurator& configurator)
+void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << programName << " [...]" << endl
-       << endl
+  cerr << "Usage: " << utils::os::basename(argv0) << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();
@@ -61,7 +62,7 @@ int main(int argc, char** argv)
     exit(1);
   }
 
-  // Initialize libprocess library
+  // Initialize libprocess.
   process::initialize();
 
   if (!conf.contains("master")) {

Modified: incubator/mesos/trunk/src/sched/sched.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/sched/sched.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/sched/sched.cpp (original)
+++ incubator/mesos/trunk/src/sched/sched.cpp Mon Apr 30 22:23:42 2012
@@ -613,6 +613,8 @@ MesosSchedulerDriver::MesosSchedulerDriv
   // Load the configuration.
   Configurator configurator;
 
+  logging::registerOptions(&configurator);
+
   if (master == "local" || master == "localquiet") {
     local::registerOptions(&configurator);
   }
@@ -626,6 +628,17 @@ MesosSchedulerDriver::MesosSchedulerDriv
     return;
   }
 
+  // Initialize libprocess.
+  process::initialize();
+
+  // TODO(benh): Consider eliminating 'localquiet' so that we don't
+  // have to have weird semantics when the 'quiet' option is set to
+  // false but 'localquiet' is being used.
+  conf->set("quiet", master == "localquiet");
+
+  // TODO(benh): Replace whitespace in framework.name() with '_'?
+  logging::initialize(framework.name(), *conf);
+
   // Initialize mutex and condition variable.
   pthread_mutexattr_t attr;
   pthread_mutexattr_init(&attr);
@@ -633,11 +646,6 @@ MesosSchedulerDriver::MesosSchedulerDriv
   pthread_mutex_init(&mutex, &attr);
   pthread_mutexattr_destroy(&attr);
   pthread_cond_init(&cond, 0);
-
-  // TODO(benh): Initialize glog.
-
-  // Initialize libprocess library (but not glog, done above).
-  process::initialize(false);
 }
 
 

Modified: incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/lxc_isolation_module.cpp Mon Apr 30 22:23:42 2012
@@ -178,10 +178,10 @@ void LxcIsolationModule::launchExecutor(
 			   frameworkInfo.user(),
                            directory,
 			   slave,
-			   conf.get("frameworks_home", ""),
-			   conf.get("hadoop_home", ""),
+			   conf.get<string>("frameworks_home", ""),
+			   conf.get<string>("hadoop_home", ""),
 			   !local,
-			   conf.get("switch_user", true),
+			   conf.get<bool>("switch_user", true),
 			   container);
 
     launcher->setupEnvironmentForLauncherMain();
@@ -204,7 +204,7 @@ void LxcIsolationModule::launchExecutor(
 
     // Determine path for mesos-launcher from Mesos home directory.
     string path =
-      conf.get("launcher_dir", MESOS_LIBEXECDIR) + "/mesos-launcher";
+      conf.get<string>("launcher_dir", MESOS_LIBEXECDIR) + "/mesos-launcher";
     args[i++] = path.c_str();
     args[i++] = NULL;
 

Modified: incubator/mesos/trunk/src/slave/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/main.cpp (original)
+++ incubator/mesos/trunk/src/slave/main.cpp Mon Apr 30 22:23:42 2012
@@ -16,11 +16,10 @@
  * limitations under the License.
  */
 
-#include <libgen.h> // For chdir.
-
 #include "common/build.hpp"
 #include "common/logging.hpp"
 #include "common/try.hpp"
+#include "common/utils.hpp"
 
 #include "configurator/configurator.hpp"
 
@@ -38,10 +37,9 @@ using std::endl;
 using std::string;
 
 
-void usage(const char *programName, const Configurator& configurator)
+void usage(const char* argv0, const Configurator& configurator)
 {
-  cerr << "Usage: " << programName << " [...]" << endl
-       << endl
+  cerr << "Usage: " << utils::os::basename(argv0) << " [...]" << endl
        << endl
        << "Supported options:" << endl
        << configurator.getUsage();
@@ -53,7 +51,9 @@ int main(int argc, char** argv)
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
   Configurator configurator;
-  Logging::registerOptions(&configurator);
+
+  logging::registerOptions(&configurator);
+
   Slave::registerOptions(&configurator);
 
   // The following options are executable specific (e.g., since we
@@ -87,18 +87,18 @@ int main(int argc, char** argv)
     exit(1);
   }
 
-  Logging::init(argv[0], conf);
-
   if (conf.contains("port")) {
-    setenv("LIBPROCESS_PORT", conf["port"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_PORT", conf["port"]);
   }
 
   if (conf.contains("ip")) {
-    setenv("LIBPROCESS_IP", conf["ip"].c_str(), 1);
+    utils::os::setenv("LIBPROCESS_IP", conf["ip"]);
   }
 
-  // Initialize libprocess library (but not glog, done above).
-  process::initialize(false);
+  // Initialize libprocess.
+  process::initialize();
+
+  logging::initialize(argv[0], conf);
 
   if (!conf.contains("master")) {
     cerr << "Missing required option --master (-m)" << endl;
@@ -119,15 +119,13 @@ int main(int argc, char** argv)
   LOG(INFO) << "Build: " << build::DATE << " by " << build::USER;
   LOG(INFO) << "Starting Mesos slave";
 
-  if (chdir(dirname(argv[0])) != 0) {
-    fatalerror("Could not chdir into %s", dirname(argv[0]));
-  }
-
   Slave* slave = new Slave(conf, false, isolationModule);
   process::spawn(slave);
 
+  bool quiet = conf.get<bool>("quiet", false);
+
   Try<MasterDetector*> detector =
-    MasterDetector::create(master, slave->self(), false, Logging::isQuiet(conf));
+    MasterDetector::create(master, slave->self(), false, quiet);
 
   CHECK(detector.isSome())
     << "Failed to create a master detector: " << detector.error();

Modified: incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp (original)
+++ incubator/mesos/trunk/src/slave/process_based_isolation_module.cpp Mon Apr 30 22:23:42 2012
@@ -190,10 +190,10 @@ ExecutorLauncher* ProcessBasedIsolationM
                               frameworkInfo.user(),
                               directory,
                               slave,
-                              conf.get("frameworks_home", ""),
-                              conf.get("hadoop_home", ""),
+                              conf.get<string>("frameworks_home", ""),
+                              conf.get<string>("hadoop_home", ""),
                               !local,
-                              conf.get("switch_user", true),
+                              conf.get<bool>("switch_user", true),
                               "");
 }
 

Modified: incubator/mesos/trunk/src/slave/slave.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.cpp (original)
+++ incubator/mesos/trunk/src/slave/slave.cpp Mon Apr 30 22:23:42 2012
@@ -1504,7 +1504,7 @@ string Slave::createUniqueWorkDirectory(
             << executorId << "' of framework " << frameworkId;
 
   std::ostringstream out(std::ios_base::app | std::ios_base::out);
-  out << conf.get("work_dir", "/tmp/mesos")
+  out << conf.get<string>("work_dir", "/tmp/mesos")
       << "/slaves/" << id
       << "/frameworks/" << frameworkId
       << "/executors/" << executorId;

Modified: incubator/mesos/trunk/src/slave/slave.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/slave.hpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/slave.hpp (original)
+++ incubator/mesos/trunk/src/slave/slave.hpp Mon Apr 30 22:23:42 2012
@@ -322,8 +322,11 @@ struct Framework
       executor.mutable_executor_id()->set_value(id);
 
       // Now determine the path to the executor.
-      Try<std::string> path = utils::os::realpath(
-          conf.get("launcher_dir", MESOS_LIBEXECDIR) + "/mesos-executor");
+      std::string directory =
+        conf.get<std::string>("launcher_dir", MESOS_LIBEXECDIR);
+
+      Try<std::string> path =
+        utils::os::realpath(directory + "/mesos-executor");
 
       if (path.isSome()) {
         executor.mutable_command()->set_value(path.get());

Modified: incubator/mesos/trunk/src/slave/webui.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/slave/webui.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/slave/webui.cpp (original)
+++ incubator/mesos/trunk/src/slave/webui.cpp Mon Apr 30 22:23:42 2012
@@ -38,9 +38,9 @@ void start(const process::PID<Slave>& sl
 {
   std::vector<std::string> args(4);
   args[0] = "--slave_port=" + utils::stringify(slave.port);
-  args[1] = "--webui_port=" + conf.get("webui_port", "8081");
-  args[2] = "--log_dir=" + conf.get("log_dir", FLAGS_log_dir);
-  args[3] = "--work_dir=" + conf.get("work_dir", "/tmp/mesos");
+  args[1] = "--webui_port=" + conf.get<std::string>("webui_port", "8081");
+  args[2] = "--log_dir=" + conf.get<std::string>("log_dir", FLAGS_log_dir);
+  args[3] = "--work_dir=" + conf.get<std::string>("work_dir", "/tmp/mesos");
 
   utils::webui::start(conf, "slave/webui.py", args);
 }

Modified: incubator/mesos/trunk/src/tests/main.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/src/tests/main.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/src/tests/main.cpp (original)
+++ incubator/mesos/trunk/src/tests/main.cpp Mon Apr 30 22:23:42 2012
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-#include <glog/logging.h>
-
 #include <gtest/gtest.h>
 
 #include <string>
 
 #include <process/process.hpp>
 
+#include "common/logging.hpp"
 #include "common/utils.hpp"
 
+#include "configurator/configuration.hpp"
 #include "configurator/configurator.hpp"
 
 #include "tests/utils.hpp"
@@ -33,26 +33,69 @@
 using namespace mesos::internal;
 using namespace mesos::internal::test;
 
+using std::cerr;
+using std::endl;
 using std::string;
 
+
+void usage(const char* argv0, const Configurator& configurator)
+{
+  cerr << "Usage: " << utils::os::basename(argv0) << " [...]" << endl
+       << endl
+       << "Supported options:" << endl
+       << configurator.getUsage();
+}
+
+
 int main(int argc, char** argv)
 {
   GOOGLE_PROTOBUF_VERIFY_VERSION;
 
-  // Initialize glog.
-  google::InitGoogleLogging("alltests");
+  Configurator configurator;
+
+  logging::registerOptions(&configurator);
 
-  if (argc >= 2 && strcmp("-v", argv[1]) == 0) {
-    google::SetStderrLogging(google::INFO);
+  // We log to stderr by default, but when running tests we'd prefer
+  // less junk to fly by, so force one to specify the verbosity.
+  configurator.addOption<bool>(
+      "verbose",
+      'v',
+      "Log all severity levels to stderr (default: serverity above ERROR)",
+      false);
+
+  configurator.addOption<bool>(
+      "help",
+      'h',
+      "Prints this usage message");
+
+  Configuration conf;
+  try {
+    conf = configurator.load(argc, argv);
+  } catch (const ConfigurationException& e) {
+    cerr << "Configuration error: " << e.what() << endl;
+    exit(1);
   }
 
+  if (conf.contains("help")) {
+    usage(argv[0], configurator);
+    cerr << endl;
+    testing::InitGoogleTest(&argc, argv); // Get usage from gtest too.
+    exit(1);
+  }
+
+  // Initialize libprocess.
+  process::initialize();
+
+  // Be quiet by default!
+  conf.set("quiet", !conf.get("verbose", false));
+
+  // Initialize logging.
+  logging::initialize(argv[0], conf);
+
   // Initialize gmock/gtest.
   testing::InitGoogleTest(&argc, argv);
   testing::FLAGS_gtest_death_test_style = "threadsafe";
 
-  // Initialize libprocess (but not glog, done above).
-  process::initialize(false);
-
   // Get the absolute path to the source (i.e., root) directory.
   Try<string> path = utils::os::realpath(SOURCE_DIR);
   CHECK(path.isSome()) << "Error getting source directory " << path.error();

Added: incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp?rev=1332449&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp (added)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/nothing.hpp Mon Apr 30 22:23:42 2012
@@ -0,0 +1,6 @@
+#ifndef __NOTHING_HPP__
+#define __NOTHING_HPP__
+
+struct Nothing {};
+
+#endif // __NOTHING_HPP__

Added: incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp?rev=1332449&view=auto
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp (added)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/once.hpp Mon Apr 30 22:23:42 2012
@@ -0,0 +1,47 @@
+#ifndef __PROCESS_ONCE_HPP__
+#define __PROCESS_ONCE_HPP__
+
+#include <process/future.hpp>
+#include <process/nothing.hpp>
+
+namespace process {
+
+// Provides a _blocking_ abstraction that's useful for performing a
+// task exactly once.
+class Once
+{
+public:
+  Once() {}
+
+  // Returns true if this Once instance has already transitioned to a
+  // 'done' state (i.e., the action you wanted to perform "once" has
+  // been completed). Note that this BLOCKS until Once::done has been
+  // called.
+  bool once()
+  {
+    if (!outer.set(&inner)) {
+      inner.future().await();
+      return true;
+    }
+
+    return false;
+  }
+
+  // Transitions this Once instance to a 'done' state.
+  void done()
+  {
+    inner.set(Nothing());
+  }
+
+private:
+  // Not copyable, not assignable.
+  Once(const Once& that);
+  Once& operator = (const Once& that);
+
+  Promise<Nothing> inner;
+  Promise<Promise<Nothing>*> outer;
+};
+
+}  // namespace process {
+
+#endif // __PROCESS_ONCE_HPP__

Modified: incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/include/process/process.hpp Mon Apr 30 22:23:42 2012
@@ -224,19 +224,14 @@ public:
 
 
 /**
- * Initialize the library.
+ * Initialize the library. Note that libprocess uses Google's glog and
+ * you can specify options for it (e.g., a logging directory) via
+ * environment variables (see the glog documentation for more
+ * information).
  *
  * @param delegate process to receive root HTTP requests
- * @param initialize_glog whether or not to initialize the
- *        Google Logging library (glog). If the application is also
- *        using glog, this should be set to false.
  */
-void initialize(const std::string& delegate = "", bool initialize_glog = true);
-
-inline void initialize(bool initialize_glog)
-{
-  initialize("", initialize_glog);
-}
+void initialize(const std::string& delegate = "");
 
 
 /**

Modified: incubator/mesos/trunk/third_party/libprocess/src/process.cpp
URL: http://svn.apache.org/viewvc/incubator/mesos/trunk/third_party/libprocess/src/process.cpp?rev=1332449&r1=1332448&r2=1332449&view=diff
==============================================================================
--- incubator/mesos/trunk/third_party/libprocess/src/process.cpp (original)
+++ incubator/mesos/trunk/third_party/libprocess/src/process.cpp Mon Apr 30 22:23:42 2012
@@ -1137,7 +1137,7 @@ void* schedule(void* arg)
 // }
 
 
-void initialize(const string& delegate, bool initialize_glog)
+void initialize(const string& delegate)
 {
   // TODO(benh): Return an error if attempting to initialize again
   // with a different delegate then originally specified.
@@ -1161,11 +1161,6 @@ void initialize(const string& delegate, 
     }
   }
 
-  if (initialize_glog) {
-    google::InitGoogleLogging("<<libprocess>>");
-    google::LogToStderr();
-  }
-
 //   // Install signal handler.
 //   struct sigaction sa;